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

[01/29] tajo git commit: TAJO-1188: Fix testcase testTimestampConstructor in TestTimestampDatum. (DaeMyung Kang via hyunsik)

Repository: tajo
Updated Branches:
  refs/heads/hbase_storage dfd7f996d -> 1526b7d10


TAJO-1188: Fix testcase testTimestampConstructor in TestTimestampDatum. (DaeMyung Kang via hyunsik)


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

Branch: refs/heads/hbase_storage
Commit: 0c97fc0319b0f33c3a76f591d48cb52f1df92567
Parents: f6e09a5
Author: Hyunsik Choi <hy...@apache.org>
Authored: Tue Nov 25 17:42:22 2014 +0900
Committer: Hyunsik Choi <hy...@apache.org>
Committed: Tue Nov 25 17:42:22 2014 +0900

----------------------------------------------------------------------
 CHANGES                                                           | 3 +++
 .../src/test/java/org/apache/tajo/datum/TestTimestampDatum.java   | 3 ++-
 2 files changed, 5 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tajo/blob/0c97fc03/CHANGES
----------------------------------------------------------------------
diff --git a/CHANGES b/CHANGES
index 06a0f47..6521c1c 100644
--- a/CHANGES
+++ b/CHANGES
@@ -62,6 +62,9 @@ Release 0.9.1 - unreleased
 
     TAJO-1163: TableDesc should use URI instead of Path. (hyunsik)
 
+    TAJO-1188: Fix testcase testTimestampConstructor in TestTimestampDatum.
+    (DaeMyung Kang via hyunsik)
+
 
   BUG FIXES
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/0c97fc03/tajo-common/src/test/java/org/apache/tajo/datum/TestTimestampDatum.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/test/java/org/apache/tajo/datum/TestTimestampDatum.java b/tajo-common/src/test/java/org/apache/tajo/datum/TestTimestampDatum.java
index 5f27cfa..5886083 100644
--- a/tajo-common/src/test/java/org/apache/tajo/datum/TestTimestampDatum.java
+++ b/tajo-common/src/test/java/org/apache/tajo/datum/TestTimestampDatum.java
@@ -151,7 +151,8 @@ public class TestTimestampDatum {
     assertEquals(datum2, datum);
 
     for (int i = 0; i < 100; i++) {
-      Calendar cal = Calendar.getInstance();
+      TimeZone timeZone = TimeZone.getTimeZone("UTC");
+      Calendar cal = Calendar.getInstance(timeZone);
       long jTime = System.currentTimeMillis();
       int uTime = (int)(jTime / 1000);
       cal.setTimeInMillis(jTime);


[07/29] tajo git commit: TAJO-1165: Needs to show error messages on query_executor.jsp. (Jihun Kang via jaehwa)

Posted by hj...@apache.org.
TAJO-1165: Needs to show error messages on query_executor.jsp. (Jihun Kang via jaehwa)


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

Branch: refs/heads/hbase_storage
Commit: 4637f7fd126c46f0e6a4bcdf2d816511b690552b
Parents: f329162
Author: JaeHwa Jung <bl...@apache.org>
Authored: Thu Nov 27 22:34:53 2014 +0900
Committer: JaeHwa Jung <bl...@apache.org>
Committed: Thu Nov 27 22:34:53 2014 +0900

----------------------------------------------------------------------
 CHANGES                                             |  2 ++
 .../apache/tajo/webapp/QueryExecutorServlet.java    | 16 ++++++++++++++++
 .../main/resources/webapps/admin/query_executor.jsp |  4 ++--
 3 files changed, 20 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tajo/blob/4637f7fd/CHANGES
----------------------------------------------------------------------
diff --git a/CHANGES b/CHANGES
index 9b15898..8bf0c1c 100644
--- a/CHANGES
+++ b/CHANGES
@@ -13,6 +13,8 @@ Release 0.9.1 - unreleased
 
   IMPROVEMENT
 
+    TAJO-1165: Needs to show error messages on query_executor.jsp. (Jihun Kang via jaehwa)
+
     TAJO-1204: Remove unused ServerName class. (DaeMyung Kang via jaehwa)
 
     TAJO-1053: ADD PARTITIONS for HCatalogStore. (jaehwa)

http://git-wip-us.apache.org/repos/asf/tajo/blob/4637f7fd/tajo-core/src/main/java/org/apache/tajo/webapp/QueryExecutorServlet.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/webapp/QueryExecutorServlet.java b/tajo-core/src/main/java/org/apache/tajo/webapp/QueryExecutorServlet.java
index 0075b04..da25fe6 100644
--- a/tajo-core/src/main/java/org/apache/tajo/webapp/QueryExecutorServlet.java
+++ b/tajo-core/src/main/java/org/apache/tajo/webapp/QueryExecutorServlet.java
@@ -316,6 +316,22 @@ public class QueryExecutorServlet extends HttpServlet {
 
             progress.set(100);
           }
+        } else if (response.getResultCode() == ClientProtos.ResultCode.ERROR) {
+          if (response.hasErrorMessage()) {
+            StringBuffer errorMessage = new StringBuffer(response.getErrorMessage());
+            String modifiedMessage;
+
+            if (errorMessage.length() > 200) {
+              modifiedMessage = errorMessage.substring(0, 200);
+            } else {
+              modifiedMessage = errorMessage.toString();
+            }
+            
+            String lineSeparator = System.getProperty("line.separator");
+            modifiedMessage = modifiedMessage.replaceAll(lineSeparator, "<br/>");
+
+            error = new Exception(modifiedMessage);
+          }
         }
       } catch (Exception e) {
         LOG.error(e.getMessage(), e);

http://git-wip-us.apache.org/repos/asf/tajo/blob/4637f7fd/tajo-core/src/main/resources/webapps/admin/query_executor.jsp
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/resources/webapps/admin/query_executor.jsp b/tajo-core/src/main/resources/webapps/admin/query_executor.jsp
index bbd1820..9ff6625 100644
--- a/tajo-core/src/main/resources/webapps/admin/query_executor.jsp
+++ b/tajo-core/src/main/resources/webapps/admin/query_executor.jsp
@@ -102,7 +102,7 @@ function runQuery() {
     var resultJson = $.parseJSON(msg);
     if(resultJson.success == "false") {
       clearTimer();
-      alert(resultJson.errorMessage);
+      alert("query execution failed.");
       return;
     }
     queryRunnerId = resultJson.queryRunnerId;
@@ -116,7 +116,7 @@ function runQuery() {
         var resultJson = $.parseJSON(msg);
         if(resultJson.success == "false") {
           clearTimer();
-          alert(resultJson.errorMessage);
+          alert("query execution failed.");
           $("#queryStatus").html(getQueryStatusHtml(resultJson));
           return;
         }


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

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

http://git-wip-us.apache.org/repos/asf/tajo/blob/0073ef23/tajo-storage/tajo-storage-common/src/test/java/org/apache/tajo/tuple/offheap/TestResizableSpec.java
----------------------------------------------------------------------
diff --cc tajo-storage/tajo-storage-common/src/test/java/org/apache/tajo/tuple/offheap/TestResizableSpec.java
index 0000000,0000000..1eb9c17
new file mode 100644
--- /dev/null
+++ b/tajo-storage/tajo-storage-common/src/test/java/org/apache/tajo/tuple/offheap/TestResizableSpec.java
@@@ -1,0 -1,0 +1,59 @@@
++/*
++ * Licensed to the Apache Software Foundation (ASF) under one
++ * or more contributor license agreements.  See the NOTICE file
++ * distributed with this work for additional information
++ * regarding copyright ownership.  The ASF licenses this file
++ * to you under the Apache License, Version 2.0 (the
++ * "License"); you may not use this file except in compliance
++ * with the License.  You may obtain a copy of the License at
++ *
++ *     http://www.apache.org/licenses/LICENSE-2.0
++ *
++ * Unless required by applicable law or agreed to in writing, software
++ * distributed under the License is distributed on an "AS IS" BASIS,
++ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
++ * See the License for the specific language governing permissions and
++ * limitations under the License.
++ */
++
++package org.apache.tajo.tuple.offheap;
++
++import org.apache.tajo.unit.StorageUnit;
++import org.junit.Test;
++
++import static org.junit.Assert.*;
++
++public class TestResizableSpec {
++
++  @Test
++  public void testResizableLimit() {
++    ResizableLimitSpec limit = new ResizableLimitSpec(10 * StorageUnit.MB, 1000 * StorageUnit.MB, 0.1f, 1.0f);
++
++    long expectedMaxSize = (long) (1000 * StorageUnit.MB + (1000 * StorageUnit.MB * 0.1f));
++
++    assertTrue(limit.limit() == 1000 * StorageUnit.MB + (1000 * StorageUnit.MB * 0.1f));
++
++    assertEquals(20971520, limit.increasedSize(10 * StorageUnit.MB));
++
++    assertEquals(expectedMaxSize, limit.increasedSize(1600 * StorageUnit.MB));
++
++    assertEquals(0.98f, limit.remainRatio(980 * StorageUnit.MB), 0.1);
++
++    assertFalse(limit.canIncrease(limit.limit()));
++  }
++
++  @Test
++  public void testFixedLimit() {
++    FixedSizeLimitSpec limit = new FixedSizeLimitSpec(100 * StorageUnit.MB, 0.0f);
++
++    assertEquals(limit.limit(), 100 * StorageUnit.MB);
++
++    assertEquals(100 * StorageUnit.MB, limit.increasedSize(1000));
++
++    assertEquals(100 * StorageUnit.MB, limit.increasedSize(1600 * StorageUnit.MB));
++
++    assertTrue(0.98f == limit.remainRatio(98 * StorageUnit.MB));
++
++    assertFalse(limit.canIncrease(limit.limit()));
++  }
++}

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

http://git-wip-us.apache.org/repos/asf/tajo/blob/0073ef23/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/avro/AvroAppender.java
----------------------------------------------------------------------
diff --cc tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/avro/AvroAppender.java
index 85f91cc,0000000..dbb8bd0
mode 100644,000000..100644
--- a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/avro/AvroAppender.java
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/avro/AvroAppender.java
@@@ -1,219 -1,0 +1,220 @@@
 +/**
 + * Licensed to the Apache Software Foundation (ASF) under one
 + * or more contributor license agreements.  See the NOTICE file
 + * distributed with this work for additional information
 + * regarding copyright ownership.  The ASF licenses this file
 + * to you under the Apache License, Version 2.0 (the
 + * "License"); you may not use this file except in compliance
 + * with the License.  You may obtain a copy of the License at
 + *
 + *     http://www.apache.org/licenses/LICENSE-2.0
 + *
 + * Unless required by applicable law or agreed to in writing, software
 + * distributed under the License is distributed on an "AS IS" BASIS,
 + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 + * See the License for the specific language governing permissions and
 + * limitations under the License.
 + */
 +
 +package org.apache.tajo.storage.avro;
 +
 +import org.apache.avro.Schema;
 +import org.apache.avro.file.DataFileWriter;
 +import org.apache.avro.generic.GenericData;
 +import org.apache.avro.generic.GenericDatumWriter;
 +import org.apache.avro.generic.GenericRecord;
 +import org.apache.avro.io.DatumWriter;
 +import org.apache.hadoop.conf.Configuration;
 +import org.apache.hadoop.fs.FSDataOutputStream;
 +import org.apache.hadoop.fs.FileSystem;
 +import org.apache.hadoop.fs.Path;
 +import org.apache.tajo.QueryUnitAttemptId;
 +import org.apache.tajo.catalog.Column;
 +import org.apache.tajo.catalog.TableMeta;
 +import org.apache.tajo.catalog.statistics.TableStats;
 +import org.apache.tajo.datum.NullDatum;
 +import org.apache.tajo.storage.FileAppender;
 +import org.apache.tajo.storage.TableStatistics;
 +import org.apache.tajo.storage.Tuple;
 +
 +import java.io.FileNotFoundException;
 +import java.io.IOException;
 +import java.nio.ByteBuffer;
 +import java.util.List;
 +
 +/**
 + * FileAppender for writing to Avro files.
 + */
 +public class AvroAppender extends FileAppender {
 +  private TableStatistics stats;
 +  private Schema avroSchema;
 +  private List<Schema.Field> avroFields;
 +  private DataFileWriter<GenericRecord> dataFileWriter;
 +
 +  /**
 +   * Creates a new AvroAppender.
 +   *
 +   * @param conf Configuration properties.
++   * @param taskAttemptId The task attempt id
 +   * @param schema The table schema.
 +   * @param meta The table metadata.
 +   * @param workDir The path of the Parquet file to write to.
 +   */
 +  public AvroAppender(Configuration conf,
 +                      QueryUnitAttemptId taskAttemptId,
 +                      org.apache.tajo.catalog.Schema schema,
 +                      TableMeta meta, Path workDir) throws IOException {
 +    super(conf, taskAttemptId, schema, meta, workDir);
 +  }
 +
 +  /**
 +   * Initializes the Appender.
 +   */
 +  public void init() throws IOException {
 +    FileSystem fs = path.getFileSystem(conf);
 +    if (!fs.exists(path.getParent())) {
 +      throw new FileNotFoundException(path.toString());
 +    }
 +    FSDataOutputStream outputStream = fs.create(path);
 +
 +    avroSchema = AvroUtil.getAvroSchema(meta, conf);
 +    avroFields = avroSchema.getFields();
 +
 +    DatumWriter<GenericRecord> datumWriter =
 +        new GenericDatumWriter<GenericRecord>(avroSchema);
 +    dataFileWriter = new DataFileWriter<GenericRecord>(datumWriter);
 +    dataFileWriter.create(avroSchema, outputStream);
 +
 +    if (enabledStats) {
 +      this.stats = new TableStatistics(schema);
 +    }
 +    super.init();
 +  }
 +
 +  /**
 +   * Gets the current offset. Tracking offsets is currenly not implemented, so
 +   * this method always returns 0.
 +   *
 +   * @return 0
 +   */
 +  @Override
 +  public long getOffset() throws IOException {
 +    return 0;
 +  }
 +
 +  private Object getPrimitive(Tuple tuple, int i, Schema.Type avroType) {
 +    if (tuple.get(i) instanceof NullDatum) {
 +      return null;
 +    }
 +    switch (avroType) {
 +      case NULL:
 +        return null;
 +      case BOOLEAN:
 +        return tuple.getBool(i);
 +      case INT:
 +        return tuple.getInt4(i);
 +      case LONG:
 +        return tuple.getInt8(i);
 +      case FLOAT:
 +        return tuple.getFloat4(i);
 +      case DOUBLE:
 +        return tuple.getFloat8(i);
 +      case BYTES:
 +      case FIXED:
 +        return ByteBuffer.wrap(tuple.getBytes(i));
 +      case STRING:
 +        return tuple.getText(i);
 +      default:
 +        throw new RuntimeException("Unknown primitive type.");
 +    }
 +  }
 +
 +  /**
 +   * Write a Tuple to the Avro file.
 +   *
 +   * @param tuple The Tuple to write.
 +   */
 +  @Override
 +  public void addTuple(Tuple tuple) throws IOException {
 +    GenericRecord record = new GenericData.Record(avroSchema);
 +    for (int i = 0; i < schema.size(); ++i) {
 +      Column column = schema.getColumn(i);
 +      if (enabledStats) {
 +        stats.analyzeField(i, tuple.get(i));
 +      }
 +      Object value;
 +      Schema.Field avroField = avroFields.get(i);
 +      Schema.Type avroType = avroField.schema().getType();
 +      switch (avroType) {
 +        case NULL:
 +        case BOOLEAN:
 +        case INT:
 +        case LONG:
 +        case FLOAT:
 +        case DOUBLE:
 +        case BYTES:
 +        case STRING:
 +        case FIXED:
 +          value = getPrimitive(tuple, i, avroType);
 +          break;
 +        case RECORD:
 +          throw new RuntimeException("Avro RECORD not supported.");
 +        case ENUM:
 +          throw new RuntimeException("Avro ENUM not supported.");
 +        case MAP:
 +          throw new RuntimeException("Avro MAP not supported.");
 +        case UNION:
 +          List<Schema> schemas = avroField.schema().getTypes();
 +          if (schemas.size() != 2) {
 +            throw new RuntimeException("Avro UNION not supported.");
 +          }
 +          if (schemas.get(0).getType().equals(Schema.Type.NULL)) {
 +            value = getPrimitive(tuple, i, schemas.get(1).getType());
 +          } else if (schemas.get(1).getType().equals(Schema.Type.NULL)) {
 +            value = getPrimitive(tuple, i, schemas.get(0).getType());
 +          } else {
 +            throw new RuntimeException("Avro UNION not supported.");
 +          }
 +          break;
 +        default:
 +          throw new RuntimeException("Unknown type: " + avroType);
 +      }
 +      record.put(i, value);
 +    }
 +    dataFileWriter.append(record);
 +
 +    if (enabledStats) {
 +      stats.incrementRow();
 +    }
 +  }
 +
 +  /**
 +   * Flushes the current state of the file.
 +   */
 +  @Override
 +  public void flush() throws IOException {
 +    dataFileWriter.flush();
 +  }
 +
 +  /**
 +   * Closes the Appender.
 +   */
 +  @Override
 +  public void close() throws IOException {
 +    dataFileWriter.close();
 +  }
 +
 +  /**
 +   * If table statistics is enabled, retrieve the table statistics.
 +   *
 +   * @return Table statistics if enabled or null otherwise.
 +   */
 +  @Override
 +  public TableStats getStats() {
 +    if (enabledStats) {
 +      return stats.getTableStat();
 +    } else {
 +      return null;
 +    }
 +  }
 +}

http://git-wip-us.apache.org/repos/asf/tajo/blob/0073ef23/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/json/JsonLineDeserializer.java
----------------------------------------------------------------------
diff --cc tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/json/JsonLineDeserializer.java
index c1f63a8,0000000..dfe36f6
mode 100644,000000..100644
--- a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/json/JsonLineDeserializer.java
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/json/JsonLineDeserializer.java
@@@ -1,220 -1,0 +1,225 @@@
 +/**
 + * Licensed to the Apache Software Foundation (ASF) under one
 + * or more contributor license agreements.  See the NOTICE file
 + * distributed with this work for additional information
 + * regarding copyright ownership.  The ASF licenses this file
 + * to you under the Apache License, Version 2.0 (the
 + * "License"); you may not use this file except in compliance
 + * with the License.  You may obtain a copy of the License at
 + *
 + *     http://www.apache.org/licenses/LICENSE-2.0
 + *
 + * Unless required by applicable law or agreed to in writing, software
 + * distributed under the License is distributed on an "AS IS" BASIS,
 + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 + * See the License for the specific language governing permissions and
 + * limitations under the License.
 + */
 +
 +package org.apache.tajo.storage.json;
 +
 +
 +import io.netty.buffer.ByteBuf;
 +import net.minidev.json.JSONArray;
 +import net.minidev.json.JSONObject;
 +import net.minidev.json.parser.JSONParser;
++import net.minidev.json.parser.ParseException;
 +import org.apache.tajo.catalog.Schema;
 +import org.apache.tajo.catalog.SchemaUtil;
 +import org.apache.tajo.catalog.TableMeta;
 +import org.apache.tajo.common.TajoDataTypes.Type;
 +import org.apache.tajo.common.exception.NotImplementedException;
 +import org.apache.tajo.datum.DatumFactory;
 +import org.apache.tajo.datum.NullDatum;
++import org.apache.tajo.datum.TextDatum;
++import org.apache.tajo.datum.protobuf.ProtobufJsonFormat;
 +import org.apache.tajo.storage.Tuple;
 +import org.apache.tajo.storage.text.TextLineDeserializer;
 +import org.apache.tajo.storage.text.TextLineParsingError;
 +
 +import java.io.IOException;
 +import java.util.Iterator;
 +
 +public class JsonLineDeserializer extends TextLineDeserializer {
 +  private JSONParser parser;
 +  private Type [] types;
 +  private String [] columnNames;
 +
 +  public JsonLineDeserializer(Schema schema, TableMeta meta, int[] targetColumnIndexes) {
 +    super(schema, meta, targetColumnIndexes);
 +  }
 +
 +  @Override
 +  public void init() {
 +    types = SchemaUtil.toTypes(schema);
 +    columnNames = SchemaUtil.toSimpleNames(schema);
 +
 +    parser = new JSONParser(JSONParser.MODE_JSON_SIMPLE);
 +  }
 +
 +  @Override
 +  public void deserialize(ByteBuf buf, Tuple output) throws IOException, TextLineParsingError {
 +    byte [] line = new byte[buf.readableBytes()];
 +    buf.readBytes(line);
 +
 +    try {
 +      JSONObject object = (JSONObject) parser.parse(line);
 +
 +      for (int i = 0; i < targetColumnIndexes.length; i++) {
 +        int actualIdx = targetColumnIndexes[i];
 +        String fieldName = columnNames[actualIdx];
 +
 +        if (!object.containsKey(fieldName)) {
 +          output.put(actualIdx, NullDatum.get());
 +          continue;
 +        }
 +
 +        switch (types[actualIdx]) {
 +        case BOOLEAN:
 +          String boolStr = object.getAsString(fieldName);
 +          if (boolStr != null) {
 +            output.put(actualIdx, DatumFactory.createBool(boolStr.equals("true")));
 +          } else {
 +            output.put(actualIdx, NullDatum.get());
 +          }
 +          break;
 +        case CHAR:
 +          String charStr = object.getAsString(fieldName);
 +          if (charStr != null) {
 +            output.put(actualIdx, DatumFactory.createChar(charStr));
 +          } else {
 +            output.put(actualIdx, NullDatum.get());
 +          }
 +          break;
 +        case INT1:
 +        case INT2:
 +          Number int2Num = object.getAsNumber(fieldName);
 +          if (int2Num != null) {
 +            output.put(actualIdx, DatumFactory.createInt2(int2Num.shortValue()));
 +          } else {
 +            output.put(actualIdx, NullDatum.get());
 +          }
 +          break;
 +        case INT4:
 +          Number int4Num = object.getAsNumber(fieldName);
 +          if (int4Num != null) {
 +            output.put(actualIdx, DatumFactory.createInt4(int4Num.intValue()));
 +          } else {
 +            output.put(actualIdx, NullDatum.get());
 +          }
 +          break;
 +        case INT8:
 +          Number int8Num = object.getAsNumber(fieldName);
 +          if (int8Num != null) {
 +            output.put(actualIdx, DatumFactory.createInt8(int8Num.longValue()));
 +          } else {
 +            output.put(actualIdx, NullDatum.get());
 +          }
 +          break;
 +        case FLOAT4:
 +          Number float4Num = object.getAsNumber(fieldName);
 +          if (float4Num != null) {
 +            output.put(actualIdx, DatumFactory.createFloat4(float4Num.floatValue()));
 +          } else {
 +            output.put(actualIdx, NullDatum.get());
 +          }
 +          break;
 +        case FLOAT8:
 +          Number float8Num = object.getAsNumber(fieldName);
 +          if (float8Num != null) {
 +            output.put(actualIdx, DatumFactory.createFloat8(float8Num.doubleValue()));
 +          } else {
 +            output.put(actualIdx, NullDatum.get());
 +          }
 +          break;
 +        case TEXT:
 +          String textStr = object.getAsString(fieldName);
 +          if (textStr != null) {
 +            output.put(actualIdx, DatumFactory.createText(textStr));
 +          } else {
 +            output.put(actualIdx, NullDatum.get());
 +          }
 +          break;
 +        case TIMESTAMP:
 +          String timestampStr = object.getAsString(fieldName);
 +          if (timestampStr != null) {
 +            output.put(actualIdx, DatumFactory.createTimestamp(timestampStr));
 +          } else {
 +            output.put(actualIdx, NullDatum.get());
 +          }
 +          break;
 +        case TIME:
 +          String timeStr = object.getAsString(fieldName);
 +          if (timeStr != null) {
 +            output.put(actualIdx, DatumFactory.createTime(timeStr));
 +          } else {
 +            output.put(actualIdx, NullDatum.get());
 +          }
 +          break;
 +        case DATE:
 +          String dateStr = object.getAsString(fieldName);
 +          if (dateStr != null) {
 +            output.put(actualIdx, DatumFactory.createDate(dateStr));
 +          } else {
 +            output.put(actualIdx, NullDatum.get());
 +          }
 +          break;
 +        case BIT:
 +        case BINARY:
 +        case VARBINARY:
 +        case BLOB: {
 +          Object jsonObject = object.get(fieldName);
 +
 +          if (jsonObject == null) {
 +            output.put(actualIdx, NullDatum.get());
 +            break;
-           } if (jsonObject instanceof String) {
-             output.put(actualIdx, DatumFactory.createBlob((String)jsonObject));
++          }
++          if (jsonObject instanceof String) {
++            output.put(actualIdx, DatumFactory.createBlob((String) jsonObject));
 +          } else if (jsonObject instanceof JSONArray) {
 +            JSONArray jsonArray = (JSONArray) jsonObject;
 +            byte[] bytes = new byte[jsonArray.size()];
 +            Iterator<Object> it = jsonArray.iterator();
 +            int arrayIdx = 0;
 +            while (it.hasNext()) {
 +              bytes[arrayIdx++] = ((Long) it.next()).byteValue();
 +            }
 +            if (bytes.length > 0) {
 +              output.put(actualIdx, DatumFactory.createBlob(bytes));
 +            } else {
 +              output.put(actualIdx, NullDatum.get());
 +            }
 +            break;
 +          } else {
 +            throw new IOException("Unknown json object: " + object.getClass().getSimpleName());
 +          }
 +          break;
 +        }
 +        case INET4:
 +          String inetStr = object.getAsString(fieldName);
 +          if (inetStr != null) {
 +            output.put(actualIdx, DatumFactory.createInet4(inetStr));
 +          } else {
 +            output.put(actualIdx, NullDatum.get());
 +          }
 +          break;
 +
 +        case NULL_TYPE:
 +          output.put(actualIdx, NullDatum.get());
 +          break;
 +
 +        default:
 +          throw new NotImplementedException(types[actualIdx].name() + " is not supported.");
 +        }
 +      }
- 
++    } catch (ParseException pe) {
++      throw new TextLineParsingError(new String(line, TextDatum.DEFAULT_CHARSET), pe);
 +    } catch (Throwable e) {
 +      throw new IOException(e);
 +    }
 +  }
 +
 +  @Override
 +  public void release() {
 +  }
 +}

http://git-wip-us.apache.org/repos/asf/tajo/blob/0073ef23/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/DelimitedTextFile.java
----------------------------------------------------------------------
diff --cc tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/DelimitedTextFile.java
index 7848198,0000000..8824e3e
mode 100644,000000..100644
--- a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/DelimitedTextFile.java
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/DelimitedTextFile.java
@@@ -1,475 -1,0 +1,481 @@@
 +/**
 + * Licensed to the Apache Software Foundation (ASF) under one
 + * or more contributor license agreements.  See the NOTICE file
 + * distributed with this work for additional information
 + * regarding copyright ownership.  The ASF licenses this file
 + * to you under the Apache License, Version 2.0 (the
 + * "License"); you may not use this file except in compliance
 + * with the License.  You may obtain a copy of the License at
 + *
 + *     http://www.apache.org/licenses/LICENSE-2.0
 + *
 + * Unless required by applicable law or agreed to in writing, software
 + * distributed under the License is distributed on an "AS IS" BASIS,
 + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 + * See the License for the specific language governing permissions and
 + * limitations under the License.
 + */
 +
 +package org.apache.tajo.storage.text;
 +
 +import io.netty.buffer.ByteBuf;
 +import org.apache.commons.logging.Log;
 +import org.apache.commons.logging.LogFactory;
 +import org.apache.hadoop.conf.Configuration;
 +import org.apache.hadoop.fs.FSDataOutputStream;
 +import org.apache.hadoop.fs.FileSystem;
 +import org.apache.hadoop.fs.Path;
 +import org.apache.hadoop.io.IOUtils;
 +import org.apache.hadoop.io.compress.CompressionCodec;
 +import org.apache.hadoop.io.compress.CompressionCodecFactory;
 +import org.apache.hadoop.io.compress.CompressionOutputStream;
 +import org.apache.hadoop.io.compress.Compressor;
 +import org.apache.tajo.QueryUnitAttemptId;
 +import org.apache.tajo.catalog.Schema;
 +import org.apache.tajo.catalog.TableMeta;
 +import org.apache.tajo.catalog.statistics.TableStats;
 +import org.apache.tajo.storage.*;
 +import org.apache.tajo.storage.compress.CodecPool;
 +import org.apache.tajo.storage.exception.AlreadyExistsStorageException;
 +import org.apache.tajo.storage.fragment.FileFragment;
 +import org.apache.tajo.storage.fragment.Fragment;
 +import org.apache.tajo.storage.rcfile.NonSyncByteArrayOutputStream;
 +import org.apache.tajo.util.ReflectionUtil;
 +
 +import java.io.BufferedOutputStream;
 +import java.io.DataOutputStream;
 +import java.io.FileNotFoundException;
 +import java.io.IOException;
 +import java.util.Arrays;
 +import java.util.Map;
 +import java.util.concurrent.ConcurrentHashMap;
 +
 +import static org.apache.tajo.storage.StorageConstants.DEFAULT_TEXT_ERROR_TOLERANCE_MAXNUM;
 +import static org.apache.tajo.storage.StorageConstants.TEXT_ERROR_TOLERANCE_MAXNUM;
 +
 +public class DelimitedTextFile {
 +
 +  public static final byte LF = '\n';
 +
 +  private static final Log LOG = LogFactory.getLog(DelimitedTextFile.class);
 +
 +  /** it caches line serde classes. */
 +  private static final Map<String, Class<? extends TextLineSerDe>> serdeClassCache =
 +      new ConcurrentHashMap<String, Class<? extends TextLineSerDe>>();
 +
 +  /**
 +   * By default, DelimitedTextFileScanner uses CSVLineSerder. If a table property 'text.serde.class' is given,
 +   * it will use the specified serder class.
 +   *
 +   * @return TextLineSerder
 +   */
 +  public static TextLineSerDe getLineSerde(TableMeta meta) {
 +    TextLineSerDe lineSerder;
 +
 +    String serDeClassName;
 +
 +    // if there is no given serde class, it will use CSV line serder.
 +    serDeClassName = meta.getOption(StorageConstants.TEXT_SERDE_CLASS, StorageConstants.DEFAULT_TEXT_SERDE_CLASS);
 +
 +    try {
 +      Class<? extends TextLineSerDe> serdeClass;
 +
 +      if (serdeClassCache.containsKey(serDeClassName)) {
 +        serdeClass = serdeClassCache.get(serDeClassName);
 +      } else {
 +        serdeClass = (Class<? extends TextLineSerDe>) Class.forName(serDeClassName);
 +        serdeClassCache.put(serDeClassName, serdeClass);
 +      }
 +      lineSerder = (TextLineSerDe) ReflectionUtil.newInstance(serdeClass);
 +    } catch (Throwable e) {
 +      throw new RuntimeException("TextLineSerde class cannot be initialized.", e);
 +    }
 +
 +    return lineSerder;
 +  }
 +
 +  public static class DelimitedTextFileAppender extends FileAppender {
 +    private final TableMeta meta;
 +    private final Schema schema;
 +    private final FileSystem fs;
 +    private FSDataOutputStream fos;
 +    private DataOutputStream outputStream;
 +    private CompressionOutputStream deflateFilter;
 +    private TableStatistics stats = null;
 +    private Compressor compressor;
 +    private CompressionCodecFactory codecFactory;
 +    private CompressionCodec codec;
 +    private Path compressedPath;
 +    private byte[] nullChars;
 +    private int BUFFER_SIZE = 128 * 1024;
 +    private int bufferedBytes = 0;
 +    private long pos = 0;
 +
 +    private NonSyncByteArrayOutputStream os;
 +    private TextLineSerializer serializer;
 +
 +    public DelimitedTextFileAppender(Configuration conf, QueryUnitAttemptId taskAttemptId,
 +                                     final Schema schema, final TableMeta meta, final Path path)
 +        throws IOException {
 +      super(conf, taskAttemptId, schema, meta, path);
 +      this.fs = path.getFileSystem(conf);
 +      this.meta = meta;
 +      this.schema = schema;
 +    }
 +
 +    public TextLineSerDe getLineSerde() {
 +      return DelimitedTextFile.getLineSerde(meta);
 +    }
 +
 +    @Override
 +    public void init() throws IOException {
 +      if (!fs.exists(path.getParent())) {
 +        throw new FileNotFoundException(path.toString());
 +      }
 +
 +      if (this.meta.containsOption(StorageConstants.COMPRESSION_CODEC)) {
 +        String codecName = this.meta.getOption(StorageConstants.COMPRESSION_CODEC);
 +        codecFactory = new CompressionCodecFactory(conf);
 +        codec = codecFactory.getCodecByClassName(codecName);
 +        compressor = CodecPool.getCompressor(codec);
 +        if (compressor != null) compressor.reset();  //builtin gzip is null
 +
 +        String extension = codec.getDefaultExtension();
 +        compressedPath = path.suffix(extension);
 +
 +        if (fs.exists(compressedPath)) {
 +          throw new AlreadyExistsStorageException(compressedPath);
 +        }
 +
 +        fos = fs.create(compressedPath);
 +        deflateFilter = codec.createOutputStream(fos, compressor);
 +        outputStream = new DataOutputStream(deflateFilter);
 +
 +      } else {
 +        if (fs.exists(path)) {
 +          throw new AlreadyExistsStorageException(path);
 +        }
 +        fos = fs.create(path);
 +        outputStream = new DataOutputStream(new BufferedOutputStream(fos));
 +      }
 +
 +      if (enabledStats) {
 +        this.stats = new TableStatistics(this.schema);
 +      }
 +
 +      serializer = getLineSerde().createSerializer(schema, meta);
 +      serializer.init();
 +
 +      if (os == null) {
 +        os = new NonSyncByteArrayOutputStream(BUFFER_SIZE);
 +      }
 +
 +      os.reset();
 +      pos = fos.getPos();
 +      bufferedBytes = 0;
 +      super.init();
 +    }
 +
 +    @Override
 +    public void addTuple(Tuple tuple) throws IOException {
 +      // write
 +      int rowBytes = serializer.serialize(os, tuple);
 +
 +      // new line
 +      os.write(LF);
 +      rowBytes += 1;
 +
 +      // update positions
 +      pos += rowBytes;
 +      bufferedBytes += rowBytes;
 +
 +      // refill buffer if necessary
 +      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 {
 +        serializer.release();
 +
 +        if(outputStream != null){
 +          flush();
 +        }
 +
 +        // Statistical section
 +        if (enabledStats) {
 +          stats.setNumBytes(getOffset());
 +        }
 +
 +        if (deflateFilter != null) {
 +          deflateFilter.finish();
 +          deflateFilter.resetState();
 +          deflateFilter = null;
 +        }
 +
 +        os.close();
 +      } finally {
 +        IOUtils.cleanup(LOG, fos);
 +        if (compressor != null) {
 +          CodecPool.returnCompressor(compressor);
 +          compressor = null;
 +        }
 +      }
 +    }
 +
 +    @Override
 +    public TableStats getStats() {
 +      if (enabledStats) {
 +        return stats.getTableStat();
 +      } else {
 +        return null;
 +      }
 +    }
 +
 +    public boolean isCompress() {
 +      return compressor != null;
 +    }
 +
 +    public String getExtension() {
 +      return codec != null ? codec.getDefaultExtension() : "";
 +    }
 +  }
 +
 +  public static class DelimitedTextFileScanner extends FileScanner {
 +    private boolean splittable = false;
 +    private final long startOffset;
 +
 +    private final long endOffset;
 +    /** The number of actual read records */
 +    private int recordCount = 0;
 +    private int[] targetColumnIndexes;
 +
 +    private DelimitedLineReader reader;
 +    private TextLineDeserializer deserializer;
 +
 +    private int errorPrintOutMaxNum = 5;
 +    /** Maximum number of permissible errors */
 +    private int errorTorrenceMaxNum;
 +    /** How many errors have occurred? */
 +    private int errorNum;
 +
 +    public DelimitedTextFileScanner(Configuration conf, final Schema schema, final TableMeta meta,
 +                                    final Fragment fragment)
 +        throws IOException {
 +      super(conf, schema, meta, fragment);
 +      reader = new DelimitedLineReader(conf, this.fragment);
 +      if (!reader.isCompressed()) {
 +        splittable = true;
 +      }
 +
 +      startOffset = this.fragment.getStartKey();
 +      endOffset = startOffset + fragment.getLength();
 +
 +      errorTorrenceMaxNum =
 +          Integer.parseInt(meta.getOption(TEXT_ERROR_TOLERANCE_MAXNUM, DEFAULT_TEXT_ERROR_TOLERANCE_MAXNUM));
 +    }
 +
 +
 +    @Override
 +    public void init() throws IOException {
 +      if (reader != null) {
 +        reader.close();
 +      }
++
 +      reader = new DelimitedLineReader(conf, fragment);
 +      reader.init();
 +      recordCount = 0;
 +
 +      if (targets == null) {
 +        targets = schema.toArray();
 +      }
 +
 +      targetColumnIndexes = new int[targets.length];
 +      for (int i = 0; i < targets.length; i++) {
 +        targetColumnIndexes[i] = schema.getColumnId(targets[i].getQualifiedName());
 +      }
 +
 +      super.init();
 +      Arrays.sort(targetColumnIndexes);
 +      if (LOG.isDebugEnabled()) {
 +        LOG.debug("DelimitedTextFileScanner open:" + fragment.getPath() + "," + startOffset + "," + endOffset);
 +      }
 +
 +      if (startOffset > 0) {
 +        reader.readLine();  // skip first line;
 +      }
 +
 +      deserializer = getLineSerde().createDeserializer(schema, meta, targetColumnIndexes);
 +      deserializer.init();
 +    }
 +
 +    public TextLineSerDe getLineSerde() {
 +      return DelimitedTextFile.getLineSerde(meta);
 +    }
 +
 +    @Override
 +    public float getProgress() {
 +      try {
 +        if (!reader.isReadable()) {
 +          return 1.0f;
 +        }
 +        long filePos = reader.getCompressedPosition();
 +        if (startOffset == filePos) {
 +          return 0.0f;
 +        } else {
 +          long readBytes = filePos - startOffset;
 +          long remainingBytes = Math.max(endOffset - filePos, 0);
 +          return Math.min(1.0f, (float) (readBytes) / (float) (readBytes + remainingBytes));
 +        }
 +      } catch (IOException e) {
 +        LOG.error(e.getMessage(), e);
 +        return 0.0f;
 +      }
 +    }
 +
 +    @Override
 +    public Tuple next() throws IOException {
++      VTuple tuple;
 +
 +      if (!reader.isReadable()) {
 +        return null;
 +      }
 +
-       if (targets.length == 0) {
-         return EmptyTuple.get();
-       }
- 
-       VTuple tuple = new VTuple(schema.size());
- 
 +      try {
 +
 +        // this loop will continue until one tuple is build or EOS (end of stream).
 +        do {
 +
 +          ByteBuf buf = reader.readLine();
++
++          // if no more line, then return EOT (end of tuple)
 +          if (buf == null) {
 +            return null;
 +          }
 +
-           try {
++          // If there is no required column, we just read each line
++          // and then return an empty tuple without parsing line.
++          if (targets.length == 0) {
++            recordCount++;
++            return EmptyTuple.get();
++          }
 +
++          tuple = new VTuple(schema.size());
++
++          try {
 +            deserializer.deserialize(buf, tuple);
 +            // if a line is read normaly, it exists this loop.
 +            break;
 +
 +          } catch (TextLineParsingError tae) {
 +
 +            errorNum++;
 +
 +            // suppress too many log prints, which probably cause performance degradation
 +            if (errorNum < errorPrintOutMaxNum) {
 +              LOG.warn("Ignore JSON Parse Error (" + errorNum + "): ", tae);
 +            }
 +
 +            // Only when the maximum error torrence limit is set (i.e., errorTorrenceMaxNum >= 0),
 +            // it checks if the number of parsing error exceeds the max limit.
 +            // Otherwise, it will ignore all parsing errors.
 +            if (errorTorrenceMaxNum >= 0 && errorNum > errorTorrenceMaxNum) {
 +              throw tae;
 +            }
 +            continue;
 +          }
 +
 +        } while (reader.isReadable()); // continue until EOS
 +
 +        // recordCount means the number of actual read records. We increment the count here.
 +        recordCount++;
 +
 +        return tuple;
 +
 +      } catch (Throwable t) {
 +        LOG.error(t);
 +        throw new IOException(t);
 +      }
 +    }
 +
 +    @Override
 +    public void reset() throws IOException {
 +      init();
 +    }
 +
 +    @Override
 +    public void close() throws IOException {
 +      try {
 +        if (deserializer != null) {
 +          deserializer.release();
 +        }
 +
 +        if (tableStats != null && reader != null) {
 +          tableStats.setReadBytes(reader.getReadBytes());  //Actual Processed Bytes. (decompressed bytes + overhead)
 +          tableStats.setNumRows(recordCount);
 +        }
 +        if (LOG.isDebugEnabled()) {
 +          LOG.debug("DelimitedTextFileScanner processed record:" + recordCount);
 +        }
 +      } finally {
 +        IOUtils.cleanup(LOG, reader);
 +        reader = null;
 +      }
 +    }
 +
 +    @Override
 +    public boolean isProjectable() {
 +      return true;
 +    }
 +
 +    @Override
 +    public boolean isSelectable() {
 +      return false;
 +    }
 +
 +    @Override
 +    public void setSearchCondition(Object expr) {
 +    }
 +
 +    @Override
 +    public boolean isSplittable() {
 +      return splittable;
 +    }
 +
 +    @Override
 +    public TableStats getInputStats() {
 +      if (tableStats != null && reader != null) {
 +        tableStats.setReadBytes(reader.getReadBytes());  //Actual Processed Bytes. (decompressed bytes + overhead)
 +        tableStats.setNumRows(recordCount);
 +        tableStats.setNumBytes(fragment.getLength());
 +      }
 +      return tableStats;
 +    }
 +  }
 +}

http://git-wip-us.apache.org/repos/asf/tajo/blob/0073ef23/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestDelimitedTextFile.java
----------------------------------------------------------------------
diff --cc tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestDelimitedTextFile.java
index 0000000,0000000..8749925
new file mode 100644
--- /dev/null
+++ b/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestDelimitedTextFile.java
@@@ -1,0 -1,0 +1,163 @@@
++/**
++ * Licensed to the Apache Software Foundation (ASF) under one
++ * or more contributor license agreements.  See the NOTICE file
++ * distributed with this work for additional information
++ * regarding copyright ownership.  The ASF licenses this file
++ * to you under the Apache License, Version 2.0 (the
++ * "License"); you may not use this file except in compliance
++ * with the License.  You may obtain a copy of the License at
++ *
++ *     http://www.apache.org/licenses/LICENSE-2.0
++ *
++ * Unless required by applicable law or agreed to in writing, software
++ * distributed under the License is distributed on an "AS IS" BASIS,
++ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
++ * See the License for the specific language governing permissions and
++ * limitations under the License.
++ */
++
++package org.apache.tajo.storage;
++
++import com.google.common.base.Preconditions;
++import org.apache.hadoop.conf.Configuration;
++import org.apache.hadoop.fs.FileStatus;
++import org.apache.hadoop.fs.FileSystem;
++import org.apache.hadoop.fs.Path;
++import org.apache.tajo.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.Type;
++import org.apache.tajo.conf.TajoConf;
++import org.apache.tajo.datum.Datum;
++import org.apache.tajo.datum.DatumFactory;
++import org.apache.tajo.storage.fragment.FileFragment;
++import org.apache.tajo.util.FileUtil;
++import org.junit.Test;
++
++import java.io.File;
++import java.io.IOException;
++import java.net.URL;
++
++import static org.junit.Assert.*;
++
++public class TestDelimitedTextFile {
++
++  private static Schema schema = new Schema();
++
++  private static Tuple baseTuple = new VTuple(10);
++
++  static {
++    schema.addColumn("col1", Type.BOOLEAN);
++    schema.addColumn("col2", Type.CHAR, 7);
++    schema.addColumn("col3", Type.INT2);
++    schema.addColumn("col4", Type.INT4);
++    schema.addColumn("col5", Type.INT8);
++    schema.addColumn("col6", Type.FLOAT4);
++    schema.addColumn("col7", Type.FLOAT8);
++    schema.addColumn("col8", Type.TEXT);
++    schema.addColumn("col9", Type.BLOB);
++    schema.addColumn("col10", Type.INET4);
++
++    baseTuple.put(new Datum[] {
++        DatumFactory.createBool(true),                // 0
++        DatumFactory.createChar("hyunsik"),           // 1
++        DatumFactory.createInt2((short) 17),          // 2
++        DatumFactory.createInt4(59),                  // 3
++        DatumFactory.createInt8(23l),                 // 4
++        DatumFactory.createFloat4(77.9f),             // 5
++        DatumFactory.createFloat8(271.9d),            // 6
++        DatumFactory.createText("hyunsik"),           // 7
++        DatumFactory.createBlob("hyunsik".getBytes()),// 8
++        DatumFactory.createInet4("192.168.0.1"),      // 9
++    });
++  }
++
++  public static Path getResourcePath(String path, String suffix) {
++    URL resultBaseURL = ClassLoader.getSystemResource(path);
++    return new Path(resultBaseURL.toString(), suffix);
++  }
++
++  public static Path getResultPath(Class clazz, String fileName) {
++    return new Path (getResourcePath("results", clazz.getSimpleName()), fileName);
++  }
++
++  public static String getResultText(Class clazz, String fileName) throws IOException {
++    FileSystem localFS = FileSystem.getLocal(new Configuration());
++    Path path = getResultPath(clazz, fileName);
++    Preconditions.checkState(localFS.exists(path) && localFS.isFile(path));
++    return FileUtil.readTextFile(new File(path.toUri()));
++  }
++
++  private static final FileFragment getFileFragment(String fileName) throws IOException {
++    TajoConf conf = new TajoConf();
++    Path tablePath = new Path(getResourcePath("dataset", "TestDelimitedTextFile"), fileName);
++    FileSystem fs = FileSystem.getLocal(conf);
++    FileStatus status = fs.getFileStatus(tablePath);
++    return new FileFragment("table", tablePath, 0, status.getLen());
++  }
++
++  @Test
++  public void testIgnoreAllErrors() throws IOException {
++    TajoConf conf = new TajoConf();
++
++    TableMeta meta = CatalogUtil.newTableMeta(CatalogProtos.StoreType.JSON);
++    meta.putOption(StorageUtil.TEXT_ERROR_TOLERANCE_MAXNUM, "-1");
++    FileFragment fragment =  getFileFragment("testErrorTolerance1.json");
++    Scanner scanner =  StorageManager.getFileStorageManager(conf).getScanner(meta, schema, fragment);
++    scanner.init();
++
++    Tuple tuple;
++    int i = 0;
++    while ((tuple = scanner.next()) != null) {
++      assertEquals(baseTuple, tuple);
++      i++;
++    }
++    assertEquals(3, i);
++    scanner.close();
++  }
++
++  @Test
++  public void testIgnoreOneErrorTolerance() throws IOException {
++
++
++    TajoConf conf = new TajoConf();
++
++    TableMeta meta = CatalogUtil.newTableMeta(CatalogProtos.StoreType.JSON);
++    meta.putOption(StorageUtil.TEXT_ERROR_TOLERANCE_MAXNUM, "1");
++    FileFragment fragment =  getFileFragment("testErrorTolerance1.json");
++    Scanner scanner =  StorageManager.getFileStorageManager(conf).getScanner(meta, schema, fragment);
++    scanner.init();
++
++    assertNotNull(scanner.next());
++    assertNotNull(scanner.next());
++    try {
++      scanner.next();
++    } catch (IOException ioe) {
++      System.out.println(ioe);
++      return;
++    } finally {
++      scanner.close();
++    }
++    fail();
++  }
++
++  @Test
++  public void testNoErrorTolerance() throws IOException {
++    TajoConf conf = new TajoConf();
++    TableMeta meta = CatalogUtil.newTableMeta(CatalogProtos.StoreType.JSON);
++    meta.putOption(StorageUtil.TEXT_ERROR_TOLERANCE_MAXNUM, "0");
++    FileFragment fragment =  getFileFragment("testErrorTolerance2.json");
++    Scanner scanner =  StorageManager.getFileStorageManager(conf).getScanner(meta, schema, fragment);
++    scanner.init();
++
++    try {
++      scanner.next();
++    } catch (IOException ioe) {
++      return;
++    } finally {
++      scanner.close();
++    }
++    fail();
++  }
++}

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

http://git-wip-us.apache.org/repos/asf/tajo/blob/0073ef23/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestSplitProcessor.java
----------------------------------------------------------------------
diff --cc tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestSplitProcessor.java
index 0000000,0000000..12ea551
new file mode 100644
--- /dev/null
+++ b/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestSplitProcessor.java
@@@ -1,0 -1,0 +1,72 @@@
++/**
++ * Licensed to the Apache Software Foundation (ASF) under one
++ * or more contributor license agreements.  See the NOTICE file
++ * distributed with this work for additional information
++ * regarding copyright ownership.  The ASF licenses this file
++ * to you under the Apache License, Version 2.0 (the
++ * "License"); you may not use this file except in compliance
++ * with the License.  You may obtain a copy of the License at
++ *
++ *     http://www.apache.org/licenses/LICENSE-2.0
++ *
++ * Unless required by applicable law or agreed to in writing, software
++ * distributed under the License is distributed on an "AS IS" BASIS,
++ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
++ * See the License for the specific language governing permissions and
++ * limitations under the License.
++ */
++
++package org.apache.tajo.storage;
++
++import io.netty.buffer.ByteBuf;
++import io.netty.buffer.Unpooled;
++import io.netty.util.CharsetUtil;
++import org.apache.tajo.storage.text.FieldSplitProcessor;
++import org.apache.tajo.storage.text.LineSplitProcessor;
++import org.junit.Test;
++
++import java.io.IOException;
++
++import static io.netty.util.ReferenceCountUtil.releaseLater;
++import static org.junit.Assert.assertEquals;
++import static org.junit.Assert.assertTrue;
++
++public class TestSplitProcessor {
++
++  @Test
++  public void testFieldSplitProcessor() throws IOException {
++    String data = "abc||de";
++    final ByteBuf buf = releaseLater(
++        Unpooled.copiedBuffer(data, CharsetUtil.ISO_8859_1));
++
++    final int len = buf.readableBytes();
++    FieldSplitProcessor processor = new FieldSplitProcessor('|');
++
++    assertEquals(3, buf.forEachByte(0, len, processor));
++    assertEquals(4, buf.forEachByte(4, len - 4, processor));
++    assertEquals(-1, buf.forEachByte(5, len - 5, processor));
++
++  }
++
++  @Test
++  public void testLineSplitProcessor() throws IOException {
++    String data = "abc\r\n\n";
++    final ByteBuf buf = releaseLater(
++        Unpooled.copiedBuffer(data, CharsetUtil.ISO_8859_1));
++
++    final int len = buf.readableBytes();
++    LineSplitProcessor processor = new LineSplitProcessor();
++
++    //find CR
++    assertEquals(3, buf.forEachByte(0, len, processor));
++
++    // find CRLF
++    assertEquals(4, buf.forEachByte(4, len - 4, processor));
++    assertEquals(buf.getByte(4), '\n');
++    // need to skip LF
++    assertTrue(processor.isPrevCharCR());
++
++    // find LF
++    assertEquals(5, buf.forEachByte(5, len - 5, processor)); //line length is zero
++  }
++}

http://git-wip-us.apache.org/repos/asf/tajo/blob/0073ef23/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/json/TestJsonSerDe.java
----------------------------------------------------------------------
diff --cc tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/json/TestJsonSerDe.java
index 0000000,0000000..70282d9
new file mode 100644
--- /dev/null
+++ b/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/json/TestJsonSerDe.java
@@@ -1,0 -1,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.json;
++
++import org.apache.hadoop.fs.FileStatus;
++import org.apache.hadoop.fs.FileSystem;
++import org.apache.hadoop.fs.Path;
++import org.apache.tajo.catalog.CatalogUtil;
++import org.apache.tajo.catalog.Schema;
++import org.apache.tajo.catalog.TableMeta;
++import org.apache.tajo.catalog.proto.CatalogProtos;
++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.datum.NullDatum;
++import org.apache.tajo.storage.Scanner;
++import org.apache.tajo.storage.StorageManager;
++import org.apache.tajo.storage.Tuple;
++import org.apache.tajo.storage.VTuple;
++import org.apache.tajo.storage.fragment.FileFragment;
++import org.junit.Test;
++
++import java.io.IOException;
++import java.net.URL;
++
++import static org.junit.Assert.*;
++
++public class TestJsonSerDe {
++  private static Schema schema = new Schema();
++
++  static {
++    schema.addColumn("col1", TajoDataTypes.Type.BOOLEAN);
++    schema.addColumn("col2", TajoDataTypes.Type.CHAR, 7);
++    schema.addColumn("col3", TajoDataTypes.Type.INT2);
++    schema.addColumn("col4", TajoDataTypes.Type.INT4);
++    schema.addColumn("col5", TajoDataTypes.Type.INT8);
++    schema.addColumn("col6", TajoDataTypes.Type.FLOAT4);
++    schema.addColumn("col7", TajoDataTypes.Type.FLOAT8);
++    schema.addColumn("col8", TajoDataTypes.Type.TEXT);
++    schema.addColumn("col9", TajoDataTypes.Type.BLOB);
++    schema.addColumn("col10", TajoDataTypes.Type.INET4);
++    schema.addColumn("col11", TajoDataTypes.Type.NULL_TYPE);
++  }
++
++  public static Path getResourcePath(String path, String suffix) {
++    URL resultBaseURL = ClassLoader.getSystemResource(path);
++    return new Path(resultBaseURL.toString(), suffix);
++  }
++
++  @Test
++  public void testVarioutType() throws IOException {
++    TajoConf conf = new TajoConf();
++
++    TableMeta meta = CatalogUtil.newTableMeta(CatalogProtos.StoreType.JSON);
++    Path tablePath = new Path(getResourcePath("dataset", "TestJsonSerDe"), "testVariousType.json");
++    FileSystem fs = FileSystem.getLocal(conf);
++    FileStatus status = fs.getFileStatus(tablePath);
++    FileFragment fragment = new FileFragment("table", tablePath, 0, status.getLen());
++    Scanner scanner =  StorageManager.getFileStorageManager(conf).getScanner(meta, schema, fragment);
++    scanner.init();
++
++    Tuple tuple = scanner.next();
++    assertNotNull(tuple);
++    assertNull(scanner.next());
++    scanner.close();
++
++    Tuple baseTuple = new VTuple(11);
++    baseTuple.put(new Datum[] {
++        DatumFactory.createBool(true),                  // 0
++        DatumFactory.createChar("hyunsik"),             // 1
++        DatumFactory.createInt2((short) 17),            // 2
++        DatumFactory.createInt4(59),                    // 3
++        DatumFactory.createInt8(23l),                   // 4
++        DatumFactory.createFloat4(77.9f),               // 5
++        DatumFactory.createFloat8(271.9d),              // 6
++        DatumFactory.createText("hyunsik"),             // 7
++        DatumFactory.createBlob("hyunsik".getBytes()),  // 8
++        DatumFactory.createInet4("192.168.0.1"),        // 9
++        NullDatum.get(),                                // 10
++    });
++
++    assertEquals(baseTuple, tuple);
++  }
++}

http://git-wip-us.apache.org/repos/asf/tajo/blob/0073ef23/tajo-storage/tajo-storage-hdfs/src/test/resources/dataset/TestDelimitedTextFile/testErrorTolerance1.json
----------------------------------------------------------------------
diff --cc tajo-storage/tajo-storage-hdfs/src/test/resources/dataset/TestDelimitedTextFile/testErrorTolerance1.json
index 0000000,0000000..739dfe7
new file mode 100644
--- /dev/null
+++ b/tajo-storage/tajo-storage-hdfs/src/test/resources/dataset/TestDelimitedTextFile/testErrorTolerance1.json
@@@ -1,0 -1,0 +1,6 @@@
++{"col1": "true", "col2": "hyunsik", "col3": 17, "col4": 59, "col5": 23, "col6": 77.9, "col7": 271.9, "col8": "hyunsik", "col9": "hyunsik", "col10": "192.168.0.1"}
++{"col1": "true", "col2": "hyunsik", "col3": 17, "col4": 59, "col5": 23, "col6": 77.9, "col7": 271.9, "col8": "hyunsik", "col9": "hyunsik", "col10": "192.168.0.1"
++{"col1": "true", "col2": "hyunsik", "col3": 17, "col4": 59, "col5": 23, "col6": 77.9, "col7": 271.9, "col8": "hyunsik", "col9": "hyunsik", "col10": "192.168.0.1"}
++{"col1": "true", "col2": "hyunsik", "col3": 17, "col4": 59, "col5": 23, "col6": 77.9, "col7": 271.9, "col8": "hyunsik", "col9": "hyunsik", "col10": "192.168.0.1"
++{"col1": "true", "col2": "hyunsik", "col3": 17, "col4": 59, "col5": 23, "col6": 77.9, "col7": 271.9, "col8": "hyunsik", "col9": "hyunsik", "col10": "192.168.0.1"
++{"col1": "true", "col2": "hyunsik", "col3": 17, "col4": 59, "col5": 23, "col6": 77.9, "col7": 271.9, "col8": "hyunsik", "col9": "hyunsik", "col10": "192.168.0.1"}

http://git-wip-us.apache.org/repos/asf/tajo/blob/0073ef23/tajo-storage/tajo-storage-hdfs/src/test/resources/dataset/TestDelimitedTextFile/testErrorTolerance2.json
----------------------------------------------------------------------
diff --cc tajo-storage/tajo-storage-hdfs/src/test/resources/dataset/TestDelimitedTextFile/testErrorTolerance2.json
index 0000000,0000000..8256b72
new file mode 100644
--- /dev/null
+++ b/tajo-storage/tajo-storage-hdfs/src/test/resources/dataset/TestDelimitedTextFile/testErrorTolerance2.json
@@@ -1,0 -1,0 +1,4 @@@
++{"col1": "true", "col2": "hyunsik", "col3": 17, "col4": 59, "col5": 23, "col6": 77.9, "col7": 271.9, "col8": "hyunsik", "col9": "hyunsik", "col10": "192.168.0.1"
++{"col1": "true", "col2": "hyunsik", "col3": 17, "col4": 59, "col5": 23, "col6": 77.9, "col7": 271.9, "col8": "hyunsik", "col9": "hyunsik", "col10": "192.168.0.1"}
++{"col1": "true", "col2": "hyunsik", "col3": 17, "col4": 59, "col5": 23, "col6": 77.9, "col7": 271.9, "col8": "hyunsik", "col9": "hyunsik", "col10": "192.168.0.1"}
++{"col1": "true", "col2": "hyunsik", "col3": 17, "col4": 59, "col5": 23, "col6": 77.9, "col7": 271.9, "col8": "hyunsik", "col9": "hyunsik", "col10": "192.168.0.1"}


[15/29] tajo git commit: TAJO-1220: Implement createStatement() and setEscapeProcessing() in JdbcConnection. (YeonSu Han via hyunsik)

Posted by hj...@apache.org.
TAJO-1220: Implement createStatement() and setEscapeProcessing() in  JdbcConnection. (YeonSu Han via hyunsik)


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

Branch: refs/heads/hbase_storage
Commit: ea2dbf82c04d609dac46abd322241a15ccf30956
Parents: f69938a
Author: Hyunsik Choi <hy...@apache.org>
Authored: Wed Dec 3 16:04:56 2014 +0900
Committer: Hyunsik Choi <hy...@apache.org>
Committed: Wed Dec 3 16:04:56 2014 +0900

----------------------------------------------------------------------
 CHANGES                                                |  3 +++
 .../main/java/org/apache/tajo/jdbc/JdbcConnection.java | 13 ++++++++++---
 .../main/java/org/apache/tajo/jdbc/TajoStatement.java  |  7 ++++---
 3 files changed, 17 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tajo/blob/ea2dbf82/CHANGES
----------------------------------------------------------------------
diff --git a/CHANGES b/CHANGES
index c26b8a9..ea8e1ca 100644
--- a/CHANGES
+++ b/CHANGES
@@ -83,6 +83,9 @@ Release 0.9.1 - unreleased
 
   BUG FIXES
 
+    TAJO-1220: Implement createStatement() and setEscapeProcessing() in 
+    JdbcConnection. (YeonSu Han via hyunsik)
+
     TAJO-1183: Keep command execution even with errors. (Jaewoong Jung via 
     hyunsik)
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/ea2dbf82/tajo-jdbc/src/main/java/org/apache/tajo/jdbc/JdbcConnection.java
----------------------------------------------------------------------
diff --git a/tajo-jdbc/src/main/java/org/apache/tajo/jdbc/JdbcConnection.java b/tajo-jdbc/src/main/java/org/apache/tajo/jdbc/JdbcConnection.java
index 4250da4..a76443d 100644
--- a/tajo-jdbc/src/main/java/org/apache/tajo/jdbc/JdbcConnection.java
+++ b/tajo-jdbc/src/main/java/org/apache/tajo/jdbc/JdbcConnection.java
@@ -181,9 +181,16 @@ public class JdbcConnection implements Connection {
   }
 
   @Override
-  public Statement createStatement(int resultSetType, int resultSetConcurrency)
-      throws SQLException {
-    throw new SQLFeatureNotSupportedException("createStatement");
+  public Statement createStatement(int resultSetType, int resultSetConcurrency) throws SQLException {
+    if (resultSetType == ResultSet.TYPE_SCROLL_SENSITIVE) {
+      throw new SQLException("TYPE_SCROLL_SENSITIVE is not supported");
+    }
+
+    if (resultSetConcurrency != ResultSet.CONCUR_READ_ONLY) {
+      throw new SQLException("CONCUR_READ_ONLY mode is not supported.");
+    }
+
+    return new TajoStatement(this, tajoClient);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/tajo/blob/ea2dbf82/tajo-jdbc/src/main/java/org/apache/tajo/jdbc/TajoStatement.java
----------------------------------------------------------------------
diff --git a/tajo-jdbc/src/main/java/org/apache/tajo/jdbc/TajoStatement.java b/tajo-jdbc/src/main/java/org/apache/tajo/jdbc/TajoStatement.java
index 8308211..eb7f8c9 100644
--- a/tajo-jdbc/src/main/java/org/apache/tajo/jdbc/TajoStatement.java
+++ b/tajo-jdbc/src/main/java/org/apache/tajo/jdbc/TajoStatement.java
@@ -311,10 +311,11 @@ public class TajoStatement implements Statement {
     throw new SQLFeatureNotSupportedException("setCursorName not supported");
   }
 
+  /**
+   * Not necessary.
+   */
   @Override
-  public void setEscapeProcessing(boolean enable) throws SQLException {
-    throw new SQLFeatureNotSupportedException("setEscapeProcessing not supported");
-  }
+  public void setEscapeProcessing(boolean enable) throws SQLException {}
 
   @Override
   public void setFetchDirection(int direction) throws SQLException {


[06/29] tajo git commit: TAJO-1204: Remove unused ServerName class. (DaeMyung Kang via jaehwa)

Posted by hj...@apache.org.
TAJO-1204: Remove unused ServerName class. (DaeMyung Kang via jaehwa)


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

Branch: refs/heads/hbase_storage
Commit: f3291621c31bbc24aff47dbc363ddfe2a1a9b21c
Parents: 72dd29c
Author: JaeHwa Jung <bl...@apache.org>
Authored: Thu Nov 27 22:28:06 2014 +0900
Committer: JaeHwa Jung <bl...@apache.org>
Committed: Thu Nov 27 22:28:06 2014 +0900

----------------------------------------------------------------------
 CHANGES                                         |   2 +
 .../apache/tajo/master/cluster/ServerName.java  | 123 -------------------
 .../org/apache/tajo/cluster/TestServerName.java | 102 ---------------
 3 files changed, 2 insertions(+), 225 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tajo/blob/f3291621/CHANGES
----------------------------------------------------------------------
diff --git a/CHANGES b/CHANGES
index 6f38f65..9b15898 100644
--- a/CHANGES
+++ b/CHANGES
@@ -13,6 +13,8 @@ Release 0.9.1 - unreleased
 
   IMPROVEMENT
 
+    TAJO-1204: Remove unused ServerName class. (DaeMyung Kang via jaehwa)
+
     TAJO-1053: ADD PARTITIONS for HCatalogStore. (jaehwa)
 
     TAJO-1195: Remove unused CachedDNSResolver Class. (DaeMyung Kang via jaehwa)

http://git-wip-us.apache.org/repos/asf/tajo/blob/f3291621/tajo-core/src/main/java/org/apache/tajo/master/cluster/ServerName.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/cluster/ServerName.java b/tajo-core/src/main/java/org/apache/tajo/master/cluster/ServerName.java
deleted file mode 100644
index 028af65..0000000
--- a/tajo-core/src/main/java/org/apache/tajo/master/cluster/ServerName.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.master.cluster;
-
-public class ServerName implements Comparable<ServerName> {
-  /**
-   * This character is used as separator between server hostname and port.
-   */
-  public static final String SERVERNAME_SEPARATOR = ":";
-
-  private final String serverName;
-  private final String hostname;
-  private final int port;
-
-
-  public ServerName(final String hostname, final int port) {
-    this.hostname = hostname;
-    this.port = port;
-    this.serverName = getServerName(hostname, port);
-  }
-
-  public ServerName(final String serverName) {
-    this(parseHostname(serverName), parsePort(serverName));
-  }
-  
-  public static ServerName create(final String serverName) {
-	  return new ServerName(serverName);
-  }
-
-  public static ServerName createWithDefaultPort(final String serverName,
-                                                 final int defaultPort) {
-    if (serverName == null || serverName.length() <= 0) {
-      throw new IllegalArgumentException("Passed hostname is null or empty ("
-          + serverName + ")");
-    }
-    int index = serverName.indexOf(SERVERNAME_SEPARATOR);
-    if (index == -1) {
-      return new ServerName(parseHostname(serverName), defaultPort);
-    } else {
-      return new ServerName(parseHostname(serverName), parsePort(serverName));
-    }
-  }
-
-  public static String parseHostname(final String serverName) {
-    if (serverName == null || serverName.length() <= 0) {
-      throw new IllegalArgumentException("Passed hostname is null or empty ("
-          + serverName + ")");
-    }
-    int index = serverName.indexOf(SERVERNAME_SEPARATOR);
-    if (index == -1) { // if a port is missing, the index will be set to -1.
-      throw new IllegalArgumentException("Passed port is missing ("
-          + serverName + ")");
-    }
-    return serverName.substring(0, index);
-  }
-
-  public static int parsePort(final String serverName) {
-    String [] split = serverName.split(SERVERNAME_SEPARATOR);
-    return Integer.parseInt(split[1]);
-  }
-
-  @Override
-  public String toString() {
-    return getServerName();
-  }
-
-  public String getServerName() {
-    return serverName;
-  }
-
-  public String getHostname() {
-    return hostname;
-  }
-
-  public int getPort() {
-    return port;
-  }
-
-  public static String getServerName(String hostName, int port) {
-    final StringBuilder name = new StringBuilder(hostName.length() + 4);
-    name.append(hostName);
-    name.append(SERVERNAME_SEPARATOR);
-    name.append(port);
-    return name.toString();
-  }
-
-  @Override
-  public int compareTo(ServerName other) {
-    int compare = this.getHostname().toLowerCase().
-      compareTo(other.getHostname().toLowerCase());
-    if (compare != 0) return compare;
-    return this.getPort() - other.getPort();        
-  }
-
-  @Override
-  public int hashCode() {
-    return getServerName().hashCode();
-  }
-
-  @Override
-  public boolean equals(Object o) {
-    if (this == o) return true;
-    if (o == null) return false;
-    if (!(o instanceof ServerName)) return false;
-    return this.compareTo((ServerName)o) == 0;
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/f3291621/tajo-core/src/test/java/org/apache/tajo/cluster/TestServerName.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/cluster/TestServerName.java b/tajo-core/src/test/java/org/apache/tajo/cluster/TestServerName.java
deleted file mode 100644
index 513187d..0000000
--- a/tajo-core/src/test/java/org/apache/tajo/cluster/TestServerName.java
+++ /dev/null
@@ -1,102 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tajo.cluster;
-
-import org.junit.Test;
-import org.apache.tajo.master.cluster.ServerName;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
-public class TestServerName {
-
-	@Test
-	public void testServerNameStringInt() {
-		ServerName server = new ServerName("ex1.com",50030);
-		assertEquals("ex1.com", server.getHostname());
-		assertEquals(50030, server.getPort());
-	}
-
-	@Test
-	public void testServerNameString() {
-		ServerName server = new ServerName("ex1.com:50030");
-		assertEquals("ex1.com", server.getHostname());
-		assertEquals(50030, server.getPort());
-	}
-
-	@Test
-	public void testParseHostname() {
-		assertEquals("ex1.com",ServerName.parseHostname("ex1.com:50030"));
-	}
-
-	@Test
-	public void testParsePort() {
-		assertEquals(50030,ServerName.parsePort("ex1.com:50030"));
-	}
-
-	@Test
-	public void testToString() {
-		ServerName server = new ServerName("ex1.com",50030);
-		assertEquals("ex1.com:50030", server.toString());
-	}
-
-	@Test
-	public void testGetServerName() {
-		ServerName server = new ServerName("ex1.com",50030);
-		assertEquals("ex1.com:50030", server.getServerName());
-	}
-
-	@Test
-	public void testGetHostname() {
-		ServerName server = new ServerName("ex1.com",50030);
-		assertEquals("ex1.com", server.getHostname());
-	}
-
-	@Test
-	public void testGetPort() {
-		ServerName server = new ServerName("ex1.com",50030);
-		assertEquals(50030, server.getPort());
-	}
-
-	@Test
-	public void testGetServerNameStringInt() {
-		assertEquals("ex2.com:50030",ServerName.getServerName("ex2.com", 50030));
-	}
-
-	@Test
-	public void testCompareTo() {
-		ServerName s1 = new ServerName("ex1.com:50030");
-		ServerName s2 = new ServerName("ex1.com:60030");
-		
-		assertTrue(s1.compareTo(s2) < 0);
-		assertTrue(s2.compareTo(s1) > 0);
-		
-		ServerName s3 = new ServerName("ex1.com:50030");
-		assertTrue(s1.compareTo(s3) == 0);
-		
-		ServerName s4 = new ServerName("ex2.com:50030");
-		assertTrue(s1.compareTo(s4) < 0);
-		assertTrue(s4.compareTo(s1) > 0);
-	}
-
-  @Test (expected = IllegalArgumentException.class)
-  public void testException() {
-    new ServerName("ex1.com");
-  }
-}


[19/29] tajo git commit: TAJO-1194: 'INSERT OVERWRITE .. SELECT' does not remove existing data when result is empty. (jaehwa)

Posted by hj...@apache.org.
TAJO-1194: 'INSERT OVERWRITE .. SELECT' does not remove existing data when result is empty. (jaehwa)

Closes #254


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

Branch: refs/heads/hbase_storage
Commit: f6da07b03587d2bce5ca4ac53d86eae3a8ba23c2
Parents: 97507e4
Author: JaeHwa Jung <bl...@apache.org>
Authored: Thu Dec 4 18:45:13 2014 +0900
Committer: JaeHwa Jung <bl...@apache.org>
Committed: Thu Dec 4 18:47:11 2014 +0900

----------------------------------------------------------------------
 CHANGES                                         |   3 +
 .../apache/tajo/master/querymaster/Query.java   |   3 +-
 .../tajo/engine/query/TestTablePartitions.java  | 243 ++++++++-----------
 3 files changed, 106 insertions(+), 143 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tajo/blob/f6da07b0/CHANGES
----------------------------------------------------------------------
diff --git a/CHANGES b/CHANGES
index a697afe..0c785ce 100644
--- a/CHANGES
+++ b/CHANGES
@@ -83,6 +83,9 @@ Release 0.9.1 - unreleased
 
   BUG FIXES
 
+    TAJO-1194: 'INSERT OVERWRITE .. SELECT' does not remove existing data when result is empty.
+    (jaehwa)
+
     TAJO-1191: Change DateDatum timezone to UTC. (Jaewoong Jung via hyunsik)
 
     TAJO-1224: When there is no projected column, json scan can be hang. 

http://git-wip-us.apache.org/repos/asf/tajo/blob/f6da07b0/tajo-core/src/main/java/org/apache/tajo/master/querymaster/Query.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/Query.java b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/Query.java
index 07b47c1..6f80171 100644
--- a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/Query.java
+++ b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/Query.java
@@ -465,8 +465,9 @@ public class Query implements EventHandler<QueryEvent> {
             boolean movedToOldTable = false;
             boolean committed = false;
             Path oldTableDir = new Path(queryContext.getStagingDir(), TajoConstants.INSERT_OVERWIRTE_OLD_TABLE_NAME);
+            ContentSummary summary = fs.getContentSummary(stagingResultDir);
 
-            if (queryContext.hasPartition()) {
+            if (queryContext.hasPartition() && summary.getFileCount() > 0L) {
               // This is a map for existing non-leaf directory to rename. A key is current directory and a value is
               // renaming directory.
               Map<Path, Path> renameDirs = TUtil.newHashMap();

http://git-wip-us.apache.org/repos/asf/tajo/blob/f6da07b0/tajo-core/src/test/java/org/apache/tajo/engine/query/TestTablePartitions.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestTablePartitions.java b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestTablePartitions.java
index cff5bfb..15cbde0 100644
--- a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestTablePartitions.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestTablePartitions.java
@@ -19,7 +19,7 @@
 package org.apache.tajo.engine.query;
 
 import com.google.common.collect.Maps;
-import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -44,7 +44,6 @@ import org.apache.tajo.jdbc.TajoResultSet;
 import org.apache.tajo.master.querymaster.QueryMasterTask;
 import org.apache.tajo.plan.logical.NodeType;
 import org.apache.tajo.storage.StorageConstants;
-import org.apache.tajo.util.CommonTestingUtil;
 import org.apache.tajo.util.KeyValueSet;
 import org.apache.tajo.worker.TajoWorker;
 import org.junit.Test;
@@ -71,7 +70,7 @@ public class TestTablePartitions extends QueryTestCaseBase {
   public final void testCreateColumnPartitionedTable() throws Exception {
     String tableName = CatalogUtil.normalizeIdentifier("testCreateColumnPartitionedTable");
     ResultSet res = executeString(
-        "create table " + tableName + " (col1 int4, col2 int4) partition by column(key float8) ");
+      "create table " + tableName + " (col1 int4, col2 int4) partition by column(key float8) ");
     res.close();
 
     assertTrue(catalog.existsTable(DEFAULT_DATABASE_NAME, tableName));
@@ -79,8 +78,8 @@ public class TestTablePartitions extends QueryTestCaseBase {
     assertEquals(3, catalog.getTableDesc(DEFAULT_DATABASE_NAME, tableName).getLogicalSchema().size());
 
     res = testBase.execute(
-        "insert overwrite into " + tableName + " select l_orderkey, l_partkey, " +
-            "l_quantity from lineitem");
+      "insert overwrite into " + tableName + " select l_orderkey, l_partkey, " +
+        "l_quantity from lineitem");
 
     MasterPlan plan = getQueryPlan(res);
     ExecutionBlock rootEB = plan.getRoot();
@@ -114,7 +113,7 @@ public class TestTablePartitions extends QueryTestCaseBase {
   public final void testCreateColumnPartitionedTableWithJoin() throws Exception {
     String tableName = CatalogUtil.normalizeIdentifier("testCreateColumnPartitionedTableWithJoin");
     ResultSet res = executeString(
-        "create table " + tableName + " (col1 int4, col2 int4) partition by column(key float8) ");
+      "create table " + tableName + " (col1 int4, col2 int4) partition by column(key float8) ");
     res.close();
 
     assertTrue(catalog.existsTable(DEFAULT_DATABASE_NAME, tableName));
@@ -122,8 +121,8 @@ public class TestTablePartitions extends QueryTestCaseBase {
     assertEquals(3, catalog.getTableDesc(DEFAULT_DATABASE_NAME, tableName).getLogicalSchema().size());
 
     res = testBase.execute(
-        "insert overwrite into " + tableName + " select l_orderkey, l_partkey, " +
-            "l_quantity from lineitem join orders on l_orderkey = o_orderkey");
+      "insert overwrite into " + tableName + " select l_orderkey, l_partkey, " +
+        "l_quantity from lineitem join orders on l_orderkey = o_orderkey");
 
     MasterPlan plan = getQueryPlan(res);
     ExecutionBlock rootEB = plan.getRoot();
@@ -159,7 +158,7 @@ public class TestTablePartitions extends QueryTestCaseBase {
   public final void testCreateColumnPartitionedTableWithSelectedColumns() throws Exception {
     String tableName = CatalogUtil.normalizeIdentifier("testCreateColumnPartitionedTableWithSelectedColumns");
     ResultSet res = executeString(
-        "create table " + tableName + " (col1 int4, col2 int4, null_col int4) partition by column(key float8) ");
+      "create table " + tableName + " (col1 int4, col2 int4, null_col int4) partition by column(key float8) ");
     res.close();
 
     assertTrue(catalog.existsTable(DEFAULT_DATABASE_NAME, tableName));
@@ -167,7 +166,7 @@ public class TestTablePartitions extends QueryTestCaseBase {
     assertEquals(4, catalog.getTableDesc(DEFAULT_DATABASE_NAME, tableName).getLogicalSchema().size());
 
     res = executeString("insert overwrite into " + tableName + " (col1, col2, key) select l_orderkey, " +
-        "l_partkey, l_quantity from lineitem");
+      "l_partkey, l_quantity from lineitem");
     res.close();
   }
 
@@ -175,20 +174,20 @@ public class TestTablePartitions extends QueryTestCaseBase {
   public final void testColumnPartitionedTableByOneColumn() throws Exception {
     String tableName = CatalogUtil.normalizeIdentifier("testColumnPartitionedTableByOneColumn");
     ResultSet res = executeString(
-        "create table " + tableName + " (col1 int4, col2 int4, null_col int4) partition by column(key float8) ");
+      "create table " + tableName + " (col1 int4, col2 int4, null_col int4) partition by column(key float8) ");
     res.close();
 
     assertTrue(catalog.existsTable(DEFAULT_DATABASE_NAME, tableName));
 
     res = executeString("insert overwrite into " + tableName
-        + " (col1, col2, key) select l_orderkey, l_partkey, l_quantity from lineitem");
+      + " (col1, col2, key) select l_orderkey, l_partkey, l_quantity from lineitem");
     res.close();
 
     TableDesc desc = catalog.getTableDesc(DEFAULT_DATABASE_NAME, tableName);
     assertPartitionDirectories(desc);
 
     res = executeString(
-        "select distinct * from " + tableName + " where (key = 45.0 or key = 38.0) and null_col is null");
+      "select distinct * from " + tableName + " where (key = 45.0 or key = 38.0) and null_col is null");
 
     Map<Double, int []> resultRows1 = Maps.newHashMap();
     resultRows1.put(45.0d, new int[]{3, 2});
@@ -220,14 +219,14 @@ public class TestTablePartitions extends QueryTestCaseBase {
   public final void testQueryCasesOnColumnPartitionedTable() throws Exception {
     String tableName = CatalogUtil.normalizeIdentifier("testQueryCasesOnColumnPartitionedTable");
     ResultSet res = executeString(
-        "create table " + tableName + " (col1 int4, col2 int4, null_col int4) partition by column(key float8) ");
+      "create table " + tableName + " (col1 int4, col2 int4, null_col int4) partition by column(key float8) ");
     res.close();
 
     assertTrue(catalog.existsTable(DEFAULT_DATABASE_NAME, tableName));
 
     res = executeString(
-        "insert overwrite into " + tableName
-            + " (col1, col2, key) select l_orderkey, l_partkey, l_quantity from lineitem");
+      "insert overwrite into " + tableName
+        + " (col1, col2, key) select l_orderkey, l_partkey, l_quantity from lineitem");
     res.close();
 
     TableDesc desc = catalog.getTableDesc(DEFAULT_DATABASE_NAME, tableName);
@@ -300,14 +299,14 @@ public class TestTablePartitions extends QueryTestCaseBase {
   public final void testColumnPartitionedTableByThreeColumns() throws Exception {
     String tableName = CatalogUtil.normalizeIdentifier("testColumnPartitionedTableByThreeColumns");
     ResultSet res = testBase.execute(
-        "create table " + tableName + " (col4 text) partition by column(col1 int4, col2 int4, col3 float8) ");
+      "create table " + tableName + " (col4 text) partition by column(col1 int4, col2 int4, col3 float8) ");
     res.close();
     TajoTestingCluster cluster = testBase.getTestingCluster();
     CatalogService catalog = cluster.getMaster().getCatalog();
     assertTrue(catalog.existsTable(DEFAULT_DATABASE_NAME, tableName));
 
     res = executeString("insert overwrite into " + tableName
-        + " select l_returnflag, l_orderkey, l_partkey, l_quantity from lineitem");
+      + " select l_returnflag, l_orderkey, l_partkey, l_quantity from lineitem");
     res.close();
 
     TableDesc desc = catalog.getTableDesc(DEFAULT_DATABASE_NAME, tableName);
@@ -364,14 +363,14 @@ public class TestTablePartitions extends QueryTestCaseBase {
   public final void testInsertIntoColumnPartitionedTableByThreeColumns() throws Exception {
     String tableName = CatalogUtil.normalizeIdentifier("testInsertIntoColumnPartitionedTableByThreeColumns");
     ResultSet res = testBase.execute(
-        "create table " + tableName + " (col4 text) partition by column(col1 int4, col2 int4, col3 float8) ");
+      "create table " + tableName + " (col4 text) partition by column(col1 int4, col2 int4, col3 float8) ");
     res.close();
     TajoTestingCluster cluster = testBase.getTestingCluster();
     CatalogService catalog = cluster.getMaster().getCatalog();
     assertTrue(catalog.existsTable(DEFAULT_DATABASE_NAME, tableName));
 
     res = executeString("insert into " + tableName
-        + " select l_returnflag, l_orderkey, l_partkey, l_quantity from lineitem");
+      + " select l_returnflag, l_orderkey, l_partkey, l_quantity from lineitem");
     res.close();
 
     TableDesc desc = catalog.getTableDesc(DEFAULT_DATABASE_NAME, tableName);
@@ -424,7 +423,7 @@ public class TestTablePartitions extends QueryTestCaseBase {
 
     // insert into already exists partitioned table
     res = executeString("insert into " + tableName
-        + " select l_returnflag, l_orderkey, l_partkey, l_quantity from lineitem");
+      + " select l_returnflag, l_orderkey, l_partkey, l_quantity from lineitem");
     res.close();
 
     desc = catalog.getTableDesc(DEFAULT_DATABASE_NAME, tableName);
@@ -447,15 +446,15 @@ public class TestTablePartitions extends QueryTestCaseBase {
       assertEquals(5, desc.getStats().getNumRows().intValue());
     }
     String expected = "N\n" +
-        "N\n" +
-        "N\n" +
-        "N\n" +
-        "N\n" +
-        "N\n" +
-        "R\n" +
-        "R\n" +
-        "R\n" +
-        "R\n";
+      "N\n" +
+      "N\n" +
+      "N\n" +
+      "N\n" +
+      "N\n" +
+      "R\n" +
+      "R\n" +
+      "R\n" +
+      "R\n";
 
     String tableData = getTableFileContents(new Path(desc.getPath()));
     assertEquals(expected, tableData);
@@ -464,30 +463,30 @@ public class TestTablePartitions extends QueryTestCaseBase {
     String resultSetData = resultSetToString(res);
     res.close();
     expected = "col4,col1,col2,col3\n" +
-        "-------------------------------\n" +
-        "N,2,2,38.0\n" +
-        "N,2,2,38.0\n" +
-        "R,3,2,45.0\n" +
-        "R,3,2,45.0\n";
+      "-------------------------------\n" +
+      "N,2,2,38.0\n" +
+      "N,2,2,38.0\n" +
+      "R,3,2,45.0\n" +
+      "R,3,2,45.0\n";
     assertEquals(expected, resultSetData);
 
     res = executeString("select * from " + tableName + " where (col1 = 2 or col1 = 3) and col2 >= 2");
     resultSetData = resultSetToString(res);
     res.close();
     expected = "col4,col1,col2,col3\n" +
-        "-------------------------------\n" +
-        "N,2,2,38.0\n" +
-        "N,2,2,38.0\n" +
-        "R,3,2,45.0\n" +
-        "R,3,2,45.0\n" +
-        "R,3,3,49.0\n" +
-        "R,3,3,49.0\n";
+      "-------------------------------\n" +
+      "N,2,2,38.0\n" +
+      "N,2,2,38.0\n" +
+      "R,3,2,45.0\n" +
+      "R,3,2,45.0\n" +
+      "R,3,3,49.0\n" +
+      "R,3,3,49.0\n";
     assertEquals(expected, resultSetData);
 
     // Check not to remove existing partition directories.
     res = executeString("insert overwrite into " + tableName
-        + " select l_returnflag, l_orderkey, l_partkey, 30.0 as l_quantity from lineitem "
-        + " where l_orderkey = 1 and l_partkey = 1 and  l_linenumber = 1");
+      + " select l_returnflag, l_orderkey, l_partkey, 30.0 as l_quantity from lineitem "
+      + " where l_orderkey = 1 and l_partkey = 1 and  l_linenumber = 1");
     res.close();
 
     desc = catalog.getTableDesc(DEFAULT_DATABASE_NAME, tableName);
@@ -514,28 +513,42 @@ public class TestTablePartitions extends QueryTestCaseBase {
     resultSetData = resultSetToString(res);
     res.close();
     expected = "col4,col1,col2,col3\n" +
-        "-------------------------------\n" +
-        "N,1,1,17.0\n" +
-        "N,1,1,17.0\n" +
-        "N,1,1,30.0\n" +
-        "N,1,1,36.0\n" +
-        "N,1,1,36.0\n";
+      "-------------------------------\n" +
+      "N,1,1,17.0\n" +
+      "N,1,1,17.0\n" +
+      "N,1,1,30.0\n" +
+      "N,1,1,36.0\n" +
+      "N,1,1,36.0\n";
 
     assertEquals(expected, resultSetData);
+
+    // insert overwrite empty result to partitioned table
+    res = executeString("insert overwrite into " + tableName
+      + " select l_returnflag, l_orderkey, l_partkey, l_quantity from lineitem where l_orderkey" +
+      " > 100");
+    res.close();
+
+    desc = catalog.getTableDesc(DEFAULT_DATABASE_NAME, tableName);
+
+    ContentSummary summary = fs.getContentSummary(new Path(desc.getPath()));
+
+    assertEquals(summary.getDirectoryCount(), 1L);
+    assertEquals(summary.getFileCount(), 0L);
+    assertEquals(summary.getLength(), 0L);
   }
 
   @Test
   public final void testColumnPartitionedTableByOneColumnsWithCompression() throws Exception {
     String tableName = CatalogUtil.normalizeIdentifier("testColumnPartitionedTableByOneColumnsWithCompression");
     ResultSet res = executeString(
-        "create table " + tableName + " (col2 int4, col3 float8) USING csv " +
-            "WITH ('csvfile.delimiter'='|','compression.codec'='org.apache.hadoop.io.compress.DeflateCodec') " +
-            "PARTITION BY column(col1 int4)");
+      "create table " + tableName + " (col2 int4, col3 float8) USING csv " +
+        "WITH ('csvfile.delimiter'='|','compression.codec'='org.apache.hadoop.io.compress.DeflateCodec') " +
+        "PARTITION BY column(col1 int4)");
     res.close();
     assertTrue(catalog.existsTable(DEFAULT_DATABASE_NAME, tableName));
 
     res = executeString(
-        "insert overwrite into " + tableName + " select l_partkey, l_quantity, l_orderkey from lineitem");
+      "insert overwrite into " + tableName + " select l_partkey, l_quantity, l_orderkey from lineitem");
     res.close();
     TableDesc desc = catalog.getTableDesc(DEFAULT_DATABASE_NAME, tableName);
     if (!testingCluster.isHCatalogStoreRunning()) {
@@ -564,15 +577,15 @@ public class TestTablePartitions extends QueryTestCaseBase {
   public final void testColumnPartitionedTableByTwoColumnsWithCompression() throws Exception {
     String tableName = CatalogUtil.normalizeIdentifier("testColumnPartitionedTableByTwoColumnsWithCompression");
     ResultSet res = executeString("create table " + tableName + " (col3 float8, col4 text) USING csv " +
-        "WITH ('csvfile.delimiter'='|','compression.codec'='org.apache.hadoop.io.compress.DeflateCodec') " +
-        "PARTITION by column(col1 int4, col2 int4)");
+      "WITH ('csvfile.delimiter'='|','compression.codec'='org.apache.hadoop.io.compress.DeflateCodec') " +
+      "PARTITION by column(col1 int4, col2 int4)");
     res.close();
 
     assertTrue(catalog.existsTable(DEFAULT_DATABASE_NAME, tableName));
 
     res = executeString(
-        "insert overwrite into " + tableName +
-            " select  l_quantity, l_returnflag, l_orderkey, l_partkey from lineitem");
+      "insert overwrite into " + tableName +
+        " select  l_quantity, l_returnflag, l_orderkey, l_partkey from lineitem");
     res.close();
     TableDesc desc = catalog.getTableDesc(DEFAULT_DATABASE_NAME, tableName);
     if (!testingCluster.isHCatalogStoreRunning()) {
@@ -608,16 +621,16 @@ public class TestTablePartitions extends QueryTestCaseBase {
   public final void testColumnPartitionedTableByThreeColumnsWithCompression() throws Exception {
     String tableName = CatalogUtil.normalizeIdentifier("testColumnPartitionedTableByThreeColumnsWithCompression");
     ResultSet res = executeString(
-        "create table " + tableName + " (col4 text) USING csv " +
-            "WITH ('csvfile.delimiter'='|','compression.codec'='org.apache.hadoop.io.compress.DeflateCodec') " +
-            "partition by column(col1 int4, col2 int4, col3 float8)");
+      "create table " + tableName + " (col4 text) USING csv " +
+        "WITH ('csvfile.delimiter'='|','compression.codec'='org.apache.hadoop.io.compress.DeflateCodec') " +
+        "partition by column(col1 int4, col2 int4, col3 float8)");
     res.close();
 
     assertTrue(catalog.existsTable(DEFAULT_DATABASE_NAME, tableName));
 
     res = executeString(
-        "insert overwrite into " + tableName +
-            " select l_returnflag, l_orderkey, l_partkey, l_quantity from lineitem");
+      "insert overwrite into " + tableName +
+        " select l_returnflag, l_orderkey, l_partkey, l_quantity from lineitem");
     res.close();
     TableDesc desc = catalog.getTableDesc(DEFAULT_DATABASE_NAME, tableName);
     if (!testingCluster.isHCatalogStoreRunning()) {
@@ -691,16 +704,16 @@ public class TestTablePartitions extends QueryTestCaseBase {
   public final void testColumnPartitionedTableNoMatchedPartition() throws Exception {
     String tableName = CatalogUtil.normalizeIdentifier("testColumnPartitionedTableNoMatchedPartition");
     ResultSet res = executeString(
-        "create table " + tableName + " (col4 text) USING csv " +
-            "WITH ('csvfile.delimiter'='|','compression.codec'='org.apache.hadoop.io.compress.DeflateCodec') " +
-            "partition by column(col1 int4, col2 int4, col3 float8)");
+      "create table " + tableName + " (col4 text) USING csv " +
+        "WITH ('csvfile.delimiter'='|','compression.codec'='org.apache.hadoop.io.compress.DeflateCodec') " +
+        "partition by column(col1 int4, col2 int4, col3 float8)");
     res.close();
 
     assertTrue(catalog.existsTable(DEFAULT_DATABASE_NAME, tableName));
 
     res = executeString(
-        "insert overwrite into " + tableName +
-            " select l_returnflag , l_orderkey, l_partkey, l_quantity from lineitem");
+      "insert overwrite into " + tableName +
+        " select l_returnflag , l_orderkey, l_partkey, l_quantity from lineitem");
     res.close();
     TableDesc desc = catalog.getTableDesc(DEFAULT_DATABASE_NAME, tableName);
     if (!testingCluster.isHCatalogStoreRunning()) {
@@ -747,13 +760,13 @@ public class TestTablePartitions extends QueryTestCaseBase {
   public final void testColumnPartitionedTableWithSmallerExpressions1() throws Exception {
     String tableName = CatalogUtil.normalizeIdentifier("testColumnPartitionedTableWithSmallerExpressions1");
     ResultSet res = executeString(
-        "create table " + tableName + " (col1 int4, col2 int4, null_col int4) partition by column(key float8) ");
+      "create table " + tableName + " (col1 int4, col2 int4, null_col int4) partition by column(key float8) ");
     res.close();
 
     assertTrue(catalog.existsTable(DEFAULT_DATABASE_NAME, tableName));
 
     ClientProtos.SubmitQueryResponse response = client.executeQuery("insert overwrite into " + tableName
-        + " select l_orderkey, l_partkey from lineitem");
+      + " select l_orderkey, l_partkey from lineitem");
 
     assertTrue(response.hasErrorMessage());
     assertEquals(response.getErrorMessage(), "INSERT has smaller expressions than target columns\n");
@@ -767,13 +780,13 @@ public class TestTablePartitions extends QueryTestCaseBase {
   public final void testColumnPartitionedTableWithSmallerExpressions2() throws Exception {
     String tableName = CatalogUtil.normalizeIdentifier("testColumnPartitionedTableWithSmallerExpressions2");
     ResultSet res = executeString(
-        "create table " + tableName + " (col1 int4, col2 int4, null_col int4) partition by column(key float8) ");
+      "create table " + tableName + " (col1 int4, col2 int4, null_col int4) partition by column(key float8) ");
     res.close();
 
     assertTrue(catalog.existsTable(DEFAULT_DATABASE_NAME, tableName));
 
     ClientProtos.SubmitQueryResponse response = client.executeQuery("insert overwrite into " + tableName
-        + " select l_returnflag , l_orderkey, l_partkey from lineitem");
+      + " select l_returnflag , l_orderkey, l_partkey from lineitem");
 
     assertTrue(response.hasErrorMessage());
     assertEquals(response.getErrorMessage(), "INSERT has smaller expressions than target columns\n");
@@ -792,11 +805,11 @@ public class TestTablePartitions extends QueryTestCaseBase {
     res.close();
 
     res = executeString("create table testinsertquery1.table1 " +
-        "(col1 int4, col2 int4, col3 float8)");
+      "(col1 int4, col2 int4, col3 float8)");
     res.close();
 
     res = executeString("create table testinsertquery2.table1 " +
-        "(col1 int4, col2 int4, col3 float8)");
+      "(col1 int4, col2 int4, col3 float8)");
     res.close();
 
     CatalogService catalog = testingCluster.getMaster().getCatalog();
@@ -804,7 +817,7 @@ public class TestTablePartitions extends QueryTestCaseBase {
     assertTrue(catalog.existsTable("testinsertquery2", "table1"));
 
     res = executeString("insert overwrite into testinsertquery1.table1 " +
-        "select l_orderkey, l_partkey, l_quantity from default.lineitem;");
+      "select l_orderkey, l_partkey, l_quantity from default.lineitem;");
     res.close();
 
     TableDesc desc = catalog.getTableDesc("testinsertquery1", "table1");
@@ -813,7 +826,7 @@ public class TestTablePartitions extends QueryTestCaseBase {
     }
 
     res = executeString("insert overwrite into testinsertquery2.table1 " +
-        "select col1, col2, col3 from testinsertquery1.table1;");
+      "select col1, col2, col3 from testinsertquery1.table1;");
     res.close();
 
     desc = catalog.getTableDesc("testinsertquery2", "table1");
@@ -822,39 +835,6 @@ public class TestTablePartitions extends QueryTestCaseBase {
     }
   }
 
-  @Test
-  public final void testColumnPartitionedTableWithSmallerExpressions5() throws Exception {
-    String tableName = CatalogUtil.normalizeIdentifier("testColumnPartitionedTableWithSmallerExpressions5");
-    ResultSet res = executeString(
-        "create table " + tableName + " (col1 text) partition by column(col2 text) ");
-    res.close();
-
-    assertTrue(catalog.existsTable(DEFAULT_DATABASE_NAME, tableName));
-
-    res = executeString("insert overwrite into " + tableName + "(col1) select l_returnflag from lineitem");
-    res.close();
-    res = executeString("select * from " + tableName);
-    assertResultSet(res);
-    res.close();
-  }
-
-  @Test
-  public final void testColumnPartitionedTableWithSmallerExpressions6() throws Exception {
-    String tableName = CatalogUtil.normalizeIdentifier("testColumnPartitionedTableWithSmallerExpressions6");
-    ResultSet res = executeString(
-        "create table " + tableName + " (col1 text) partition by column(col2 text) ");
-    res.close();
-
-    assertTrue(catalog.existsTable(DEFAULT_DATABASE_NAME, tableName));
-
-    res = executeString(
-        "insert overwrite into " + tableName + "(col1) select l_returnflag from lineitem where l_orderkey = 1");
-    res.close();
-    res = executeString("select * from " + tableName);
-    assertResultSet(res);
-    res.close();
-  }
-
   private MasterPlan getQueryPlan(ResultSet res) {
     QueryId queryId = ((TajoResultSet)res).getQueryId();
     for (TajoWorker eachWorker: testingCluster.getTajoWorkers()) {
@@ -885,8 +865,8 @@ public class TestTablePartitions extends QueryTestCaseBase {
       int totalBytes = 0;
       Random rand = new Random(System.currentTimeMillis());
       String col2Data = "Column-2Column-2Column-2Column-2Column-2Column-2Column-2Column-2Column-2Column-2Column-2" +
-          "Column-2Column-2Column-2Column-2Column-2Column-2Column-2Column-2Column-2Column-2Column-2" +
-          "Column-2Column-2Column-2Column-2Column-2Column-2Column-2Column-2Column-2Column-2Column-2";
+        "Column-2Column-2Column-2Column-2Column-2Column-2Column-2Column-2Column-2Column-2Column-2" +
+        "Column-2Column-2Column-2Column-2Column-2Column-2Column-2Column-2Column-2Column-2Column-2";
 
       int index = 0;
       while(true) {
@@ -921,9 +901,9 @@ public class TestTablePartitions extends QueryTestCaseBase {
 
     } finally {
       testingCluster.setAllTajoDaemonConfValue(TajoConf.ConfVars.$DIST_QUERY_TABLE_PARTITION_VOLUME.varname,
-          TajoConf.ConfVars.$DIST_QUERY_TABLE_PARTITION_VOLUME.defaultVal);
+        TajoConf.ConfVars.$DIST_QUERY_TABLE_PARTITION_VOLUME.defaultVal);
       testingCluster.setAllTajoDaemonConfValue(TajoConf.ConfVars.SHUFFLE_HASH_APPENDER_PAGE_VOLUME.varname,
-          TajoConf.ConfVars.SHUFFLE_HASH_APPENDER_PAGE_VOLUME.defaultVal);
+        TajoConf.ConfVars.SHUFFLE_HASH_APPENDER_PAGE_VOLUME.defaultVal);
       executeString("DROP TABLE test_partition PURGE").close();
       executeString("DROP TABLE testScatteredHashShuffle PURGE").close();
     }
@@ -936,17 +916,17 @@ public class TestTablePartitions extends QueryTestCaseBase {
     executeDDL("lineitemspecial_ddl.sql", "lineitemspecial.tbl");
 
     executeString("CREATE TABLE IF NOT EXISTS pTable947 (id int, name text) PARTITION BY COLUMN (type text)")
-        .close();
+      .close();
     executeString("INSERT OVERWRITE INTO pTable947 SELECT l_orderkey, l_shipinstruct, l_shipmode FROM lineitemspecial")
-        .close();
+      .close();
     ResultSet res = executeString("select * from pTable947 where type='RA:*?><I/L#%S' or type='AIR'");
 
     String resStr = resultSetToString(res);
     String expected =
-        "id,name,type\n" +
-            "-------------------------------\n"
-            + "3,NONE,AIR\n"
-            + "3,TEST SPECIAL CHARS,RA:*?><I/L#%S\n";
+      "id,name,type\n" +
+        "-------------------------------\n"
+        + "3,NONE,AIR\n"
+        + "3,TEST SPECIAL CHARS,RA:*?><I/L#%S\n";
 
     assertEquals(expected, resStr);
     cleanupQuery(res);
@@ -959,9 +939,9 @@ public class TestTablePartitions extends QueryTestCaseBase {
     executeDDL("lineitemspecial_ddl.sql", "lineitemspecial.tbl");
 
     executeString("CREATE TABLE IF NOT EXISTS pTable948 (id int, name text) PARTITION BY COLUMN (type text)")
-        .close();
+      .close();
     executeString("INSERT OVERWRITE INTO pTable948 SELECT l_orderkey, l_shipinstruct, l_shipmode FROM lineitemspecial")
-        .close();
+      .close();
 
     ResultSet res = executeString("select * from pTable948 where type='RA:*?><I/L#%S'");
     assertResultSet(res);
@@ -971,25 +951,4 @@ public class TestTablePartitions extends QueryTestCaseBase {
     assertResultSet(res);
     cleanupQuery(res);
   }
-
-  @Test
-  public final void testIgnoreFilesInIntermediateDir() throws Exception {
-    // See - TAJO-1219: Files located in intermediate directories of partitioned table should be ignored
-    // It verifies that Tajo ignores files located in intermediate directories of partitioned table.
-
-    Path testDir = CommonTestingUtil.getTestDir();
-
-    executeString(
-        "CREATE EXTERNAL TABLE testIgnoreFilesInIntermediateDir (col1 int) USING CSV PARTITION BY COLUMN (col2 text) " +
-        "LOCATION '" + testDir + "'");
-
-    FileSystem fs = testDir.getFileSystem(conf);
-    FSDataOutputStream fos = fs.create(new Path(testDir, "table1.data"));
-    fos.write("a|b|c".getBytes());
-    fos.close();
-
-    ResultSet res = executeString("select * from testIgnoreFilesInIntermediateDir;");
-    assertFalse(res.next());
-    res.close();
-  }
 }


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

Posted by hj...@apache.org.
http://git-wip-us.apache.org/repos/asf/tajo/blob/940546a0/tajo-plan/src/main/java/org/apache/tajo/plan/Target.java
----------------------------------------------------------------------
diff --cc tajo-plan/src/main/java/org/apache/tajo/plan/Target.java
index f49a93d,f49a93d..0524240
--- a/tajo-plan/src/main/java/org/apache/tajo/plan/Target.java
+++ b/tajo-plan/src/main/java/org/apache/tajo/plan/Target.java
@@@ -21,7 -21,7 +21,7 @@@ package org.apache.tajo.plan
  import com.google.gson.annotations.Expose;
  import org.apache.tajo.catalog.Column;
  import org.apache.tajo.common.TajoDataTypes.DataType;
--import org.apache.tajo.json.GsonObject;
++import org.apache.tajo.storage.json.GsonObject;
  import org.apache.tajo.plan.expr.EvalNode;
  import org.apache.tajo.plan.expr.FieldEval;
  import org.apache.tajo.plan.serder.PlanGsonHelper;

http://git-wip-us.apache.org/repos/asf/tajo/blob/940546a0/tajo-plan/src/main/java/org/apache/tajo/plan/expr/CaseWhenEval.java
----------------------------------------------------------------------
diff --cc tajo-plan/src/main/java/org/apache/tajo/plan/expr/CaseWhenEval.java
index 4321d02,4321d02..24101e5
--- a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/CaseWhenEval.java
+++ b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/CaseWhenEval.java
@@@ -27,7 -27,7 +27,7 @@@ import org.apache.tajo.common.TajoDataT
  import org.apache.tajo.common.TajoDataTypes.Type;
  import org.apache.tajo.datum.Datum;
  import org.apache.tajo.datum.NullDatum;
--import org.apache.tajo.json.GsonObject;
++import org.apache.tajo.storage.json.GsonObject;
  import org.apache.tajo.plan.serder.PlanGsonHelper;
  import org.apache.tajo.storage.Tuple;
  import org.apache.tajo.util.TUtil;

http://git-wip-us.apache.org/repos/asf/tajo/blob/940546a0/tajo-plan/src/main/java/org/apache/tajo/plan/expr/EvalNode.java
----------------------------------------------------------------------
diff --cc tajo-plan/src/main/java/org/apache/tajo/plan/expr/EvalNode.java
index 638383a,638383a..e028481
--- a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/EvalNode.java
+++ b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/EvalNode.java
@@@ -22,7 -22,7 +22,7 @@@ import com.google.gson.annotations.Expo
  import org.apache.tajo.catalog.Schema;
  import org.apache.tajo.common.TajoDataTypes.DataType;
  import org.apache.tajo.datum.Datum;
--import org.apache.tajo.json.GsonObject;
++import org.apache.tajo.storage.json.GsonObject;
  import org.apache.tajo.plan.serder.PlanGsonHelper;
  import org.apache.tajo.storage.Tuple;
  

http://git-wip-us.apache.org/repos/asf/tajo/blob/940546a0/tajo-plan/src/main/java/org/apache/tajo/plan/function/GeneralFunction.java
----------------------------------------------------------------------
diff --cc tajo-plan/src/main/java/org/apache/tajo/plan/function/GeneralFunction.java
index 006449f,006449f..5a83f48
--- a/tajo-plan/src/main/java/org/apache/tajo/plan/function/GeneralFunction.java
+++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/GeneralFunction.java
@@@ -23,7 -23,7 +23,7 @@@ import org.apache.tajo.catalog.json.Cat
  import org.apache.tajo.catalog.proto.CatalogProtos;
  import org.apache.tajo.datum.Datum;
  import org.apache.tajo.function.Function;
--import org.apache.tajo.json.GsonObject;
++import org.apache.tajo.storage.json.GsonObject;
  import org.apache.tajo.plan.expr.FunctionEval;
  import org.apache.tajo.storage.Tuple;
  

http://git-wip-us.apache.org/repos/asf/tajo/blob/940546a0/tajo-plan/src/main/java/org/apache/tajo/plan/logical/BinaryNode.java
----------------------------------------------------------------------
diff --cc tajo-plan/src/main/java/org/apache/tajo/plan/logical/BinaryNode.java
index 709ef34,709ef34..55b6871
--- a/tajo-plan/src/main/java/org/apache/tajo/plan/logical/BinaryNode.java
+++ b/tajo-plan/src/main/java/org/apache/tajo/plan/logical/BinaryNode.java
@@@ -19,7 -19,7 +19,7 @@@
  package org.apache.tajo.plan.logical;
  
  import com.google.gson.annotations.Expose;
--import org.apache.tajo.json.GsonObject;
++import org.apache.tajo.storage.json.GsonObject;
  
  public abstract class BinaryNode extends LogicalNode implements Cloneable, GsonObject {
  	@Expose LogicalNode leftChild = null;

http://git-wip-us.apache.org/repos/asf/tajo/blob/940546a0/tajo-plan/src/main/java/org/apache/tajo/plan/logical/LogicalNode.java
----------------------------------------------------------------------
diff --cc tajo-plan/src/main/java/org/apache/tajo/plan/logical/LogicalNode.java
index c42a05e,c42a05e..beb2a8f
--- a/tajo-plan/src/main/java/org/apache/tajo/plan/logical/LogicalNode.java
+++ b/tajo-plan/src/main/java/org/apache/tajo/plan/logical/LogicalNode.java
@@@ -23,7 -23,7 +23,7 @@@ package org.apache.tajo.plan.logical
  
  import com.google.gson.annotations.Expose;
  import org.apache.tajo.catalog.Schema;
--import org.apache.tajo.json.GsonObject;
++import org.apache.tajo.storage.json.GsonObject;
  import org.apache.tajo.plan.PlanString;
  import org.apache.tajo.plan.util.PlannerUtil;
  import org.apache.tajo.plan.serder.PlanGsonHelper;

http://git-wip-us.apache.org/repos/asf/tajo/blob/940546a0/tajo-plan/src/main/java/org/apache/tajo/plan/serder/EvalNodeAdapter.java
----------------------------------------------------------------------
diff --cc tajo-plan/src/main/java/org/apache/tajo/plan/serder/EvalNodeAdapter.java
index 5a75e58,5a75e58..8003af5
--- a/tajo-plan/src/main/java/org/apache/tajo/plan/serder/EvalNodeAdapter.java
+++ b/tajo-plan/src/main/java/org/apache/tajo/plan/serder/EvalNodeAdapter.java
@@@ -22,8 -22,8 +22,8 @@@
  package org.apache.tajo.plan.serder;
  
  import com.google.gson.*;
--import org.apache.tajo.json.CommonGsonHelper;
--import org.apache.tajo.json.GsonSerDerAdapter;
++import org.apache.tajo.storage.json.CommonGsonHelper;
++import org.apache.tajo.storage.json.GsonSerDerAdapter;
  import org.apache.tajo.plan.expr.EvalNode;
  import org.apache.tajo.plan.expr.EvalType;
  

http://git-wip-us.apache.org/repos/asf/tajo/blob/940546a0/tajo-plan/src/main/java/org/apache/tajo/plan/serder/LogicalNodeAdapter.java
----------------------------------------------------------------------
diff --cc tajo-plan/src/main/java/org/apache/tajo/plan/serder/LogicalNodeAdapter.java
index f86ba5f,f86ba5f..20c16bb
--- a/tajo-plan/src/main/java/org/apache/tajo/plan/serder/LogicalNodeAdapter.java
+++ b/tajo-plan/src/main/java/org/apache/tajo/plan/serder/LogicalNodeAdapter.java
@@@ -22,10 -22,10 +22,10 @@@
  package org.apache.tajo.plan.serder;
  
  import com.google.gson.*;
--import org.apache.tajo.json.CommonGsonHelper;
++import org.apache.tajo.storage.json.CommonGsonHelper;
  import org.apache.tajo.plan.logical.LogicalNode;
  import org.apache.tajo.plan.logical.NodeType;
--import org.apache.tajo.json.GsonSerDerAdapter;
++import org.apache.tajo.storage.json.GsonSerDerAdapter;
  
  import java.lang.reflect.Type;
  

http://git-wip-us.apache.org/repos/asf/tajo/blob/940546a0/tajo-plan/src/main/java/org/apache/tajo/plan/serder/PlanGsonHelper.java
----------------------------------------------------------------------
diff --cc tajo-plan/src/main/java/org/apache/tajo/plan/serder/PlanGsonHelper.java
index 8cafbd0,8cafbd0..e0a7ad5
--- a/tajo-plan/src/main/java/org/apache/tajo/plan/serder/PlanGsonHelper.java
+++ b/tajo-plan/src/main/java/org/apache/tajo/plan/serder/PlanGsonHelper.java
@@@ -27,7 -27,7 +27,7 @@@ import org.apache.tajo.catalog.json.Tab
  import org.apache.tajo.common.TajoDataTypes.DataType;
  import org.apache.tajo.datum.Datum;
  import org.apache.tajo.function.Function;
--import org.apache.tajo.json.*;
++import org.apache.tajo.storage.json.*;
  import org.apache.tajo.plan.expr.EvalNode;
  import org.apache.tajo.plan.function.AggFunction;
  import org.apache.tajo.plan.function.GeneralFunction;

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

http://git-wip-us.apache.org/repos/asf/tajo/blob/940546a0/tajo-storage/tajo-storage-common/src/main/resources/storage-default.xml
----------------------------------------------------------------------
diff --cc tajo-storage/tajo-storage-common/src/main/resources/storage-default.xml
index 47d11c7,0000000..67033ed
mode 100644,000000..100644
--- a/tajo-storage/tajo-storage-common/src/main/resources/storage-default.xml
+++ b/tajo-storage/tajo-storage-common/src/main/resources/storage-default.xml
@@@ -1,184 -1,0 +1,198 @@@
 +<?xml version="1.0"?>
 +<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
 +
 +<!--
 +  Licensed to the Apache Software Foundation (ASF) under one
 +  or more contributor license agreements.  See the NOTICE file
 +  distributed with this work for additional information
 +  regarding copyright ownership.  The ASF licenses this file
 +  to you under the Apache License, Version 2.0 (the
 +  "License"); you may not use this file except in compliance
 +  with the License.  You may obtain a copy of the License at
 +
 +      http://www.apache.org/licenses/LICENSE-2.0
 +
 +  Unless required by applicable law or agreed to in writing, software
 +  distributed under the License is distributed on an "AS IS" BASIS,
 +  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 +  See the License for the specific language governing permissions and
 +  limitations under the License.
 +  -->
 +
 +<configuration>
 +  <!-- Storage Manager Configuration -->
 +  <property>
 +    <name>tajo.storage.manager.hdfs.class</name>
 +    <value>org.apache.tajo.storage.FileStorageManager</value>
 +  </property>
 +  <property>
 +    <name>tajo.storage.manager.hbase.class</name>
 +    <value>org.apache.tajo.storage.hbase.HBaseStorageManager</value>
 +  </property>
 +
 +  <property>
 +    <name>tajo.storage.manager.concurrency.perDisk</name>
 +    <value>1</value>
 +    <description></description>
 +  </property>
 +
 +  <!--- Registered Scanner Handler -->
 +  <property>
 +    <name>tajo.storage.scanner-handler</name>
-     <value>textfile,csv,raw,rcfile,row,parquet,sequencefile,avro,hbase</value>
++    <value>textfile,csv,json,raw,rcfile,row,parquet,sequencefile,avro,hbase</value>
 +  </property>
 +
 +  <!--- Fragment Class Configurations -->
 +  <property>
 +    <name>tajo.storage.fragment.textfile.class</name>
 +    <value>org.apache.tajo.storage.fragment.FileFragment</value>
 +  </property>
 +  <property>
 +    <name>tajo.storage.fragment.csv.class</name>
 +    <value>org.apache.tajo.storage.fragment.FileFragment</value>
 +  </property>
 +  <property>
++    <name>tajo.storage.fragment.json.class</name>
++    <value>org.apache.tajo.storage.fragment.FileFragment</value>
++  </property>
++  <property>
 +    <name>tajo.storage.fragment.raw.class</name>
 +    <value>org.apache.tajo.storage.fragment.FileFragment</value>
 +  </property>
 +  <property>
 +    <name>tajo.storage.fragment.rcfile.class</name>
 +    <value>org.apache.tajo.storage.fragment.FileFragment</value>
 +  </property>
 +  <property>
 +    <name>tajo.storage.fragment.row.class</name>
 +    <value>org.apache.tajo.storage.fragment.FileFragment</value>
 +  </property>
 +  <property>
 +    <name>tajo.storage.fragment.parquet.class</name>
 +    <value>org.apache.tajo.storage.fragment.FileFragment</value>
 +  </property>
 +  <property>
 +    <name>tajo.storage.fragment.sequencefile.class</name>
 +    <value>org.apache.tajo.storage.fragment.FileFragment</value>
 +  </property>
 +  <property>
 +    <name>tajo.storage.fragment.avro.class</name>
 +    <value>org.apache.tajo.storage.fragment.FileFragment</value>
 +  </property>
 +  <property>
 +    <name>tajo.storage.fragment.hbase.class</name>
 +    <value>org.apache.tajo.storage.hbase.HBaseFragment</value>
 +  </property>
 +
 +  <!--- Scanner Handler -->
 +  <property>
 +    <name>tajo.storage.scanner-handler.textfile.class</name>
 +    <value>org.apache.tajo.storage.text.DelimitedTextFile$DelimitedTextFileScanner</value>
 +  </property>
 +
 +  <property>
 +    <name>tajo.storage.scanner-handler.csv.class</name>
 +    <value>org.apache.tajo.storage.CSVFile$CSVScanner</value>
 +  </property>
 +
 +  <property>
++    <name>tajo.storage.scanner-handler.json.class</name>
++    <value>org.apache.tajo.storage.text.DelimitedTextFile$DelimitedTextFileScanner</value>
++  </property>
++
++  <property>
 +    <name>tajo.storage.scanner-handler.raw.class</name>
 +    <value>org.apache.tajo.storage.RawFile$RawFileScanner</value>
 +  </property>
 +
 +  <property>
 +    <name>tajo.storage.scanner-handler.rcfile.class</name>
 +    <value>org.apache.tajo.storage.rcfile.RCFile$RCFileScanner</value>
 +  </property>
 +
 +  <property>
 +    <name>tajo.storage.scanner-handler.rowfile.class</name>
 +    <value>org.apache.tajo.storage.RowFile$RowFileScanner</value>
 +  </property>
 +
 +  <property>
 +    <name>tajo.storage.scanner-handler.parquet.class</name>
 +    <value>org.apache.tajo.storage.parquet.ParquetScanner</value>
 +  </property>
 +
 +  <property>
 +    <name>tajo.storage.scanner-handler.sequencefile.class</name>
 +    <value>org.apache.tajo.storage.sequencefile.SequenceFileScanner</value>
 +  </property>
 +
 +  <property>
 +    <name>tajo.storage.scanner-handler.avro.class</name>
 +    <value>org.apache.tajo.storage.avro.AvroScanner</value>
 +  </property>
 +
 +  <property>
 +    <name>tajo.storage.scanner-handler.hbase.class</name>
 +    <value>org.apache.tajo.storage.hbase.HBaseScanner</value>
 +  </property>
 +  
 +  <!--- Appender Handler -->
 +  <property>
 +    <name>tajo.storage.appender-handler</name>
 +    <value>textfile,csv,raw,rcfile,row,parquet,sequencefile,avro,hbase</value>
 +  </property>
 +
 +  <property>
 +    <name>tajo.storage.appender-handler.textfile.class</name>
 +    <value>org.apache.tajo.storage.text.DelimitedTextFile$DelimitedTextFileAppender</value>
 +  </property>
 +
 +  <property>
 +    <name>tajo.storage.appender-handler.csv.class</name>
 +    <value>org.apache.tajo.storage.CSVFile$CSVAppender</value>
 +  </property>
 +
 +  <property>
++    <name>tajo.storage.appender-handler.json.class</name>
++    <value>org.apache.tajo.storage.text.DelimitedTextFile$DelimitedTextFileAppender</value>
++  </property>
++
++  <property>
 +    <name>tajo.storage.appender-handler.raw.class</name>
 +    <value>org.apache.tajo.storage.RawFile$RawFileAppender</value>
 +  </property>
 +
 +  <property>
 +    <name>tajo.storage.appender-handler.rcfile.class</name>
 +    <value>org.apache.tajo.storage.rcfile.RCFile$RCFileAppender</value>
 +  </property>
 +
 +  <property>
 +    <name>tajo.storage.appender-handler.rowfile.class</name>
 +    <value>org.apache.tajo.storage.RowFile$RowFileAppender</value>
 +  </property>
 +
 +  <property>
 +    <name>tajo.storage.appender-handler.parquet.class</name>
 +    <value>org.apache.tajo.storage.parquet.ParquetAppender</value>
 +  </property>
 +
 +  <property>
 +    <name>tajo.storage.appender-handler.sequencefile.class</name>
 +    <value>org.apache.tajo.storage.sequencefile.SequenceFileAppender</value>
 +  </property>
 +
 +  <property>
 +    <name>tajo.storage.appender-handler.avro.class</name>
 +    <value>org.apache.tajo.storage.avro.AvroAppender</value>
 +  </property>
 +
 +  <property>
 +    <name>tajo.storage.appender-handler.hbase.class</name>
 +    <value>org.apache.tajo.storage.hbase.HFileAppender</value>
 +  </property>
 +
 +  <property>
 +    <name>tajo.storage.appender-handler.hfile.class</name>
 +    <value>org.apache.tajo.storage.hbase.HFileAppender</value>
 +  </property>
 +</configuration>

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


[02/29] tajo git commit: TAJO-1166: S3 related storage causes compilation error in Hadoop 2.6.0-SNAPSHOT. (jaehwa)

Posted by hj...@apache.org.
http://git-wip-us.apache.org/repos/asf/tajo/blob/3d485ecb/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoWorkerResourceManager.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoWorkerResourceManager.java b/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoWorkerResourceManager.java
index 0e3ccad..0cc87fc 100644
--- a/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoWorkerResourceManager.java
+++ b/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoWorkerResourceManager.java
@@ -33,6 +33,7 @@ import org.apache.hadoop.yarn.event.AsyncDispatcher;
 import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.tajo.QueryId;
 import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.ipc.ContainerProtocol;
 import org.apache.tajo.ipc.TajoMasterProtocol;
 import org.apache.tajo.master.TajoMaster;
 import org.apache.tajo.master.querymaster.QueryInProgress;
@@ -48,7 +49,6 @@ import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
 
-import static org.apache.hadoop.yarn.proto.YarnProtos.ContainerIdProto;
 import static org.apache.tajo.ipc.TajoMasterProtocol.*;
 
 
@@ -80,7 +80,8 @@ public class TajoWorkerResourceManager extends CompositeService implements Worke
 
   private TajoConf systemConf;
 
-  private ConcurrentMap<ContainerIdProto, AllocatedWorkerResource> allocatedResourceMap = Maps.newConcurrentMap();
+  private ConcurrentMap<ContainerProtocol.TajoContainerIdProto, AllocatedWorkerResource> allocatedResourceMap = Maps
+    .newConcurrentMap();
 
   /** It receives status messages from workers and their resources. */
   private TajoResourceTracker resourceTracker;
@@ -194,7 +195,7 @@ public class TajoWorkerResourceManager extends CompositeService implements Worke
 
   private WorkerResourceAllocationRequest createQMResourceRequest(QueryId queryId) {
     float queryMasterDefaultDiskSlot = masterContext.getConf().getFloatVar(
-        TajoConf.ConfVars.TAJO_QUERYMASTER_DISK_SLOT);
+      TajoConf.ConfVars.TAJO_QUERYMASTER_DISK_SLOT);
     int queryMasterDefaultMemoryMB = masterContext.getConf().getIntVar(TajoConf.ConfVars.TAJO_QUERYMASTER_MEMORY_MB);
 
     WorkerResourceAllocationRequest.Builder builder = WorkerResourceAllocationRequest.newBuilder();
@@ -235,7 +236,7 @@ public class TajoWorkerResourceManager extends CompositeService implements Worke
     return resource;
   }
 
-  private void registerQueryMaster(QueryId queryId, ContainerIdProto containerId) {
+  private void registerQueryMaster(QueryId queryId, ContainerProtocol.TajoContainerIdProto containerId) {
     rmContext.getQueryMasterContainer().putIfAbsent(queryId, containerId);
   }
 
@@ -256,9 +257,9 @@ public class TajoWorkerResourceManager extends CompositeService implements Worke
     WorkerResourceAllocationRequest request;
     RpcCallback<WorkerResourceAllocationResponse> callBack;
     WorkerResourceRequest(
-        QueryId queryId,
-        boolean queryMasterRequest, WorkerResourceAllocationRequest request,
-        RpcCallback<WorkerResourceAllocationResponse> callBack) {
+      QueryId queryId,
+      boolean queryMasterRequest, WorkerResourceAllocationRequest request,
+      RpcCallback<WorkerResourceAllocationResponse> callBack) {
       this.queryId = queryId;
       this.queryMasterRequest = queryMasterRequest;
       this.request = request;
@@ -282,14 +283,14 @@ public class TajoWorkerResourceManager extends CompositeService implements Worke
 
           if (LOG.isDebugEnabled()) {
             LOG.debug("allocateWorkerResources:" +
-                (new QueryId(resourceRequest.request.getQueryId())) +
-                ", requiredMemory:" + resourceRequest.request.getMinMemoryMBPerContainer() +
-                "~" + resourceRequest.request.getMaxMemoryMBPerContainer() +
-                ", requiredContainers:" + resourceRequest.request.getNumContainers() +
-                ", requiredDiskSlots:" + resourceRequest.request.getMinDiskSlotPerContainer() +
-                "~" + resourceRequest.request.getMaxDiskSlotPerContainer() +
-                ", queryMasterRequest=" + resourceRequest.queryMasterRequest +
-                ", liveWorkers=" + rmContext.getWorkers().size());
+              (new QueryId(resourceRequest.request.getQueryId())) +
+              ", requiredMemory:" + resourceRequest.request.getMinMemoryMBPerContainer() +
+              "~" + resourceRequest.request.getMaxMemoryMBPerContainer() +
+              ", requiredContainers:" + resourceRequest.request.getNumContainers() +
+              ", requiredDiskSlots:" + resourceRequest.request.getMinDiskSlotPerContainer() +
+              "~" + resourceRequest.request.getMaxDiskSlotPerContainer() +
+              ", queryMasterRequest=" + resourceRequest.queryMasterRequest +
+              ", liveWorkers=" + rmContext.getWorkers().size());
           }
 
           // TajoWorkerResourceManager can't return allocated disk slots occasionally.
@@ -300,25 +301,25 @@ public class TajoWorkerResourceManager extends CompositeService implements Worke
 
             if(allocatedWorkerResources.size() > 0) {
               List<WorkerAllocatedResource> allocatedResources =
-                  new ArrayList<WorkerAllocatedResource>();
+                new ArrayList<WorkerAllocatedResource>();
 
               for(AllocatedWorkerResource allocatedResource: allocatedWorkerResources) {
                 NodeId nodeId = NodeId.newInstance(allocatedResource.worker.getConnectionInfo().getHost(),
-                    allocatedResource.worker.getConnectionInfo().getPeerRpcPort());
+                  allocatedResource.worker.getConnectionInfo().getPeerRpcPort());
 
                 TajoWorkerContainerId containerId = new TajoWorkerContainerId();
 
                 containerId.setApplicationAttemptId(
-                    ApplicationIdUtils.createApplicationAttemptId(resourceRequest.queryId));
+                  ApplicationIdUtils.createApplicationAttemptId(resourceRequest.queryId));
                 containerId.setId(containerIdSeq.incrementAndGet());
 
-                ContainerIdProto containerIdProto = containerId.getProto();
+                ContainerProtocol.TajoContainerIdProto containerIdProto = containerId.getProto();
                 allocatedResources.add(WorkerAllocatedResource.newBuilder()
-                    .setContainerId(containerIdProto)
-                    .setConnectionInfo(allocatedResource.worker.getConnectionInfo().getProto())
-                    .setAllocatedMemoryMB(allocatedResource.allocatedMemoryMB)
-                    .setAllocatedDiskSlots(allocatedResource.allocatedDiskSlots)
-                    .build());
+                  .setContainerId(containerIdProto)
+                  .setConnectionInfo(allocatedResource.worker.getConnectionInfo().getProto())
+                  .setAllocatedMemoryMB(allocatedResource.allocatedMemoryMB)
+                  .setAllocatedDiskSlots(allocatedResource.allocatedDiskSlots)
+                  .build());
 
 
                 allocatedResourceMap.putIfAbsent(containerIdProto, allocatedResource);
@@ -358,7 +359,7 @@ public class TajoWorkerResourceManager extends CompositeService implements Worke
     int allocatedResources = 0;
 
     TajoMasterProtocol.ResourceRequestPriority resourceRequestPriority
-        = resourceRequest.request.getResourceRequestPriority();
+      = resourceRequest.request.getResourceRequestPriority();
 
     if(resourceRequestPriority == TajoMasterProtocol.ResourceRequestPriority.MEMORY) {
       synchronized(rmContext) {
@@ -369,7 +370,7 @@ public class TajoWorkerResourceManager extends CompositeService implements Worke
         int minMemoryMB = resourceRequest.request.getMinMemoryMBPerContainer();
         int maxMemoryMB = resourceRequest.request.getMaxMemoryMBPerContainer();
         float diskSlot = Math.max(resourceRequest.request.getMaxDiskSlotPerContainer(),
-            resourceRequest.request.getMinDiskSlotPerContainer());
+          resourceRequest.request.getMinDiskSlotPerContainer());
 
         int liveWorkerSize = randomWorkers.size();
         Set<Integer> insufficientWorkers = new HashSet<Integer>();
@@ -418,7 +419,7 @@ public class TajoWorkerResourceManager extends CompositeService implements Worke
               }
 
               workerResource.allocateResource(allocatedWorkerResource.allocatedDiskSlots,
-                  allocatedWorkerResource.allocatedMemoryMB);
+                allocatedWorkerResource.allocatedMemoryMB);
 
               selectedWorkers.add(allocatedWorkerResource);
 
@@ -438,7 +439,7 @@ public class TajoWorkerResourceManager extends CompositeService implements Worke
         float minDiskSlots = resourceRequest.request.getMinDiskSlotPerContainer();
         float maxDiskSlots = resourceRequest.request.getMaxDiskSlotPerContainer();
         int memoryMB = Math.max(resourceRequest.request.getMaxMemoryMBPerContainer(),
-            resourceRequest.request.getMinMemoryMBPerContainer());
+          resourceRequest.request.getMinMemoryMBPerContainer());
 
         int liveWorkerSize = randomWorkers.size();
         Set<Integer> insufficientWorkers = new HashSet<Integer>();
@@ -487,7 +488,7 @@ public class TajoWorkerResourceManager extends CompositeService implements Worke
                 allocatedWorkerResource.allocatedMemoryMB = workerResource.getAvailableMemoryMB();
               }
               workerResource.allocateResource(allocatedWorkerResource.allocatedDiskSlots,
-                  allocatedWorkerResource.allocatedMemoryMB);
+                allocatedWorkerResource.allocatedMemoryMB);
 
               selectedWorkers.add(allocatedWorkerResource);
 
@@ -508,7 +509,7 @@ public class TajoWorkerResourceManager extends CompositeService implements Worke
    * @param containerId ContainerIdProto to be released
    */
   @Override
-  public void releaseWorkerResource(ContainerIdProto containerId) {
+  public void releaseWorkerResource(ContainerProtocol.TajoContainerIdProto containerId) {
     AllocatedWorkerResource allocated = allocatedResourceMap.get(containerId);
     if(allocated != null) {
       LOG.info("Release Resource: " + allocated.allocatedDiskSlots + "," + allocated.allocatedMemoryMB);
@@ -530,7 +531,7 @@ public class TajoWorkerResourceManager extends CompositeService implements Worke
       LOG.warn("No QueryMaster resource info for " + queryId);
       return;
     } else {
-      ContainerIdProto containerId = rmContext.getQueryMasterContainer().remove(queryId);
+      ContainerProtocol.TajoContainerIdProto containerId = rmContext.getQueryMasterContainer().remove(queryId);
       releaseWorkerResource(containerId);
       rmContext.getStoppedQueryIds().add(queryId);
       LOG.info(String.format("Released QueryMaster (%s) resource." , queryId.toString()));

http://git-wip-us.apache.org/repos/asf/tajo/blob/3d485ecb/tajo-core/src/main/java/org/apache/tajo/master/rm/WorkerResourceManager.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/rm/WorkerResourceManager.java b/tajo-core/src/main/java/org/apache/tajo/master/rm/WorkerResourceManager.java
index 8e8ac51..9c2b71b 100644
--- a/tajo-core/src/main/java/org/apache/tajo/master/rm/WorkerResourceManager.java
+++ b/tajo-core/src/main/java/org/apache/tajo/master/rm/WorkerResourceManager.java
@@ -20,8 +20,8 @@ package org.apache.tajo.master.rm;
 
 import com.google.protobuf.RpcCallback;
 import org.apache.hadoop.service.Service;
-import org.apache.hadoop.yarn.proto.YarnProtos.ContainerIdProto;
 import org.apache.tajo.QueryId;
+import org.apache.tajo.ipc.ContainerProtocol;
 import org.apache.tajo.ipc.TajoMasterProtocol;
 import org.apache.tajo.master.querymaster.QueryInProgress;
 
@@ -63,7 +63,7 @@ public interface WorkerResourceManager extends Service {
    *
    * @param containerId ContainerIdProto to be released
    */
-  public void releaseWorkerResource(ContainerIdProto containerId);
+  public void releaseWorkerResource(ContainerProtocol.TajoContainerIdProto containerId);
 
   public String getSeedQueryId() throws IOException;
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/3d485ecb/tajo-core/src/main/java/org/apache/tajo/worker/AbstractResourceAllocator.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/AbstractResourceAllocator.java b/tajo-core/src/main/java/org/apache/tajo/worker/AbstractResourceAllocator.java
index ca71c53..68c57f2 100644
--- a/tajo-core/src/main/java/org/apache/tajo/worker/AbstractResourceAllocator.java
+++ b/tajo-core/src/main/java/org/apache/tajo/worker/AbstractResourceAllocator.java
@@ -20,9 +20,9 @@ package org.apache.tajo.worker;
 
 import com.google.common.collect.Maps;
 import org.apache.hadoop.service.CompositeService;
-import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.tajo.master.ContainerProxy;
 import org.apache.tajo.master.cluster.WorkerConnectionInfo;
+import org.apache.tajo.master.container.TajoContainerId;
 
 import java.util.Map;
 import java.util.concurrent.ConcurrentMap;
@@ -41,29 +41,29 @@ public abstract class AbstractResourceAllocator extends CompositeService impleme
     workerInfoMap.putIfAbsent(connectionInfo.getId(), connectionInfo);
   }
 
-  private Map<ContainerId, ContainerProxy> containers = Maps.newConcurrentMap();
+  private Map<TajoContainerId, ContainerProxy> containers = Maps.newConcurrentMap();
 
   public AbstractResourceAllocator() {
     super(AbstractResourceAllocator.class.getName());
   }
 
-  public void addContainer(ContainerId cId, ContainerProxy container) {
+  public void addContainer(TajoContainerId cId, ContainerProxy container) {
     containers.put(cId, container);
   }
 
-  public void removeContainer(ContainerId cId) {
+  public void removeContainer(TajoContainerId cId) {
     containers.remove(cId);
   }
 
-  public boolean containsContainer(ContainerId cId) {
+  public boolean containsContainer(TajoContainerId cId) {
     return containers.containsKey(cId);
   }
 
-  public ContainerProxy getContainer(ContainerId cId) {
+  public ContainerProxy getContainer(TajoContainerId cId) {
     return containers.get(cId);
   }
 
-  public Map<ContainerId, ContainerProxy> getContainers() {
+  public Map<TajoContainerId, ContainerProxy> getContainers() {
     return containers;
   }
 }

http://git-wip-us.apache.org/repos/asf/tajo/blob/3d485ecb/tajo-core/src/main/java/org/apache/tajo/worker/ResourceAllocator.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/ResourceAllocator.java b/tajo-core/src/main/java/org/apache/tajo/worker/ResourceAllocator.java
index 8b9219c..b713e70 100644
--- a/tajo-core/src/main/java/org/apache/tajo/worker/ResourceAllocator.java
+++ b/tajo-core/src/main/java/org/apache/tajo/worker/ResourceAllocator.java
@@ -18,12 +18,12 @@
 
 package org.apache.tajo.worker;
 
-import org.apache.hadoop.yarn.api.records.ContainerId;
-import org.apache.hadoop.yarn.proto.YarnProtos;
+import org.apache.tajo.ipc.ContainerProtocol;
+import org.apache.tajo.master.container.TajoContainerId;
 
 public interface ResourceAllocator {
   public void allocateTaskWorker();
-  public ContainerId makeContainerId(YarnProtos.ContainerIdProto containerId);
+  public TajoContainerId makeContainerId(ContainerProtocol.TajoContainerIdProto containerId);
   public int calculateNumRequestContainers(TajoWorker.WorkerContext workerContext,
                                            int numTasks, int memoryMBPerTask);
 }

http://git-wip-us.apache.org/repos/asf/tajo/blob/3d485ecb/tajo-core/src/main/java/org/apache/tajo/worker/TajoResourceAllocator.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TajoResourceAllocator.java b/tajo-core/src/main/java/org/apache/tajo/worker/TajoResourceAllocator.java
index 2220089..9345885 100644
--- a/tajo-core/src/main/java/org/apache/tajo/worker/TajoResourceAllocator.java
+++ b/tajo-core/src/main/java/org/apache/tajo/worker/TajoResourceAllocator.java
@@ -23,28 +23,25 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
-import org.apache.hadoop.yarn.api.records.Container;
-import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationAttemptIdPBImpl;
 import org.apache.hadoop.yarn.event.EventHandler;
-import org.apache.hadoop.yarn.proto.YarnProtos;
 import org.apache.tajo.ExecutionBlockId;
 import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.ipc.ContainerProtocol;
 import org.apache.tajo.ipc.TajoMasterProtocol;
 import org.apache.tajo.ipc.TajoWorkerProtocol;
 import org.apache.tajo.master.*;
 import org.apache.tajo.master.cluster.WorkerConnectionInfo;
+import org.apache.tajo.master.container.TajoContainer;
+import org.apache.tajo.master.container.TajoContainerId;
 import org.apache.tajo.master.event.ContainerAllocationEvent;
 import org.apache.tajo.master.event.ContainerAllocatorEventType;
 import org.apache.tajo.master.event.SubQueryContainerAllocationEvent;
 import org.apache.tajo.master.querymaster.QueryMasterTask;
 import org.apache.tajo.master.querymaster.SubQuery;
 import org.apache.tajo.master.querymaster.SubQueryState;
-import org.apache.tajo.master.rm.TajoWorkerContainer;
-import org.apache.tajo.master.rm.TajoWorkerContainerId;
-import org.apache.tajo.master.rm.Worker;
-import org.apache.tajo.master.rm.WorkerResource;
+import org.apache.tajo.master.rm.*;
 import org.apache.tajo.rpc.CallFuture;
 import org.apache.tajo.rpc.NettyClientBase;
 import org.apache.tajo.rpc.NullCallback;
@@ -72,11 +69,11 @@ public class TajoResourceAllocator extends AbstractResourceAllocator {
   public TajoResourceAllocator(QueryMasterTask.QueryMasterTaskContext queryTaskContext) {
     this.queryTaskContext = queryTaskContext;
     executorService = Executors.newFixedThreadPool(
-        queryTaskContext.getConf().getIntVar(TajoConf.ConfVars.YARN_RM_TASKRUNNER_LAUNCH_PARALLEL_NUM));
+      queryTaskContext.getConf().getIntVar(TajoConf.ConfVars.YARN_RM_TASKRUNNER_LAUNCH_PARALLEL_NUM));
   }
 
   @Override
-  public ContainerId makeContainerId(YarnProtos.ContainerIdProto containerIdProto) {
+  public TajoContainerId makeContainerId(ContainerProtocol.TajoContainerIdProto containerIdProto) {
     TajoWorkerContainerId containerId = new TajoWorkerContainerId();
     ApplicationAttemptId appAttemptId = new ApplicationAttemptIdPBImpl(containerIdProto.getAppAttemptId());
     containerId.setApplicationAttemptId(appAttemptId);
@@ -98,7 +95,7 @@ public class TajoResourceAllocator extends AbstractResourceAllocator {
     int clusterSlots = clusterResource == null ? 0 : clusterResource.getTotalMemoryMB() / memoryMBPerTask;
     clusterSlots =  Math.max(1, clusterSlots - 1); // reserve query master slot
     LOG.info("CalculateNumberRequestContainer - Number of Tasks=" + numTasks +
-        ", Number of Cluster Slots=" + clusterSlots);
+      ", Number of Cluster Slots=" + clusterSlots);
     return  Math.min(numTasks, clusterSlots);
   }
 
@@ -121,7 +118,8 @@ public class TajoResourceAllocator extends AbstractResourceAllocator {
 
     executorService.shutdownNow();
 
-    Map<ContainerId, ContainerProxy> containers = queryTaskContext.getResourceAllocator().getContainers();
+    Map<TajoContainerId, ContainerProxy> containers = queryTaskContext.getResourceAllocator()
+      .getContainers();
     List<ContainerProxy> list = new ArrayList<ContainerProxy>(containers.values());
     for(ContainerProxy eachProxy: list) {
       try {
@@ -156,16 +154,17 @@ public class TajoResourceAllocator extends AbstractResourceAllocator {
   private void launchTaskRunners(LaunchTaskRunnersEvent event) {
     // Query in standby mode doesn't need launch Worker.
     // But, Assign ExecutionBlock to assigned tajo worker
-    for(Container eachContainer: event.getContainers()) {
+    for(TajoContainer eachContainer: event.getContainers()) {
       TajoContainerProxy containerProxy = new TajoContainerProxy(queryTaskContext, tajoConf,
-          eachContainer, event.getQueryContext(), event.getExecutionBlockId(), event.getPlanJson());
+        eachContainer, event.getQueryContext(), event.getExecutionBlockId(), event.getPlanJson());
       executorService.submit(new LaunchRunner(eachContainer.getId(), containerProxy));
     }
   }
 
-  public void stopExecutionBlock(final ExecutionBlockId executionBlockId, Collection<Container> containers) {
+  public void stopExecutionBlock(final ExecutionBlockId executionBlockId,
+                                 Collection<TajoContainer> containers) {
     Set<NodeId> workers = Sets.newHashSet();
-    for (Container container : containers){
+    for (TajoContainer container : containers){
       workers.add(container.getNodeId());
     }
 
@@ -196,8 +195,8 @@ public class TajoResourceAllocator extends AbstractResourceAllocator {
 
   protected static class LaunchRunner implements Runnable {
     private final ContainerProxy proxy;
-    private final ContainerId id;
-    public LaunchRunner(ContainerId id, ContainerProxy proxy) {
+    private final TajoContainerId id;
+    public LaunchRunner(TajoContainerId id, ContainerProxy proxy) {
       this.proxy = proxy;
       this.id = id;
     }
@@ -210,8 +209,8 @@ public class TajoResourceAllocator extends AbstractResourceAllocator {
     }
   }
 
-  private void stopContainers(Collection<Container> containers) {
-    for (Container container : containers) {
+  private void stopContainers(Collection<TajoContainer> containers) {
+    for (TajoContainer container : containers) {
       final ContainerProxy proxy = queryTaskContext.getResourceAllocator().getContainer(container.getId());
       executorService.submit(new StopContainerRunner(container.getId(), proxy));
     }
@@ -219,8 +218,8 @@ public class TajoResourceAllocator extends AbstractResourceAllocator {
 
   private static class StopContainerRunner implements Runnable {
     private final ContainerProxy proxy;
-    private final ContainerId id;
-    public StopContainerRunner(ContainerId id, ContainerProxy proxy) {
+    private final TajoContainerId id;
+    public StopContainerRunner(TajoContainerId id, ContainerProxy proxy) {
       this.id = id;
       this.proxy = proxy;
     }
@@ -251,23 +250,23 @@ public class TajoResourceAllocator extends AbstractResourceAllocator {
     public void run() {
       LOG.info("Start TajoWorkerAllocationThread");
       CallFuture<TajoMasterProtocol.WorkerResourceAllocationResponse> callBack =
-          new CallFuture<TajoMasterProtocol.WorkerResourceAllocationResponse>();
+        new CallFuture<TajoMasterProtocol.WorkerResourceAllocationResponse>();
 
       //TODO consider task's resource usage pattern
       int requiredMemoryMB = tajoConf.getIntVar(TajoConf.ConfVars.TASK_DEFAULT_MEMORY);
       float requiredDiskSlots = tajoConf.getFloatVar(TajoConf.ConfVars.TASK_DEFAULT_DISK);
 
       TajoMasterProtocol.WorkerResourceAllocationRequest request =
-          TajoMasterProtocol.WorkerResourceAllocationRequest.newBuilder()
-              .setMinMemoryMBPerContainer(requiredMemoryMB)
-              .setMaxMemoryMBPerContainer(requiredMemoryMB)
-              .setNumContainers(event.getRequiredNum())
-              .setResourceRequestPriority(!event.isLeafQuery() ? TajoMasterProtocol.ResourceRequestPriority.MEMORY
-                  : TajoMasterProtocol.ResourceRequestPriority.DISK)
-              .setMinDiskSlotPerContainer(requiredDiskSlots)
-              .setMaxDiskSlotPerContainer(requiredDiskSlots)
-              .setQueryId(event.getExecutionBlockId().getQueryId().getProto())
-              .build();
+        TajoMasterProtocol.WorkerResourceAllocationRequest.newBuilder()
+          .setMinMemoryMBPerContainer(requiredMemoryMB)
+          .setMaxMemoryMBPerContainer(requiredMemoryMB)
+          .setNumContainers(event.getRequiredNum())
+          .setResourceRequestPriority(!event.isLeafQuery() ? TajoMasterProtocol.ResourceRequestPriority.MEMORY
+            : TajoMasterProtocol.ResourceRequestPriority.DISK)
+          .setMinDiskSlotPerContainer(requiredDiskSlots)
+          .setMaxDiskSlotPerContainer(requiredDiskSlots)
+          .setQueryId(event.getExecutionBlockId().getQueryId().getProto())
+          .build();
 
       RpcConnectionPool connPool = RpcConnectionPool.getPool(queryTaskContext.getConf());
       NettyClientBase tmClient = null;
@@ -280,21 +279,21 @@ public class TajoResourceAllocator extends AbstractResourceAllocator {
         if (tajoConf.getBoolVar(TajoConf.ConfVars.TAJO_MASTER_HA_ENABLE)) {
           try {
             tmClient = connPool.getConnection(
-                queryTaskContext.getQueryMasterContext().getWorkerContext().getTajoMasterAddress(),
-                TajoMasterProtocol.class, true);
+              queryTaskContext.getQueryMasterContext().getWorkerContext().getTajoMasterAddress(),
+              TajoMasterProtocol.class, true);
           } catch (Exception e) {
             queryTaskContext.getQueryMasterContext().getWorkerContext().
-                setWorkerResourceTrackerAddr(HAServiceUtil.getResourceTrackerAddress(tajoConf));
+              setWorkerResourceTrackerAddr(HAServiceUtil.getResourceTrackerAddress(tajoConf));
             queryTaskContext.getQueryMasterContext().getWorkerContext().
-                setTajoMasterAddress(HAServiceUtil.getMasterUmbilicalAddress(tajoConf));
+              setTajoMasterAddress(HAServiceUtil.getMasterUmbilicalAddress(tajoConf));
             tmClient = connPool.getConnection(
-                queryTaskContext.getQueryMasterContext().getWorkerContext().getTajoMasterAddress(),
-                TajoMasterProtocol.class, true);
+              queryTaskContext.getQueryMasterContext().getWorkerContext().getTajoMasterAddress(),
+              TajoMasterProtocol.class, true);
           }
         } else {
           tmClient = connPool.getConnection(
-              queryTaskContext.getQueryMasterContext().getWorkerContext().getTajoMasterAddress(),
-              TajoMasterProtocol.class, true);
+            queryTaskContext.getQueryMasterContext().getWorkerContext().getTajoMasterAddress(),
+            TajoMasterProtocol.class, true);
         }
 
         TajoMasterProtocol.TajoMasterProtocolService masterClientService = tmClient.getStub();
@@ -325,17 +324,17 @@ public class TajoResourceAllocator extends AbstractResourceAllocator {
         List<TajoMasterProtocol.WorkerAllocatedResource> allocatedResources = response.getWorkerAllocatedResourceList();
         ExecutionBlockId executionBlockId = event.getExecutionBlockId();
 
-        List<Container> containers = new ArrayList<Container>();
+        List<TajoContainer> containers = new ArrayList<TajoContainer>();
         for(TajoMasterProtocol.WorkerAllocatedResource eachAllocatedResource: allocatedResources) {
           TajoWorkerContainer container = new TajoWorkerContainer();
           NodeId nodeId = NodeId.newInstance(eachAllocatedResource.getConnectionInfo().getHost(),
-              eachAllocatedResource.getConnectionInfo().getPeerRpcPort());
+            eachAllocatedResource.getConnectionInfo().getPeerRpcPort());
 
           TajoWorkerContainerId containerId = new TajoWorkerContainerId();
 
           containerId.setApplicationAttemptId(
-              ApplicationIdUtils.createApplicationAttemptId(executionBlockId.getQueryId(),
-                  eachAllocatedResource.getContainerId().getAppAttemptId().getAttemptId()));
+            ApplicationIdUtils.createApplicationAttemptId(executionBlockId.getQueryId(),
+              eachAllocatedResource.getContainerId().getAppAttemptId().getAttemptId()));
           containerId.setId(eachAllocatedResource.getContainerId().getId());
 
           container.setId(containerId);
@@ -347,7 +346,7 @@ public class TajoResourceAllocator extends AbstractResourceAllocator {
           workerResource.setDiskSlots(eachAllocatedResource.getAllocatedDiskSlots());
 
           Worker worker = new Worker(null, workerResource,
-              new WorkerConnectionInfo(eachAllocatedResource.getConnectionInfo()));
+            new WorkerConnectionInfo(eachAllocatedResource.getConnectionInfo()));
           container.setWorkerResource(worker);
           addWorkerConnectionInfo(worker.getConnectionInfo());
           containers.add(container);
@@ -356,8 +355,8 @@ public class TajoResourceAllocator extends AbstractResourceAllocator {
         SubQueryState state = queryTaskContext.getSubQuery(executionBlockId).getSynchronizedState();
         if (!SubQuery.isRunningState(state)) {
           try {
-            List<ContainerId> containerIds = new ArrayList<ContainerId>();
-            for(Container eachContainer: containers) {
+            List<TajoContainerId> containerIds = new ArrayList<TajoContainerId>();
+            for(TajoContainer eachContainer: containers) {
               containerIds.add(eachContainer.getId());
             }
             TajoContainerProxy.releaseWorkerResource(queryTaskContext, executionBlockId, containerIds);
@@ -378,10 +377,10 @@ public class TajoResourceAllocator extends AbstractResourceAllocator {
       }
       if(event.getRequiredNum() > numAllocatedContainers) {
         ContainerAllocationEvent shortRequestEvent = new ContainerAllocationEvent(
-            event.getType(), event.getExecutionBlockId(), event.getPriority(),
-            event.getResource(),
-            event.getRequiredNum() - numAllocatedContainers,
-            event.isLeafQuery(), event.getProgress()
+          event.getType(), event.getExecutionBlockId(), event.getPriority(),
+          event.getResource(),
+          event.getRequiredNum() - numAllocatedContainers,
+          event.isLeafQuery(), event.getProgress()
         );
         queryTaskContext.getEventHandler().handle(shortRequestEvent);
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/3d485ecb/tajo-core/src/main/java/org/apache/tajo/worker/TaskRunner.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TaskRunner.java b/tajo-core/src/main/java/org/apache/tajo/worker/TaskRunner.java
index 1910575..4e9860b 100644
--- a/tajo-core/src/main/java/org/apache/tajo/worker/TaskRunner.java
+++ b/tajo-core/src/main/java/org/apache/tajo/worker/TaskRunner.java
@@ -24,15 +24,15 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.service.AbstractService;
-import org.apache.hadoop.yarn.api.records.ContainerId;
-import org.apache.hadoop.yarn.api.records.impl.pb.ContainerIdPBImpl;
-import org.apache.hadoop.yarn.util.ConverterUtils;
 import org.apache.tajo.ExecutionBlockId;
 import org.apache.tajo.QueryUnitAttemptId;
 import org.apache.tajo.conf.TajoConf;
 import org.apache.tajo.conf.TajoConf.ConfVars;
 import org.apache.tajo.engine.query.QueryUnitRequestImpl;
 import org.apache.tajo.ipc.QueryMasterProtocol.QueryMasterProtocolService;
+import org.apache.tajo.master.container.TajoContainerId;
+import org.apache.tajo.master.container.impl.pb.TajoContainerIdPBImpl;
+import org.apache.tajo.master.container.TajoConverterUtils;
 import org.apache.tajo.rpc.CallFuture;
 import org.apache.tajo.rpc.NullCallback;
 import org.jboss.netty.channel.ConnectTimeoutException;
@@ -53,7 +53,7 @@ public class TaskRunner extends AbstractService {
   private volatile boolean stopped = false;
   private Path baseDirPath;
 
-  private ContainerId containerId;
+  private TajoContainerId containerId;
 
   // for Fetcher
   private ExecutorService fetchLauncher;
@@ -77,7 +77,7 @@ public class TaskRunner extends AbstractService {
     this.fetchLauncher = Executors.newFixedThreadPool(
         systemConf.getIntVar(ConfVars.SHUFFLE_FETCHER_PARALLEL_EXECUTION_MAX_NUM), fetcherFactory);
     try {
-      this.containerId = ConverterUtils.toContainerId(containerId);
+      this.containerId = TajoConverterUtils.toTajoContainerId(containerId);
       this.executionBlockContext = executionBlockContext;
       this.history = executionBlockContext.createTaskRunnerHistory(this);
       this.history.setState(getServiceState());
@@ -91,11 +91,11 @@ public class TaskRunner extends AbstractService {
     return getId(getContext().getExecutionBlockId(), containerId);
   }
 
-  public ContainerId getContainerId(){
+  public TajoContainerId getContainerId(){
     return containerId;
   }
 
-  public static String getId(ExecutionBlockId executionBlockId, ContainerId containerId) {
+  public static String getId(ExecutionBlockId executionBlockId, TajoContainerId containerId) {
     return executionBlockId + "," + containerId;
   }
 
@@ -211,7 +211,7 @@ public class TaskRunner extends AbstractService {
                 LOG.info("Request GetTask: " + getId());
                 GetTaskRequestProto request = GetTaskRequestProto.newBuilder()
                     .setExecutionBlockId(getExecutionBlockId().getProto())
-                    .setContainerId(((ContainerIdPBImpl) containerId).getProto())
+                    .setContainerId(((TajoContainerIdPBImpl) containerId).getProto())
                     .setWorkerId(getContext().getWorkerContext().getConnectionInfo().getId())
                     .build();
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/3d485ecb/tajo-core/src/main/java/org/apache/tajo/worker/TaskRunnerHistory.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TaskRunnerHistory.java b/tajo-core/src/main/java/org/apache/tajo/worker/TaskRunnerHistory.java
index a8a11c1..364348f 100644
--- a/tajo-core/src/main/java/org/apache/tajo/worker/TaskRunnerHistory.java
+++ b/tajo-core/src/main/java/org/apache/tajo/worker/TaskRunnerHistory.java
@@ -21,11 +21,11 @@ package org.apache.tajo.worker;
 import com.google.common.base.Objects;
 import com.google.common.collect.Maps;
 import org.apache.hadoop.service.Service;
-import org.apache.hadoop.yarn.api.records.ContainerId;
-import org.apache.hadoop.yarn.util.ConverterUtils;
 import org.apache.tajo.ExecutionBlockId;
 import org.apache.tajo.QueryUnitAttemptId;
 import org.apache.tajo.common.ProtoObject;
+import org.apache.tajo.master.container.TajoContainerId;
+import org.apache.tajo.master.container.TajoConverterUtils;
 
 import java.util.Collections;
 import java.util.Map;
@@ -39,13 +39,13 @@ import static org.apache.tajo.ipc.TajoWorkerProtocol.TaskRunnerHistoryProto;
 public class TaskRunnerHistory implements ProtoObject<TaskRunnerHistoryProto> {
 
   private Service.STATE state;
-  private ContainerId containerId;
+  private TajoContainerId containerId;
   private long startTime;
   private long finishTime;
   private ExecutionBlockId executionBlockId;
   private Map<QueryUnitAttemptId, TaskHistory> taskHistoryMap = null;
 
-  public TaskRunnerHistory(ContainerId containerId, ExecutionBlockId executionBlockId) {
+  public TaskRunnerHistory(TajoContainerId containerId, ExecutionBlockId executionBlockId) {
     init();
     this.containerId = containerId;
     this.executionBlockId = executionBlockId;
@@ -53,7 +53,7 @@ public class TaskRunnerHistory implements ProtoObject<TaskRunnerHistoryProto> {
 
   public TaskRunnerHistory(TaskRunnerHistoryProto proto) {
     this.state = Service.STATE.valueOf(proto.getState());
-    this.containerId = ConverterUtils.toContainerId(proto.getContainerId());
+    this.containerId = TajoConverterUtils.toTajoContainerId(proto.getContainerId());
     this.startTime = proto.getStartTime();
     this.finishTime = proto.getFinishTime();
     this.executionBlockId = new ExecutionBlockId(proto.getExecutionBlockId());
@@ -129,11 +129,11 @@ public class TaskRunnerHistory implements ProtoObject<TaskRunnerHistoryProto> {
     this.state = state;
   }
 
-  public ContainerId getContainerId() {
+  public TajoContainerId getContainerId() {
     return containerId;
   }
 
-  public void setContainerId(ContainerId containerId) {
+  public void setContainerId(TajoContainerId containerId) {
     this.containerId = containerId;
   }
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/3d485ecb/tajo-core/src/main/proto/ContainerProtocol.proto
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/proto/ContainerProtocol.proto b/tajo-core/src/main/proto/ContainerProtocol.proto
new file mode 100644
index 0000000..df7a450
--- /dev/null
+++ b/tajo-core/src/main/proto/ContainerProtocol.proto
@@ -0,0 +1,48 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/**
+ * These .proto interfaces are public and stable.
+ * Please see http://wiki.apache.org/hadoop/Compatibility
+ * for what changes are allowed for a *stable* .proto interface.
+ */
+
+option java_package = "org.apache.tajo.ipc";
+option java_outer_classname = "ContainerProtocol";
+option java_generic_services = true;
+option java_generate_equals_and_hash = true;
+
+package hadoop.yarn;
+
+import "Security.proto";
+import "yarn_protos.proto";
+
+message TajoContainerIdProto {
+  optional ApplicationIdProto app_id = 1;
+  optional ApplicationAttemptIdProto app_attempt_id = 2;
+  optional int32 id = 3;
+}
+
+message TajoContainerProto {
+  optional TajoContainerIdProto id = 1;
+  optional NodeIdProto nodeId = 2;
+  optional string node_http_address = 3;
+  optional ResourceProto resource = 4;
+  optional PriorityProto priority = 5;
+  optional hadoop.common.TokenProto container_token = 6;
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/3d485ecb/tajo-core/src/main/proto/QueryMasterProtocol.proto
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/proto/QueryMasterProtocol.proto b/tajo-core/src/main/proto/QueryMasterProtocol.proto
index 06d2a42..494d296 100644
--- a/tajo-core/src/main/proto/QueryMasterProtocol.proto
+++ b/tajo-core/src/main/proto/QueryMasterProtocol.proto
@@ -27,6 +27,9 @@ import "TajoIdProtos.proto";
 import "CatalogProtos.proto";
 import "PrimitiveProtos.proto";
 import "TajoWorkerProtocol.proto";
+import "ContainerProtocol.proto";
+
+package hadoop.yarn;
 
 service QueryMasterProtocolService {
   //from Worker

http://git-wip-us.apache.org/repos/asf/tajo/blob/3d485ecb/tajo-core/src/main/proto/ResourceTrackerProtocol.proto
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/proto/ResourceTrackerProtocol.proto b/tajo-core/src/main/proto/ResourceTrackerProtocol.proto
index b117cac..b2db46a 100644
--- a/tajo-core/src/main/proto/ResourceTrackerProtocol.proto
+++ b/tajo-core/src/main/proto/ResourceTrackerProtocol.proto
@@ -23,8 +23,11 @@ option java_generic_services = true;
 option java_generate_equals_and_hash = true;
 
 import "TajoMasterProtocol.proto";
+import "ContainerProtocol.proto";
 import "tajo_protos.proto";
 
+package hadoop.yarn;
+
 message NodeHeartbeat {
   required WorkerConnectionInfoProto connectionInfo = 1;
   optional ServerStatusProto serverStatus = 2;

http://git-wip-us.apache.org/repos/asf/tajo/blob/3d485ecb/tajo-core/src/main/proto/TajoMasterProtocol.proto
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/proto/TajoMasterProtocol.proto b/tajo-core/src/main/proto/TajoMasterProtocol.proto
index 7283543..e5eab4f 100644
--- a/tajo-core/src/main/proto/TajoMasterProtocol.proto
+++ b/tajo-core/src/main/proto/TajoMasterProtocol.proto
@@ -28,6 +28,9 @@ import "tajo_protos.proto";
 import "TajoIdProtos.proto";
 import "CatalogProtos.proto";
 import "PrimitiveProtos.proto";
+import "ContainerProtocol.proto";
+
+package hadoop.yarn;
 
 message ServerStatusProto {
     message System {
@@ -119,11 +122,11 @@ message WorkerResourcesRequest {
 
 message WorkerResourceReleaseRequest {
     required ExecutionBlockIdProto executionBlockId = 1;
-    repeated hadoop.yarn.ContainerIdProto containerIds = 2;
+    repeated TajoContainerIdProto containerIds = 2;
 }
 
 message WorkerAllocatedResource {
-    required hadoop.yarn.ContainerIdProto containerId = 1;
+    required TajoContainerIdProto containerId = 1;
     required WorkerConnectionInfoProto connectionInfo = 2;
 
     required int32 allocatedMemoryMB = 3;

http://git-wip-us.apache.org/repos/asf/tajo/blob/3d485ecb/tajo-core/src/main/proto/TajoWorkerProtocol.proto
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/proto/TajoWorkerProtocol.proto b/tajo-core/src/main/proto/TajoWorkerProtocol.proto
index e515438..989b0e3 100644
--- a/tajo-core/src/main/proto/TajoWorkerProtocol.proto
+++ b/tajo-core/src/main/proto/TajoWorkerProtocol.proto
@@ -28,6 +28,9 @@ import "TajoIdProtos.proto";
 import "CatalogProtos.proto";
 import "PrimitiveProtos.proto";
 import "Plan.proto";
+import "ContainerProtocol.proto";
+
+package hadoop.yarn;
 
 message SessionProto {
   required string session_id = 1;
@@ -170,7 +173,7 @@ message QueryExecutionRequestProto {
 
 message GetTaskRequestProto {
     required int32 workerId = 1;
-    required hadoop.yarn.ContainerIdProto containerId = 2;
+    required TajoContainerIdProto containerId = 2;
     required ExecutionBlockIdProto executionBlockId = 3;
 }
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/3d485ecb/tajo-core/src/test/java/org/apache/tajo/master/rm/TestTajoResourceManager.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/master/rm/TestTajoResourceManager.java b/tajo-core/src/test/java/org/apache/tajo/master/rm/TestTajoResourceManager.java
index 0423894..b8fbd67 100644
--- a/tajo-core/src/test/java/org/apache/tajo/master/rm/TestTajoResourceManager.java
+++ b/tajo-core/src/test/java/org/apache/tajo/master/rm/TestTajoResourceManager.java
@@ -23,6 +23,7 @@ import org.apache.hadoop.yarn.proto.YarnProtos;
 import org.apache.tajo.QueryId;
 import org.apache.tajo.QueryIdFactory;
 import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.ipc.ContainerProtocol;
 import org.apache.tajo.ipc.TajoMasterProtocol.*;
 import org.apache.tajo.master.cluster.WorkerConnectionInfo;
 import org.apache.tajo.rpc.NullCallback;
@@ -150,7 +151,8 @@ public class TestTajoResourceManager {
           .build();
 
       final CountDownLatch barrier = new CountDownLatch(1);
-      final List<YarnProtos.ContainerIdProto> containerIds = new ArrayList<YarnProtos.ContainerIdProto>();
+      final List<ContainerProtocol.TajoContainerIdProto> containerIds = new
+        ArrayList<ContainerProtocol.TajoContainerIdProto>();
 
       RpcCallback<WorkerResourceAllocationResponse> callBack = new RpcCallback<WorkerResourceAllocationResponse>() {
 
@@ -190,7 +192,7 @@ public class TestTajoResourceManager {
         containerIds.add(eachResource.getContainerId());
       }
 
-      for(YarnProtos.ContainerIdProto eachContainerId: containerIds) {
+      for(ContainerProtocol.TajoContainerIdProto eachContainerId: containerIds) {
         tajoWorkerResourceManager.releaseWorkerResource(eachContainerId);
       }
 
@@ -318,7 +320,8 @@ public class TestTajoResourceManager {
           .build();
 
       final CountDownLatch barrier = new CountDownLatch(1);
-      final List<YarnProtos.ContainerIdProto> containerIds = new ArrayList<YarnProtos.ContainerIdProto>();
+      final List<ContainerProtocol.TajoContainerIdProto> containerIds = new
+        ArrayList<ContainerProtocol.TajoContainerIdProto>();
 
 
       RpcCallback<WorkerResourceAllocationResponse> callBack = new RpcCallback<WorkerResourceAllocationResponse>() {
@@ -356,7 +359,7 @@ public class TestTajoResourceManager {
 
       assertEquals(numWorkers * 3, response.getWorkerAllocatedResourceList().size());
 
-      for(YarnProtos.ContainerIdProto eachContainerId: containerIds) {
+      for(ContainerProtocol.TajoContainerIdProto eachContainerId: containerIds) {
         tajoWorkerResourceManager.releaseWorkerResource(eachContainerId);
       }
 
@@ -399,7 +402,8 @@ public class TestTajoResourceManager {
           .build();
 
       final CountDownLatch barrier = new CountDownLatch(1);
-      final List<YarnProtos.ContainerIdProto> containerIds = new ArrayList<YarnProtos.ContainerIdProto>();
+      final List<ContainerProtocol.TajoContainerIdProto> containerIds = new
+        ArrayList<ContainerProtocol.TajoContainerIdProto>();
 
 
       RpcCallback<WorkerResourceAllocationResponse> callBack = new RpcCallback<WorkerResourceAllocationResponse>() {
@@ -431,7 +435,7 @@ public class TestTajoResourceManager {
 
       assertEquals(0, totalUsedDisks, 0);
 
-      for(YarnProtos.ContainerIdProto eachContainerId: containerIds) {
+      for(ContainerProtocol.TajoContainerIdProto eachContainerId: containerIds) {
         tajoWorkerResourceManager.releaseWorkerResource(eachContainerId);
       }
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/3d485ecb/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
index 87b4197..220eb6c 100644
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/StorageManager.java
+++ b/tajo-storage/src/main/java/org/apache/tajo/storage/StorageManager.java
@@ -567,6 +567,7 @@ public class StorageManager {
 
     for (Path p : inputs) {
       FileSystem fs = p.getFileSystem(conf);
+
       ArrayList<FileStatus> files = Lists.newArrayList();
       if (fs.isFile(p)) {
         files.addAll(Lists.newArrayList(fs.getFileStatus(p)));

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

http://git-wip-us.apache.org/repos/asf/tajo/blob/3d485ecb/tajo-storage/src/test/java/org/apache/tajo/storage/s3/INode.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/test/java/org/apache/tajo/storage/s3/INode.java b/tajo-storage/src/test/java/org/apache/tajo/storage/s3/INode.java
deleted file mode 100644
index 7b09937..0000000
--- a/tajo-storage/src/test/java/org/apache/tajo/storage/s3/INode.java
+++ /dev/null
@@ -1,124 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tajo.storage.s3;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.fs.s3.Block;
-import org.apache.hadoop.io.IOUtils;
-
-import java.io.*;
-
-/**
- * Holds file metadata including type (regular file, or directory),
- * and the list of blocks that are pointers to the data.
- */
-@InterfaceAudience.Private
-@InterfaceStability.Unstable
-public class INode {
-
-  enum FileType {
-    DIRECTORY, FILE
-  }
-
-  public static final FileType[] FILE_TYPES = {
-      FileType.DIRECTORY,
-      FileType.FILE
-  };
-
-  public static final INode DIRECTORY_INODE = new INode(FileType.DIRECTORY, null);
-
-  private FileType fileType;
-  private Block[] blocks;
-
-  public INode(FileType fileType, Block[] blocks) {
-    this.fileType = fileType;
-    if (isDirectory() && blocks != null) {
-      throw new IllegalArgumentException("A directory cannot contain blocks.");
-    }
-    this.blocks = blocks;
-  }
-
-  public Block[] getBlocks() {
-    return blocks;
-  }
-
-  public FileType getFileType() {
-    return fileType;
-  }
-
-  public boolean isDirectory() {
-    return fileType == FileType.DIRECTORY;
-  }
-
-  public boolean isFile() {
-    return fileType == FileType.FILE;
-  }
-
-  public long getSerializedLength() {
-    return 1L + (blocks == null ? 0 : 4 + blocks.length * 16);
-  }
-
-
-  public InputStream serialize() throws IOException {
-    ByteArrayOutputStream bytes = new ByteArrayOutputStream();
-    DataOutputStream out = new DataOutputStream(bytes);
-    try {
-      out.writeByte(fileType.ordinal());
-      if (isFile()) {
-        out.writeInt(blocks.length);
-        for (int i = 0; i < blocks.length; i++) {
-          out.writeLong(blocks[i].getId());
-          out.writeLong(blocks[i].getLength());
-        }
-      }
-      out.close();
-      out = null;
-    } finally {
-      IOUtils.closeStream(out);
-    }
-    return new ByteArrayInputStream(bytes.toByteArray());
-  }
-
-  public static INode deserialize(InputStream in) throws IOException {
-    if (in == null) {
-      return null;
-    }
-    DataInputStream dataIn = new DataInputStream(in);
-    FileType fileType = INode.FILE_TYPES[dataIn.readByte()];
-    switch (fileType) {
-      case DIRECTORY:
-        in.close();
-        return INode.DIRECTORY_INODE;
-      case FILE:
-        int numBlocks = dataIn.readInt();
-        Block[] blocks = new Block[numBlocks];
-        for (int i = 0; i < numBlocks; i++) {
-          long id = dataIn.readLong();
-          long length = dataIn.readLong();
-          blocks[i] = new Block(id, length);
-        }
-        in.close();
-        return new INode(fileType, blocks);
-      default:
-        throw new IllegalArgumentException("Cannot deserialize inode.");
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/3d485ecb/tajo-storage/src/test/java/org/apache/tajo/storage/s3/InMemoryFileSystemStore.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/test/java/org/apache/tajo/storage/s3/InMemoryFileSystemStore.java b/tajo-storage/src/test/java/org/apache/tajo/storage/s3/InMemoryFileSystemStore.java
deleted file mode 100644
index 40decc2..0000000
--- a/tajo-storage/src/test/java/org/apache/tajo/storage/s3/InMemoryFileSystemStore.java
+++ /dev/null
@@ -1,175 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tajo.storage.s3;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.s3.Block;
-import org.apache.hadoop.fs.s3.FileSystemStore;
-import org.apache.hadoop.fs.s3.INode;
-import org.apache.tajo.common.exception.NotImplementedException;
-
-import java.io.*;
-import java.net.URI;
-import java.util.*;
-
-/**
- * A stub implementation of {@link FileSystemStore} for testing
- * {@link S3FileSystem} without actually connecting to S3.
- */
-public class InMemoryFileSystemStore implements FileSystemStore {
-
-  private Configuration conf;
-  private SortedMap<Path, INode> inodes = new TreeMap<Path, INode>();
-  private Map<Long, byte[]> blocks = new HashMap<Long, byte[]>();
-
-  @Override
-  public void initialize(URI uri, Configuration conf) {
-    this.conf = conf;
-  }
-
-  @Override
-  public String getVersion() throws IOException {
-    return "0";
-  }
-
-  @Override
-  public void deleteINode(Path path) throws IOException {
-    inodes.remove(normalize(path));
-  }
-
-  @Override
-  public void deleteBlock(Block block) throws IOException {
-    blocks.remove(block.getId());
-  }
-
-  @Override
-  public boolean inodeExists(Path path) throws IOException {
-    return inodes.containsKey(normalize(path));
-  }
-
-  @Override
-  public boolean blockExists(long blockId) throws IOException {
-    return blocks.containsKey(blockId);
-  }
-
-  @Override
-  public INode retrieveINode(Path path) throws IOException {
-    return inodes.get(normalize(path));
-  }
-
-  @Override
-  public File retrieveBlock(Block block, long byteRangeStart) throws IOException {
-    byte[] data = blocks.get(block.getId());
-    File file = createTempFile();
-    BufferedOutputStream out = null;
-    try {
-      out = new BufferedOutputStream(new FileOutputStream(file));
-      out.write(data, (int) byteRangeStart, data.length - (int) byteRangeStart);
-    } finally {
-      if (out != null) {
-        out.close();
-      }
-    }
-    return file;
-  }
-
-  private File createTempFile() throws IOException {
-    File dir = new File(conf.get("fs.s3.buffer.dir"));
-    if (!dir.exists() && !dir.mkdirs()) {
-      throw new IOException("Cannot create S3 buffer directory: " + dir);
-    }
-    File result = File.createTempFile("test-", ".tmp", dir);
-    result.deleteOnExit();
-    return result;
-  }
-
-  @Override
-  public Set<Path> listSubPaths(Path path) throws IOException {
-    Path normalizedPath = normalize(path);
-    // This is inefficient but more than adequate for testing purposes.
-    Set<Path> subPaths = new LinkedHashSet<Path>();
-    for (Path p : inodes.tailMap(normalizedPath).keySet()) {
-      if (normalizedPath.equals(p.getParent())) {
-        subPaths.add(p);
-      }
-    }
-    return subPaths;
-  }
-
-  @Override
-  public Set<Path> listDeepSubPaths(Path path) throws IOException {
-    Path normalizedPath = normalize(path);
-    String pathString = normalizedPath.toUri().getPath();
-    if (!pathString.endsWith("/")) {
-      pathString += "/";
-    }
-    // This is inefficient but more than adequate for testing purposes.
-    Set<Path> subPaths = new LinkedHashSet<Path>();
-    for (Path p : inodes.tailMap(normalizedPath).keySet()) {
-      if (p.toUri().getPath().startsWith(pathString)) {
-        subPaths.add(p);
-      }
-    }
-    return subPaths;
-  }
-
-  @Override
-  public void storeINode(Path path, INode inode) throws IOException {
-    inodes.put(normalize(path), inode);
-  }
-
-  @Override
-  public void storeBlock(Block block, File file) throws IOException {
-    ByteArrayOutputStream out = new ByteArrayOutputStream();
-    byte[] buf = new byte[8192];
-    int numRead;
-    BufferedInputStream in = null;
-    try {
-      in = new BufferedInputStream(new FileInputStream(file));
-      while ((numRead = in.read(buf)) >= 0) {
-        out.write(buf, 0, numRead);
-      }
-    } finally {
-      if (in != null) {
-        in.close();
-      }
-    }
-    blocks.put(block.getId(), out.toByteArray());
-  }
-
-  private Path normalize(Path path) {
-    if (!path.isAbsolute()) {
-      throw new IllegalArgumentException("Path must be absolute: " + path);
-    }
-    return new Path(path.toUri().getPath());
-  }
-
-  @Override
-  public void purge() throws IOException {
-    inodes.clear();
-    blocks.clear();
-  }
-
-  @Override
-  public void dump() throws IOException {
-    throw new NotImplementedException();
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/3d485ecb/tajo-storage/src/test/java/org/apache/tajo/storage/s3/S3OutputStream.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/test/java/org/apache/tajo/storage/s3/S3OutputStream.java b/tajo-storage/src/test/java/org/apache/tajo/storage/s3/S3OutputStream.java
deleted file mode 100644
index d4034b9..0000000
--- a/tajo-storage/src/test/java/org/apache/tajo/storage/s3/S3OutputStream.java
+++ /dev/null
@@ -1,234 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tajo.storage.s3;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.s3.Block;
-import org.apache.hadoop.fs.s3.FileSystemStore;
-import org.apache.hadoop.fs.s3.INode;
-import org.apache.hadoop.util.Progressable;
-
-import java.io.File;
-import java.io.FileOutputStream;
-import java.io.IOException;
-import java.io.OutputStream;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Random;
-
-@InterfaceAudience.Private
-@InterfaceStability.Unstable
-class S3OutputStream extends OutputStream {
-
-  private Configuration conf;
-
-  private int bufferSize;
-
-  private FileSystemStore store;
-
-  private Path path;
-
-  private long blockSize;
-
-  private File backupFile;
-
-  private OutputStream backupStream;
-
-  private Random r = new Random();
-
-  private boolean closed;
-
-  private int pos = 0;
-
-  private long filePos = 0;
-
-  private int bytesWrittenToBlock = 0;
-
-  private byte[] outBuf;
-
-  private List<Block> blocks = new ArrayList<Block>();
-
-  private Block nextBlock;
-
-  private static final Log LOG =
-      LogFactory.getLog(S3OutputStream.class.getName());
-
-
-  public S3OutputStream(Configuration conf, FileSystemStore store,
-                        Path path, long blockSize, Progressable progress,
-                        int buffersize) throws IOException {
-
-    this.conf = conf;
-    this.store = store;
-    this.path = path;
-    this.blockSize = blockSize;
-    this.backupFile = newBackupFile();
-    this.backupStream = new FileOutputStream(backupFile);
-    this.bufferSize = buffersize;
-    this.outBuf = new byte[bufferSize];
-
-  }
-
-  private File newBackupFile() throws IOException {
-    File dir = new File(conf.get("fs.s3.buffer.dir"));
-    if (!dir.exists() && !dir.mkdirs()) {
-      throw new IOException("Cannot create S3 buffer directory: " + dir);
-    }
-    File result = File.createTempFile("output-", ".tmp", dir);
-    result.deleteOnExit();
-    return result;
-  }
-
-  public long getPos() throws IOException {
-    return filePos;
-  }
-
-  @Override
-  public synchronized void write(int b) throws IOException {
-    if (closed) {
-      throw new IOException("Stream closed");
-    }
-
-    if ((bytesWrittenToBlock + pos == blockSize) || (pos >= bufferSize)) {
-      flush();
-    }
-    outBuf[pos++] = (byte) b;
-    filePos++;
-  }
-
-  @Override
-  public synchronized void write(byte b[], int off, int len) throws IOException {
-    if (closed) {
-      throw new IOException("Stream closed");
-    }
-    while (len > 0) {
-      int remaining = bufferSize - pos;
-      int toWrite = Math.min(remaining, len);
-      System.arraycopy(b, off, outBuf, pos, toWrite);
-      pos += toWrite;
-      off += toWrite;
-      len -= toWrite;
-      filePos += toWrite;
-
-      if ((bytesWrittenToBlock + pos >= blockSize) || (pos == bufferSize)) {
-        flush();
-      }
-    }
-  }
-
-  @Override
-  public synchronized void flush() throws IOException {
-    if (closed) {
-      throw new IOException("Stream closed");
-    }
-
-    if (bytesWrittenToBlock + pos >= blockSize) {
-      flushData((int) blockSize - bytesWrittenToBlock);
-    }
-    if (bytesWrittenToBlock == blockSize) {
-      endBlock();
-    }
-    flushData(pos);
-  }
-
-  private synchronized void flushData(int maxPos) throws IOException {
-    int workingPos = Math.min(pos, maxPos);
-
-    if (workingPos > 0) {
-      //
-      // To the local block backup, write just the bytes
-      //
-      backupStream.write(outBuf, 0, workingPos);
-
-      //
-      // Track position
-      //
-      bytesWrittenToBlock += workingPos;
-      System.arraycopy(outBuf, workingPos, outBuf, 0, pos - workingPos);
-      pos -= workingPos;
-    }
-  }
-
-  private synchronized void endBlock() throws IOException {
-    //
-    // Done with local copy
-    //
-    backupStream.close();
-
-    //
-    // Send it to S3
-    //
-    // TODO: Use passed in Progressable to report progress.
-    nextBlockOutputStream();
-    store.storeBlock(nextBlock, backupFile);
-    Block[] arr = new Block[blocks.size()];
-    arr = blocks.toArray(arr);
-    store.storeINode(path, new INode(INode.FILE_TYPES[1], arr));
-
-    //
-    // Delete local backup, start new one
-    //
-    boolean b = backupFile.delete();
-    if (!b) {
-      LOG.warn("Ignoring failed delete");
-    }
-    backupFile = newBackupFile();
-    backupStream = new FileOutputStream(backupFile);
-    bytesWrittenToBlock = 0;
-  }
-
-  private synchronized void nextBlockOutputStream() throws IOException {
-    long blockId = r.nextLong();
-    while (store.blockExists(blockId)) {
-      blockId = r.nextLong();
-    }
-    nextBlock = new Block(blockId, bytesWrittenToBlock);
-    blocks.add(nextBlock);
-    bytesWrittenToBlock = 0;
-  }
-
-
-  @Override
-  public synchronized void close() throws IOException {
-    if (closed) {
-      return;
-    }
-
-    flush();
-    if (filePos == 0 || bytesWrittenToBlock != 0) {
-      endBlock();
-    }
-
-    backupStream.close();
-    boolean b = backupFile.delete();
-    if (!b) {
-      LOG.warn("Ignoring failed delete");
-    }
-
-    super.close();
-
-    closed = true;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/3d485ecb/tajo-storage/src/test/java/org/apache/tajo/storage/s3/SmallBlockS3FileSystem.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/test/java/org/apache/tajo/storage/s3/SmallBlockS3FileSystem.java b/tajo-storage/src/test/java/org/apache/tajo/storage/s3/SmallBlockS3FileSystem.java
deleted file mode 100644
index fc1c908..0000000
--- a/tajo-storage/src/test/java/org/apache/tajo/storage/s3/SmallBlockS3FileSystem.java
+++ /dev/null
@@ -1,314 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tajo.storage.s3;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FSDataOutputStream;
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.permission.FsPermission;
-import org.apache.hadoop.fs.s3.Block;
-import org.apache.hadoop.fs.s3.FileSystemStore;
-import org.apache.hadoop.fs.s3.INode;
-import org.apache.hadoop.fs.s3.S3FileSystem;
-import org.apache.hadoop.util.Progressable;
-
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.net.URI;
-import java.util.ArrayList;
-import java.util.List;
-
-public class SmallBlockS3FileSystem extends S3FileSystem {
-
-  private URI uri;
-
-  private FileSystemStore store;
-
-  private Path workingDir;
-
-  static class Holder {
-    private static InMemoryFileSystemStore s;
-
-    public synchronized static FileSystemStore get() {
-      if(s != null) {
-        return s;
-      }
-      s = new InMemoryFileSystemStore();
-      return s;
-    }
-
-    public synchronized static void set(InMemoryFileSystemStore inMemoryFileSystemStore) {
-      s = inMemoryFileSystemStore;
-    }
-  }
-
-  public SmallBlockS3FileSystem() {
-  }
-
-
-  public SmallBlockS3FileSystem(
-      InMemoryFileSystemStore inMemoryFileSystemStore) {
-    Holder.set(inMemoryFileSystemStore);
-    this.store = inMemoryFileSystemStore;
-  }
-
-  @Override
-  public URI getUri() {
-    return uri;
-  }
-  @Override
-  public long getDefaultBlockSize() {
-    return 10;
-  }
-
-  @Override
-  public void initialize(URI uri, Configuration conf) throws IOException {
-    if (store == null) {
-      store = Holder.get();
-    }
-    store.initialize(uri, conf);
-    setConf(conf);
-    this.uri = URI.create(uri.getScheme() + "://" + uri.getAuthority());
-    this.workingDir =
-        new Path("/user", System.getProperty("user.name")).makeQualified(this);
-  }
-  @Override
-  public boolean isFile(Path path) throws IOException {
-    INode inode = store.retrieveINode(makeAbsolute(path));
-    if (inode == null) {
-      return false;
-    }
-    return inode.isFile();
-  }
-
-  private INode checkFile(Path path) throws IOException {
-    INode inode = store.retrieveINode(makeAbsolute(path));
-    if (inode == null) {
-      throw new IOException("No such file.");
-    }
-    if (inode.isDirectory()) {
-      throw new IOException("Path " + path + " is a directory.");
-    }
-    return inode;
-  }
-
-  @Override
-  public FileStatus[] listStatus(Path f) throws IOException {
-    Path absolutePath = makeAbsolute(f);
-    INode inode = store.retrieveINode(absolutePath);
-    if (inode == null) {
-      throw new FileNotFoundException("File " + f + " does not exist.");
-    }
-    if (inode.isFile()) {
-      return new FileStatus[] {
-          new S3FileStatus(f.makeQualified(this), inode)
-      };
-    }
-    ArrayList<FileStatus> ret = new ArrayList<FileStatus>();
-    for (Path p : store.listSubPaths(absolutePath)) {
-      ret.add(getFileStatus(p.makeQualified(this)));
-    }
-    return ret.toArray(new FileStatus[0]);
-  }
-  @Override
-  public FSDataOutputStream create(Path file, FsPermission permission,
-                                   boolean overwrite, int bufferSize,
-                                   short replication, long blockSize, Progressable progress)
-      throws IOException {
-
-    INode inode = store.retrieveINode(makeAbsolute(file));
-    if (inode != null) {
-      if (overwrite) {
-        delete(file, true);
-      } else {
-        throw new IOException("File already exists: " + file);
-      }
-    } else {
-      Path parent = file.getParent();
-      if (parent != null) {
-        if (!mkdirs(parent)) {
-          throw new IOException("Mkdirs failed to create " + parent.toString());
-        }
-      }
-    }
-    return new FSDataOutputStream
-        (new S3OutputStream(getConf(), store, makeAbsolute(file),
-            blockSize, progress, bufferSize),
-            statistics);
-  }
-  @Override
-  public boolean mkdirs(Path path, FsPermission permission) throws IOException {
-    Path absolutePath = makeAbsolute(path);
-    List<Path> paths = new ArrayList<Path>();
-    do {
-      paths.add(0, absolutePath);
-      absolutePath = absolutePath.getParent();
-    } while (absolutePath != null);
-
-    boolean result = true;
-    for (Path p : paths) {
-      result &= mkdir(p);
-    }
-    return result;
-  }
-
-  @Override
-  public Path getWorkingDirectory() {
-    return workingDir;
-  }
-
-  @Override
-  public boolean rename(Path src, Path dst) throws IOException {
-    Path absoluteSrc = makeAbsolute(src);
-    INode srcINode = store.retrieveINode(absoluteSrc);
-    if (srcINode == null) {
-      // src path doesn't exist
-      return false;
-    }
-    Path absoluteDst = makeAbsolute(dst);
-    INode dstINode = store.retrieveINode(absoluteDst);
-    if (dstINode != null && dstINode.isDirectory()) {
-      absoluteDst = new Path(absoluteDst, absoluteSrc.getName());
-      dstINode = store.retrieveINode(absoluteDst);
-    }
-    if (dstINode != null) {
-      // dst path already exists - can't overwrite
-      return false;
-    }
-    Path dstParent = absoluteDst.getParent();
-    if (dstParent != null) {
-      INode dstParentINode = store.retrieveINode(dstParent);
-      if (dstParentINode == null || dstParentINode.isFile()) {
-        // dst parent doesn't exist or is a file
-        return false;
-      }
-    }
-    return renameRecursive(absoluteSrc, absoluteDst);
-  }
-
-  private boolean renameRecursive(Path src, Path dst) throws IOException {
-    INode srcINode = store.retrieveINode(src);
-    store.storeINode(dst, srcINode);
-    store.deleteINode(src);
-    if (srcINode.isDirectory()) {
-      for (Path oldSrc : store.listDeepSubPaths(src)) {
-        INode inode = store.retrieveINode(oldSrc);
-        if (inode == null) {
-          return false;
-        }
-        String oldSrcPath = oldSrc.toUri().getPath();
-        String srcPath = src.toUri().getPath();
-        String dstPath = dst.toUri().getPath();
-        Path newDst = new Path(oldSrcPath.replaceFirst(srcPath, dstPath));
-        store.storeINode(newDst, inode);
-        store.deleteINode(oldSrc);
-      }
-    }
-    return true;
-  }
-
-  @Override
-  public boolean delete(Path path, boolean recursive) throws IOException {
-    Path absolutePath = makeAbsolute(path);
-    INode inode = store.retrieveINode(absolutePath);
-    if (inode == null) {
-      return false;
-    }
-    if (inode.isFile()) {
-      store.deleteINode(absolutePath);
-      for (Block block: inode.getBlocks()) {
-        store.deleteBlock(block);
-      }
-    } else {
-      FileStatus[] contents = null;
-      try {
-        contents = listStatus(absolutePath);
-      } catch(FileNotFoundException fnfe) {
-        return false;
-      }
-
-      if ((contents.length !=0) && (!recursive)) {
-        throw new IOException("Directory " + path.toString()
-            + " is not empty.");
-      }
-      for (FileStatus p:contents) {
-        if (!delete(p.getPath(), recursive)) {
-          return false;
-        }
-      }
-      store.deleteINode(absolutePath);
-    }
-    return true;
-  }
-
-  /**
-   * FileStatus for S3 file systems.
-   */
-  @Override
-  public FileStatus getFileStatus(Path f)  throws IOException {
-    INode inode = store.retrieveINode(makeAbsolute(f));
-    if (inode == null) {
-      throw new FileNotFoundException(f + ": No such file or directory.");
-    }
-    return new S3FileStatus(f.makeQualified(this), inode);
-  }
-  private boolean mkdir(Path path) throws IOException {
-    Path absolutePath = makeAbsolute(path);
-    INode inode = store.retrieveINode(absolutePath);
-    if (inode == null) {
-      store.storeINode(absolutePath, INode.DIRECTORY_INODE);
-    } else if (inode.isFile()) {
-      throw new IOException(String.format(
-          "Can't make directory for path %s since it is a file.",
-          absolutePath));
-    }
-    return true;
-  }
-  private Path makeAbsolute(Path path) {
-    if (path.isAbsolute()) {
-      return path;
-    }
-    return new Path(workingDir, path);
-  }
-
-  private static class S3FileStatus extends FileStatus {
-
-    S3FileStatus(Path f, INode inode) throws IOException {
-      super(findLength(inode), inode.isDirectory(), 1,
-          findBlocksize(inode), 0, f);
-    }
-
-    private static long findLength(INode inode) {
-      if (!inode.isDirectory()) {
-        long length = 0L;
-        for (Block block : inode.getBlocks()) {
-          length += block.getLength();
-        }
-        return length;
-      }
-      return 0;
-    }
-
-    private static long findBlocksize(INode inode) {
-      final Block[] ret = inode.getBlocks();
-      return ret == null ? 0L : ret[0].getLength();
-    }
-  }
-}
\ No newline at end of file


[16/29] tajo git commit: TAJO-1223: Wrong query verification against asterisk and more expressions in select list.

Posted by hj...@apache.org.
TAJO-1223: Wrong query verification against asterisk and more expressions in select list.

Closes #279


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

Branch: refs/heads/hbase_storage
Commit: 20d1f0145b47e6c478bb55c79d670926e66f137c
Parents: ea2dbf8
Author: Hyunsik Choi <hy...@apache.org>
Authored: Wed Dec 3 17:34:28 2014 +0900
Committer: Hyunsik Choi <hy...@apache.org>
Committed: Wed Dec 3 18:34:33 2014 +0900

----------------------------------------------------------------------
 .../tajo/engine/query/TestInsertQuery.java      | 22 +++++++
 .../TestInsertQuery/lineitem_year_month_ddl.sql | 18 ++++++
 .../load_to_lineitem_year_month.sql             |  1 +
 .../testInsertOverwriteWithAsteriskAndMore.sql  |  1 +
 ...estInsertOverwriteWithAsteriskAndMore.result |  7 +++
 .../plan/verifier/PreLogicalPlanVerifier.java   | 64 ++++++++++++--------
 6 files changed, 87 insertions(+), 26 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tajo/blob/20d1f014/tajo-core/src/test/java/org/apache/tajo/engine/query/TestInsertQuery.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestInsertQuery.java b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestInsertQuery.java
index 117f186..cc7dced 100644
--- a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestInsertQuery.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestInsertQuery.java
@@ -392,6 +392,28 @@ public class TestInsertQuery extends QueryTestCaseBase {
   }
 
   @Test
+  public final void testInsertOverwriteWithAsteriskAndMore() throws Exception {
+    ResultSet res = executeFile("lineitem_year_month_ddl.sql");
+    res.close();
+
+    CatalogService catalog = testingCluster.getMaster().getCatalog();
+    assertTrue(catalog.existsTable(getCurrentDatabase(), "lineitem_year_month"));
+
+    res = executeFile("load_to_lineitem_year_month.sql");
+    res.close();
+    TableDesc desc = catalog.getTableDesc(getCurrentDatabase(), "lineitem_year_month");
+    if (!testingCluster.isHCatalogStoreRunning()) {
+      assertEquals(5, desc.getStats().getNumRows().intValue());
+    }
+
+    res = executeQuery();
+    assertResultSet(res);
+    res.close();
+
+    executeString("DROP TABLE lineitem_year_month PURGE");
+  }
+
+  @Test
   public final void testInsertOverwriteIntoSelect() throws Exception {
     String tableName = CatalogUtil.normalizeIdentifier("insertoverwriteintoselect");
     ResultSet res = executeString("create table " + tableName + " as select l_orderkey from default.lineitem");

http://git-wip-us.apache.org/repos/asf/tajo/blob/20d1f014/tajo-core/src/test/resources/queries/TestInsertQuery/lineitem_year_month_ddl.sql
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/resources/queries/TestInsertQuery/lineitem_year_month_ddl.sql b/tajo-core/src/test/resources/queries/TestInsertQuery/lineitem_year_month_ddl.sql
new file mode 100644
index 0000000..fb18ad8
--- /dev/null
+++ b/tajo-core/src/test/resources/queries/TestInsertQuery/lineitem_year_month_ddl.sql
@@ -0,0 +1,18 @@
+create table lineitem_year_month (
+ l_orderkey bigint,
+ l_partkey bigint,
+ l_suppkey bigint,
+ l_linenumber int,
+ l_quantity float8,
+ l_extendedprice float8,
+ l_discount float8,
+ l_tax float8,
+ l_returnflag text,
+ l_linestatus text,
+ l_shipdate text,
+ l_commitdate text,
+ l_receiptdate text,
+ l_shipinstruct text,
+ l_shipmode text,
+ l_comment text
+) PARTITION BY COLUMN (year TEXT, MONTH TEXT);
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/20d1f014/tajo-core/src/test/resources/queries/TestInsertQuery/load_to_lineitem_year_month.sql
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/resources/queries/TestInsertQuery/load_to_lineitem_year_month.sql b/tajo-core/src/test/resources/queries/TestInsertQuery/load_to_lineitem_year_month.sql
new file mode 100644
index 0000000..601af12
--- /dev/null
+++ b/tajo-core/src/test/resources/queries/TestInsertQuery/load_to_lineitem_year_month.sql
@@ -0,0 +1 @@
+INSERT INTO lineitem_year_month SELECT *, SUBSTR(l_shipdate, 1,4) as year, SUBSTR(l_shipdate, 6, 2) as month FROM default.lineitem;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/20d1f014/tajo-core/src/test/resources/queries/TestInsertQuery/testInsertOverwriteWithAsteriskAndMore.sql
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/resources/queries/TestInsertQuery/testInsertOverwriteWithAsteriskAndMore.sql b/tajo-core/src/test/resources/queries/TestInsertQuery/testInsertOverwriteWithAsteriskAndMore.sql
new file mode 100644
index 0000000..07c904d
--- /dev/null
+++ b/tajo-core/src/test/resources/queries/TestInsertQuery/testInsertOverwriteWithAsteriskAndMore.sql
@@ -0,0 +1 @@
+select * from lineitem_year_month;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/20d1f014/tajo-core/src/test/resources/results/TestInsertQuery/testInsertOverwriteWithAsteriskAndMore.result
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/resources/results/TestInsertQuery/testInsertOverwriteWithAsteriskAndMore.result b/tajo-core/src/test/resources/results/TestInsertQuery/testInsertOverwriteWithAsteriskAndMore.result
new file mode 100644
index 0000000..bb797ba
--- /dev/null
+++ b/tajo-core/src/test/resources/results/TestInsertQuery/testInsertOverwriteWithAsteriskAndMore.result
@@ -0,0 +1,7 @@
+l_orderkey,l_partkey,l_suppkey,l_linenumber,l_quantity,l_extendedprice,l_discount,l_tax,l_returnflag,l_linestatus,l_shipdate,l_commitdate,l_receiptdate,l_shipinstruct,l_shipmode,l_comment,year,MONTH
+-------------------------------
+3,3,6540,2,49.0,46796.47,0.1,0.0,R,F,1993-11-09,1993-12-20,1993-11-24,TAKE BACK RETURN,RAIL, unusual accounts. eve,1993,11
+3,2,1798,1,45.0,54058.05,0.06,0.0,R,F,1994-02-02,1994-01-04,1994-02-23,NONE,AIR,ongside of the furiously brave acco,1994,02
+1,1,7706,1,17.0,21168.23,0.04,0.02,N,O,1996-03-13,1996-02-12,1996-03-22,DELIVER IN PERSON,TRUCK,egular courts above the,1996,03
+1,1,7311,2,36.0,45983.16,0.09,0.06,N,O,1996-04-12,1996-02-28,1996-04-20,TAKE BACK RETURN,MAIL,ly final dependencies: slyly bold ,1996,04
+2,2,1191,1,38.0,44694.46,0.0,0.05,N,O,1997-01-28,1997-01-14,1997-02-02,TAKE BACK RETURN,RAIL,ven requests. deposits breach a,1997,01
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/20d1f014/tajo-plan/src/main/java/org/apache/tajo/plan/verifier/PreLogicalPlanVerifier.java
----------------------------------------------------------------------
diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/verifier/PreLogicalPlanVerifier.java b/tajo-plan/src/main/java/org/apache/tajo/plan/verifier/PreLogicalPlanVerifier.java
index 95e0f30..f6d04ba 100644
--- a/tajo-plan/src/main/java/org/apache/tajo/plan/verifier/PreLogicalPlanVerifier.java
+++ b/tajo-plan/src/main/java/org/apache/tajo/plan/verifier/PreLogicalPlanVerifier.java
@@ -254,36 +254,48 @@ public class PreLogicalPlanVerifier extends BaseAlgebraVisitor<PreLogicalPlanVer
 
     if (child != null && child.getType() == OpType.Projection) {
       Projection projection = (Projection) child;
-      int projectColumnNum = projection.getNamedExprs().length;
 
-      if (expr.hasTargetColumns()) {
-        int targetColumnNum = expr.getTargetColumns().length;
+      // checking if at least one asterisk exists in target list
+      boolean includeAsterisk = false;
+      for (NamedExpr namedExpr : projection.getNamedExprs()) {
+        includeAsterisk |= namedExpr.getExpr().getType() == OpType.Asterisk;
+      }
 
-        if (targetColumnNum > projectColumnNum)  {
-          context.state.addVerification("INSERT has more target columns than expressions");
-        } else if (targetColumnNum < projectColumnNum) {
-          context.state.addVerification("INSERT has more expressions than target columns");
-        }
-      } else {
-        if (expr.hasTableName()) {
-          String qualifiedName = expr.getTableName();
-          if (TajoConstants.EMPTY_STRING.equals(CatalogUtil.extractQualifier(expr.getTableName()))) {
-            qualifiedName = CatalogUtil.buildFQName(context.queryContext.get(SessionVars.CURRENT_DATABASE),
-                expr.getTableName());
-          }
+      // If one asterisk expression exists, we verify the match between the target exprs and output exprs.
+      // This verification will be in LogicalPlanVerifier.
+      if (!includeAsterisk) {
+
+        int projectColumnNum = projection.getNamedExprs().length;
+
+        if (expr.hasTargetColumns()) {
+          int targetColumnNum = expr.getTargetColumns().length;
 
-          TableDesc table = catalog.getTableDesc(qualifiedName);
-          if (table == null) {
-            context.state.addVerification(String.format("relation \"%s\" does not exist", qualifiedName));
-            return null;
+          if (targetColumnNum > projectColumnNum) {
+            context.state.addVerification("INSERT has more target columns than expressions");
+          } else if (targetColumnNum < projectColumnNum) {
+            context.state.addVerification("INSERT has more expressions than target columns");
           }
-          if (table.hasPartition()) {
-            int columnSize = table.getSchema().getColumns().size();
-            columnSize += table.getPartitionMethod().getExpressionSchema().getColumns().size();
-            if (projectColumnNum < columnSize) {
-              context.state.addVerification("INSERT has smaller expressions than target columns");
-            } else if (projectColumnNum > columnSize) {
-              context.state.addVerification("INSERT has more expressions than target columns");
+        } else {
+          if (expr.hasTableName()) {
+            String qualifiedName = expr.getTableName();
+            if (TajoConstants.EMPTY_STRING.equals(CatalogUtil.extractQualifier(expr.getTableName()))) {
+              qualifiedName = CatalogUtil.buildFQName(context.queryContext.get(SessionVars.CURRENT_DATABASE),
+                  expr.getTableName());
+            }
+
+            TableDesc table = catalog.getTableDesc(qualifiedName);
+            if (table == null) {
+              context.state.addVerification(String.format("relation \"%s\" does not exist", qualifiedName));
+              return null;
+            }
+            if (table.hasPartition()) {
+              int columnSize = table.getSchema().getColumns().size();
+              columnSize += table.getPartitionMethod().getExpressionSchema().getColumns().size();
+              if (projectColumnNum < columnSize) {
+                context.state.addVerification("INSERT has smaller expressions than target columns");
+              } else if (projectColumnNum > columnSize) {
+                context.state.addVerification("INSERT has more expressions than target columns");
+              }
             }
           }
         }


[18/29] tajo git commit: TAJO-1191: Change DateDatum timezone to UTC. (Jaewoong Jung via hyunsik)

Posted by hj...@apache.org.
TAJO-1191: Change DateDatum timezone to UTC. (Jaewoong Jung via hyunsik)

Closes #278


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

Branch: refs/heads/hbase_storage
Commit: 97507e45883c9db2fec2f5a9a9e544384a86ccd0
Parents: 9f8be1a
Author: Hyunsik Choi <hy...@apache.org>
Authored: Thu Dec 4 16:12:25 2014 +0900
Committer: Hyunsik Choi <hy...@apache.org>
Committed: Thu Dec 4 16:12:25 2014 +0900

----------------------------------------------------------------------
 CHANGES                                                  |  2 ++
 .../src/main/java/org/apache/tajo/datum/DateDatum.java   | 11 +----------
 .../main/java/org/apache/tajo/datum/IntervalDatum.java   |  1 -
 .../src/main/java/org/apache/tajo/datum/TimeDatum.java   |  4 ----
 .../java/org/apache/tajo/datum/TestIntervalDatum.java    |  8 ++++----
 5 files changed, 7 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tajo/blob/97507e45/CHANGES
----------------------------------------------------------------------
diff --git a/CHANGES b/CHANGES
index 60aa3e0..a697afe 100644
--- a/CHANGES
+++ b/CHANGES
@@ -83,6 +83,8 @@ Release 0.9.1 - unreleased
 
   BUG FIXES
 
+    TAJO-1191: Change DateDatum timezone to UTC. (Jaewoong Jung via hyunsik)
+
     TAJO-1224: When there is no projected column, json scan can be hang. 
     (hyunsik) 
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/97507e45/tajo-common/src/main/java/org/apache/tajo/datum/DateDatum.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/datum/DateDatum.java b/tajo-common/src/main/java/org/apache/tajo/datum/DateDatum.java
index 3296d4f..188d226 100644
--- a/tajo-common/src/main/java/org/apache/tajo/datum/DateDatum.java
+++ b/tajo-common/src/main/java/org/apache/tajo/datum/DateDatum.java
@@ -31,6 +31,7 @@ import org.apache.tajo.util.datetime.TimeMeta;
 public class DateDatum extends Datum {
   public static final int SIZE = 4;
 
+  // Dates are stored in UTC.
   private int year;
   private int monthOfYear;
   private int dayOfMonth;
@@ -119,17 +120,12 @@ public class DateDatum extends Datum {
         if (interval.getMonths() > 0) {
           tm.plusMonths(interval.getMonths());
         }
-        DateTimeUtil.toUTCTimezone(tm);
         return new TimestampDatum(DateTimeUtil.toJulianTimestamp(tm));
       }
       case TIME: {
         TimeMeta tm1 = toTimeMeta();
-
         TimeMeta tm2 = ((TimeDatum)datum).toTimeMeta();
-        DateTimeUtil.toUserTimezone(tm2);     //TimeDatum is UTC
-
         tm1.plusTime(DateTimeUtil.toTime(tm2));
-        DateTimeUtil.toUTCTimezone(tm1);
         return new TimestampDatum(DateTimeUtil.toJulianTimestamp(tm1));
       }
       default:
@@ -155,17 +151,12 @@ public class DateDatum extends Datum {
           tm.plusMonths(0 - interval.getMonths());
         }
         tm.plusMillis(0 - interval.getMilliSeconds());
-        DateTimeUtil.toUTCTimezone(tm);
         return new TimestampDatum(DateTimeUtil.toJulianTimestamp(tm));
       }
       case TIME: {
         TimeMeta tm1 = toTimeMeta();
-
         TimeMeta tm2 = ((TimeDatum)datum).toTimeMeta();
-        DateTimeUtil.toUserTimezone(tm2);     //TimeDatum is UTC
-
         tm1.plusTime(0 - DateTimeUtil.toTime(tm2));
-        DateTimeUtil.toUTCTimezone(tm1);
         return new TimestampDatum(DateTimeUtil.toJulianTimestamp(tm1));
       }
       case DATE: {

http://git-wip-us.apache.org/repos/asf/tajo/blob/97507e45/tajo-common/src/main/java/org/apache/tajo/datum/IntervalDatum.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/datum/IntervalDatum.java b/tajo-common/src/main/java/org/apache/tajo/datum/IntervalDatum.java
index c6f3922..6207891 100644
--- a/tajo-common/src/main/java/org/apache/tajo/datum/IntervalDatum.java
+++ b/tajo-common/src/main/java/org/apache/tajo/datum/IntervalDatum.java
@@ -236,7 +236,6 @@ public class IntervalDatum extends Datum {
         if (getMonths() > 0) {
           tm.plusMonths(getMonths());
         }
-        DateTimeUtil.toUTCTimezone(tm);
         return new TimestampDatum(DateTimeUtil.toJulianTimestamp(tm));
       }
       case TIME: {

http://git-wip-us.apache.org/repos/asf/tajo/blob/97507e45/tajo-common/src/main/java/org/apache/tajo/datum/TimeDatum.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/datum/TimeDatum.java b/tajo-common/src/main/java/org/apache/tajo/datum/TimeDatum.java
index 37e5e78..6cac586 100644
--- a/tajo-common/src/main/java/org/apache/tajo/datum/TimeDatum.java
+++ b/tajo-common/src/main/java/org/apache/tajo/datum/TimeDatum.java
@@ -128,13 +128,9 @@ public class TimeDatum extends Datum {
       }
       case DATE: {
         TimeMeta tm = toTimeMeta();
-        DateTimeUtil.toUserTimezone(tm);     //TimeDatum is UTC
-
         DateDatum dateDatum = (DateDatum) datum;
         TimeMeta dateTm = dateDatum.toTimeMeta();
         dateTm.plusTime(DateTimeUtil.toTime(tm));
-
-        DateTimeUtil.toUTCTimezone(dateTm);
         return new TimestampDatum(DateTimeUtil.toJulianTimestamp(dateTm));
       }
       default:

http://git-wip-us.apache.org/repos/asf/tajo/blob/97507e45/tajo-common/src/test/java/org/apache/tajo/datum/TestIntervalDatum.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/test/java/org/apache/tajo/datum/TestIntervalDatum.java b/tajo-common/src/test/java/org/apache/tajo/datum/TestIntervalDatum.java
index 511b356..2646ee7 100644
--- a/tajo-common/src/test/java/org/apache/tajo/datum/TestIntervalDatum.java
+++ b/tajo-common/src/test/java/org/apache/tajo/datum/TestIntervalDatum.java
@@ -79,13 +79,13 @@ public class TestIntervalDatum {
     datum = DatumFactory.createDate(2001, 9, 28);
     Datum result = datum.plus(new IntervalDatum(60 * 60 * 1000));
     assertEquals(TajoDataTypes.Type.TIMESTAMP, result.type());
-    assertEquals("2001-09-28 01:00:00", ((TimestampDatum)result).asChars(TajoConf.getCurrentTimeZone(), false));
+    assertEquals("2001-09-28 01:00:00", result.asChars());
 
     // interval '1 hour' +  date '2001-09-28'	==> timestamp '2001-09-28 01:00:00'
     datum = new IntervalDatum(60 * 60 * 1000);
     result = datum.plus(DatumFactory.createDate(2001, 9, 28));
     assertEquals(TajoDataTypes.Type.TIMESTAMP, result.type());
-    assertEquals("2001-09-28 01:00:00", ((TimestampDatum)result).asChars(TajoConf.getCurrentTimeZone(), false));
+    assertEquals("2001-09-28 01:00:00", result.asChars());
 
     // date '2001-09-28' + time '03:00' ==> timestamp '2001-09-28 03:00:00'
     datum = DatumFactory.createDate(2001, 9, 28);
@@ -133,14 +133,14 @@ public class TestIntervalDatum {
     datum = DatumFactory.createDate(2001, 9, 28);
     result = datum.minus(new IntervalDatum(1 * 60 * 60 * 1000));
     assertEquals(TajoDataTypes.Type.TIMESTAMP, result.type());
-    assertEquals("2001-09-27 23:00:00", ((TimestampDatum)result).asChars(TajoConf.getCurrentTimeZone(), false));
+    assertEquals("2001-09-27 23:00:00", result.asChars());
 
     // date '2001-09-28' - interval '1 day 1 hour' ==> timestamp '2001-09-26 23:00:00'
     // In this case all datums are UTC
     datum = DatumFactory.createDate(2001, 9, 28);
     result = datum.minus(new IntervalDatum(IntervalDatum.DAY_MILLIS + 1 * 60 * 60 * 1000));
     assertEquals(TajoDataTypes.Type.TIMESTAMP, result.type());
-    assertEquals("2001-09-26 23:00:00",  ((TimestampDatum)result).asChars(TajoConf.getCurrentTimeZone(), false));
+    assertEquals("2001-09-26 23:00:00",  result.asChars());
 
     // time '05:00' - time '03:00' ==>	interval '02:00:00'
     datum = new TimeDatum(DateTimeUtil.toTime(5, 0, 0, 0));


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

Posted by hj...@apache.org.
http://git-wip-us.apache.org/repos/asf/tajo/blob/940546a0/tajo-core/src/main/java/org/apache/tajo/master/querymaster/Query.java
----------------------------------------------------------------------
diff --cc tajo-core/src/main/java/org/apache/tajo/master/querymaster/Query.java
index bc3be04,07b47c1..b68eb2e
--- a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/Query.java
+++ b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/Query.java
@@@ -438,28 -427,356 +434,28 @@@ public class Query implements EventHand
        return finalState;
      }
  
-     private boolean finalizeQuery(Query query, QueryCompletedEvent event) {
+     private QueryState finalizeQuery(Query query, QueryCompletedEvent event) {
 -      MasterPlan masterPlan = query.getPlan();
 +      SubQuery lastStage = query.getSubQuery(event.getExecutionBlockId());
 +      StoreType storeType = lastStage.getTableMeta().getStoreType();
 +      try {
 +        LogicalRootNode rootNode = lastStage.getMasterPlan().getLogicalPlan().getRootBlock().getRoot();
 +        CatalogService catalog = lastStage.getContext().getQueryMasterContext().getWorkerContext().getCatalog();
 +        TableDesc tableDesc =  PlannerUtil.getTableDesc(catalog, rootNode.getChild());
  
 -      ExecutionBlock terminal = query.getPlan().getTerminalBlock();
 -      DataChannel finalChannel = masterPlan.getChannel(event.getExecutionBlockId(), terminal.getId());
 +        Path finalOutputDir = StorageManager.getStorageManager(query.systemConf, storeType)
 +            .commitOutputData(query.context.getQueryContext(),
 +                lastStage.getId(), lastStage.getMasterPlan().getLogicalPlan(), lastStage.getSchema(), tableDesc);
  
 -      QueryHookExecutor hookExecutor = new QueryHookExecutor(query.context.getQueryMasterContext());
 -      try {
 -        Path finalOutputDir = commitOutputData(query);
 +        QueryHookExecutor hookExecutor = new QueryHookExecutor(query.context.getQueryMasterContext());
-           hookExecutor.execute(query.context.getQueryContext(), query, event.getExecutionBlockId(), finalOutputDir);
-         return true;
+         hookExecutor.execute(query.context.getQueryContext(), query, event.getExecutionBlockId(), finalOutputDir);
 -      } catch (Throwable t) {
 -        query.eventHandler.handle(new QueryDiagnosticsUpdateEvent(query.id, ExceptionUtils.getStackTrace(t)));
 +      } catch (Exception e) {
-         LOG.error(e.getMessage(), e);
 +        query.eventHandler.handle(new QueryDiagnosticsUpdateEvent(query.id, ExceptionUtils.getStackTrace(e)));
-         return false;
+         return QueryState.QUERY_ERROR;
        }
 -
++      
+       return QueryState.QUERY_SUCCEEDED;
      }
  
 -    /**
 -     * It moves a result data stored in a staging output dir into a final output dir.
 -     */
 -    public Path commitOutputData(Query query) throws IOException {
 -      QueryContext queryContext = query.context.getQueryContext();
 -      Path stagingResultDir = new Path(queryContext.getStagingDir(), TajoConstants.RESULT_DIR_NAME);
 -      Path finalOutputDir;
 -      if (queryContext.hasOutputPath()) {
 -        finalOutputDir = queryContext.getOutputPath();
 -        try {
 -          FileSystem fs = stagingResultDir.getFileSystem(query.systemConf);
 -
 -          if (queryContext.isOutputOverwrite()) { // INSERT OVERWRITE INTO
 -
 -            // It moves the original table into the temporary location.
 -            // Then it moves the new result table into the original table location.
 -            // Upon failed, it recovers the original table if possible.
 -            boolean movedToOldTable = false;
 -            boolean committed = false;
 -            Path oldTableDir = new Path(queryContext.getStagingDir(), TajoConstants.INSERT_OVERWIRTE_OLD_TABLE_NAME);
 -
 -            if (queryContext.hasPartition()) {
 -              // This is a map for existing non-leaf directory to rename. A key is current directory and a value is
 -              // renaming directory.
 -              Map<Path, Path> renameDirs = TUtil.newHashMap();
 -              // This is a map for recovering existing partition directory. A key is current directory and a value is
 -              // temporary directory to back up.
 -              Map<Path, Path> recoveryDirs = TUtil.newHashMap();
 -
 -              try {
 -                if (!fs.exists(finalOutputDir)) {
 -                  fs.mkdirs(finalOutputDir);
 -                }
 -
 -                visitPartitionedDirectory(fs, stagingResultDir, finalOutputDir, stagingResultDir.toString(),
 -                    renameDirs, oldTableDir);
 -
 -                // Rename target partition directories
 -                for(Map.Entry<Path, Path> entry : renameDirs.entrySet()) {
 -                  // Backup existing data files for recovering
 -                  if (fs.exists(entry.getValue())) {
 -                    String recoveryPathString = entry.getValue().toString().replaceAll(finalOutputDir.toString(),
 -                        oldTableDir.toString());
 -                    Path recoveryPath = new Path(recoveryPathString);
 -                    fs.rename(entry.getValue(), recoveryPath);
 -                    fs.exists(recoveryPath);
 -                    recoveryDirs.put(entry.getValue(), recoveryPath);
 -                  }
 -                  // Delete existing directory
 -                  fs.delete(entry.getValue(), true);
 -                  // Rename staging directory to final output directory
 -                  fs.rename(entry.getKey(), entry.getValue());
 -                }
 -
 -              } catch (IOException ioe) {
 -                // Remove created dirs
 -                for(Map.Entry<Path, Path> entry : renameDirs.entrySet()) {
 -                  fs.delete(entry.getValue(), true);
 -                }
 -
 -                // Recovery renamed dirs
 -                for(Map.Entry<Path, Path> entry : recoveryDirs.entrySet()) {
 -                  fs.delete(entry.getValue(), true);
 -                  fs.rename(entry.getValue(), entry.getKey());
 -                }
 -
 -                throw new IOException(ioe.getMessage());
 -              }
 -            } else { // no partition
 -              try {
 -
 -                // if the final output dir exists, move all contents to the temporary table dir.
 -                // Otherwise, just make the final output dir. As a result, the final output dir will be empty.
 -                if (fs.exists(finalOutputDir)) {
 -                  fs.mkdirs(oldTableDir);
 -
 -                  for (FileStatus status : fs.listStatus(finalOutputDir, StorageManager.hiddenFileFilter)) {
 -                    fs.rename(status.getPath(), oldTableDir);
 -                  }
 -
 -                  movedToOldTable = fs.exists(oldTableDir);
 -                } else { // if the parent does not exist, make its parent directory.
 -                  fs.mkdirs(finalOutputDir);
 -                }
 -
 -                // Move the results to the final output dir.
 -                for (FileStatus status : fs.listStatus(stagingResultDir)) {
 -                  fs.rename(status.getPath(), finalOutputDir);
 -                }
 -
 -                // Check the final output dir
 -                committed = fs.exists(finalOutputDir);
 -
 -              } catch (IOException ioe) {
 -                // recover the old table
 -                if (movedToOldTable && !committed) {
 -
 -                  // if commit is failed, recover the old data
 -                  for (FileStatus status : fs.listStatus(finalOutputDir, StorageManager.hiddenFileFilter)) {
 -                    fs.delete(status.getPath(), true);
 -                  }
 -
 -                  for (FileStatus status : fs.listStatus(oldTableDir)) {
 -                    fs.rename(status.getPath(), finalOutputDir);
 -                  }
 -                }
 -
 -                throw new IOException(ioe.getMessage());
 -              }
 -            }
 -          } else {
 -            NodeType queryType = queryContext.getCommandType();
 -
 -            if (queryType == NodeType.INSERT) { // INSERT INTO an existing table
 -
 -              NumberFormat fmt = NumberFormat.getInstance();
 -              fmt.setGroupingUsed(false);
 -              fmt.setMinimumIntegerDigits(3);
 -
 -              if (queryContext.hasPartition()) {
 -                for(FileStatus eachFile: fs.listStatus(stagingResultDir)) {
 -                  if (eachFile.isFile()) {
 -                    LOG.warn("Partition table can't have file in a staging dir: " + eachFile.getPath());
 -                    continue;
 -                  }
 -                  moveResultFromStageToFinal(fs, stagingResultDir, eachFile, finalOutputDir, fmt, -1);
 -                }
 -              } else {
 -                int maxSeq = StorageUtil.getMaxFileSequence(fs, finalOutputDir, false) + 1;
 -                for(FileStatus eachFile: fs.listStatus(stagingResultDir)) {
 -                  moveResultFromStageToFinal(fs, stagingResultDir, eachFile, finalOutputDir, fmt, maxSeq++);
 -                }
 -              }
 -              // checking all file moved and remove empty dir
 -              verifyAllFileMoved(fs, stagingResultDir);
 -              FileStatus[] files = fs.listStatus(stagingResultDir);
 -              if (files != null && files.length != 0) {
 -                for (FileStatus eachFile: files) {
 -                  LOG.error("There are some unmoved files in staging dir:" + eachFile.getPath());
 -                }
 -              }
 -            } else { // CREATE TABLE AS SELECT (CTAS)
 -              if (fs.exists(finalOutputDir)) {
 -                for (FileStatus status : fs.listStatus(stagingResultDir)) {
 -                  fs.rename(status.getPath(), finalOutputDir);
 -                }
 -              } else {
 -                fs.rename(stagingResultDir, finalOutputDir);
 -              }
 -              LOG.info("Moved from the staging dir to the output directory '" + finalOutputDir);
 -            }
 -          }
 -
 -          // remove the staging directory if the final output dir is given.
 -          Path stagingDirRoot = queryContext.getStagingDir().getParent();
 -          fs.delete(stagingDirRoot, true);
 -
 -        } catch (Throwable t) {
 -          LOG.error(t);
 -          throw new IOException(t);
 -        }
 -      } else {
 -        finalOutputDir = new Path(queryContext.getStagingDir(), TajoConstants.RESULT_DIR_NAME);
 -      }
 -
 -      return finalOutputDir;
 -    }
 -
 -    /**
 -     * This method sets a rename map which includes renamed staging directory to final output directory recursively.
 -     * If there exists some data files, this delete it for duplicate data.
 -     *
 -     *
 -     * @param fs
 -     * @param stagingPath
 -     * @param outputPath
 -     * @param stagingParentPathString
 -     * @throws IOException
 -     */
 -    private void visitPartitionedDirectory(FileSystem fs, Path stagingPath, Path outputPath,
 -                                        String stagingParentPathString,
 -                                        Map<Path, Path> renameDirs, Path oldTableDir) throws IOException {
 -      FileStatus[] files = fs.listStatus(stagingPath);
 -
 -      for(FileStatus eachFile : files) {
 -        if (eachFile.isDirectory()) {
 -          Path oldPath = eachFile.getPath();
 -
 -          // Make recover directory.
 -          String recoverPathString = oldPath.toString().replaceAll(stagingParentPathString,
 -          oldTableDir.toString());
 -          Path recoveryPath = new Path(recoverPathString);
 -          if (!fs.exists(recoveryPath)) {
 -            fs.mkdirs(recoveryPath);
 -          }
 -
 -          visitPartitionedDirectory(fs, eachFile.getPath(), outputPath, stagingParentPathString,
 -          renameDirs, oldTableDir);
 -          // Find last order partition for renaming
 -          String newPathString = oldPath.toString().replaceAll(stagingParentPathString,
 -          outputPath.toString());
 -          Path newPath = new Path(newPathString);
 -          if (!isLeafDirectory(fs, eachFile.getPath())) {
 -           renameDirs.put(eachFile.getPath(), newPath);
 -          } else {
 -            if (!fs.exists(newPath)) {
 -             fs.mkdirs(newPath);
 -            }
 -          }
 -        }
 -      }
 -    }
 -
 -    private boolean isLeafDirectory(FileSystem fs, Path path) throws IOException {
 -      boolean retValue = false;
 -
 -      FileStatus[] files = fs.listStatus(path);
 -      for (FileStatus file : files) {
 -        if (fs.isDirectory(file.getPath())) {
 -          retValue = true;
 -          break;
 -        }
 -      }
 -
 -      return retValue;
 -    }
 -
 -    private boolean verifyAllFileMoved(FileSystem fs, Path stagingPath) throws IOException {
 -      FileStatus[] files = fs.listStatus(stagingPath);
 -      if (files != null && files.length != 0) {
 -        for (FileStatus eachFile: files) {
 -          if (eachFile.isFile()) {
 -            LOG.error("There are some unmoved files in staging dir:" + eachFile.getPath());
 -            return false;
 -          } else {
 -            if (verifyAllFileMoved(fs, eachFile.getPath())) {
 -              fs.delete(eachFile.getPath(), false);
 -            } else {
 -              return false;
 -            }
 -          }
 -        }
 -      }
 -
 -      return true;
 -    }
 -
 -    /**
 -     * Attach the sequence number to a path.
 -     *
 -     * @param path Path
 -     * @param seq sequence number
 -     * @param nf Number format
 -     * @return New path attached with sequence number
 -     * @throws IOException
 -     */
 -    private String replaceFileNameSeq(Path path, int seq, NumberFormat nf) throws IOException {
 -      String[] tokens = path.getName().split("-");
 -      if (tokens.length != 4) {
 -        throw new IOException("Wrong result file name:" + path);
 -      }
 -      return tokens[0] + "-" + tokens[1] + "-" + tokens[2] + "-" + nf.format(seq);
 -    }
 -
 -    /**
 -     * Attach the sequence number to the output file name and than move the file into the final result path.
 -     *
 -     * @param fs FileSystem
 -     * @param stagingResultDir The staging result dir
 -     * @param fileStatus The file status
 -     * @param finalOutputPath Final output path
 -     * @param nf Number format
 -     * @param fileSeq The sequence number
 -     * @throws IOException
 -     */
 -    private void moveResultFromStageToFinal(FileSystem fs, Path stagingResultDir,
 -                                            FileStatus fileStatus, Path finalOutputPath,
 -                                            NumberFormat nf,
 -                                            int fileSeq) throws IOException {
 -      if (fileStatus.isDirectory()) {
 -        String subPath = extractSubPath(stagingResultDir, fileStatus.getPath());
 -        if (subPath != null) {
 -          Path finalSubPath = new Path(finalOutputPath, subPath);
 -          if (!fs.exists(finalSubPath)) {
 -            fs.mkdirs(finalSubPath);
 -          }
 -          int maxSeq = StorageUtil.getMaxFileSequence(fs, finalSubPath, false);
 -          for (FileStatus eachFile : fs.listStatus(fileStatus.getPath())) {
 -            moveResultFromStageToFinal(fs, stagingResultDir, eachFile, finalOutputPath, nf, ++maxSeq);
 -          }
 -        } else {
 -          throw new IOException("Wrong staging dir:" + stagingResultDir + "," + fileStatus.getPath());
 -        }
 -      } else {
 -        String subPath = extractSubPath(stagingResultDir, fileStatus.getPath());
 -        if (subPath != null) {
 -          Path finalSubPath = new Path(finalOutputPath, subPath);
 -          finalSubPath = new Path(finalSubPath.getParent(), replaceFileNameSeq(finalSubPath, fileSeq, nf));
 -          if (!fs.exists(finalSubPath.getParent())) {
 -            fs.mkdirs(finalSubPath.getParent());
 -          }
 -          if (fs.exists(finalSubPath)) {
 -            throw new IOException("Already exists data file:" + finalSubPath);
 -          }
 -          boolean success = fs.rename(fileStatus.getPath(), finalSubPath);
 -          if (success) {
 -            LOG.info("Moving staging file[" + fileStatus.getPath() + "] + " +
 -                "to final output[" + finalSubPath + "]");
 -          } else {
 -            LOG.error("Can't move staging file[" + fileStatus.getPath() + "] + " +
 -                "to final output[" + finalSubPath + "]");
 -          }
 -        }
 -      }
 -    }
 -
 -    private String extractSubPath(Path parentPath, Path childPath) {
 -      String parentPathStr = parentPath.toUri().getPath();
 -      String childPathStr = childPath.toUri().getPath();
 -
 -      if (parentPathStr.length() > childPathStr.length()) {
 -        return null;
 -      }
 -
 -      int index = childPathStr.indexOf(parentPathStr);
 -      if (index != 0) {
 -        return null;
 -      }
 -
 -      return childPathStr.substring(parentPathStr.length() + 1);
 -    }
 -
      private static interface QueryHook {
        boolean isEligible(QueryContext queryContext, Query query, ExecutionBlockId finalExecBlockId, Path finalOutputDir);
        void execute(QueryMaster.QueryMasterContext context, QueryContext queryContext, Query query,

http://git-wip-us.apache.org/repos/asf/tajo/blob/940546a0/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryInfo.java
----------------------------------------------------------------------
diff --cc tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryInfo.java
index 00b95ac,00b95ac..d4e0752
--- a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryInfo.java
+++ b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryInfo.java
@@@ -24,7 -24,7 +24,7 @@@ import org.apache.tajo.QueryId
  import org.apache.tajo.TajoProtos;
  import org.apache.tajo.engine.json.CoreGsonHelper;
  import org.apache.tajo.ipc.ClientProtos.QueryInfoProto;
--import org.apache.tajo.json.GsonObject;
++import org.apache.tajo.storage.json.GsonObject;
  import org.apache.tajo.util.TajoIdUtils;
  import org.apache.tajo.util.history.History;
  

http://git-wip-us.apache.org/repos/asf/tajo/blob/940546a0/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryMasterTask.java
----------------------------------------------------------------------
diff --cc tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryMasterTask.java
index 6b61d04,75d8ab6..1eaef0f
--- a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryMasterTask.java
+++ b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryMasterTask.java
@@@ -62,7 -58,7 +62,8 @@@ import org.apache.tajo.rpc.CallFuture
  import org.apache.tajo.rpc.NettyClientBase;
  import org.apache.tajo.rpc.RpcConnectionPool;
  import org.apache.tajo.storage.StorageManager;
 +import org.apache.tajo.storage.StorageProperty;
+ import org.apache.tajo.storage.StorageUtil;
  import org.apache.tajo.util.HAServiceUtil;
  import org.apache.tajo.util.metrics.TajoMetrics;
  import org.apache.tajo.util.metrics.reporter.MetricsConsoleReporter;
@@@ -476,7 -441,14 +479,20 @@@ public class QueryMasterTask extends Co
      // Create Output Directory
      ////////////////////////////////////////////
  
-     stagingDir = new Path(TajoConf.getStagingDir(conf), queryId);
++    String outputPath = context.get(QueryVars.OUTPUT_TABLE_PATH, "");
+     if (context.isCreateTable() || context.isInsert()) {
 -      stagingDir = StorageUtil.concatPath(context.getOutputPath(), TMP_STAGING_DIR_PREFIX, queryId);
++      if (outputPath == null || outputPath.isEmpty()) {
++        // hbase
++        stagingDir = new Path(TajoConf.getDefaultRootStagingDir(conf), queryId);
++      } else {
++        stagingDir = StorageUtil.concatPath(context.getOutputPath(), TMP_STAGING_DIR_PREFIX, queryId);
++      }
+     } else {
+       stagingDir = new Path(TajoConf.getDefaultRootStagingDir(conf), queryId);
+     }
+ 
+     // initializ
+     fs = stagingDir.getFileSystem(conf);
  
      if (fs.exists(stagingDir)) {
        throw new IOException("The staging directory '" + stagingDir + "' already exists");

http://git-wip-us.apache.org/repos/asf/tajo/blob/940546a0/tajo-core/src/main/java/org/apache/tajo/master/querymaster/SubQuery.java
----------------------------------------------------------------------
diff --cc tajo-core/src/main/java/org/apache/tajo/master/querymaster/SubQuery.java
index 745456a,39bb7ed..7f05fa4
--- a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/SubQuery.java
+++ b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/SubQuery.java
@@@ -57,7 -57,8 +57,9 @@@ import org.apache.tajo.master.TaskRunne
  import org.apache.tajo.master.event.*;
  import org.apache.tajo.master.event.QueryUnitAttemptScheduleEvent.QueryUnitAttemptScheduleContext;
  import org.apache.tajo.master.querymaster.QueryUnit.IntermediateEntry;
+ import org.apache.tajo.master.container.TajoContainer;
+ import org.apache.tajo.master.container.TajoContainerId;
 +import org.apache.tajo.storage.FileStorageManager;
  import org.apache.tajo.plan.util.PlannerUtil;
  import org.apache.tajo.plan.logical.*;
  import org.apache.tajo.storage.StorageManager;

http://git-wip-us.apache.org/repos/asf/tajo/blob/940546a0/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoWorkerResourceManager.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/tajo/blob/940546a0/tajo-core/src/main/java/org/apache/tajo/util/history/QueryHistory.java
----------------------------------------------------------------------
diff --cc tajo-core/src/main/java/org/apache/tajo/util/history/QueryHistory.java
index 7a81b4b,7a81b4b..aaf5754
--- a/tajo-core/src/main/java/org/apache/tajo/util/history/QueryHistory.java
+++ b/tajo-core/src/main/java/org/apache/tajo/util/history/QueryHistory.java
@@@ -22,7 -22,7 +22,7 @@@ import com.google.gson.annotations.Expo
  import org.apache.tajo.engine.json.CoreGsonHelper;
  import org.apache.tajo.ipc.ClientProtos.QueryHistoryProto;
  import org.apache.tajo.ipc.ClientProtos.SubQueryHistoryProto;
--import org.apache.tajo.json.GsonObject;
++import org.apache.tajo.storage.json.GsonObject;
  import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos.KeyValueProto;
  
  import java.util.ArrayList;

http://git-wip-us.apache.org/repos/asf/tajo/blob/940546a0/tajo-core/src/main/java/org/apache/tajo/util/history/QueryUnitHistory.java
----------------------------------------------------------------------
diff --cc tajo-core/src/main/java/org/apache/tajo/util/history/QueryUnitHistory.java
index 556a971,556a971..126e3fe
--- a/tajo-core/src/main/java/org/apache/tajo/util/history/QueryUnitHistory.java
+++ b/tajo-core/src/main/java/org/apache/tajo/util/history/QueryUnitHistory.java
@@@ -20,7 -20,7 +20,7 @@@ package org.apache.tajo.util.history
  
  import com.google.gson.annotations.Expose;
  import org.apache.tajo.engine.json.CoreGsonHelper;
--import org.apache.tajo.json.GsonObject;
++import org.apache.tajo.storage.json.GsonObject;
  
  public class QueryUnitHistory implements GsonObject {
    @Expose private String id;

http://git-wip-us.apache.org/repos/asf/tajo/blob/940546a0/tajo-core/src/main/java/org/apache/tajo/util/history/SubQueryHistory.java
----------------------------------------------------------------------
diff --cc tajo-core/src/main/java/org/apache/tajo/util/history/SubQueryHistory.java
index b3ac4d2,b3ac4d2..17b9ec7
--- a/tajo-core/src/main/java/org/apache/tajo/util/history/SubQueryHistory.java
+++ b/tajo-core/src/main/java/org/apache/tajo/util/history/SubQueryHistory.java
@@@ -22,7 -22,7 +22,7 @@@ import com.google.gson.annotations.Expo
  import com.google.gson.reflect.TypeToken;
  import org.apache.tajo.engine.json.CoreGsonHelper;
  import org.apache.tajo.ipc.ClientProtos.SubQueryHistoryProto;
--import org.apache.tajo.json.GsonObject;
++import org.apache.tajo.storage.json.GsonObject;
  
  import java.util.ArrayList;
  import java.util.List;

http://git-wip-us.apache.org/repos/asf/tajo/blob/940546a0/tajo-core/src/test/java/org/apache/tajo/engine/query/TestHBaseTable.java
----------------------------------------------------------------------
diff --cc tajo-core/src/test/java/org/apache/tajo/engine/query/TestHBaseTable.java
index cb9aa74,0000000..db8eb84
mode 100644,000000..100644
--- a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestHBaseTable.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestHBaseTable.java
@@@ -1,1474 -1,0 +1,1469 @@@
 +/**
 + * Licensed to the Apache Software Foundation (ASF) under one
 + * or more contributor license agreements.  See the NOTICE file
 + * distributed with this work for additional information
 + * regarding copyright ownership.  The ASF licenses this file
 + * to you under the Apache License, Version 2.0 (the
 + * "License"); you may not use this file except in compliance
 + * with the License.  You may obtain a copy of the License at
 + *
 + *     http://www.apache.org/licenses/LICENSE-2.0
 + *
 + * Unless required by applicable law or agreed to in writing, software
 + * distributed under the License is distributed on an "AS IS" BASIS,
 + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 + * See the License for the specific language governing permissions and
 + * limitations under the License.
 + */
 +
 +package org.apache.tajo.engine.query;
 +
 +import org.apache.commons.logging.Log;
 +import org.apache.commons.logging.LogFactory;
 +import org.apache.hadoop.fs.FileStatus;
 +import org.apache.hadoop.fs.FileSystem;
 +import org.apache.hadoop.fs.Path;
 +import org.apache.hadoop.hbase.*;
 +import org.apache.hadoop.hbase.client.*;
 +import org.apache.hadoop.hbase.filter.Filter;
 +import org.apache.hadoop.hbase.filter.InclusiveStopFilter;
 +import org.apache.tajo.IntegrationTest;
 +import org.apache.tajo.QueryTestCaseBase;
 +import org.apache.tajo.TajoTestingCluster;
 +import org.apache.tajo.catalog.Schema;
 +import org.apache.tajo.catalog.TableDesc;
 +import org.apache.tajo.catalog.proto.CatalogProtos.StoreType;
 +import org.apache.tajo.common.TajoDataTypes.Type;
 +import org.apache.tajo.datum.TextDatum;
 +import org.apache.tajo.plan.expr.*;
 +import org.apache.tajo.plan.logical.ScanNode;
 +import org.apache.tajo.storage.StorageConstants;
 +import org.apache.tajo.storage.StorageManager;
 +import org.apache.tajo.storage.fragment.Fragment;
 +import org.apache.tajo.storage.hbase.*;
 +import org.apache.tajo.util.Bytes;
 +import org.apache.tajo.util.KeyValueSet;
 +import org.apache.tajo.util.TUtil;
 +import org.junit.AfterClass;
 +import org.junit.BeforeClass;
 +import org.junit.Test;
 +import org.junit.experimental.categories.Category;
 +
 +import java.net.InetAddress;
 +import java.sql.ResultSet;
 +import java.text.DecimalFormat;
 +import java.util.*;
 +
 +import static org.junit.Assert.*;
 +import static org.junit.Assert.assertEquals;
 +
 +@Category(IntegrationTest.class)
 +public class TestHBaseTable extends QueryTestCaseBase {
 +  private static final Log LOG = LogFactory.getLog(TestHBaseTable.class);
 +
 +  @BeforeClass
 +  public static void beforeClass() {
 +    try {
 +      testingCluster.getHBaseUtil().startHBaseCluster();
 +    } catch (Exception e) {
 +      e.printStackTrace();
 +    }
 +  }
 +
 +  @AfterClass
 +  public static void afterClass() {
 +    try {
 +      testingCluster.getHBaseUtil().stopHBaseCluster();
 +    } catch (Exception e) {
 +      e.printStackTrace();
 +    }
 +  }
 +
 +  @Test
 +  public void testVerifyCreateHBaseTableRequiredMeta() throws Exception {
 +    try {
 +      executeString("CREATE TABLE hbase_mapped_table1 (col1 text, col2 text) " +
 +          "USING hbase").close();
 +
 +      fail("hbase table must have 'table' meta");
 +    } catch (Exception e) {
 +      assertTrue(e.getMessage().indexOf("HBase mapped table") >= 0);
 +    }
 +
 +    try {
 +      executeString("CREATE TABLE hbase_mapped_table1 (col1 text, col2 text) " +
 +          "USING hbase " +
 +          "WITH ('table'='hbase_table')").close();
 +
 +      fail("hbase table must have 'columns' meta");
 +    } catch (Exception e) {
 +      assertTrue(e.getMessage().indexOf("'columns' property is required") >= 0);
 +    }
 +
 +    try {
 +      executeString("CREATE TABLE hbase_mapped_table1 (col1 text, col2 text) " +
 +          "USING hbase " +
 +          "WITH ('table'='hbase_table', 'columns'='col1:,col2:')").close();
 +
 +      fail("hbase table must have 'hbase.zookeeper.quorum' meta");
 +    } catch (Exception e) {
 +      assertTrue(e.getMessage().indexOf("HBase mapped table") >= 0);
 +    }
 +  }
 +
 +  @Test
 +  public void testCreateHBaseTable() throws Exception {
 +    String hostName = InetAddress.getLocalHost().getHostName();
 +    String zkPort = testingCluster.getHBaseUtil().getConf().get(HConstants.ZOOKEEPER_CLIENT_PORT);
 +    assertNotNull(zkPort);
 +
 +    executeString("CREATE TABLE hbase_mapped_table1 (col1 text, col2 text, col3 text, col4 text) " +
 +        "USING hbase WITH ('table'='hbase_table', 'columns'=':key,col2:a,col3:,col2:b', " +
 +        "'" + HConstants.ZOOKEEPER_QUORUM + "'='" + hostName + "'," +
 +        "'" + HConstants.ZOOKEEPER_CLIENT_PORT + "'='" + zkPort + "')").close();
 +
 +    assertTableExists("hbase_mapped_table1");
 +
 +    HTableDescriptor hTableDesc = testingCluster.getHBaseUtil().getTableDescriptor("hbase_table");
 +    assertNotNull(hTableDesc);
 +    assertEquals("hbase_table", hTableDesc.getNameAsString());
 +
 +    HColumnDescriptor[] hColumns = hTableDesc.getColumnFamilies();
 +    // col1 is mapped to rowkey
 +    assertEquals(2, hColumns.length);
 +    assertEquals("col2", hColumns[0].getNameAsString());
 +    assertEquals("col3", hColumns[1].getNameAsString());
 +
 +    executeString("DROP TABLE hbase_mapped_table1 PURGE").close();
 +
 +    HBaseAdmin hAdmin =  new HBaseAdmin(testingCluster.getHBaseUtil().getConf());
 +    try {
 +      assertFalse(hAdmin.tableExists("hbase_table"));
 +    } finally {
 +      hAdmin.close();
 +    }
 +  }
 +
 +  @Test
 +  public void testCreateNotExistsExternalHBaseTable() throws Exception {
 +    String hostName = InetAddress.getLocalHost().getHostName();
 +    String zkPort = testingCluster.getHBaseUtil().getConf().get(HConstants.ZOOKEEPER_CLIENT_PORT);
 +    assertNotNull(zkPort);
 +
 +    try {
 +      executeString("CREATE EXTERNAL TABLE external_hbase_mapped_table1 (col1 text, col2 text, col3 text, col4 text) " +
 +          "USING hbase WITH ('table'='external_hbase_table', 'columns'=':key,col2:a,col3:,col2:b', " +
 +          "'" + HConstants.ZOOKEEPER_QUORUM + "'='" + hostName + "'," +
 +          "'" + HConstants.ZOOKEEPER_CLIENT_PORT + "'='" + zkPort + "')").close();
 +      fail("External table should be a existed table.");
 +    } catch (Exception e) {
 +      assertTrue(e.getMessage().indexOf("External table should be a existed table.") >= 0);
 +    }
 +  }
 +
 +  @Test
 +  public void testCreateRowFieldWithNonText() throws Exception {
 +    String hostName = InetAddress.getLocalHost().getHostName();
 +    String zkPort = testingCluster.getHBaseUtil().getConf().get(HConstants.ZOOKEEPER_CLIENT_PORT);
 +    assertNotNull(zkPort);
 +
 +    try {
 +      executeString("CREATE TABLE hbase_mapped_table2 (rk1 int4, rk2 text, col3 text, col4 text) " +
 +          "USING hbase WITH ('table'='hbase_table', 'columns'='0:key#b,1:key,col3:,col2:b', " +
 +          "'hbase.rowkey.delimiter'='_', " +
 +          "'" + HConstants.ZOOKEEPER_QUORUM + "'='" + hostName + "'," +
 +          "'" + HConstants.ZOOKEEPER_CLIENT_PORT + "'='" + zkPort + "')").close();
 +      fail("Key field type should be TEXT type");
 +    } catch (Exception e) {
 +      assertTrue(e.getMessage().indexOf("Key field type should be TEXT type") >= 0);
 +    }
 +  }
 +
 +  @Test
 +  public void testCreateExternalHBaseTable() throws Exception {
 +    HTableDescriptor hTableDesc = new HTableDescriptor(TableName.valueOf("external_hbase_table_not_purge"));
 +    hTableDesc.addFamily(new HColumnDescriptor("col1"));
 +    hTableDesc.addFamily(new HColumnDescriptor("col2"));
 +    hTableDesc.addFamily(new HColumnDescriptor("col3"));
 +    testingCluster.getHBaseUtil().createTable(hTableDesc);
 +
 +    String hostName = InetAddress.getLocalHost().getHostName();
 +    String zkPort = testingCluster.getHBaseUtil().getConf().get(HConstants.ZOOKEEPER_CLIENT_PORT);
 +    assertNotNull(zkPort);
 +
 +    executeString("CREATE EXTERNAL TABLE external_hbase_mapped_table (rk text, col1 text, col2 text, col3 text) " +
 +        "USING hbase WITH ('table'='external_hbase_table_not_purge', 'columns'=':key,col1:a,col2:,col3:b', " +
 +        "'" + HConstants.ZOOKEEPER_QUORUM + "'='" + hostName + "'," +
 +        "'" + HConstants.ZOOKEEPER_CLIENT_PORT + "'='" + zkPort + "')").close();
 +
 +    assertTableExists("external_hbase_mapped_table");
 +
 +    executeString("DROP TABLE external_hbase_mapped_table").close();
 +
 +    HBaseAdmin hAdmin =  new HBaseAdmin(testingCluster.getHBaseUtil().getConf());
 +    try {
 +      assertTrue(hAdmin.tableExists("external_hbase_table_not_purge"));
 +      hAdmin.disableTable("external_hbase_table_not_purge");
 +      hAdmin.deleteTable("external_hbase_table_not_purge");
 +    } finally {
 +      hAdmin.close();
 +    }
 +  }
 +
 +  @Test
 +  public void testSimpleSelectQuery() throws Exception {
 +    HTableDescriptor hTableDesc = new HTableDescriptor(TableName.valueOf("external_hbase_table"));
 +    hTableDesc.addFamily(new HColumnDescriptor("col1"));
 +    hTableDesc.addFamily(new HColumnDescriptor("col2"));
 +    hTableDesc.addFamily(new HColumnDescriptor("col3"));
 +    testingCluster.getHBaseUtil().createTable(hTableDesc);
 +
 +    String hostName = InetAddress.getLocalHost().getHostName();
 +    String zkPort = testingCluster.getHBaseUtil().getConf().get(HConstants.ZOOKEEPER_CLIENT_PORT);
 +    assertNotNull(zkPort);
 +
 +    executeString("CREATE EXTERNAL TABLE external_hbase_mapped_table (rk text, col1 text, col2 text, col3 text) " +
 +        "USING hbase WITH ('table'='external_hbase_table', 'columns'=':key,col1:a,col2:,col3:b', " +
 +        "'" + HConstants.ZOOKEEPER_QUORUM + "'='" + hostName + "'," +
 +        "'" + HConstants.ZOOKEEPER_CLIENT_PORT + "'='" + zkPort + "')").close();
 +
 +    assertTableExists("external_hbase_mapped_table");
 +
 +    HConnection hconn = ((HBaseStorageManager)StorageManager.getStorageManager(conf, StoreType.HBASE))
 +        .getConnection(testingCluster.getHBaseUtil().getConf());
 +    HTableInterface htable = hconn.getTable("external_hbase_table");
 +
 +    try {
 +      for (int i = 0; i < 100; i++) {
 +        Put put = new Put(String.valueOf(i).getBytes());
 +        put.add("col1".getBytes(), "a".getBytes(), ("a-" + i).getBytes());
 +        put.add("col1".getBytes(), "b".getBytes(), ("b-" + i).getBytes());
 +        put.add("col2".getBytes(), "k1".getBytes(), ("k1-" + i).getBytes());
 +        put.add("col2".getBytes(), "k2".getBytes(), ("k2-" + i).getBytes());
 +        put.add("col3".getBytes(), "b".getBytes(), ("b-" + i).getBytes());
 +        htable.put(put);
 +      }
 +
 +      ResultSet res = executeString("select * from external_hbase_mapped_table where rk > '20'");
 +      assertResultSet(res);
 +      cleanupQuery(res);
 +    } finally {
 +      executeString("DROP TABLE external_hbase_mapped_table PURGE").close();
 +      htable.close();
 +    }
 +  }
 +
 +  @Test
 +  public void testBinaryMappedQuery() throws Exception {
 +    HTableDescriptor hTableDesc = new HTableDescriptor(TableName.valueOf("external_hbase_table"));
 +    hTableDesc.addFamily(new HColumnDescriptor("col1"));
 +    hTableDesc.addFamily(new HColumnDescriptor("col2"));
 +    hTableDesc.addFamily(new HColumnDescriptor("col3"));
 +    testingCluster.getHBaseUtil().createTable(hTableDesc);
 +
 +    String hostName = InetAddress.getLocalHost().getHostName();
 +    String zkPort = testingCluster.getHBaseUtil().getConf().get(HConstants.ZOOKEEPER_CLIENT_PORT);
 +    assertNotNull(zkPort);
 +
 +    executeString("CREATE EXTERNAL TABLE external_hbase_mapped_table (rk int8, col1 text, col2 text, col3 int4)\n " +
 +        "USING hbase WITH ('table'='external_hbase_table', 'columns'=':key#b,col1:a,col2:,col3:b#b', \n" +
 +        "'" + HConstants.ZOOKEEPER_QUORUM + "'='" + hostName + "', \n" +
 +        "'" + HConstants.ZOOKEEPER_CLIENT_PORT + "'='" + zkPort + "')").close();
 +
 +    assertTableExists("external_hbase_mapped_table");
 +
 +    HConnection hconn = ((HBaseStorageManager)StorageManager.getStorageManager(conf, StoreType.HBASE))
 +        .getConnection(testingCluster.getHBaseUtil().getConf());
 +    HTableInterface htable = hconn.getTable("external_hbase_table");
 +
 +    try {
 +      for (int i = 0; i < 100; i++) {
 +        Put put = new Put(Bytes.toBytes((long) i));
 +        put.add("col1".getBytes(), "a".getBytes(), ("a-" + i).getBytes());
 +        put.add("col1".getBytes(), "b".getBytes(), ("b-" + i).getBytes());
 +        put.add("col2".getBytes(), "k1".getBytes(), ("k1-" + i).getBytes());
 +        put.add("col2".getBytes(), "k2".getBytes(), ("k2-" + i).getBytes());
 +        put.add("col3".getBytes(), "b".getBytes(), Bytes.toBytes(i));
 +        htable.put(put);
 +      }
 +
 +      ResultSet res = executeString("select * from external_hbase_mapped_table where rk > 20");
 +      assertResultSet(res);
 +      res.close();
 +
 +      //Projection
 +      res = executeString("select col3, col2, rk from external_hbase_mapped_table where rk > 95");
 +
 +      String expected = "col3,col2,rk\n" +
 +          "-------------------------------\n" +
 +          "96,{\"k1\":\"k1-96\", \"k2\":\"k2-96\"},96\n" +
 +          "97,{\"k1\":\"k1-97\", \"k2\":\"k2-97\"},97\n" +
 +          "98,{\"k1\":\"k1-98\", \"k2\":\"k2-98\"},98\n" +
 +          "99,{\"k1\":\"k1-99\", \"k2\":\"k2-99\"},99\n";
 +
 +      assertEquals(expected, resultSetToString(res));
 +      res.close();
 +
 +    } finally {
 +      executeString("DROP TABLE external_hbase_mapped_table PURGE").close();
 +      htable.close();
 +    }
 +  }
 +
 +  @Test
 +  public void testColumnKeyValueSelectQuery() throws Exception {
 +    HTableDescriptor hTableDesc = new HTableDescriptor(TableName.valueOf("external_hbase_table"));
 +    hTableDesc.addFamily(new HColumnDescriptor("col2"));
 +    hTableDesc.addFamily(new HColumnDescriptor("col3"));
 +    testingCluster.getHBaseUtil().createTable(hTableDesc);
 +
 +    String hostName = InetAddress.getLocalHost().getHostName();
 +    String zkPort = testingCluster.getHBaseUtil().getConf().get(HConstants.ZOOKEEPER_CLIENT_PORT);
 +    assertNotNull(zkPort);
 +
 +    executeString("CREATE EXTERNAL TABLE external_hbase_mapped_table (rk1 text, col2_key text, col2_value text, col3 text) " +
 +        "USING hbase WITH ('table'='external_hbase_table', 'columns'=':key,col2:key:,col2:value:,col3:', " +
 +        "'hbase.rowkey.delimiter'='_', " +
 +        "'" + HConstants.ZOOKEEPER_QUORUM + "'='" + hostName + "'," +
 +        "'" + HConstants.ZOOKEEPER_CLIENT_PORT + "'='" + zkPort + "')").close();
 +
 +    assertTableExists("external_hbase_mapped_table");
 +
 +    HConnection hconn = ((HBaseStorageManager)StorageManager.getStorageManager(conf, StoreType.HBASE))
 +        .getConnection(testingCluster.getHBaseUtil().getConf());
 +    HTableInterface htable = hconn.getTable("external_hbase_table");
 +
 +    try {
 +      for (int i = 0; i < 10; i++) {
 +        Put put = new Put(Bytes.toBytes("rk-" + i));
 +        for (int j = 0; j < 5; j++) {
 +          put.add("col2".getBytes(), ("key-" + j).getBytes(), Bytes.toBytes("value-" + j));
 +        }
 +        put.add("col3".getBytes(), "".getBytes(), ("col3-value-" + i).getBytes());
 +        htable.put(put);
 +      }
 +
 +      ResultSet res = executeString("select * from external_hbase_mapped_table where rk1 >= 'rk-0'");
 +      assertResultSet(res);
 +      cleanupQuery(res);
 +    } finally {
 +      executeString("DROP TABLE external_hbase_mapped_table PURGE").close();
 +      htable.close();
 +    }
 +  }
 +
 +  @Test
 +  public void testRowFieldSelectQuery() throws Exception {
 +    HTableDescriptor hTableDesc = new HTableDescriptor(TableName.valueOf("external_hbase_table"));
 +    hTableDesc.addFamily(new HColumnDescriptor("col3"));
 +    testingCluster.getHBaseUtil().createTable(hTableDesc);
 +
 +    String hostName = InetAddress.getLocalHost().getHostName();
 +    String zkPort = testingCluster.getHBaseUtil().getConf().get(HConstants.ZOOKEEPER_CLIENT_PORT);
 +    assertNotNull(zkPort);
 +
 +    executeString("CREATE EXTERNAL TABLE external_hbase_mapped_table (rk1 text, rk2 text, col3 text) " +
 +        "USING hbase WITH ('table'='external_hbase_table', 'columns'='0:key,1:key,col3:a', " +
 +        "'hbase.rowkey.delimiter'='_', " +
 +        "'" + HConstants.ZOOKEEPER_QUORUM + "'='" + hostName + "'," +
 +        "'" + HConstants.ZOOKEEPER_CLIENT_PORT + "'='" + zkPort + "')").close();
 +
 +    assertTableExists("external_hbase_mapped_table");
 +
 +    HConnection hconn = ((HBaseStorageManager)StorageManager.getStorageManager(conf, StoreType.HBASE))
 +        .getConnection(testingCluster.getHBaseUtil().getConf());
 +    HTableInterface htable = hconn.getTable("external_hbase_table");
 +
 +    try {
 +      for (int i = 0; i < 100; i++) {
 +        Put put = new Put(("field1-" + i + "_field2-" + i).getBytes());
 +        put.add("col3".getBytes(), "a".getBytes(), ("a-" + i).getBytes());
 +        htable.put(put);
 +      }
 +
 +      ResultSet res = executeString("select * from external_hbase_mapped_table where rk1 > 'field1-20'");
 +      assertResultSet(res);
 +      cleanupQuery(res);
 +    } finally {
 +      executeString("DROP TABLE external_hbase_mapped_table PURGE").close();
 +      htable.close();
 +    }
 +  }
 +
 +  @Test
 +  public void testIndexPredication() throws Exception {
 +    String hostName = InetAddress.getLocalHost().getHostName();
 +    String zkPort = testingCluster.getHBaseUtil().getConf().get(HConstants.ZOOKEEPER_CLIENT_PORT);
 +    assertNotNull(zkPort);
 +
 +    executeString("CREATE TABLE hbase_mapped_table (rk text, col1 text, col2 text, col3 text) " +
 +        "USING hbase WITH ('table'='hbase_table', 'columns'=':key,col1:a,col2:,col3:b', " +
 +        "'hbase.split.rowkeys'='010,040,060,080', " +
 +        "'" + HConstants.ZOOKEEPER_QUORUM + "'='" + hostName + "'," +
 +        "'" + HConstants.ZOOKEEPER_CLIENT_PORT + "'='" + zkPort + "')").close();
 +
 +
 +    assertTableExists("hbase_mapped_table");
 +    HBaseAdmin hAdmin = new HBaseAdmin(testingCluster.getHBaseUtil().getConf());
 +    hAdmin.tableExists("hbase_table");
 +
 +    HTable htable = new HTable(testingCluster.getHBaseUtil().getConf(), "hbase_table");
 +    try {
 +      org.apache.hadoop.hbase.util.Pair<byte[][], byte[][]> keys = htable.getStartEndKeys();
 +      assertEquals(5, keys.getFirst().length);
 +
 +      DecimalFormat df = new DecimalFormat("000");
 +      for (int i = 0; i < 100; i++) {
 +        Put put = new Put(String.valueOf(df.format(i)).getBytes());
 +        put.add("col1".getBytes(), "a".getBytes(), ("a-" + i).getBytes());
 +        put.add("col1".getBytes(), "b".getBytes(), ("b-" + i).getBytes());
 +        put.add("col2".getBytes(), "k1".getBytes(), ("k1-" + i).getBytes());
 +        put.add("col2".getBytes(), "k2".getBytes(), ("k2-" + i).getBytes());
 +        put.add("col3".getBytes(), "b".getBytes(), ("b-" + i).getBytes());
 +        htable.put(put);
 +      }
 +      assertIndexPredication(false);
 +
 +      ResultSet res = executeString("select * from hbase_mapped_table where rk >= '020' and rk <= '055'");
 +      assertResultSet(res);
 +      res.close();
 +
 +      res = executeString("select * from hbase_mapped_table where rk = '021'");
 +      String expected = "rk,col1,col2,col3\n" +
 +          "-------------------------------\n" +
 +          "021,a-21,{\"k1\":\"k1-21\", \"k2\":\"k2-21\"},b-21\n";
 +
 +      assertEquals(expected, resultSetToString(res));
 +      res.close();
 +    } finally {
 +      executeString("DROP TABLE hbase_mapped_table PURGE").close();
 +      htable.close();
 +      hAdmin.close();
 +    }
 +  }
 +
 +  @Test
 +  public void testCompositeRowIndexPredication() throws Exception {
 +    String hostName = InetAddress.getLocalHost().getHostName();
 +    String zkPort = testingCluster.getHBaseUtil().getConf().get(HConstants.ZOOKEEPER_CLIENT_PORT);
 +    assertNotNull(zkPort);
 +
 +    executeString("CREATE TABLE hbase_mapped_table (rk text, rk2 text, col1 text, col2 text, col3 text) " +
 +        "USING hbase WITH ('table'='hbase_table', 'columns'='0:key,1:key,col1:a,col2:,col3:b', " +
 +        "'hbase.split.rowkeys'='010,040,060,080', " +
 +        "'hbase.rowkey.delimiter'='_', " +
 +        "'" + HConstants.ZOOKEEPER_QUORUM + "'='" + hostName + "'," +
 +        "'" + HConstants.ZOOKEEPER_CLIENT_PORT + "'='" + zkPort + "')").close();
 +
 +
 +    assertTableExists("hbase_mapped_table");
 +    HBaseAdmin hAdmin = new HBaseAdmin(testingCluster.getHBaseUtil().getConf());
 +    hAdmin.tableExists("hbase_table");
 +
 +    HTable htable = new HTable(testingCluster.getHBaseUtil().getConf(), "hbase_table");
 +    try {
 +      org.apache.hadoop.hbase.util.Pair<byte[][], byte[][]> keys = htable.getStartEndKeys();
 +      assertEquals(5, keys.getFirst().length);
 +
 +      DecimalFormat df = new DecimalFormat("000");
 +      for (int i = 0; i < 100; i++) {
 +        Put put = new Put((df.format(i) + "_" + df.format(i)).getBytes());
 +        put.add("col1".getBytes(), "a".getBytes(), ("a-" + i).getBytes());
 +        put.add("col1".getBytes(), "b".getBytes(), ("b-" + i).getBytes());
 +        put.add("col2".getBytes(), "k1".getBytes(), ("k1-" + i).getBytes());
 +        put.add("col2".getBytes(), "k2".getBytes(), ("k2-" + i).getBytes());
 +        put.add("col3".getBytes(), "b".getBytes(), ("b-" + i).getBytes());
 +        htable.put(put);
 +      }
 +
 +      Scan scan = new Scan();
 +      scan.setStartRow("021".getBytes());
 +      scan.setStopRow(("021_" + new String(new char[]{Character.MAX_VALUE})).getBytes());
 +      Filter filter = new InclusiveStopFilter(scan.getStopRow());
 +      scan.setFilter(filter);
 +
 +      ResultScanner scanner = htable.getScanner(scan);
 +      Result result = scanner.next();
 +      assertNotNull(result);
 +      assertEquals("021_021", new String(result.getRow()));
 +      scanner.close();
 +
 +      assertIndexPredication(true);
 +
 +      ResultSet res = executeString("select * from hbase_mapped_table where rk = '021'");
 +      String expected = "rk,rk2,col1,col2,col3\n" +
 +          "-------------------------------\n" +
 +          "021,021,a-21,{\"k1\":\"k1-21\", \"k2\":\"k2-21\"},b-21\n";
 +
 +      assertEquals(expected, resultSetToString(res));
 +      res.close();
 +    } finally {
 +      executeString("DROP TABLE hbase_mapped_table PURGE").close();
 +      htable.close();
 +      hAdmin.close();
 +    }
 +  }
 +
 +  private void assertIndexPredication(boolean isCompositeRowKey) throws Exception {
 +    String postFix = isCompositeRowKey ? "_" + new String(new char[]{Character.MAX_VALUE}) : "";
 +    TableDesc tableDesc = catalog.getTableDesc(getCurrentDatabase(), "hbase_mapped_table");
 +
 +    ScanNode scanNode = new ScanNode(1);
 +
 +    // where rk = '021'
 +    EvalNode evalNodeEq = new BinaryEval(EvalType.EQUAL, new FieldEval(tableDesc.getLogicalSchema().getColumn("rk")),
 +        new ConstEval(new TextDatum("021")));
 +    scanNode.setQual(evalNodeEq);
 +    StorageManager storageManager = StorageManager.getStorageManager(conf, StoreType.HBASE);
 +    List<Fragment> fragments = storageManager.getSplits("hbase_mapped_table", tableDesc, scanNode);
 +    assertEquals(1, fragments.size());
 +    assertEquals("021", new String(((HBaseFragment)fragments.get(0)).getStartRow()));
 +    assertEquals("021" + postFix, new String(((HBaseFragment)fragments.get(0)).getStopRow()));
 +
 +    // where rk >= '020' and rk <= '055'
 +    EvalNode evalNode1 = new BinaryEval(EvalType.GEQ, new FieldEval(tableDesc.getLogicalSchema().getColumn("rk")),
 +        new ConstEval(new TextDatum("020")));
 +    EvalNode evalNode2 = new BinaryEval(EvalType.LEQ, new FieldEval(tableDesc.getLogicalSchema().getColumn("rk")),
 +        new ConstEval(new TextDatum("055")));
 +    EvalNode evalNodeA = new BinaryEval(EvalType.AND, evalNode1, evalNode2);
 +    scanNode.setQual(evalNodeA);
 +
 +    fragments = storageManager.getSplits("hbase_mapped_table", tableDesc, scanNode);
 +    assertEquals(2, fragments.size());
 +    HBaseFragment fragment1 = (HBaseFragment) fragments.get(0);
 +    assertEquals("020", new String(fragment1.getStartRow()));
 +    assertEquals("040", new String(fragment1.getStopRow()));
 +
 +    HBaseFragment fragment2 = (HBaseFragment) fragments.get(1);
 +    assertEquals("040", new String(fragment2.getStartRow()));
 +    assertEquals("055" + postFix, new String(fragment2.getStopRow()));
 +
 +    // where (rk >= '020' and rk <= '055') or rk = '075'
 +    EvalNode evalNode3 = new BinaryEval(EvalType.EQUAL, new FieldEval(tableDesc.getLogicalSchema().getColumn("rk")),
 +        new ConstEval(new TextDatum("075")));
 +    EvalNode evalNodeB = new BinaryEval(EvalType.OR, evalNodeA, evalNode3);
 +    scanNode.setQual(evalNodeB);
 +    fragments = storageManager.getSplits("hbase_mapped_table", tableDesc, scanNode);
 +    assertEquals(3, fragments.size());
 +    fragment1 = (HBaseFragment) fragments.get(0);
 +    assertEquals("020", new String(fragment1.getStartRow()));
 +    assertEquals("040", new String(fragment1.getStopRow()));
 +
 +    fragment2 = (HBaseFragment) fragments.get(1);
 +    assertEquals("040", new String(fragment2.getStartRow()));
 +    assertEquals("055" + postFix, new String(fragment2.getStopRow()));
 +
 +    HBaseFragment fragment3 = (HBaseFragment) fragments.get(2);
 +    assertEquals("075", new String(fragment3.getStartRow()));
 +    assertEquals("075" + postFix, new String(fragment3.getStopRow()));
 +
 +
 +    // where (rk >= '020' and rk <= '055') or (rk >= '072' and rk <= '078')
 +    EvalNode evalNode4 = new BinaryEval(EvalType.GEQ, new FieldEval(tableDesc.getLogicalSchema().getColumn("rk")),
 +        new ConstEval(new TextDatum("072")));
 +    EvalNode evalNode5 = new BinaryEval(EvalType.LEQ, new FieldEval(tableDesc.getLogicalSchema().getColumn("rk")),
 +        new ConstEval(new TextDatum("078")));
 +    EvalNode evalNodeC = new BinaryEval(EvalType.AND, evalNode4, evalNode5);
 +    EvalNode evalNodeD = new BinaryEval(EvalType.OR, evalNodeA, evalNodeC);
 +    scanNode.setQual(evalNodeD);
 +    fragments = storageManager.getSplits("hbase_mapped_table", tableDesc, scanNode);
 +    assertEquals(3, fragments.size());
 +
 +    fragment1 = (HBaseFragment) fragments.get(0);
 +    assertEquals("020", new String(fragment1.getStartRow()));
 +    assertEquals("040", new String(fragment1.getStopRow()));
 +
 +    fragment2 = (HBaseFragment) fragments.get(1);
 +    assertEquals("040", new String(fragment2.getStartRow()));
 +    assertEquals("055" + postFix, new String(fragment2.getStopRow()));
 +
 +    fragment3 = (HBaseFragment) fragments.get(2);
 +    assertEquals("072", new String(fragment3.getStartRow()));
 +    assertEquals("078" + postFix, new String(fragment3.getStopRow()));
 +
 +    // where (rk >= '020' and rk <= '055') or (rk >= '057' and rk <= '059')
 +    evalNode4 = new BinaryEval(EvalType.GEQ, new FieldEval(tableDesc.getLogicalSchema().getColumn("rk")),
 +        new ConstEval(new TextDatum("057")));
 +    evalNode5 = new BinaryEval(EvalType.LEQ, new FieldEval(tableDesc.getLogicalSchema().getColumn("rk")),
 +        new ConstEval(new TextDatum("059")));
 +    evalNodeC = new BinaryEval(EvalType.AND, evalNode4, evalNode5);
 +    evalNodeD = new BinaryEval(EvalType.OR, evalNodeA, evalNodeC);
 +    scanNode.setQual(evalNodeD);
 +    fragments = storageManager.getSplits("hbase_mapped_table", tableDesc, scanNode);
 +    assertEquals(2, fragments.size());
 +
 +    fragment1 = (HBaseFragment) fragments.get(0);
 +    assertEquals("020", new String(fragment1.getStartRow()));
 +    assertEquals("040", new String(fragment1.getStopRow()));
 +
 +    fragment2 = (HBaseFragment) fragments.get(1);
 +    assertEquals("040", new String(fragment2.getStartRow()));
 +    assertEquals("059" + postFix, new String(fragment2.getStopRow()));
 +  }
 +
 +  @Test
 +  public void testNonForwardQuery() throws Exception {
 +    String hostName = InetAddress.getLocalHost().getHostName();
 +    String zkPort = testingCluster.getHBaseUtil().getConf().get(HConstants.ZOOKEEPER_CLIENT_PORT);
 +    assertNotNull(zkPort);
 +
 +    executeString("CREATE TABLE hbase_mapped_table (rk text, col1 text, col2 text, col3 int) " +
 +        "USING hbase WITH ('table'='hbase_table', 'columns'=':key,col1:a,col2:,col3:#b', " +
 +        "'hbase.split.rowkeys'='010,040,060,080', " +
 +        "'" + HConstants.ZOOKEEPER_QUORUM + "'='" + hostName + "'," +
 +        "'" + HConstants.ZOOKEEPER_CLIENT_PORT + "'='" + zkPort + "')").close();
 +
 +
 +    assertTableExists("hbase_mapped_table");
 +    HBaseAdmin hAdmin =  new HBaseAdmin(testingCluster.getHBaseUtil().getConf());
 +    HTable htable = null;
 +    try {
 +      hAdmin.tableExists("hbase_table");
 +      htable = new HTable(testingCluster.getHBaseUtil().getConf(), "hbase_table");
 +      org.apache.hadoop.hbase.util.Pair<byte[][], byte[][]> keys = htable.getStartEndKeys();
 +      assertEquals(5, keys.getFirst().length);
 +
 +      DecimalFormat df = new DecimalFormat("000");
 +      for (int i = 0; i < 100; i++) {
 +        Put put = new Put(String.valueOf(df.format(i)).getBytes());
 +        put.add("col1".getBytes(), "a".getBytes(), ("a-" + i).getBytes());
 +        put.add("col1".getBytes(), "b".getBytes(), ("b-" + i).getBytes());
 +        put.add("col2".getBytes(), "k1".getBytes(), ("k1-" + i).getBytes());
 +        put.add("col2".getBytes(), "k2".getBytes(), ("k2-" + i).getBytes());
 +        put.add("col3".getBytes(), "".getBytes(), Bytes.toBytes(i));
 +        htable.put(put);
 +      }
 +
 +      ResultSet res = executeString("select * from hbase_mapped_table");
 +      assertResultSet(res);
 +      res.close();
 +    } finally {
 +      executeString("DROP TABLE hbase_mapped_table PURGE").close();
 +      hAdmin.close();
 +      if (htable == null) {
 +        htable.close();
 +      }
 +    }
 +  }
 +
 +  @Test
 +  public void testJoin() throws Exception {
 +    String hostName = InetAddress.getLocalHost().getHostName();
 +    String zkPort = testingCluster.getHBaseUtil().getConf().get(HConstants.ZOOKEEPER_CLIENT_PORT);
 +    assertNotNull(zkPort);
 +
 +    executeString("CREATE TABLE hbase_mapped_table (rk text, col1 text, col2 text, col3 int8) " +
 +        "USING hbase WITH ('table'='hbase_table', 'columns'=':key,col1:a,col2:,col3:b#b', " +
 +        "'hbase.split.rowkeys'='010,040,060,080', " +
 +        "'" + HConstants.ZOOKEEPER_QUORUM + "'='" + hostName + "'," +
 +        "'" + HConstants.ZOOKEEPER_CLIENT_PORT + "'='" + zkPort + "')").close();
 +
 +
 +    assertTableExists("hbase_mapped_table");
 +    HBaseAdmin hAdmin =  new HBaseAdmin(testingCluster.getHBaseUtil().getConf());
 +    HTable htable = null;
 +    try {
 +      hAdmin.tableExists("hbase_table");
 +      htable = new HTable(testingCluster.getHBaseUtil().getConf(), "hbase_table");
 +      org.apache.hadoop.hbase.util.Pair<byte[][], byte[][]> keys = htable.getStartEndKeys();
 +      assertEquals(5, keys.getFirst().length);
 +
 +      DecimalFormat df = new DecimalFormat("000");
 +      for (int i = 0; i < 100; i++) {
 +        Put put = new Put(String.valueOf(df.format(i)).getBytes());
 +        put.add("col1".getBytes(), "a".getBytes(), ("a-" + i).getBytes());
 +        put.add("col1".getBytes(), "b".getBytes(), ("b-" + i).getBytes());
 +        put.add("col2".getBytes(), "k1".getBytes(), ("k1-" + i).getBytes());
 +        put.add("col2".getBytes(), "k2".getBytes(), ("k2-" + i).getBytes());
 +        put.add("col3".getBytes(), "b".getBytes(), Bytes.toBytes((long) i));
 +        htable.put(put);
 +      }
 +
 +      ResultSet res = executeString("select a.rk, a.col1, a.col2, a.col3, b.l_orderkey, b.l_linestatus " +
 +          "from hbase_mapped_table a " +
 +          "join default.lineitem b on a.col3 = b.l_orderkey");
 +      assertResultSet(res);
 +      res.close();
 +    } finally {
 +      executeString("DROP TABLE hbase_mapped_table PURGE").close();
 +      hAdmin.close();
 +      if (htable != null) {
 +        htable.close();
 +      }
 +    }
 +  }
 +
 +  @Test
 +  public void testInsertInto() throws Exception {
 +    String hostName = InetAddress.getLocalHost().getHostName();
 +    String zkPort = testingCluster.getHBaseUtil().getConf().get(HConstants.ZOOKEEPER_CLIENT_PORT);
 +    assertNotNull(zkPort);
 +
 +    executeString("CREATE TABLE hbase_mapped_table (rk text, col1 text, col2 text, col3 int4) " +
 +        "USING hbase WITH ('table'='hbase_table', 'columns'=':key,col1:a,col2:,col3:b#b', " +
 +        "'" + HConstants.ZOOKEEPER_QUORUM + "'='" + hostName + "'," +
 +        "'" + HConstants.ZOOKEEPER_CLIENT_PORT + "'='" + zkPort + "')").close();
 +
 +    assertTableExists("hbase_mapped_table");
 +    TableDesc tableDesc = catalog.getTableDesc(getCurrentDatabase(), "hbase_mapped_table");
 +
 +    executeString("insert into hbase_mapped_table " +
 +        "select l_orderkey::text, l_shipdate, l_returnflag, l_suppkey from default.lineitem ").close();
 +
 +    HTable htable = null;
 +    ResultScanner scanner = null;
 +    try {
 +      htable = new HTable(testingCluster.getHBaseUtil().getConf(), "hbase_table");
 +
 +      Scan scan = new Scan();
 +      scan.addFamily(Bytes.toBytes("col1"));
 +      scan.addFamily(Bytes.toBytes("col2"));
 +      scan.addFamily(Bytes.toBytes("col3"));
 +      scanner = htable.getScanner(scan);
 +
 +      assertStrings(resultSetToString(scanner,
 +          new byte[][]{null, Bytes.toBytes("col1"), Bytes.toBytes("col2"), Bytes.toBytes("col3")},
 +          new byte[][]{null, Bytes.toBytes("a"), null, Bytes.toBytes("b")},
 +          new boolean[]{false, false, false, true}, tableDesc.getSchema()));
 +
 +    } finally {
 +      executeString("DROP TABLE hbase_mapped_table PURGE").close();
 +
 +      if (scanner != null) {
 +        scanner.close();
 +      }
 +
 +      if (htable != null) {
 +        htable.close();
 +      }
 +    }
 +  }
 +
 +  @Test
 +  public void testInsertIntoMultiRegion() throws Exception {
 +    String hostName = InetAddress.getLocalHost().getHostName();
 +    String zkPort = testingCluster.getHBaseUtil().getConf().get(HConstants.ZOOKEEPER_CLIENT_PORT);
 +    assertNotNull(zkPort);
 +
 +    executeString("CREATE TABLE hbase_mapped_table (rk text, col1 text) " +
 +        "USING hbase WITH ('table'='hbase_table', 'columns'=':key,col1:a', " +
 +        "'hbase.split.rowkeys'='010,040,060,080', " +
 +        "'" + HConstants.ZOOKEEPER_QUORUM + "'='" + hostName + "'," +
 +        "'" + HConstants.ZOOKEEPER_CLIENT_PORT + "'='" + zkPort + "')").close();
 +
 +    assertTableExists("hbase_mapped_table");
 +    TableDesc tableDesc = catalog.getTableDesc(getCurrentDatabase(), "hbase_mapped_table");
 +
 +    // create test table
 +    KeyValueSet tableOptions = new KeyValueSet();
 +    tableOptions.set(StorageConstants.CSVFILE_DELIMITER, StorageConstants.DEFAULT_FIELD_DELIMITER);
 +    tableOptions.set(StorageConstants.CSVFILE_NULL, "\\\\N");
 +
 +    Schema schema = new Schema();
 +    schema.addColumn("id", Type.TEXT);
 +    schema.addColumn("name", Type.TEXT);
 +    List<String> datas = new ArrayList<String>();
 +    DecimalFormat df = new DecimalFormat("000");
 +    for (int i = 99; i >= 0; i--) {
 +      datas.add(df.format(i) + "|value" + i);
 +    }
 +    TajoTestingCluster.createTable(getCurrentDatabase() + ".base_table",
 +        schema, tableOptions, datas.toArray(new String[]{}), 2);
 +
 +    executeString("insert into hbase_mapped_table " +
 +        "select id, name from base_table ").close();
 +
 +    HTable htable = null;
 +    ResultScanner scanner = null;
 +    try {
 +      htable = new HTable(testingCluster.getHBaseUtil().getConf(), "hbase_table");
 +
 +      Scan scan = new Scan();
 +      scan.addFamily(Bytes.toBytes("col1"));
 +      scanner = htable.getScanner(scan);
 +
 +      assertStrings(resultSetToString(scanner,
 +          new byte[][]{null, Bytes.toBytes("col1")},
 +          new byte[][]{null, Bytes.toBytes("a")},
 +          new boolean[]{false, false}, tableDesc.getSchema()));
 +
 +    } finally {
 +      executeString("DROP TABLE base_table PURGE").close();
 +      executeString("DROP TABLE hbase_mapped_table PURGE").close();
 +
 +      if (scanner != null) {
 +        scanner.close();
 +      }
 +
 +      if (htable != null) {
 +        htable.close();
 +      }
 +    }
 +  }
 +
 +  @Test
 +  public void testInsertIntoMultiRegion2() throws Exception {
 +    String hostName = InetAddress.getLocalHost().getHostName();
 +    String zkPort = testingCluster.getHBaseUtil().getConf().get(HConstants.ZOOKEEPER_CLIENT_PORT);
 +    assertNotNull(zkPort);
 +
 +    executeString("CREATE TABLE hbase_mapped_table (rk text, col1 text) " +
 +        "USING hbase WITH ('table'='hbase_table', 'columns'=':key,col1:a', " +
 +        "'hbase.split.rowkeys'='1,2,3,4,5,6,7,8,9', " +
 +        "'" + HConstants.ZOOKEEPER_QUORUM + "'='" + hostName + "'," +
 +        "'" + HConstants.ZOOKEEPER_CLIENT_PORT + "'='" + zkPort + "')").close();
 +
 +    assertTableExists("hbase_mapped_table");
 +    TableDesc tableDesc = catalog.getTableDesc(getCurrentDatabase(), "hbase_mapped_table");
 +
 +    // create test table
 +    KeyValueSet tableOptions = new KeyValueSet();
 +    tableOptions.set(StorageConstants.CSVFILE_DELIMITER, StorageConstants.DEFAULT_FIELD_DELIMITER);
 +    tableOptions.set(StorageConstants.CSVFILE_NULL, "\\\\N");
 +
 +    Schema schema = new Schema();
 +    schema.addColumn("id", Type.TEXT);
 +    schema.addColumn("name", Type.TEXT);
 +    List<String> datas = new ArrayList<String>();
 +    for (int i = 99; i >= 0; i--) {
 +      datas.add(i + "|value" + i);
 +    }
 +    TajoTestingCluster.createTable(getCurrentDatabase() + ".base_table",
 +        schema, tableOptions, datas.toArray(new String[]{}), 2);
 +
 +    executeString("insert into hbase_mapped_table " +
 +        "select id, name from base_table ").close();
 +
 +    HTable htable = null;
 +    ResultScanner scanner = null;
 +    try {
 +      htable = new HTable(testingCluster.getHBaseUtil().getConf(), "hbase_table");
 +
 +      Scan scan = new Scan();
 +      scan.addFamily(Bytes.toBytes("col1"));
 +      scanner = htable.getScanner(scan);
 +
 +      assertStrings(resultSetToString(scanner,
 +          new byte[][]{null, Bytes.toBytes("col1")},
 +          new byte[][]{null, Bytes.toBytes("a")},
 +          new boolean[]{false, false}, tableDesc.getSchema()));
 +
 +    } finally {
 +      executeString("DROP TABLE base_table PURGE").close();
 +      executeString("DROP TABLE hbase_mapped_table PURGE").close();
 +
 +      if (scanner != null) {
 +        scanner.close();
 +      }
 +
 +      if (htable != null) {
 +        htable.close();
 +      }
 +    }
 +  }
 +
 +  @Test
 +  public void testInsertIntoMultiRegionWithSplitFile() throws Exception {
 +    String hostName = InetAddress.getLocalHost().getHostName();
 +    String zkPort = testingCluster.getHBaseUtil().getConf().get(HConstants.ZOOKEEPER_CLIENT_PORT);
 +    assertNotNull(zkPort);
 +
 +    String splitFilePath = currentDatasetPath + "/splits.data";
 +
 +    executeString("CREATE TABLE hbase_mapped_table (rk text, col1 text) " +
 +        "USING hbase WITH ('table'='hbase_table', 'columns'=':key,col1:a', " +
 +        "'hbase.split.rowkeys.file'='" + splitFilePath + "', " +
 +        "'" + HConstants.ZOOKEEPER_QUORUM + "'='" + hostName + "'," +
 +        "'" + HConstants.ZOOKEEPER_CLIENT_PORT + "'='" + zkPort + "')").close();
 +
 +    assertTableExists("hbase_mapped_table");
 +    TableDesc tableDesc = catalog.getTableDesc(getCurrentDatabase(), "hbase_mapped_table");
 +
 +    // create test table
 +    KeyValueSet tableOptions = new KeyValueSet();
 +    tableOptions.set(StorageConstants.CSVFILE_DELIMITER, StorageConstants.DEFAULT_FIELD_DELIMITER);
 +    tableOptions.set(StorageConstants.CSVFILE_NULL, "\\\\N");
 +
 +    Schema schema = new Schema();
 +    schema.addColumn("id", Type.TEXT);
 +    schema.addColumn("name", Type.TEXT);
 +    List<String> datas = new ArrayList<String>();
 +    DecimalFormat df = new DecimalFormat("000");
 +    for (int i = 99; i >= 0; i--) {
 +      datas.add(df.format(i) + "|value" + i);
 +    }
 +    TajoTestingCluster.createTable(getCurrentDatabase() + ".base_table",
 +        schema, tableOptions, datas.toArray(new String[]{}), 2);
 +
 +    executeString("insert into hbase_mapped_table " +
 +        "select id, name from base_table ").close();
 +
 +    HTable htable = null;
 +    ResultScanner scanner = null;
 +    try {
 +      htable = new HTable(testingCluster.getHBaseUtil().getConf(), "hbase_table");
 +
 +      Scan scan = new Scan();
 +      scan.addFamily(Bytes.toBytes("col1"));
 +      scanner = htable.getScanner(scan);
 +
 +      assertStrings(resultSetToString(scanner,
 +          new byte[][]{null, Bytes.toBytes("col1")},
 +          new byte[][]{null, Bytes.toBytes("a")},
 +          new boolean[]{false, false}, tableDesc.getSchema()));
 +
 +    } finally {
 +      executeString("DROP TABLE base_table PURGE").close();
 +      executeString("DROP TABLE hbase_mapped_table PURGE").close();
 +
 +      if (scanner != null) {
 +        scanner.close();
 +      }
 +
 +      if (htable != null) {
 +        htable.close();
 +      }
 +    }
 +  }
 +
 +  @Test
 +  public void testInsertIntoMultiRegionMultiRowFields() throws Exception {
 +    String hostName = InetAddress.getLocalHost().getHostName();
 +    String zkPort = testingCluster.getHBaseUtil().getConf().get(HConstants.ZOOKEEPER_CLIENT_PORT);
 +    assertNotNull(zkPort);
 +
 +    executeString("CREATE TABLE hbase_mapped_table (rk1 text, rk2 text, col1 text) " +
 +        "USING hbase WITH ('table'='hbase_table', 'columns'='0:key,1:key,col1:a', " +
 +        "'hbase.split.rowkeys'='001,002,003,004,005,006,007,008,009', " +
 +        "'hbase.rowkey.delimiter'='_', " +
 +        "'" + HConstants.ZOOKEEPER_QUORUM + "'='" + hostName + "'," +
 +        "'" + HConstants.ZOOKEEPER_CLIENT_PORT + "'='" + zkPort + "')").close();
 +
 +    assertTableExists("hbase_mapped_table");
 +    TableDesc tableDesc = catalog.getTableDesc(getCurrentDatabase(), "hbase_mapped_table");
 +
 +    // create test table
 +    KeyValueSet tableOptions = new KeyValueSet();
 +    tableOptions.set(StorageConstants.CSVFILE_DELIMITER, StorageConstants.DEFAULT_FIELD_DELIMITER);
 +    tableOptions.set(StorageConstants.CSVFILE_NULL, "\\\\N");
 +
 +    Schema schema = new Schema();
 +    schema.addColumn("id1", Type.TEXT);
 +    schema.addColumn("id2", Type.TEXT);
 +    schema.addColumn("name", Type.TEXT);
 +    DecimalFormat df = new DecimalFormat("000");
 +    List<String> datas = new ArrayList<String>();
 +    for (int i = 99; i >= 0; i--) {
 +      datas.add(df.format(i) + "|" + (i + 100) + "|value" + i);
 +    }
 +    TajoTestingCluster.createTable(getCurrentDatabase() + ".base_table",
 +        schema, tableOptions, datas.toArray(new String[]{}), 2);
 +
 +    executeString("insert into hbase_mapped_table " +
 +        "select id1, id2, name from base_table ").close();
 +
 +    HTable htable = null;
 +    ResultScanner scanner = null;
 +    try {
 +      htable = new HTable(testingCluster.getHBaseUtil().getConf(), "hbase_table");
 +
 +      Scan scan = new Scan();
 +      scan.addFamily(Bytes.toBytes("col1"));
 +      scanner = htable.getScanner(scan);
 +
 +      assertStrings(resultSetToString(scanner,
 +          new byte[][]{null, null, Bytes.toBytes("col1")},
 +          new byte[][]{null, null, Bytes.toBytes("a")},
 +          new boolean[]{false, false, false}, tableDesc.getSchema()));
 +
 +    } finally {
 +      executeString("DROP TABLE base_table PURGE").close();
 +      executeString("DROP TABLE hbase_mapped_table PURGE").close();
 +
 +      if (scanner != null) {
 +        scanner.close();
 +      }
 +
 +      if (htable != null) {
 +        htable.close();
 +      }
 +    }
 +  }
 +
 +  @Test
 +  public void testInsertIntoBinaryMultiRegion() throws Exception {
 +    String hostName = InetAddress.getLocalHost().getHostName();
 +    String zkPort = testingCluster.getHBaseUtil().getConf().get(HConstants.ZOOKEEPER_CLIENT_PORT);
 +    assertNotNull(zkPort);
 +
 +    executeString("CREATE TABLE hbase_mapped_table (rk int4, col1 text) " +
 +        "USING hbase WITH ('table'='hbase_table', 'columns'=':key#b,col1:a', " +
 +        "'hbase.split.rowkeys'='1,2,3,4,5,6,7,8,9', " +
 +        "'" + HConstants.ZOOKEEPER_QUORUM + "'='" + hostName + "'," +
 +        "'" + HConstants.ZOOKEEPER_CLIENT_PORT + "'='" + zkPort + "')").close();
 +
 +    assertTableExists("hbase_mapped_table");
 +    TableDesc tableDesc = catalog.getTableDesc(getCurrentDatabase(), "hbase_mapped_table");
 +
 +    // create test table
 +    KeyValueSet tableOptions = new KeyValueSet();
 +    tableOptions.set(StorageConstants.CSVFILE_DELIMITER, StorageConstants.DEFAULT_FIELD_DELIMITER);
 +    tableOptions.set(StorageConstants.CSVFILE_NULL, "\\\\N");
 +
 +    Schema schema = new Schema();
 +    schema.addColumn("id", Type.INT4);
 +    schema.addColumn("name", Type.TEXT);
 +    List<String> datas = new ArrayList<String>();
 +    for (int i = 99; i >= 0; i--) {
 +      datas.add(i + "|value" + i);
 +    }
 +    TajoTestingCluster.createTable(getCurrentDatabase() + ".base_table",
 +        schema, tableOptions, datas.toArray(new String[]{}), 2);
 +
 +    executeString("insert into hbase_mapped_table " +
 +        "select id, name from base_table ").close();
 +
 +    HTable htable = null;
 +    ResultScanner scanner = null;
 +    try {
 +      htable = new HTable(testingCluster.getHBaseUtil().getConf(), "hbase_table");
 +
 +      Scan scan = new Scan();
 +      scan.addFamily(Bytes.toBytes("col1"));
 +      scanner = htable.getScanner(scan);
 +
 +      assertStrings(resultSetToString(scanner,
 +          new byte[][]{null, Bytes.toBytes("col1")},
 +          new byte[][]{null, Bytes.toBytes("a")},
 +          new boolean[]{true, false}, tableDesc.getSchema()));
 +
 +    } finally {
 +      executeString("DROP TABLE base_table PURGE").close();
 +      executeString("DROP TABLE hbase_mapped_table PURGE").close();
 +
 +      if (scanner != null) {
 +        scanner.close();
 +      }
 +
 +      if (htable != null) {
 +        htable.close();
 +      }
 +    }
 +  }
 +
 +  @Test
 +  public void testInsertIntoColumnKeyValue() throws Exception {
 +    String hostName = InetAddress.getLocalHost().getHostName();
 +    String zkPort = testingCluster.getHBaseUtil().getConf().get(HConstants.ZOOKEEPER_CLIENT_PORT);
 +    assertNotNull(zkPort);
 +
 +    executeString("CREATE TABLE hbase_mapped_table (rk text, col2_key text, col2_value text, col3 text) " +
 +        "USING hbase WITH ('table'='hbase_table', 'columns'=':key,col2:key:,col2:value:,col3:', " +
 +        "'hbase.rowkey.delimiter'='_', " +
 +        "'" + HConstants.ZOOKEEPER_QUORUM + "'='" + hostName + "'," +
 +        "'" + HConstants.ZOOKEEPER_CLIENT_PORT + "'='" + zkPort + "')").close();
 +
 +    assertTableExists("hbase_mapped_table");
 +    TableDesc tableDesc = catalog.getTableDesc(getCurrentDatabase(), "hbase_mapped_table");
 +
 +    // create test table
 +    KeyValueSet tableOptions = new KeyValueSet();
 +    tableOptions.set(StorageConstants.CSVFILE_DELIMITER, StorageConstants.DEFAULT_FIELD_DELIMITER);
 +    tableOptions.set(StorageConstants.CSVFILE_NULL, "\\\\N");
 +
 +    Schema schema = new Schema();
 +    schema.addColumn("rk", Type.TEXT);
 +    schema.addColumn("col2_key", Type.TEXT);
 +    schema.addColumn("col2_value", Type.TEXT);
 +    schema.addColumn("col3", Type.TEXT);
 +    List<String> datas = new ArrayList<String>();
 +    for (int i = 20; i >= 0; i--) {
 +      for (int j = 0; j < 3; j++) {
 +        datas.add(i + "|ck-" + j + "|value-" + j + "|col3-" + i);
 +      }
 +    }
 +    TajoTestingCluster.createTable(getCurrentDatabase() + ".base_table",
 +        schema, tableOptions, datas.toArray(new String[]{}), 2);
 +
 +    executeString("insert into hbase_mapped_table " +
 +        "select rk, col2_key, col2_value, col3 from base_table ").close();
 +
 +    HTable htable = null;
 +    ResultScanner scanner = null;
 +    try {
 +      htable = new HTable(testingCluster.getHBaseUtil().getConf(), "hbase_table");
 +
 +      Scan scan = new Scan();
 +      scan.addFamily(Bytes.toBytes("col2"));
 +      scan.addFamily(Bytes.toBytes("col3"));
 +      scanner = htable.getScanner(scan);
 +
 +      assertStrings(resultSetToString(scanner,
 +          new byte[][]{null, Bytes.toBytes("col2"), Bytes.toBytes("col3")},
 +          new byte[][]{null, null, null},
 +          new boolean[]{false, false, false}, tableDesc.getSchema()));
 +
 +      ResultSet res = executeString("select * from hbase_mapped_table");
 +
 +      String expected = "rk,col2_key,col2_value,col3\n" +
 +          "-------------------------------\n" +
 +          "0,[\"ck-0\", \"ck-1\", \"ck-2\"],[\"value-0\", \"value-1\", \"value-2\"],col3-0\n" +
 +          "1,[\"ck-0\", \"ck-1\", \"ck-2\"],[\"value-0\", \"value-1\", \"value-2\"],col3-1\n" +
 +          "10,[\"ck-0\", \"ck-1\", \"ck-2\"],[\"value-0\", \"value-1\", \"value-2\"],col3-10\n" +
 +          "11,[\"ck-0\", \"ck-1\", \"ck-2\"],[\"value-0\", \"value-1\", \"value-2\"],col3-11\n" +
 +          "12,[\"ck-0\", \"ck-1\", \"ck-2\"],[\"value-0\", \"value-1\", \"value-2\"],col3-12\n" +
 +          "13,[\"ck-0\", \"ck-1\", \"ck-2\"],[\"value-0\", \"value-1\", \"value-2\"],col3-13\n" +
 +          "14,[\"ck-0\", \"ck-1\", \"ck-2\"],[\"value-0\", \"value-1\", \"value-2\"],col3-14\n" +
 +          "15,[\"ck-0\", \"ck-1\", \"ck-2\"],[\"value-0\", \"value-1\", \"value-2\"],col3-15\n" +
 +          "16,[\"ck-0\", \"ck-1\", \"ck-2\"],[\"value-0\", \"value-1\", \"value-2\"],col3-16\n" +
 +          "17,[\"ck-0\", \"ck-1\", \"ck-2\"],[\"value-0\", \"value-1\", \"value-2\"],col3-17\n" +
 +          "18,[\"ck-0\", \"ck-1\", \"ck-2\"],[\"value-0\", \"value-1\", \"value-2\"],col3-18\n" +
 +          "19,[\"ck-0\", \"ck-1\", \"ck-2\"],[\"value-0\", \"value-1\", \"value-2\"],col3-19\n" +
 +          "2,[\"ck-0\", \"ck-1\", \"ck-2\"],[\"value-0\", \"value-1\", \"value-2\"],col3-2\n" +
 +          "20,[\"ck-0\", \"ck-1\", \"ck-2\"],[\"value-0\", \"value-1\", \"value-2\"],col3-20\n" +
 +          "3,[\"ck-0\", \"ck-1\", \"ck-2\"],[\"value-0\", \"value-1\", \"value-2\"],col3-3\n" +
 +          "4,[\"ck-0\", \"ck-1\", \"ck-2\"],[\"value-0\", \"value-1\", \"value-2\"],col3-4\n" +
 +          "5,[\"ck-0\", \"ck-1\", \"ck-2\"],[\"value-0\", \"value-1\", \"value-2\"],col3-5\n" +
 +          "6,[\"ck-0\", \"ck-1\", \"ck-2\"],[\"value-0\", \"value-1\", \"value-2\"],col3-6\n" +
 +          "7,[\"ck-0\", \"ck-1\", \"ck-2\"],[\"value-0\", \"value-1\", \"value-2\"],col3-7\n" +
 +          "8,[\"ck-0\", \"ck-1\", \"ck-2\"],[\"value-0\", \"value-1\", \"value-2\"],col3-8\n" +
 +          "9,[\"ck-0\", \"ck-1\", \"ck-2\"],[\"value-0\", \"value-1\", \"value-2\"],col3-9\n";
 +
 +      assertEquals(expected, resultSetToString(res));
 +      res.close();
 +
 +    } finally {
 +      executeString("DROP TABLE base_table PURGE").close();
 +      executeString("DROP TABLE hbase_mapped_table PURGE").close();
 +
 +      if (scanner != null) {
 +        scanner.close();
 +      }
 +
 +      if (htable != null) {
 +        htable.close();
 +      }
 +    }
 +  }
 +
 +  @Test
 +  public void testInsertIntoDifferentType() throws Exception {
 +    String hostName = InetAddress.getLocalHost().getHostName();
 +    String zkPort = testingCluster.getHBaseUtil().getConf().get(HConstants.ZOOKEEPER_CLIENT_PORT);
 +    assertNotNull(zkPort);
 +
 +    executeString("CREATE TABLE hbase_mapped_table (rk text, col1 text) " +
 +        "USING hbase WITH ('table'='hbase_table', 'columns'=':key,col1:a', " +
 +        "'hbase.split.rowkeys'='1,2,3,4,5,6,7,8,9', " +
 +        "'" + HConstants.ZOOKEEPER_QUORUM + "'='" + hostName + "'," +
 +        "'" + HConstants.ZOOKEEPER_CLIENT_PORT + "'='" + zkPort + "')").close();
 +
 +    assertTableExists("hbase_mapped_table");
 +
 +    // create test table
 +    KeyValueSet tableOptions = new KeyValueSet();
 +    tableOptions.set(StorageConstants.CSVFILE_DELIMITER, StorageConstants.DEFAULT_FIELD_DELIMITER);
 +    tableOptions.set(StorageConstants.CSVFILE_NULL, "\\\\N");
 +
 +    Schema schema = new Schema();
 +    schema.addColumn("id", Type.INT4);
 +    schema.addColumn("name", Type.TEXT);
 +    List<String> datas = new ArrayList<String>();
 +    for (int i = 99; i >= 0; i--) {
 +      datas.add(i + "|value" + i);
 +    }
 +    TajoTestingCluster.createTable(getCurrentDatabase() + ".base_table",
 +        schema, tableOptions, datas.toArray(new String[]{}), 2);
 +
 +    try {
 +      executeString("insert into hbase_mapped_table " +
 +          "select id, name from base_table ").close();
 +      fail("If inserting data type different with target table data type, should throw exception");
 +    } catch (Exception e) {
 +      assertTrue(e.getMessage().indexOf("VerifyException") >= 0);
 +    } finally {
 +      executeString("DROP TABLE base_table PURGE").close();
 +      executeString("DROP TABLE hbase_mapped_table PURGE").close();
 +    }
 +  }
 +
 +  @Test
 +  public void testInsertIntoRowField() throws Exception {
 +    String hostName = InetAddress.getLocalHost().getHostName();
 +    String zkPort = testingCluster.getHBaseUtil().getConf().get(HConstants.ZOOKEEPER_CLIENT_PORT);
 +    assertNotNull(zkPort);
 +
 +    executeString("CREATE TABLE hbase_mapped_table (rk1 text, rk2 text, col1 text, col2 text, col3 text) " +
 +        "USING hbase WITH ('table'='hbase_table', 'columns'='0:key,1:key,col1:a,col2:,col3:b', " +
 +        "'hbase.rowkey.delimiter'='_', " +
 +        "'" + HConstants.ZOOKEEPER_QUORUM + "'='" + hostName + "'," +
 +        "'" + HConstants.ZOOKEEPER_CLIENT_PORT + "'='" + zkPort + "')").close();
 +
 +
 +    assertTableExists("hbase_mapped_table");
 +    TableDesc tableDesc = catalog.getTableDesc(getCurrentDatabase(), "hbase_mapped_table");
 +
 +    executeString("insert into hbase_mapped_table " +
 +        "select l_orderkey::text, l_partkey::text, l_shipdate, l_returnflag, l_suppkey::text from default.lineitem ");
 +
 +    HTable htable = null;
 +    ResultScanner scanner = null;
 +    try {
 +      htable = new HTable(testingCluster.getHBaseUtil().getConf(), "hbase_table");
 +
 +      Scan scan = new Scan();
 +      scan.addFamily(Bytes.toBytes("col1"));
 +      scan.addFamily(Bytes.toBytes("col2"));
 +      scan.addFamily(Bytes.toBytes("col3"));
 +      scanner = htable.getScanner(scan);
 +
 +      assertStrings(resultSetToString(scanner,
 +          new byte[][]{null, Bytes.toBytes("col1"), Bytes.toBytes("col2"), Bytes.toBytes("col3")},
 +          new byte[][]{null, Bytes.toBytes("a"), Bytes.toBytes(""), Bytes.toBytes("b")},
 +          new boolean[]{false, false, false, false}, tableDesc.getSchema()));
 +
 +    } finally {
 +      executeString("DROP TABLE hbase_mapped_table PURGE").close();
 +
 +      if (scanner != null) {
 +        scanner.close();
 +      }
 +
 +      if (htable != null) {
 +        htable.close();
 +      }
 +    }
 +  }
 +
 +  @Test
 +  public void testCATS() throws Exception {
 +    String hostName = InetAddress.getLocalHost().getHostName();
 +    String zkPort = testingCluster.getHBaseUtil().getConf().get(HConstants.ZOOKEEPER_CLIENT_PORT);
 +    assertNotNull(zkPort);
 +
 +    // create test table
 +    KeyValueSet tableOptions = new KeyValueSet();
 +    tableOptions.set(StorageConstants.CSVFILE_DELIMITER, StorageConstants.DEFAULT_FIELD_DELIMITER);
 +    tableOptions.set(StorageConstants.CSVFILE_NULL, "\\\\N");
 +
 +    Schema schema = new Schema();
 +    schema.addColumn("id", Type.TEXT);
 +    schema.addColumn("name", Type.TEXT);
 +    List<String> datas = new ArrayList<String>();
 +    DecimalFormat df = new DecimalFormat("000");
 +    for (int i = 99; i >= 0; i--) {
 +      datas.add(df.format(i) + "|value" + i);
 +    }
 +    TajoTestingCluster.createTable(getCurrentDatabase() + ".base_table",
 +        schema, tableOptions, datas.toArray(new String[]{}), 2);
 +
 +    executeString("CREATE TABLE hbase_mapped_table (rk text, col1 text) " +
 +        "USING hbase WITH ('table'='hbase_table', 'columns'=':key,col1:a', " +
 +        "'hbase.split.rowkeys'='010,040,060,080', " +
 +        "'" + HConstants.ZOOKEEPER_QUORUM + "'='" + hostName + "'," +
 +        "'" + HConstants.ZOOKEEPER_CLIENT_PORT + "'='" + zkPort + "')" +
 +        " as " +
 +        "select id, name from base_table"
 +    ).close();
 +
 +    assertTableExists("hbase_mapped_table");
 +    TableDesc tableDesc = catalog.getTableDesc(getCurrentDatabase(), "hbase_mapped_table");
 +
 +    HTable htable = null;
 +    ResultScanner scanner = null;
 +    try {
 +      htable = new HTable(testingCluster.getHBaseUtil().getConf(), "hbase_table");
 +
 +      Scan scan = new Scan();
 +      scan.addFamily(Bytes.toBytes("col1"));
 +      scanner = htable.getScanner(scan);
 +
 +      assertStrings(resultSetToString(scanner,
 +          new byte[][]{null, Bytes.toBytes("col1")},
 +          new byte[][]{null, Bytes.toBytes("a")},
 +          new boolean[]{false, false}, tableDesc.getSchema()));
 +
 +    } finally {
 +      executeString("DROP TABLE base_table PURGE").close();
 +      executeString("DROP TABLE hbase_mapped_table PURGE").close();
 +
 +      if (scanner != null) {
 +        scanner.close();
 +      }
 +
 +      if (htable != null) {
 +        htable.close();
 +      }
 +    }
 +  }
 +
 +  @Test
 +  public void testInsertIntoUsingPut() throws Exception {
 +    String hostName = InetAddress.getLocalHost().getHostName();
 +    String zkPort = testingCluster.getHBaseUtil().getConf().get(HConstants.ZOOKEEPER_CLIENT_PORT);
 +    assertNotNull(zkPort);
 +
 +    executeString("CREATE TABLE hbase_mapped_table (rk text, col1 text, col2 text, col3 int4) " +
 +        "USING hbase WITH ('table'='hbase_table', 'columns'=':key,col1:a,col2:,col3:b#b', " +
 +        "'" + HConstants.ZOOKEEPER_QUORUM + "'='" + hostName + "'," +
 +        "'" + HConstants.ZOOKEEPER_CLIENT_PORT + "'='" + zkPort + "')").close();
 +
 +    assertTableExists("hbase_mapped_table");
 +    TableDesc tableDesc = catalog.getTableDesc(getCurrentDatabase(), "hbase_mapped_table");
 +
 +    Map<String, String> sessions = new HashMap<String, String>();
 +    sessions.put(HBaseStorageConstants.INSERT_PUT_MODE, "true");
 +    client.updateSessionVariables(sessions);
 +
 +    HTable htable = null;
 +    ResultScanner scanner = null;
 +    try {
 +      executeString("insert into hbase_mapped_table " +
 +          "select l_orderkey::text, l_shipdate, l_returnflag, l_suppkey from default.lineitem ").close();
 +
 +      htable = new HTable(testingCluster.getHBaseUtil().getConf(), "hbase_table");
 +
 +      Scan scan = new Scan();
 +      scan.addFamily(Bytes.toBytes("col1"));
 +      scan.addFamily(Bytes.toBytes("col2"));
 +      scan.addFamily(Bytes.toBytes("col3"));
 +      scanner = htable.getScanner(scan);
 +
 +      // result is dirrerent with testInsertInto because l_orderkey is not unique.
 +      assertStrings(resultSetToString(scanner,
 +          new byte[][]{null, Bytes.toBytes("col1"), Bytes.toBytes("col2"), Bytes.toBytes("col3")},
 +          new byte[][]{null, Bytes.toBytes("a"), null, Bytes.toBytes("b")},
 +          new boolean[]{false, false, false, true}, tableDesc.getSchema()));
 +
 +    } finally {
 +      executeString("DROP TABLE hbase_mapped_table PURGE").close();
 +
 +      client.unsetSessionVariables(TUtil.newList(HBaseStorageConstants.INSERT_PUT_MODE));
 +
 +      if (scanner != null) {
 +        scanner.close();
 +      }
 +
 +      if (htable != null) {
 +        htable.close();
 +      }
 +    }
 +  }
 +
 +  @Test
 +  public void testInsertIntoLocation() throws Exception {
 +    String hostName = InetAddress.getLocalHost().getHostName();
 +    String zkPort = testingCluster.getHBaseUtil().getConf().get(HConstants.ZOOKEEPER_CLIENT_PORT);
 +    assertNotNull(zkPort);
 +
 +    executeString("CREATE TABLE hbase_mapped_table (rk text, col1 text, col2 text) " +
 +        "USING hbase WITH ('table'='hbase_table', 'columns'=':key,col1:a,col2:', " +
 +        "'hbase.split.rowkeys'='010,040,060,080', " +
 +        "'" + HConstants.ZOOKEEPER_QUORUM + "'='" + hostName + "'," +
 +        "'" + HConstants.ZOOKEEPER_CLIENT_PORT + "'='" + zkPort + "')").close();
 +
 +    assertTableExists("hbase_mapped_table");
 +
 +    try {
 +      // create test table
 +      KeyValueSet tableOptions = new KeyValueSet();
 +      tableOptions.set(StorageConstants.CSVFILE_DELIMITER, StorageConstants.DEFAULT_FIELD_DELIMITER);
 +      tableOptions.set(StorageConstants.CSVFILE_NULL, "\\\\N");
 +
 +      Schema schema = new Schema();
 +      schema.addColumn("id", Type.TEXT);
 +      schema.addColumn("name", Type.TEXT);
 +      schema.addColumn("comment", Type.TEXT);
 +      List<String> datas = new ArrayList<String>();
 +      DecimalFormat df = new DecimalFormat("000");
 +      for (int i = 99; i >= 0; i--) {
 +        datas.add(df.format(i) + "|value" + i + "|comment-" + i);
 +      }
 +      TajoTestingCluster.createTable(getCurrentDatabase() + ".base_table",
 +          schema, tableOptions, datas.toArray(new String[]{}), 2);
 +
 +      executeString("insert into location '/tmp/hfile_test' " +
-           "USING hbase WITH ('table'='hbase_table', 'columns'=':key,col1:a,col2:', " +
-           "'" + HConstants.ZOOKEEPER_QUORUM + "'='" + hostName + "'," +
-           "'" + HConstants.ZOOKEEPER_CLIENT_PORT + "'='" + zkPort + "')" +
 +          "select id, name, comment from base_table ").close();
 +
 +      FileSystem fs = testingCluster.getDefaultFileSystem();
 +      Path path = new Path("/tmp/hfile_test");
 +      assertTrue(fs.exists(path));
 +
 +      FileStatus[] files = fs.listStatus(path);
 +      assertNotNull(files);
 +      assertEquals(2, files.length);
 +
-       assertEquals("/tmp/hfile_test/col2", files[1].getPath().toUri().getPath());
- 
-       int index = 1;
++      int index = 0;
 +      for (FileStatus eachFile: files) {
-         assertEquals("/tmp/hfile_test/col" + index, eachFile.getPath().toUri().getPath());
++        assertEquals("/tmp/hfile_test/part-01-00000" + index + "-00" + index, eachFile.getPath().toUri().getPath());
 +        for (FileStatus subFile: fs.listStatus(eachFile.getPath())) {
 +          assertTrue(subFile.isFile());
 +          assertTrue(subFile.getLen() > 0);
 +        }
 +        index++;
 +      }
 +    } finally {
 +      executeString("DROP TABLE base_table PURGE").close();
 +      executeString("DROP TABLE hbase_mapped_table PURGE").close();
 +    }
 +  }
 +
 +  private String resultSetToString(ResultScanner scanner,
 +                                   byte[][] cfNames, byte[][] qualifiers,
 +                                   boolean[] binaries,
 +                                   Schema schema) throws Exception {
 +    StringBuilder sb = new StringBuilder();
 +    Result result = null;
 +    while ( (result = scanner.next()) != null ) {
 +      if (binaries[0]) {
 +        sb.append(HBaseBinarySerializerDeserializer.deserialize(schema.getColumn(0), result.getRow()).asChar());
 +      } else {
 +        sb.append(new String(result.getRow()));
 +      }
 +
 +      for (int i = 0; i < cfNames.length; i++) {
 +        if (cfNames[i] == null) {
 +          //rowkey
 +          continue;
 +        }
 +        if (qualifiers[i] == null) {
 +          Map<byte[], byte[]> values = result.getFamilyMap(cfNames[i]);
 +          if (values == null) {
 +            sb.append(", null");
 +          } else {
 +            sb.append(", {");
 +            String delim = "";
 +            for (Map.Entry<byte[], byte[]> valueEntry: values.entrySet()) {
 +              byte[] keyBytes = valueEntry.getKey();
 +              byte[] valueBytes = valueEntry.getValue();
 +
 +              if (binaries[i]) {
 +                sb.append(delim).append("\"").append(keyBytes == null ? "" : Bytes.toLong(keyBytes)).append("\"");
 +                sb.append(": \"").append(HBaseBinarySerializerDeserializer.deserialize(schema.getColumn(i), valueBytes)).append("\"");
 +              } else {
 +                sb.append(delim).append("\"").append(keyBytes == null ? "" : new String(keyBytes)).append("\"");
 +                sb.append(": \"").append(HBaseTextSerializerDeserializer.deserialize(schema.getColumn(i), valueBytes)).append("\"");
 +              }
 +              delim = ", ";
 +            }
 +            sb.append("}");
 +          }
 +        } else {
 +          byte[] value = result.getValue(cfNames[i], qualifiers[i]);
 +          if (value == null) {
 +            sb.append(", null");
 +          } else {
 +            if (binaries[i]) {
 +              sb.append(", ").append(HBaseBinarySerializerDeserializer.deserialize(schema.getColumn(i), value));
 +            } else {
 +              sb.append(", ").append(HBaseTextSerializerDeserializer.deserialize(schema.getColumn(i), value));
 +            }
 +          }
 +        }
 +      }
 +      sb.append("\n");
 +    }
 +
 +    return sb.toString();
 +  }
 +}


[11/29] tajo git commit: TAJO-1211: Staging directory for CTAS and INSERT should be in the output dir.

Posted by hj...@apache.org.
TAJO-1211: Staging directory for CTAS and INSERT should be in the output dir.

Closes #274


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

Branch: refs/heads/hbase_storage
Commit: b4adc18cd25de550fe04a43ef69d715c146976db
Parents: bf68b77
Author: Hyunsik Choi <hy...@apache.org>
Authored: Mon Dec 1 17:23:35 2014 +0900
Committer: Hyunsik Choi <hy...@apache.org>
Committed: Mon Dec 1 17:23:35 2014 +0900

----------------------------------------------------------------------
 CHANGES                                         |  3 +
 .../java/org/apache/tajo/conf/TajoConf.java     | 12 +++-
 .../org/apache/tajo/master/GlobalEngine.java    |  7 +-
 .../java/org/apache/tajo/master/TajoMaster.java |  2 +-
 .../apache/tajo/master/querymaster/Query.java   | 74 +++++++++++++++-----
 .../master/querymaster/QueryMasterTask.java     | 23 ++++--
 .../src/main/resources/webapps/admin/index.jsp  |  2 +-
 .../apache/tajo/engine/query/TestCTASQuery.java |  1 +
 .../tajo/engine/query/TestInsertQuery.java      | 14 +++-
 9 files changed, 105 insertions(+), 33 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tajo/blob/b4adc18c/CHANGES
----------------------------------------------------------------------
diff --git a/CHANGES b/CHANGES
index 33983bc..8d51d44 100644
--- a/CHANGES
+++ b/CHANGES
@@ -79,6 +79,9 @@ Release 0.9.1 - unreleased
 
   BUG FIXES
 
+    TAJO-1211: Staging directory for CTAS and INSERT should be in 
+    the output dir. (hyunsik)
+
     TAJO-1210: ByteBufLineReader does not handle the end of file, 
     if newline is not appeared. (jinho)
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/b4adc18c/tajo-common/src/main/java/org/apache/tajo/conf/TajoConf.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/conf/TajoConf.java b/tajo-common/src/main/java/org/apache/tajo/conf/TajoConf.java
index f3ae453..312abfb 100644
--- a/tajo-common/src/main/java/org/apache/tajo/conf/TajoConf.java
+++ b/tajo-common/src/main/java/org/apache/tajo/conf/TajoConf.java
@@ -671,7 +671,15 @@ public class TajoConf extends Configuration {
     return path.indexOf("file:/") == 0 || path.indexOf("hdfs:/") == 0;
   }
 
-  public static Path getStagingDir(TajoConf conf) throws IOException {
+  /**
+   * It returns the default root staging directory used by queries without a target table or
+   * a specified output directory. An example query is <pre>SELECT a,b,c FROM XXX;</pre>.
+   *
+   * @param conf TajoConf
+   * @return Path which points the default staging directory
+   * @throws IOException
+   */
+  public static Path getDefaultRootStagingDir(TajoConf conf) throws IOException {
     String stagingDirString = conf.getVar(ConfVars.STAGING_ROOT_DIR);
     if (!hasScheme(stagingDirString)) {
       Path warehousePath = getWarehouseDir(conf);
@@ -686,7 +694,7 @@ public class TajoConf extends Configuration {
   public static Path getQueryHistoryDir(TajoConf conf) throws IOException {
     String historyDirString = conf.getVar(ConfVars.HISTORY_QUERY_DIR);
     if (!hasScheme(historyDirString)) {
-      Path stagingPath = getStagingDir(conf);
+      Path stagingPath = getDefaultRootStagingDir(conf);
       FileSystem fs = stagingPath.getFileSystem(conf);
       Path path = new Path(fs.getUri().toString(), historyDirString);
       conf.setVar(ConfVars.HISTORY_QUERY_DIR, path.toString());

http://git-wip-us.apache.org/repos/asf/tajo/blob/b4adc18c/tajo-core/src/main/java/org/apache/tajo/master/GlobalEngine.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/GlobalEngine.java b/tajo-core/src/main/java/org/apache/tajo/master/GlobalEngine.java
index 47a8750..9bf9a75 100644
--- a/tajo-core/src/main/java/org/apache/tajo/master/GlobalEngine.java
+++ b/tajo-core/src/main/java/org/apache/tajo/master/GlobalEngine.java
@@ -335,10 +335,8 @@ public class GlobalEngine extends AbstractService {
     String queryId = nodeUniqName + "_" + System.currentTimeMillis();
 
     FileSystem fs = TajoConf.getWarehouseDir(context.getConf()).getFileSystem(context.getConf());
-    Path stagingDir = QueryMasterTask.initStagingDir(context.getConf(), fs, queryId.toString());
-
+    Path stagingDir = QueryMasterTask.initStagingDir(context.getConf(), queryId.toString(), queryContext);
     Path stagingResultDir = new Path(stagingDir, TajoConstants.RESULT_DIR_NAME);
-    fs.mkdirs(stagingResultDir);
 
     TableDesc tableDesc = null;
     Path finalOutputDir = null;
@@ -349,8 +347,7 @@ public class GlobalEngine extends AbstractService {
       finalOutputDir = insertNode.getPath();
     }
 
-    TaskAttemptContext taskAttemptContext =
-        new TaskAttemptContext(queryContext, null, null, (CatalogProtos.FragmentProto[]) null, stagingDir);
+    TaskAttemptContext taskAttemptContext = new TaskAttemptContext(queryContext, null, null, null, stagingDir);
     taskAttemptContext.setOutputPath(new Path(stagingResultDir, "part-01-000000"));
 
     EvalExprExec evalExprExec = new EvalExprExec(taskAttemptContext, (EvalExprNode) insertNode.getChild());

http://git-wip-us.apache.org/repos/asf/tajo/blob/b4adc18c/tajo-core/src/main/java/org/apache/tajo/master/TajoMaster.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/TajoMaster.java b/tajo-core/src/main/java/org/apache/tajo/master/TajoMaster.java
index 6e585af..795983d 100644
--- a/tajo-core/src/main/java/org/apache/tajo/master/TajoMaster.java
+++ b/tajo-core/src/main/java/org/apache/tajo/master/TajoMaster.java
@@ -271,7 +271,7 @@ public class TajoMaster extends CompositeService {
       LOG.info("Warehouse dir '" + wareHousePath + "' is created");
     }
 
-    Path stagingPath = TajoConf.getStagingDir(systemConf);
+    Path stagingPath = TajoConf.getDefaultRootStagingDir(systemConf);
     LOG.info("Staging dir: " + wareHousePath);
     if (!defaultFS.exists(stagingPath)) {
       defaultFS.mkdirs(stagingPath, new FsPermission(STAGING_ROOTDIR_PERMISSION));

http://git-wip-us.apache.org/repos/asf/tajo/blob/b4adc18c/tajo-core/src/main/java/org/apache/tajo/master/querymaster/Query.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/Query.java b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/Query.java
index 7db6d8b..07b47c1 100644
--- a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/Query.java
+++ b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/Query.java
@@ -411,9 +411,9 @@ public class Query implements EventHandler<QueryEvent> {
     public QueryState transition(Query query, QueryEvent queryEvent) {
       QueryCompletedEvent subQueryEvent = (QueryCompletedEvent) queryEvent;
       QueryState finalState;
+
       if (subQueryEvent.getState() == SubQueryState.SUCCEEDED) {
-        finalizeQuery(query, subQueryEvent);
-        finalState = QueryState.QUERY_SUCCEEDED;
+        finalState = finalizeQuery(query, subQueryEvent);
       } else if (subQueryEvent.getState() == SubQueryState.FAILED) {
         finalState = QueryState.QUERY_FAILED;
       } else if (subQueryEvent.getState() == SubQueryState.KILLED) {
@@ -427,26 +427,28 @@ public class Query implements EventHandler<QueryEvent> {
       return finalState;
     }
 
-    private void finalizeQuery(Query query, QueryCompletedEvent event) {
+    private QueryState finalizeQuery(Query query, QueryCompletedEvent event) {
       MasterPlan masterPlan = query.getPlan();
 
       ExecutionBlock terminal = query.getPlan().getTerminalBlock();
       DataChannel finalChannel = masterPlan.getChannel(event.getExecutionBlockId(), terminal.getId());
-      Path finalOutputDir = commitOutputData(query);
 
       QueryHookExecutor hookExecutor = new QueryHookExecutor(query.context.getQueryMasterContext());
       try {
-        hookExecutor.execute(query.context.getQueryContext(), query, event.getExecutionBlockId(),
-            finalOutputDir);
-      } catch (Exception e) {
-        query.eventHandler.handle(new QueryDiagnosticsUpdateEvent(query.id, ExceptionUtils.getStackTrace(e)));
+        Path finalOutputDir = commitOutputData(query);
+        hookExecutor.execute(query.context.getQueryContext(), query, event.getExecutionBlockId(), finalOutputDir);
+      } catch (Throwable t) {
+        query.eventHandler.handle(new QueryDiagnosticsUpdateEvent(query.id, ExceptionUtils.getStackTrace(t)));
+        return QueryState.QUERY_ERROR;
       }
+
+      return QueryState.QUERY_SUCCEEDED;
     }
 
     /**
      * It moves a result data stored in a staging output dir into a final output dir.
      */
-    public Path commitOutputData(Query query) {
+    public Path commitOutputData(Query query) throws IOException {
       QueryContext queryContext = query.context.getQueryContext();
       Path stagingResultDir = new Path(queryContext.getStagingDir(), TajoConstants.RESULT_DIR_NAME);
       Path finalOutputDir;
@@ -508,24 +510,49 @@ public class Query implements EventHandler<QueryEvent> {
                   fs.delete(entry.getValue(), true);
                   fs.rename(entry.getValue(), entry.getKey());
                 }
+
                 throw new IOException(ioe.getMessage());
               }
-            } else {
+            } else { // no partition
               try {
+
+                // if the final output dir exists, move all contents to the temporary table dir.
+                // Otherwise, just make the final output dir. As a result, the final output dir will be empty.
                 if (fs.exists(finalOutputDir)) {
-                  fs.rename(finalOutputDir, oldTableDir);
+                  fs.mkdirs(oldTableDir);
+
+                  for (FileStatus status : fs.listStatus(finalOutputDir, StorageManager.hiddenFileFilter)) {
+                    fs.rename(status.getPath(), oldTableDir);
+                  }
+
                   movedToOldTable = fs.exists(oldTableDir);
                 } else { // if the parent does not exist, make its parent directory.
-                  fs.mkdirs(finalOutputDir.getParent());
+                  fs.mkdirs(finalOutputDir);
                 }
 
-                fs.rename(stagingResultDir, finalOutputDir);
+                // Move the results to the final output dir.
+                for (FileStatus status : fs.listStatus(stagingResultDir)) {
+                  fs.rename(status.getPath(), finalOutputDir);
+                }
+
+                // Check the final output dir
                 committed = fs.exists(finalOutputDir);
+
               } catch (IOException ioe) {
                 // recover the old table
                 if (movedToOldTable && !committed) {
-                  fs.rename(oldTableDir, finalOutputDir);
+
+                  // if commit is failed, recover the old data
+                  for (FileStatus status : fs.listStatus(finalOutputDir, StorageManager.hiddenFileFilter)) {
+                    fs.delete(status.getPath(), true);
+                  }
+
+                  for (FileStatus status : fs.listStatus(oldTableDir)) {
+                    fs.rename(status.getPath(), finalOutputDir);
+                  }
                 }
+
+                throw new IOException(ioe.getMessage());
               }
             }
           } else {
@@ -560,13 +587,24 @@ public class Query implements EventHandler<QueryEvent> {
                 }
               }
             } else { // CREATE TABLE AS SELECT (CTAS)
-              fs.rename(stagingResultDir, finalOutputDir);
+              if (fs.exists(finalOutputDir)) {
+                for (FileStatus status : fs.listStatus(stagingResultDir)) {
+                  fs.rename(status.getPath(), finalOutputDir);
+                }
+              } else {
+                fs.rename(stagingResultDir, finalOutputDir);
+              }
               LOG.info("Moved from the staging dir to the output directory '" + finalOutputDir);
             }
           }
-        } catch (IOException e) {
-          // TODO report to client
-          e.printStackTrace();
+
+          // remove the staging directory if the final output dir is given.
+          Path stagingDirRoot = queryContext.getStagingDir().getParent();
+          fs.delete(stagingDirRoot, true);
+
+        } catch (Throwable t) {
+          LOG.error(t);
+          throw new IOException(t);
         }
       } else {
         finalOutputDir = new Path(queryContext.getStagingDir(), TajoConstants.RESULT_DIR_NAME);

http://git-wip-us.apache.org/repos/asf/tajo/blob/b4adc18c/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryMasterTask.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryMasterTask.java b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryMasterTask.java
index 5cf3df5..75d8ab6 100644
--- a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryMasterTask.java
+++ b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryMasterTask.java
@@ -58,6 +58,7 @@ import org.apache.tajo.rpc.CallFuture;
 import org.apache.tajo.rpc.NettyClientBase;
 import org.apache.tajo.rpc.RpcConnectionPool;
 import org.apache.tajo.storage.StorageManager;
+import org.apache.tajo.storage.StorageUtil;
 import org.apache.tajo.util.HAServiceUtil;
 import org.apache.tajo.util.metrics.TajoMetrics;
 import org.apache.tajo.util.metrics.reporter.MetricsConsoleReporter;
@@ -79,6 +80,8 @@ public class QueryMasterTask extends CompositeService {
   final public static FsPermission STAGING_DIR_PERMISSION =
       FsPermission.createImmutable((short) 0700); // rwx--------
 
+  public static final String TMP_STAGING_DIR_PREFIX = ".staging";
+
   private QueryId queryId;
 
   private Session session;
@@ -399,8 +402,7 @@ public class QueryMasterTask extends CompositeService {
 
     try {
 
-      stagingDir = initStagingDir(systemConf, defaultFS, queryId.toString());
-      defaultFS.mkdirs(new Path(stagingDir, TajoConstants.RESULT_DIR_NAME));
+      stagingDir = initStagingDir(systemConf, queryId.toString(), queryContext);
 
       // Create a subdirectories
       LOG.info("The staging dir '" + stagingDir + "' is created.");
@@ -423,7 +425,7 @@ public class QueryMasterTask extends CompositeService {
    * It initializes the final output and staging directory and sets
    * them to variables.
    */
-  public static Path initStagingDir(TajoConf conf, FileSystem fs, String queryId) throws IOException {
+  public static Path initStagingDir(TajoConf conf, String queryId, QueryContext context) throws IOException {
 
     String realUser;
     String currentUser;
@@ -432,13 +434,21 @@ public class QueryMasterTask extends CompositeService {
     realUser = ugi.getShortUserName();
     currentUser = UserGroupInformation.getCurrentUser().getShortUserName();
 
-    Path stagingDir = null;
+    FileSystem fs;
+    Path stagingDir;
 
     ////////////////////////////////////////////
     // Create Output Directory
     ////////////////////////////////////////////
 
-    stagingDir = new Path(TajoConf.getStagingDir(conf), queryId);
+    if (context.isCreateTable() || context.isInsert()) {
+      stagingDir = StorageUtil.concatPath(context.getOutputPath(), TMP_STAGING_DIR_PREFIX, queryId);
+    } else {
+      stagingDir = new Path(TajoConf.getDefaultRootStagingDir(conf), queryId);
+    }
+
+    // initializ
+    fs = stagingDir.getFileSystem(conf);
 
     if (fs.exists(stagingDir)) {
       throw new IOException("The staging directory '" + stagingDir + "' already exists");
@@ -462,6 +472,9 @@ public class QueryMasterTask extends CompositeService {
       fs.setPermission(stagingDir, new FsPermission(STAGING_DIR_PERMISSION));
     }
 
+    Path stagingResultDir = new Path(stagingDir, TajoConstants.RESULT_DIR_NAME);
+    fs.mkdirs(stagingResultDir);
+
     return stagingDir;
   }
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/b4adc18c/tajo-core/src/main/resources/webapps/admin/index.jsp
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/resources/webapps/admin/index.jsp b/tajo-core/src/main/resources/webapps/admin/index.jsp
index 30cbf88..6778725 100644
--- a/tajo-core/src/main/resources/webapps/admin/index.jsp
+++ b/tajo-core/src/main/resources/webapps/admin/index.jsp
@@ -131,7 +131,7 @@
     <tr><td width='150'>Root dir:</td><td><%=TajoConf.getTajoRootDir(master.getContext().getConf())%></td></tr>
     <tr><td width='150'>System dir:</td><td><%=TajoConf.getSystemDir(master.getContext().getConf())%></td></tr>
     <tr><td width='150'>Warehouse dir:</td><td><%=TajoConf.getWarehouseDir(master.getContext().getConf())%></td></tr>
-    <tr><td width='150'>Staging dir:</td><td><%=TajoConf.getStagingDir(master.getContext().getConf())%></td></tr>
+    <tr><td width='150'>Staging dir:</td><td><%=TajoConf.getDefaultRootStagingDir(master.getContext().getConf())%></td></tr>
     <tr><td width='150'>Client Service:</td><td><%=NetUtils.normalizeInetSocketAddress(master.getTajoMasterClientService().getBindAddress())%></td></tr>
     <tr><td width='150'>Catalog Service:</td><td><%=master.getCatalogServer().getCatalogServerName()%></td></tr>
     <tr><td width='150'>Heap(Free/Total/Max): </td><td><%=Runtime.getRuntime().freeMemory()/1024/1024%> MB / <%=Runtime.getRuntime().totalMemory()/1024/1024%> MB / <%=Runtime.getRuntime().maxMemory()/1024/1024%> MB</td>

http://git-wip-us.apache.org/repos/asf/tajo/blob/b4adc18c/tajo-core/src/test/java/org/apache/tajo/engine/query/TestCTASQuery.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestCTASQuery.java b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestCTASQuery.java
index 0e02079..0e89803 100644
--- a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestCTASQuery.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestCTASQuery.java
@@ -28,6 +28,7 @@ import org.apache.tajo.TajoConstants;
 import org.apache.tajo.TajoTestingCluster;
 import org.apache.tajo.catalog.CatalogService;
 import org.apache.tajo.catalog.CatalogUtil;
+import org.apache.tajo.util.CommonTestingUtil;
 import org.apache.tajo.util.KeyValueSet;
 import org.apache.tajo.catalog.TableDesc;
 import org.apache.tajo.catalog.partition.PartitionMethodDesc;

http://git-wip-us.apache.org/repos/asf/tajo/blob/b4adc18c/tajo-core/src/test/java/org/apache/tajo/engine/query/TestInsertQuery.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestInsertQuery.java b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestInsertQuery.java
index 9c97a55..117f186 100644
--- a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestInsertQuery.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestInsertQuery.java
@@ -29,6 +29,7 @@ import org.apache.tajo.QueryTestCaseBase;
 import org.apache.tajo.catalog.CatalogService;
 import org.apache.tajo.catalog.CatalogUtil;
 import org.apache.tajo.catalog.TableDesc;
+import org.apache.tajo.util.CommonTestingUtil;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
@@ -114,8 +115,19 @@ public class TestInsertQuery extends QueryTestCaseBase {
 
   @Test
   public final void testInsertIntoLocation() throws Exception {
+    Path dfsPath = new Path("/tajo-data/testInsertIntoLocation");
+    assertTestInsertIntoLocation(dfsPath);
+  }
+
+  @Test
+  public final void testInsertIntoLocationDifferentFSs() throws Exception {
+    Path localPath = CommonTestingUtil.getTestDir();
+    assertTestInsertIntoLocation(localPath);
+  }
+
+  public final void assertTestInsertIntoLocation(Path path) throws Exception {
     FileSystem fs = null;
-    Path path = new Path("/tajo-data/testInsertIntoLocation");
+
     try {
       executeString("insert into location '" + path + "' select l_orderkey, l_partkey, l_linenumber from default.lineitem").close();
 


[26/29] tajo git commit: TAJO-1231: Implicit table properties in session is not stored in table property.

Posted by hj...@apache.org.
TAJO-1231: Implicit table properties in session is not stored in table property.

Closes #287


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

Branch: refs/heads/hbase_storage
Commit: ab2efce8fb4015523a9a818f0e03fa57c6366585
Parents: 2a69bcc
Author: Hyunsik Choi <hy...@apache.org>
Authored: Fri Dec 5 11:43:15 2014 +0900
Committer: Hyunsik Choi <hy...@apache.org>
Committed: Fri Dec 5 16:36:12 2014 +0900

----------------------------------------------------------------------
 CHANGES                                         |  7 ++--
 .../org/apache/tajo/client/TestTajoClient.java  | 37 +++++++++++++++++---
 .../org/apache/tajo/plan/LogicalPlanner.java    | 20 ++++++++---
 .../org/apache/tajo/plan/util/PlannerUtil.java  | 17 +++++++++
 4 files changed, 70 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tajo/blob/ab2efce8/CHANGES
----------------------------------------------------------------------
diff --git a/CHANGES b/CHANGES
index 0c785ce..8ecd795 100644
--- a/CHANGES
+++ b/CHANGES
@@ -83,8 +83,11 @@ Release 0.9.1 - unreleased
 
   BUG FIXES
 
-    TAJO-1194: 'INSERT OVERWRITE .. SELECT' does not remove existing data when result is empty.
-    (jaehwa)
+    TAJO-1231: Implicit table properties in session are not stored in 
+    table property. (hyunsik)
+
+    TAJO-1194: 'INSERT OVERWRITE .. SELECT' does not remove existing data 
+    when result is empty. (jaehwa)
 
     TAJO-1191: Change DateDatum timezone to UTC. (Jaewoong Jung via hyunsik)
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/ab2efce8/tajo-core/src/test/java/org/apache/tajo/client/TestTajoClient.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/client/TestTajoClient.java b/tajo-core/src/test/java/org/apache/tajo/client/TestTajoClient.java
index 616799f..21dd7d5 100644
--- a/tajo-core/src/test/java/org/apache/tajo/client/TestTajoClient.java
+++ b/tajo-core/src/test/java/org/apache/tajo/client/TestTajoClient.java
@@ -680,7 +680,7 @@ public class TestTajoClient {
   }
 
   @Test
-  public void testSetCvsNull() throws Exception {
+  public void testNullCharSession() throws Exception {
     String sql =
         "select\n" +
             "  c_custkey,\n" +
@@ -692,17 +692,44 @@ public class TestTajoClient {
             "  c_custkey,\n" +
             "  orders.o_orderkey;\n";
 
-    TajoConf tajoConf = TpchTestBase.getInstance().getTestingCluster().getConfiguration();
-
     Map<String, String> variables = new HashMap<String, String>();
     variables.put(SessionVars.NULL_CHAR.keyname(), "\\\\T");
     client.updateSessionVariables(variables);
+    TajoResultSet resultDesc = (TajoResultSet)client.executeQueryAndGetResult(sql);
+    resultDesc.close();
+    assertNullCharSessionVar(resultDesc.getTableDesc());
+  }
+
+  @Test
+  public void testNullCharSessionInCTAS() throws Exception {
+    String sql =
+        "create table nullcharsession as select\n" +
+            "  c_custkey,\n" +
+            "  orders.o_orderkey,\n" +
+            "  orders.o_orderstatus \n" +
+            "from\n" +
+            "  orders full outer join customer on c_custkey = o_orderkey\n" +
+            "order by\n" +
+            "  c_custkey,\n" +
+            "  orders.o_orderkey;\n";
 
+    Map<String, String> variables = new HashMap<String, String>();
+    variables.put(SessionVars.NULL_CHAR.keyname(), "\\\\T");
+    client.updateSessionVariables(variables);
     TajoResultSet res = (TajoResultSet)client.executeQueryAndGetResult(sql);
+    res.close();
+
+    TableDesc resultDesc = client.getTableDesc("nullcharsession");
+    assertNullCharSessionVar(resultDesc);
+  }
+
+
+  public void assertNullCharSessionVar(TableDesc resultDesc) throws Exception {
+    TajoConf tajoConf = TpchTestBase.getInstance().getTestingCluster().getConfiguration();
 
-    assertEquals(res.getTableDesc().getMeta().getOption(StorageConstants.TEXT_NULL), "\\\\T");
+    assertEquals(resultDesc.getMeta().getOption(StorageConstants.TEXT_NULL), "\\\\T");
 
-    Path path = new Path(res.getTableDesc().getPath());
+    Path path = new Path(resultDesc.getPath());
     FileSystem fs = path.getFileSystem(tajoConf);
 
     FileStatus[] files = fs.listStatus(path);

http://git-wip-us.apache.org/repos/asf/tajo/blob/ab2efce8/tajo-plan/src/main/java/org/apache/tajo/plan/LogicalPlanner.java
----------------------------------------------------------------------
diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/LogicalPlanner.java b/tajo-plan/src/main/java/org/apache/tajo/plan/LogicalPlanner.java
index 3b1b4e3..69c0e4b 100644
--- a/tajo-plan/src/main/java/org/apache/tajo/plan/LogicalPlanner.java
+++ b/tajo-plan/src/main/java/org/apache/tajo/plan/LogicalPlanner.java
@@ -1706,13 +1706,25 @@ public class LogicalPlanner extends BaseAlgebraVisitor<LogicalPlanner.PlanContex
       createTableNode.setStorageType(CatalogProtos.StoreType.CSV);
     }
 
-    // Set default storage properties to be created.
-    KeyValueSet keyValueSet = CatalogUtil.newPhysicalProperties(createTableNode.getStorageType());
+
+
+    // Set default storage properties to table
+    KeyValueSet properties = CatalogUtil.newPhysicalProperties(createTableNode.getStorageType());
+
+    // Priority to apply table properties
+    // 1. Explicit table properties specified in WITH clause
+    // 2. Session variables
+
+    // Set session variables to properties
+    PlannerUtil.applySessionToTableProperties(context.queryContext, createTableNode.getStorageType(), properties);
+    // Set table properties specified in WITH clause
     if (expr.hasParams()) {
-      keyValueSet.putAll(expr.getParams());
+      properties.putAll(expr.getParams());
     }
 
-    createTableNode.setOptions(keyValueSet);
+    createTableNode.setOptions(properties);
+
+
 
     if (expr.hasPartition()) {
       if (expr.getPartitionMethod().getPartitionType().equals(PartitionType.COLUMN)) {

http://git-wip-us.apache.org/repos/asf/tajo/blob/ab2efce8/tajo-plan/src/main/java/org/apache/tajo/plan/util/PlannerUtil.java
----------------------------------------------------------------------
diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/util/PlannerUtil.java b/tajo-plan/src/main/java/org/apache/tajo/plan/util/PlannerUtil.java
index 6868b6c..c55c203 100644
--- a/tajo-plan/src/main/java/org/apache/tajo/plan/util/PlannerUtil.java
+++ b/tajo-plan/src/main/java/org/apache/tajo/plan/util/PlannerUtil.java
@@ -21,6 +21,8 @@ package org.apache.tajo.plan.util;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Sets;
+import org.apache.tajo.OverridableConf;
+import org.apache.tajo.SessionVars;
 import org.apache.tajo.algebra.*;
 import org.apache.tajo.annotation.Nullable;
 import org.apache.tajo.catalog.Column;
@@ -35,10 +37,15 @@ import org.apache.tajo.plan.logical.*;
 import org.apache.tajo.plan.visitor.BasicLogicalPlanVisitor;
 import org.apache.tajo.plan.visitor.ExplainLogicalPlanVisitor;
 import org.apache.tajo.plan.visitor.SimpleAlgebraVisitor;
+import org.apache.tajo.storage.StorageConstants;
+import org.apache.tajo.util.KeyValueSet;
 import org.apache.tajo.util.TUtil;
 
 import java.util.*;
 
+import static org.apache.tajo.catalog.proto.CatalogProtos.StoreType.CSV;
+import static org.apache.tajo.catalog.proto.CatalogProtos.StoreType.TEXTFILE;
+
 public class PlannerUtil {
 
   public static boolean checkIfDDLPlan(LogicalNode node) {
@@ -776,4 +783,14 @@ public class PlannerUtil {
 
     return explains.toString();
   }
+
+  public static void applySessionToTableProperties(OverridableConf sessionVars,
+                                                   CatalogProtos.StoreType storeType,
+                                                   KeyValueSet tableProperties) {
+    if (storeType == CSV || storeType == TEXTFILE) {
+      if (sessionVars.containsKey(SessionVars.NULL_CHAR)) {
+        tableProperties.set(StorageConstants.TEXT_NULL, sessionVars.get(SessionVars.NULL_CHAR));
+      }
+    }
+  }
 }


[14/29] tajo git commit: TAJO-1222: DelimitedTextFile should be tolerant against parsing errors.

Posted by hj...@apache.org.
TAJO-1222: DelimitedTextFile should be tolerant against parsing errors.

Closes #277


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

Branch: refs/heads/hbase_storage
Commit: f69938abecd3d53968e318d97aba53d9acd3de40
Parents: 5066ac3
Author: Hyunsik Choi <hy...@apache.org>
Authored: Wed Dec 3 15:44:05 2014 +0900
Committer: Hyunsik Choi <hy...@apache.org>
Committed: Wed Dec 3 15:44:05 2014 +0900

----------------------------------------------------------------------
 CHANGES                                         |   3 +
 .../apache/tajo/storage/StorageConstants.java   |  14 +-
 .../src/main/sphinx/table_management/csv.rst    |   5 +
 .../storage/FieldSerializerDeserializer.java    |   4 +-
 .../tajo/storage/json/JsonLineDeserializer.java |  13 +-
 .../tajo/storage/text/CSVLineDeserializer.java  |   2 +-
 .../tajo/storage/text/DelimitedTextFile.java    |  84 +++++++---
 .../tajo/storage/text/TextLineDeserializer.java |   2 +-
 .../tajo/storage/text/TextLineParsingError.java |  31 ++++
 .../tajo/storage/TestDelimitedTextFile.java     | 164 +++++++++++++++++++
 .../testErrorTolerance1.json                    |   6 +
 .../testErrorTolerance2.json                    |   4 +
 12 files changed, 303 insertions(+), 29 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tajo/blob/f69938ab/CHANGES
----------------------------------------------------------------------
diff --git a/CHANGES b/CHANGES
index c03b72b..c26b8a9 100644
--- a/CHANGES
+++ b/CHANGES
@@ -5,6 +5,9 @@ Release 0.9.1 - unreleased
 
   NEW FEATURES
 
+    TAJO-1222: DelimitedTextFile should be tolerant against parsing errors.
+    (hyunsik)
+
     TAJO-1026: Implement Query history persistency manager.(Hyoungjun Kim)
 
     TAJO-233: Support PostgreSQL CatalogStore. (Jihun Kang via hyunsik)

http://git-wip-us.apache.org/repos/asf/tajo/blob/f69938ab/tajo-common/src/main/java/org/apache/tajo/storage/StorageConstants.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/storage/StorageConstants.java b/tajo-common/src/main/java/org/apache/tajo/storage/StorageConstants.java
index a3d8de0..459c9c9 100644
--- a/tajo-common/src/main/java/org/apache/tajo/storage/StorageConstants.java
+++ b/tajo-common/src/main/java/org/apache/tajo/storage/StorageConstants.java
@@ -32,8 +32,20 @@ public class StorageConstants {
 
   public static final String TEXT_DELIMITER = "text.delimiter";
   public static final String TEXT_NULL = "text.null";
-  public static final String TEXT_SERDE_CLASS = "text.serde.class";
+  public static final String TEXT_SERDE_CLASS = "text.serde";
   public static final String DEFAULT_TEXT_SERDE_CLASS = "org.apache.tajo.storage.text.CSVLineSerDe";
+  /**
+   * It's the maximum number of parsing error torrence.
+   *
+   * <ul>
+   *   <li>If it is -1, it is always torrent against any parsing error.</li>
+   *   <li>If it is 0, it does not permit any parsing error.</li>
+   *   <li>If it is some positive integer (i.e., > 0), the given number of parsing errors in each
+   *       task will be permissible</li>
+   * </ul>
+   **/
+  public static final String TEXT_ERROR_TOLERANCE_MAXNUM = "text.error-tolerance.max-num";
+  public static final String DEFAULT_TEXT_ERROR_TOLERANCE_MAXNUM = "0";
 
   @Deprecated
   public static final String SEQUENCEFILE_DELIMITER = "sequencefile.delimiter";

http://git-wip-us.apache.org/repos/asf/tajo/blob/f69938ab/tajo-docs/src/main/sphinx/table_management/csv.rst
----------------------------------------------------------------------
diff --git a/tajo-docs/src/main/sphinx/table_management/csv.rst b/tajo-docs/src/main/sphinx/table_management/csv.rst
index 3aba2ba..71313d6 100644
--- a/tajo-docs/src/main/sphinx/table_management/csv.rst
+++ b/tajo-docs/src/main/sphinx/table_management/csv.rst
@@ -40,6 +40,11 @@ Now, the CSV storage format provides the following physical properties.
 * ``text.null``: NULL character. The default NULL character is an empty string ``''``. Hive's default NULL character is ``'\\N'``.
 * ``compression.codec``: Compression codec. You can enable compression feature and set specified compression algorithm. The compression algorithm used to compress files. The compression codec name should be the fully qualified class name inherited from `org.apache.hadoop.io.compress.CompressionCodec <https://hadoop.apache.org/docs/current/api/org/apache/hadoop/io/compress/CompressionCodec.html>`_. By default, compression is disabled.
 * ``csvfile.serde``: custom (De)serializer class. ``org.apache.tajo.storage.TextSerializerDeserializer`` is the default (De)serializer class.
+* ``text.error-tolerance.max-num``: the maximum number of permissible parsing errors. This value should be an integer value. By default, ``text.error-tolerance.max-num`` is ``0``. According to the value, parsing errors will be handled in different ways.
+
+  * If ``text.error-tolerance.max-num < 0``, all parsing errors are ignored.
+  * If ``text.error-tolerance.max-num == 0``, any parsing error is not allowed. If any error occurs, the query will be failed. (default)
+  * If ``text.error-tolerance.max-num > 0``, the given number of parsing errors in each task will be pemissible.
 
 The following example is to set a custom field delimiter, NULL character, and compression codec:
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/f69938ab/tajo-storage/src/main/java/org/apache/tajo/storage/FieldSerializerDeserializer.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/FieldSerializerDeserializer.java b/tajo-storage/src/main/java/org/apache/tajo/storage/FieldSerializerDeserializer.java
index 7df4584..0b3755d 100644
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/FieldSerializerDeserializer.java
+++ b/tajo-storage/src/main/java/org/apache/tajo/storage/FieldSerializerDeserializer.java
@@ -21,6 +21,7 @@ package org.apache.tajo.storage;
 import io.netty.buffer.ByteBuf;
 import org.apache.tajo.catalog.Column;
 import org.apache.tajo.datum.Datum;
+import org.apache.tajo.storage.text.TextLineParsingError;
 
 import java.io.IOException;
 import java.io.OutputStream;
@@ -30,6 +31,7 @@ public interface FieldSerializerDeserializer {
 
   public int serialize(OutputStream out, Datum datum, Column col, int columnIndex, byte[] nullChars) throws IOException;
 
-  public Datum deserialize(ByteBuf buf, Column col, int columnIndex, ByteBuf nullChars) throws IOException;
+  public Datum deserialize(ByteBuf buf, Column col, int columnIndex, ByteBuf nullChars)
+      throws IOException, TextLineParsingError;
 
 }

http://git-wip-us.apache.org/repos/asf/tajo/blob/f69938ab/tajo-storage/src/main/java/org/apache/tajo/storage/json/JsonLineDeserializer.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/json/JsonLineDeserializer.java b/tajo-storage/src/main/java/org/apache/tajo/storage/json/JsonLineDeserializer.java
index 37cd9f3..dfe36f6 100644
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/json/JsonLineDeserializer.java
+++ b/tajo-storage/src/main/java/org/apache/tajo/storage/json/JsonLineDeserializer.java
@@ -23,6 +23,7 @@ import io.netty.buffer.ByteBuf;
 import net.minidev.json.JSONArray;
 import net.minidev.json.JSONObject;
 import net.minidev.json.parser.JSONParser;
+import net.minidev.json.parser.ParseException;
 import org.apache.tajo.catalog.Schema;
 import org.apache.tajo.catalog.SchemaUtil;
 import org.apache.tajo.catalog.TableMeta;
@@ -30,9 +31,11 @@ import org.apache.tajo.common.TajoDataTypes.Type;
 import org.apache.tajo.common.exception.NotImplementedException;
 import org.apache.tajo.datum.DatumFactory;
 import org.apache.tajo.datum.NullDatum;
+import org.apache.tajo.datum.TextDatum;
 import org.apache.tajo.datum.protobuf.ProtobufJsonFormat;
 import org.apache.tajo.storage.Tuple;
 import org.apache.tajo.storage.text.TextLineDeserializer;
+import org.apache.tajo.storage.text.TextLineParsingError;
 
 import java.io.IOException;
 import java.util.Iterator;
@@ -55,7 +58,7 @@ public class JsonLineDeserializer extends TextLineDeserializer {
   }
 
   @Override
-  public void deserialize(ByteBuf buf, Tuple output) throws IOException {
+  public void deserialize(ByteBuf buf, Tuple output) throws IOException, TextLineParsingError {
     byte [] line = new byte[buf.readableBytes()];
     buf.readBytes(line);
 
@@ -170,8 +173,9 @@ public class JsonLineDeserializer extends TextLineDeserializer {
           if (jsonObject == null) {
             output.put(actualIdx, NullDatum.get());
             break;
-          } if (jsonObject instanceof String) {
-            output.put(actualIdx, DatumFactory.createBlob((String)jsonObject));
+          }
+          if (jsonObject instanceof String) {
+            output.put(actualIdx, DatumFactory.createBlob((String) jsonObject));
           } else if (jsonObject instanceof JSONArray) {
             JSONArray jsonArray = (JSONArray) jsonObject;
             byte[] bytes = new byte[jsonArray.size()];
@@ -208,7 +212,8 @@ public class JsonLineDeserializer extends TextLineDeserializer {
           throw new NotImplementedException(types[actualIdx].name() + " is not supported.");
         }
       }
-
+    } catch (ParseException pe) {
+      throw new TextLineParsingError(new String(line, TextDatum.DEFAULT_CHARSET), pe);
     } catch (Throwable e) {
       throw new IOException(e);
     }

http://git-wip-us.apache.org/repos/asf/tajo/blob/f69938ab/tajo-storage/src/main/java/org/apache/tajo/storage/text/CSVLineDeserializer.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/text/CSVLineDeserializer.java b/tajo-storage/src/main/java/org/apache/tajo/storage/text/CSVLineDeserializer.java
index 0e2dfb0..f2eebc6 100644
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/text/CSVLineDeserializer.java
+++ b/tajo-storage/src/main/java/org/apache/tajo/storage/text/CSVLineDeserializer.java
@@ -48,7 +48,7 @@ public class CSVLineDeserializer extends TextLineDeserializer {
     fieldSerDer = new TextFieldSerializerDeserializer();
   }
 
-  public void deserialize(final ByteBuf lineBuf, Tuple output) throws IOException {
+  public void deserialize(final ByteBuf lineBuf, Tuple output) throws IOException, TextLineParsingError {
     int[] projection = targetColumnIndexes;
     if (lineBuf == null || targetColumnIndexes == null || targetColumnIndexes.length == 0) {
       return;

http://git-wip-us.apache.org/repos/asf/tajo/blob/f69938ab/tajo-storage/src/main/java/org/apache/tajo/storage/text/DelimitedTextFile.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/text/DelimitedTextFile.java b/tajo-storage/src/main/java/org/apache/tajo/storage/text/DelimitedTextFile.java
index 2218fae..c54131b 100644
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/text/DelimitedTextFile.java
+++ b/tajo-storage/src/main/java/org/apache/tajo/storage/text/DelimitedTextFile.java
@@ -48,6 +48,9 @@ import java.util.Arrays;
 import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
 
+import static org.apache.tajo.storage.StorageConstants.DEFAULT_TEXT_ERROR_TOLERANCE_MAXNUM;
+import static org.apache.tajo.storage.StorageConstants.TEXT_ERROR_TOLERANCE_MAXNUM;
+
 public class DelimitedTextFile {
 
   public static final byte LF = '\n';
@@ -267,12 +270,19 @@ public class DelimitedTextFile {
     private final long startOffset;
 
     private final long endOffset;
+    /** The number of actual read records */
     private int recordCount = 0;
     private int[] targetColumnIndexes;
 
     private DelimitedLineReader reader;
     private TextLineDeserializer deserializer;
 
+    private int errorPrintOutMaxNum = 5;
+    /** Maximum number of permissible errors */
+    private int errorTorrenceMaxNum;
+    /** How many errors have occurred? */
+    private int errorNum;
+
     public DelimitedTextFileScanner(Configuration conf, final Schema schema, final TableMeta meta,
                                     final FileFragment fragment)
         throws IOException {
@@ -284,10 +294,9 @@ public class DelimitedTextFile {
 
       startOffset = fragment.getStartKey();
       endOffset = startOffset + fragment.getEndKey();
-    }
 
-    public TextLineSerDe getLineSerde() {
-      return DelimitedTextFile.getLineSerde(meta);
+      errorTorrenceMaxNum =
+          Integer.parseInt(meta.getOption(TEXT_ERROR_TOLERANCE_MAXNUM, DEFAULT_TEXT_ERROR_TOLERANCE_MAXNUM));
     }
 
     @Override
@@ -295,6 +304,7 @@ public class DelimitedTextFile {
       if (reader != null) {
         reader.close();
       }
+
       reader = new DelimitedLineReader(conf, fragment);
       reader.init();
       recordCount = 0;
@@ -322,15 +332,8 @@ public class DelimitedTextFile {
       deserializer.init();
     }
 
-    public ByteBuf readLine() throws IOException {
-      ByteBuf buf = reader.readLine();
-      if (buf == null) {
-        return null;
-      } else {
-        recordCount++;
-      }
-
-      return buf;
+    public TextLineSerDe getLineSerde() {
+      return DelimitedTextFile.getLineSerde(meta);
     }
 
     @Override
@@ -355,21 +358,60 @@ public class DelimitedTextFile {
 
     @Override
     public Tuple next() throws IOException {
+
+      if (!reader.isReadable()) {
+        return null;
+      }
+
+      if (targets.length == 0) {
+        return EmptyTuple.get();
+      }
+
+      VTuple tuple = new VTuple(schema.size());
+
       try {
-        if (!reader.isReadable()) return null;
 
-        ByteBuf buf = readLine();
-        if (buf == null) return null;
+        // this loop will continue until one tuple is build or EOS (end of stream).
+        do {
 
-        if (targets.length == 0) {
-          return EmptyTuple.get();
-        }
+          ByteBuf buf = reader.readLine();
+          if (buf == null) {
+            return null;
+          }
+
+          try {
+
+            deserializer.deserialize(buf, tuple);
+            // if a line is read normaly, it exists this loop.
+            break;
+
+          } catch (TextLineParsingError tae) {
+
+            errorNum++;
+
+            // suppress too many log prints, which probably cause performance degradation
+            if (errorNum < errorPrintOutMaxNum) {
+              LOG.warn("Ignore JSON Parse Error (" + errorNum + "): ", tae);
+            }
+
+            // Only when the maximum error torrence limit is set (i.e., errorTorrenceMaxNum >= 0),
+            // it checks if the number of parsing error exceeds the max limit.
+            // Otherwise, it will ignore all parsing errors.
+            if (errorTorrenceMaxNum >= 0 && errorNum > errorTorrenceMaxNum) {
+              throw tae;
+            }
+            continue;
+          }
+
+        } while (reader.isReadable()); // continue until EOS
+
+        // recordCount means the number of actual read records. We increment the count here.
+        recordCount++;
 
-        VTuple tuple = new VTuple(schema.size());
-        deserializer.deserialize(buf, tuple);
         return tuple;
+
       } catch (Throwable t) {
-        LOG.error("Tuple list current index: " + recordCount + " file offset:" + reader.getCompressedPosition(), t);
+        LOG.error(t);
         throw new IOException(t);
       }
     }

http://git-wip-us.apache.org/repos/asf/tajo/blob/f69938ab/tajo-storage/src/main/java/org/apache/tajo/storage/text/TextLineDeserializer.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/text/TextLineDeserializer.java b/tajo-storage/src/main/java/org/apache/tajo/storage/text/TextLineDeserializer.java
index b0d3c3a..7ebfa79 100644
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/text/TextLineDeserializer.java
+++ b/tajo-storage/src/main/java/org/apache/tajo/storage/text/TextLineDeserializer.java
@@ -51,7 +51,7 @@ public abstract class TextLineDeserializer {
    * @param output Tuple to be filled with read fields
    * @throws java.io.IOException
    */
-  public abstract void deserialize(final ByteBuf buf, Tuple output) throws IOException;
+  public abstract void deserialize(final ByteBuf buf, Tuple output) throws IOException, TextLineParsingError;
 
   /**
    * Release external resources

http://git-wip-us.apache.org/repos/asf/tajo/blob/f69938ab/tajo-storage/src/main/java/org/apache/tajo/storage/text/TextLineParsingError.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/text/TextLineParsingError.java b/tajo-storage/src/main/java/org/apache/tajo/storage/text/TextLineParsingError.java
new file mode 100644
index 0000000..f0bae5e
--- /dev/null
+++ b/tajo-storage/src/main/java/org/apache/tajo/storage/text/TextLineParsingError.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.text;
+
+public class TextLineParsingError extends Exception {
+
+  public TextLineParsingError(Throwable t) {
+    super(t);
+  }
+
+  public TextLineParsingError(String message, Throwable t) {
+    super(t.getMessage() + ", Error line: " + message);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/f69938ab/tajo-storage/src/test/java/org/apache/tajo/storage/TestDelimitedTextFile.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/test/java/org/apache/tajo/storage/TestDelimitedTextFile.java b/tajo-storage/src/test/java/org/apache/tajo/storage/TestDelimitedTextFile.java
new file mode 100644
index 0000000..93fb12b
--- /dev/null
+++ b/tajo-storage/src/test/java/org/apache/tajo/storage/TestDelimitedTextFile.java
@@ -0,0 +1,164 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.tajo.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.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.fragment.FileFragment;
+import org.apache.tajo.util.FileUtil;
+import org.junit.Test;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.URL;
+
+import static org.junit.Assert.*;
+
+public class TestDelimitedTextFile {
+
+  private static Schema schema = new Schema();
+
+  private static Tuple baseTuple = new VTuple(10);
+
+  static {
+    schema.addColumn("col1", Type.BOOLEAN);
+    schema.addColumn("col2", Type.CHAR, 7);
+    schema.addColumn("col3", Type.INT2);
+    schema.addColumn("col4", Type.INT4);
+    schema.addColumn("col5", Type.INT8);
+    schema.addColumn("col6", Type.FLOAT4);
+    schema.addColumn("col7", Type.FLOAT8);
+    schema.addColumn("col8", Type.TEXT);
+    schema.addColumn("col9", Type.BLOB);
+    schema.addColumn("col10", Type.INET4);
+
+    baseTuple.put(new Datum[] {
+        DatumFactory.createBool(true),                // 0
+        DatumFactory.createChar("hyunsik"),           // 1
+        DatumFactory.createInt2((short) 17),          // 2
+        DatumFactory.createInt4(59),                  // 3
+        DatumFactory.createInt8(23l),                 // 4
+        DatumFactory.createFloat4(77.9f),             // 5
+        DatumFactory.createFloat8(271.9d),            // 6
+        DatumFactory.createText("hyunsik"),           // 7
+        DatumFactory.createBlob("hyunsik".getBytes()),// 8
+        DatumFactory.createInet4("192.168.0.1"),      // 9
+    });
+  }
+
+  public static Path getResourcePath(String path, String suffix) {
+    URL resultBaseURL = ClassLoader.getSystemResource(path);
+    return new Path(resultBaseURL.toString(), suffix);
+  }
+
+  public static Path getResultPath(Class clazz, String fileName) {
+    return new Path (getResourcePath("results", clazz.getSimpleName()), fileName);
+  }
+
+  public static String getResultText(Class clazz, String fileName) throws IOException {
+    FileSystem localFS = FileSystem.getLocal(new Configuration());
+    Path path = getResultPath(clazz, fileName);
+    Preconditions.checkState(localFS.exists(path) && localFS.isFile(path));
+    return FileUtil.readTextFile(new File(path.toUri()));
+  }
+
+  private static final FileFragment getFileFragment(String fileName) throws IOException {
+    TajoConf conf = new TajoConf();
+    Path tablePath = new Path(getResourcePath("dataset", "TestDelimitedTextFile"), fileName);
+    FileSystem fs = FileSystem.getLocal(conf);
+    FileStatus status = fs.getFileStatus(tablePath);
+    return new FileFragment("table", tablePath, 0, status.getLen());
+  }
+
+  @Test
+  public void testIgnoreAllErrors() throws IOException {
+    TajoConf conf = new TajoConf();
+
+    TableMeta meta = CatalogUtil.newTableMeta(CatalogProtos.StoreType.JSON);
+    meta.putOption(StorageUtil.TEXT_ERROR_TOLERANCE_MAXNUM, "-1");
+    FileFragment fragment =  getFileFragment("testErrorTolerance1.json");
+    Scanner scanner =  StorageManager.getStorageManager(conf).getScanner(meta, schema, fragment);
+    scanner.init();
+
+    Tuple tuple;
+    int i = 0;
+    while ((tuple = scanner.next()) != null) {
+      assertEquals(baseTuple, tuple);
+      i++;
+    }
+    assertEquals(3, i);
+    scanner.close();
+  }
+
+  @Test
+  public void testIgnoreOneErrorTolerance() throws IOException {
+
+
+    TajoConf conf = new TajoConf();
+
+    TableMeta meta = CatalogUtil.newTableMeta(CatalogProtos.StoreType.JSON);
+    meta.putOption(StorageUtil.TEXT_ERROR_TOLERANCE_MAXNUM, "1");
+    FileFragment fragment =  getFileFragment("testErrorTolerance1.json");
+    Scanner scanner =  StorageManager.getStorageManager(conf).getScanner(meta, schema, fragment);
+    scanner.init();
+
+    assertNotNull(scanner.next());
+    assertNotNull(scanner.next());
+    try {
+      scanner.next();
+    } catch (IOException ioe) {
+      System.out.println(ioe);
+      return;
+    } finally {
+      scanner.close();
+    }
+    fail();
+  }
+
+  @Test
+  public void testNoErrorTolerance() throws IOException {
+    TajoConf conf = new TajoConf();
+    TableMeta meta = CatalogUtil.newTableMeta(CatalogProtos.StoreType.JSON);
+    meta.putOption(StorageUtil.TEXT_ERROR_TOLERANCE_MAXNUM, "0");
+    FileFragment fragment =  getFileFragment("testErrorTolerance2.json");
+    Scanner scanner =  StorageManager.getStorageManager(conf).getScanner(meta, schema, fragment);
+    scanner.init();
+
+    try {
+      scanner.next();
+    } catch (IOException ioe) {
+      return;
+    } finally {
+      scanner.close();
+    }
+    fail();
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/f69938ab/tajo-storage/src/test/resources/dataset/TestDelimitedTextFile/testErrorTolerance1.json
----------------------------------------------------------------------
diff --git a/tajo-storage/src/test/resources/dataset/TestDelimitedTextFile/testErrorTolerance1.json b/tajo-storage/src/test/resources/dataset/TestDelimitedTextFile/testErrorTolerance1.json
new file mode 100644
index 0000000..739dfe7
--- /dev/null
+++ b/tajo-storage/src/test/resources/dataset/TestDelimitedTextFile/testErrorTolerance1.json
@@ -0,0 +1,6 @@
+{"col1": "true", "col2": "hyunsik", "col3": 17, "col4": 59, "col5": 23, "col6": 77.9, "col7": 271.9, "col8": "hyunsik", "col9": "hyunsik", "col10": "192.168.0.1"}
+{"col1": "true", "col2": "hyunsik", "col3": 17, "col4": 59, "col5": 23, "col6": 77.9, "col7": 271.9, "col8": "hyunsik", "col9": "hyunsik", "col10": "192.168.0.1"
+{"col1": "true", "col2": "hyunsik", "col3": 17, "col4": 59, "col5": 23, "col6": 77.9, "col7": 271.9, "col8": "hyunsik", "col9": "hyunsik", "col10": "192.168.0.1"}
+{"col1": "true", "col2": "hyunsik", "col3": 17, "col4": 59, "col5": 23, "col6": 77.9, "col7": 271.9, "col8": "hyunsik", "col9": "hyunsik", "col10": "192.168.0.1"
+{"col1": "true", "col2": "hyunsik", "col3": 17, "col4": 59, "col5": 23, "col6": 77.9, "col7": 271.9, "col8": "hyunsik", "col9": "hyunsik", "col10": "192.168.0.1"
+{"col1": "true", "col2": "hyunsik", "col3": 17, "col4": 59, "col5": 23, "col6": 77.9, "col7": 271.9, "col8": "hyunsik", "col9": "hyunsik", "col10": "192.168.0.1"}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/f69938ab/tajo-storage/src/test/resources/dataset/TestDelimitedTextFile/testErrorTolerance2.json
----------------------------------------------------------------------
diff --git a/tajo-storage/src/test/resources/dataset/TestDelimitedTextFile/testErrorTolerance2.json b/tajo-storage/src/test/resources/dataset/TestDelimitedTextFile/testErrorTolerance2.json
new file mode 100644
index 0000000..8256b72
--- /dev/null
+++ b/tajo-storage/src/test/resources/dataset/TestDelimitedTextFile/testErrorTolerance2.json
@@ -0,0 +1,4 @@
+{"col1": "true", "col2": "hyunsik", "col3": 17, "col4": 59, "col5": 23, "col6": 77.9, "col7": 271.9, "col8": "hyunsik", "col9": "hyunsik", "col10": "192.168.0.1"
+{"col1": "true", "col2": "hyunsik", "col3": 17, "col4": 59, "col5": 23, "col6": 77.9, "col7": 271.9, "col8": "hyunsik", "col9": "hyunsik", "col10": "192.168.0.1"}
+{"col1": "true", "col2": "hyunsik", "col3": 17, "col4": 59, "col5": 23, "col6": 77.9, "col7": 271.9, "col8": "hyunsik", "col9": "hyunsik", "col10": "192.168.0.1"}
+{"col1": "true", "col2": "hyunsik", "col3": 17, "col4": 59, "col5": 23, "col6": 77.9, "col7": 271.9, "col8": "hyunsik", "col9": "hyunsik", "col10": "192.168.0.1"}
\ No newline at end of file


[05/29] tajo git commit: TAJO-1209: Pluggable line (de)serializer for DelimitedTextFile.

Posted by hj...@apache.org.
TAJO-1209: Pluggable line (de)serializer for DelimitedTextFile.

Closes #1209


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

Branch: refs/heads/hbase_storage
Commit: 72dd29c520981a3ffaac2150ee7306ca41192893
Parents: 3ae44b1
Author: Hyunsik Choi <hy...@apache.org>
Authored: Thu Nov 27 19:46:58 2014 +0900
Committer: Hyunsik Choi <hy...@apache.org>
Committed: Thu Nov 27 19:46:58 2014 +0900

----------------------------------------------------------------------
 CHANGES                                         |   3 +
 .../apache/tajo/storage/StorageConstants.java   |   3 +-
 .../org/apache/tajo/util/ReflectionUtil.java    |   4 +-
 .../tajo/storage/text/CSVLineDeserializer.java  |  96 +++++++++++
 .../apache/tajo/storage/text/CSVLineSerDe.java  |  45 +++++
 .../tajo/storage/text/CSVLineSerializer.java    |  68 ++++++++
 .../tajo/storage/text/DelimitedTextFile.java    | 163 +++++++------------
 .../tajo/storage/text/TextLineDeserializer.java |  60 +++++++
 .../apache/tajo/storage/text/TextLineSerDe.java |  65 ++++++++
 .../tajo/storage/text/TextLineSerializer.java   |  45 +++++
 10 files changed, 449 insertions(+), 103 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tajo/blob/72dd29c5/CHANGES
----------------------------------------------------------------------
diff --git a/CHANGES b/CHANGES
index f2141a6..6f38f65 100644
--- a/CHANGES
+++ b/CHANGES
@@ -67,6 +67,9 @@ Release 0.9.1 - unreleased
     TAJO-1188: Fix testcase testTimestampConstructor in TestTimestampDatum.
     (DaeMyung Kang via hyunsik)
 
+    TAJO-1209: Pluggable line (de)serializer for DelimitedTextFile.
+    (hyunsik)
+
 
   BUG FIXES
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/72dd29c5/tajo-common/src/main/java/org/apache/tajo/storage/StorageConstants.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/storage/StorageConstants.java b/tajo-common/src/main/java/org/apache/tajo/storage/StorageConstants.java
index 11ac9b7..3065d31 100644
--- a/tajo-common/src/main/java/org/apache/tajo/storage/StorageConstants.java
+++ b/tajo-common/src/main/java/org/apache/tajo/storage/StorageConstants.java
@@ -32,7 +32,8 @@ public class StorageConstants {
 
   public static final String TEXT_DELIMITER = "text.delimiter";
   public static final String TEXT_NULL = "text.null";
-  public static final String TEXTFILE_SERDE = "textfile.serde";
+  public static final String TEXT_SERDE_CLASS = "text.serde.class";
+  public static final String DEFAULT_TEXT_SERDE_CLASS = "org.apache.tajo.storage.text.CSVLineSerde";
 
   @Deprecated
   public static final String SEQUENCEFILE_DELIMITER = "sequencefile.delimiter";

http://git-wip-us.apache.org/repos/asf/tajo/blob/72dd29c5/tajo-common/src/main/java/org/apache/tajo/util/ReflectionUtil.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/util/ReflectionUtil.java b/tajo-common/src/main/java/org/apache/tajo/util/ReflectionUtil.java
index 410815f..eccc61f 100644
--- a/tajo-common/src/main/java/org/apache/tajo/util/ReflectionUtil.java
+++ b/tajo-common/src/main/java/org/apache/tajo/util/ReflectionUtil.java
@@ -32,8 +32,8 @@ public class ReflectionUtil {
   private static final Map<Class<?>, Constructor<?>> CONSTRUCTOR_CACHE =
       new ConcurrentHashMap<Class<?>, Constructor<?>>();
 
-	public static Object newInstance(Class<?> clazz) 
-			throws InstantiationException, IllegalAccessException {         
+	public static Object newInstance(Class<?> clazz)
+			throws InstantiationException, IllegalAccessException {
 		return clazz.newInstance();
 	}
 }

http://git-wip-us.apache.org/repos/asf/tajo/blob/72dd29c5/tajo-storage/src/main/java/org/apache/tajo/storage/text/CSVLineDeserializer.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/text/CSVLineDeserializer.java b/tajo-storage/src/main/java/org/apache/tajo/storage/text/CSVLineDeserializer.java
new file mode 100644
index 0000000..f580da1
--- /dev/null
+++ b/tajo-storage/src/main/java/org/apache/tajo/storage/text/CSVLineDeserializer.java
@@ -0,0 +1,96 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.storage.text;
+
+import io.netty.buffer.ByteBuf;
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.catalog.TableMeta;
+import org.apache.tajo.datum.Datum;
+import org.apache.tajo.storage.FieldSerializerDeserializer;
+import org.apache.tajo.storage.Tuple;
+
+import java.io.IOException;
+
+public class CSVLineDeserializer extends TextLineDeserializer {
+  private FieldSplitProcessor processor;
+  private FieldSerializerDeserializer fieldSerDer;
+  private ByteBuf nullChars;
+
+  public CSVLineDeserializer(Schema schema, TableMeta meta, int[] targetColumnIndexes) {
+    super(schema, meta, targetColumnIndexes);
+  }
+
+  @Override
+  public void init() {
+    this.processor = new FieldSplitProcessor(CSVLineSerDe.getFieldDelimiter(meta));
+
+    if (nullChars != null) {
+      nullChars.release();
+    }
+    nullChars = TextLineSerDe.getNullChars(meta);
+
+    fieldSerDer = new TextFieldSerializerDeserializer();
+  }
+
+  public void deserialize(final ByteBuf lineBuf, Tuple tuple) throws IOException {
+    int[] projection = targetColumnIndexes;
+    if (lineBuf == null || targetColumnIndexes == null || targetColumnIndexes.length == 0) {
+      return;
+    }
+
+    final int rowLength = lineBuf.readableBytes();
+    int start = 0, fieldLength = 0, end = 0;
+
+    //Projection
+    int currentTarget = 0;
+    int currentIndex = 0;
+
+    while (end != -1) {
+      end = lineBuf.forEachByte(start, rowLength - start, processor);
+
+      if (end < 0) {
+        fieldLength = rowLength - start;
+      } else {
+        fieldLength = end - start;
+      }
+
+      if (projection.length > currentTarget && currentIndex == projection[currentTarget]) {
+        lineBuf.setIndex(start, start + fieldLength);
+        Datum datum = fieldSerDer.deserialize(lineBuf, schema.getColumn(currentIndex), currentIndex, nullChars);
+        tuple.put(currentIndex, datum);
+        currentTarget++;
+      }
+
+      if (projection.length == currentTarget) {
+        break;
+      }
+
+      start = end + 1;
+      currentIndex++;
+    }
+  }
+
+  @Override
+  public void release() {
+    if (nullChars != null) {
+      nullChars.release();
+      nullChars = null;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/72dd29c5/tajo-storage/src/main/java/org/apache/tajo/storage/text/CSVLineSerDe.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/text/CSVLineSerDe.java b/tajo-storage/src/main/java/org/apache/tajo/storage/text/CSVLineSerDe.java
new file mode 100644
index 0000000..e2686a6
--- /dev/null
+++ b/tajo-storage/src/main/java/org/apache/tajo/storage/text/CSVLineSerDe.java
@@ -0,0 +1,45 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.storage.text;
+
+import org.apache.commons.lang.StringEscapeUtils;
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.catalog.TableMeta;
+import org.apache.tajo.storage.StorageConstants;
+
+public class CSVLineSerDe extends TextLineSerDe {
+
+  public CSVLineSerDe() {
+  }
+
+  @Override
+  public TextLineDeserializer createDeserializer(Schema schema, TableMeta meta, int[] targetColumnIndexes) {
+    return new CSVLineDeserializer(schema, meta, targetColumnIndexes);
+  }
+
+  @Override
+  public TextLineSerializer createSerializer(Schema schema, TableMeta meta) {
+    return new CSVLineSerializer(schema, meta);
+  }
+
+  public static char getFieldDelimiter(TableMeta meta) {
+    return StringEscapeUtils.unescapeJava(meta.getOption(StorageConstants.TEXT_DELIMITER,
+        StorageConstants.DEFAULT_FIELD_DELIMITER)).charAt(0);
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/72dd29c5/tajo-storage/src/main/java/org/apache/tajo/storage/text/CSVLineSerializer.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/text/CSVLineSerializer.java b/tajo-storage/src/main/java/org/apache/tajo/storage/text/CSVLineSerializer.java
new file mode 100644
index 0000000..684519c
--- /dev/null
+++ b/tajo-storage/src/main/java/org/apache/tajo/storage/text/CSVLineSerializer.java
@@ -0,0 +1,68 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.storage.text;
+
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.catalog.TableMeta;
+import org.apache.tajo.datum.Datum;
+import org.apache.tajo.storage.FieldSerializerDeserializer;
+import org.apache.tajo.storage.Tuple;
+
+import java.io.IOException;
+import java.io.OutputStream;
+
+public class CSVLineSerializer extends TextLineSerializer {
+  private FieldSerializerDeserializer serde;
+
+  private byte [] nullChars;
+  private char delimiter;
+
+  public CSVLineSerializer(Schema schema, TableMeta meta) {
+    super(schema, meta);
+  }
+
+  @Override
+  public void init() {
+    nullChars = TextLineSerDe.getNullCharsAsBytes(meta);
+    delimiter = CSVLineSerDe.getFieldDelimiter(meta);
+
+    serde = new TextFieldSerializerDeserializer();
+  }
+
+  @Override
+  public int serialize(OutputStream out, Tuple input) throws IOException {
+    int rowBytes = 0;
+
+    for (int i = 0; i < schema.size(); i++) {
+      Datum datum = input.get(i);
+      rowBytes += serde.serialize(out, datum, schema.getColumn(i), i, nullChars);
+
+      if (schema.size() - 1 > i) {
+        out.write((byte) delimiter);
+        rowBytes += 1;
+      }
+    }
+
+    return rowBytes;
+  }
+
+  @Override
+  public void release() {
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/72dd29c5/tajo-storage/src/main/java/org/apache/tajo/storage/text/DelimitedTextFile.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/text/DelimitedTextFile.java b/tajo-storage/src/main/java/org/apache/tajo/storage/text/DelimitedTextFile.java
index 68d89e7..d15f394 100644
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/text/DelimitedTextFile.java
+++ b/tajo-storage/src/main/java/org/apache/tajo/storage/text/DelimitedTextFile.java
@@ -19,8 +19,6 @@
 package org.apache.tajo.storage.text;
 
 import io.netty.buffer.ByteBuf;
-import org.apache.commons.lang.StringEscapeUtils;
-import org.apache.commons.lang.StringUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -35,19 +33,20 @@ import org.apache.hadoop.io.compress.Compressor;
 import org.apache.tajo.catalog.Schema;
 import org.apache.tajo.catalog.TableMeta;
 import org.apache.tajo.catalog.statistics.TableStats;
-import org.apache.tajo.datum.Datum;
-import org.apache.tajo.datum.NullDatum;
 import org.apache.tajo.storage.*;
 import org.apache.tajo.storage.compress.CodecPool;
 import org.apache.tajo.storage.exception.AlreadyExistsStorageException;
 import org.apache.tajo.storage.fragment.FileFragment;
 import org.apache.tajo.storage.rcfile.NonSyncByteArrayOutputStream;
+import org.apache.tajo.util.ReflectionUtil;
 
 import java.io.BufferedOutputStream;
 import java.io.DataOutputStream;
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.util.Arrays;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
 
 public class DelimitedTextFile {
 
@@ -56,15 +55,48 @@ public class DelimitedTextFile {
 
   private static final Log LOG = LogFactory.getLog(DelimitedTextFile.class);
 
+  /** it caches line serde classes. */
+  private static final Map<String, Class<? extends TextLineSerDe>> serdeClassCache =
+      new ConcurrentHashMap<String, Class<? extends TextLineSerDe>>();
+
+  /**
+   * By default, DelimitedTextFileScanner uses CSVLineSerder. If a table property 'text.serde.class' is given,
+   * it will use the specified serder class.
+   *
+   * @return TextLineSerder
+   */
+  public static TextLineSerDe getLineSerde(TableMeta meta) {
+    TextLineSerDe lineSerder;
+
+    String serDeClassName;
+
+    // if there is no given serde class, it will use CSV line serder.
+    serDeClassName = meta.getOption(StorageConstants.TEXT_SERDE_CLASS, StorageConstants.DEFAULT_TEXT_SERDE_CLASS);
+
+    try {
+      Class<? extends TextLineSerDe> serdeClass;
+
+      if (serdeClassCache.containsKey(serDeClassName)) {
+        serdeClass = serdeClassCache.get(serDeClassName);
+      } else {
+        serdeClass = (Class<? extends TextLineSerDe>) Class.forName(CSVLineSerDe.class.getName());
+        serdeClassCache.put(serDeClassName, serdeClass);
+      }
+      lineSerder = (TextLineSerDe) ReflectionUtil.newInstance(serdeClass);
+    } catch (Throwable e) {
+      throw new RuntimeException("TextLineSerde class cannot be initialized");
+    }
+
+    return lineSerder;
+  }
+
   public static class DelimitedTextFileAppender extends FileAppender {
     private final TableMeta meta;
     private final Schema schema;
-    private final int columnNum;
     private final FileSystem fs;
     private FSDataOutputStream fos;
     private DataOutputStream outputStream;
     private CompressionOutputStream deflateFilter;
-    private char delimiter;
     private TableStatistics stats = null;
     private Compressor compressor;
     private CompressionCodecFactory codecFactory;
@@ -76,7 +108,7 @@ public class DelimitedTextFile {
     private long pos = 0;
 
     private NonSyncByteArrayOutputStream os;
-    private FieldSerializerDeserializer serde;
+    private TextLineSerializer serializer;
 
     public DelimitedTextFileAppender(Configuration conf, final Schema schema, final TableMeta meta, final Path path)
         throws IOException {
@@ -84,17 +116,10 @@ public class DelimitedTextFile {
       this.fs = path.getFileSystem(conf);
       this.meta = meta;
       this.schema = schema;
-      this.delimiter = StringEscapeUtils.unescapeJava(this.meta.getOption(StorageConstants.TEXT_DELIMITER,
-          StorageConstants.DEFAULT_FIELD_DELIMITER)).charAt(0);
-      this.columnNum = schema.size();
-
-      String nullCharacters = StringEscapeUtils.unescapeJava(this.meta.getOption(StorageConstants.TEXT_NULL,
-          NullDatum.DEFAULT_TEXT));
-      if (StringUtils.isEmpty(nullCharacters)) {
-        nullChars = NullDatum.get().asTextBytes();
-      } else {
-        nullChars = nullCharacters.getBytes();
-      }
+    }
+
+    public TextLineSerDe getLineSerde() {
+      return DelimitedTextFile.getLineSerde(meta);
     }
 
     @Override
@@ -133,7 +158,8 @@ public class DelimitedTextFile {
         this.stats = new TableStatistics(this.schema);
       }
 
-      serde = new TextFieldSerializerDeserializer();
+      serializer = getLineSerde().createSerializer(schema, meta);
+      serializer.init();
 
       if (os == null) {
         os = new NonSyncByteArrayOutputStream(BUFFER_SIZE);
@@ -145,26 +171,20 @@ public class DelimitedTextFile {
       super.init();
     }
 
-
     @Override
     public void addTuple(Tuple tuple) throws IOException {
-      Datum datum;
-      int rowBytes = 0;
+      // write
+      int rowBytes = serializer.serialize(os, tuple);
 
-      for (int i = 0; i < columnNum; i++) {
-        datum = tuple.get(i);
-        rowBytes += serde.serialize(os, datum, schema.getColumn(i), i, nullChars);
-
-        if (columnNum - 1 > i) {
-          os.write((byte) delimiter);
-          rowBytes += 1;
-        }
-      }
+      // new line
       os.write(LF);
       rowBytes += 1;
 
+      // update positions
       pos += rowBytes;
       bufferedBytes += rowBytes;
+
+      // refill buffer if necessary
       if (bufferedBytes > BUFFER_SIZE) {
         flushBuffer();
       }
@@ -197,6 +217,8 @@ public class DelimitedTextFile {
     public void close() throws IOException {
 
       try {
+        serializer.release();
+
         if(outputStream != null){
           flush();
         }
@@ -241,18 +263,15 @@ public class DelimitedTextFile {
   }
 
   public static class DelimitedTextFileScanner extends FileScanner {
-
     private boolean splittable = false;
     private final long startOffset;
-    private final long endOffset;
 
+    private final long endOffset;
     private int recordCount = 0;
     private int[] targetColumnIndexes;
 
-    private ByteBuf nullChars;
-    private FieldSerializerDeserializer serde;
     private DelimitedLineReader reader;
-    private FieldSplitProcessor processor;
+    private TextLineDeserializer deserializer;
 
     public DelimitedTextFileScanner(Configuration conf, final Schema schema, final TableMeta meta,
                                     final FileFragment fragment)
@@ -265,30 +284,14 @@ public class DelimitedTextFile {
 
       startOffset = fragment.getStartKey();
       endOffset = startOffset + fragment.getEndKey();
+    }
 
-      //Delimiter
-      String delim = meta.getOption(StorageConstants.TEXT_DELIMITER, StorageConstants.DEFAULT_FIELD_DELIMITER);
-      this.processor = new FieldSplitProcessor(StringEscapeUtils.unescapeJava(delim).charAt(0));
+    public TextLineSerDe getLineSerde() {
+      return DelimitedTextFile.getLineSerde(meta);
     }
 
     @Override
     public void init() throws IOException {
-      if (nullChars != null) {
-        nullChars.release();
-      }
-
-      String nullCharacters = StringEscapeUtils.unescapeJava(meta.getOption(StorageConstants.TEXT_NULL,
-          NullDatum.DEFAULT_TEXT));
-      byte[] bytes;
-      if (StringUtils.isEmpty(nullCharacters)) {
-        bytes = NullDatum.get().asTextBytes();
-      } else {
-        bytes = nullCharacters.getBytes();
-      }
-
-      nullChars = BufferPool.directBuffer(bytes.length, bytes.length);
-      nullChars.writeBytes(bytes);
-
       if (reader != null) {
         reader.close();
       }
@@ -305,8 +308,6 @@ public class DelimitedTextFile {
         targetColumnIndexes[i] = schema.getColumnId(targets[i].getQualifiedName());
       }
 
-      serde = new TextFieldSerializerDeserializer();
-
       super.init();
       Arrays.sort(targetColumnIndexes);
       if (LOG.isDebugEnabled()) {
@@ -316,6 +317,9 @@ public class DelimitedTextFile {
       if (startOffset > 0) {
         reader.readLine();  // skip first line;
       }
+
+      deserializer = getLineSerde().createDeserializer(schema, meta, targetColumnIndexes);
+      deserializer.init();
     }
 
     public ByteBuf readLine() throws IOException {
@@ -362,7 +366,7 @@ public class DelimitedTextFile {
         }
 
         VTuple tuple = new VTuple(schema.size());
-        fillTuple(schema, tuple, buf, targetColumnIndexes);
+        deserializer.deserialize(buf, tuple);
         return tuple;
       } catch (Throwable t) {
         LOG.error("Tuple list current index: " + recordCount + " file offset:" + reader.getCompressedPosition(), t);
@@ -370,44 +374,6 @@ public class DelimitedTextFile {
       }
     }
 
-    private void fillTuple(Schema schema, Tuple dst, ByteBuf lineBuf, int[] target) throws IOException {
-      int[] projection = target;
-      if (lineBuf == null || target == null || target.length == 0) {
-        return;
-      }
-
-      final int rowLength = lineBuf.readableBytes();
-      int start = 0, fieldLength = 0, end = 0;
-
-      //Projection
-      int currentTarget = 0;
-      int currentIndex = 0;
-
-      while (end != -1) {
-        end = lineBuf.forEachByte(start, rowLength - start, processor);
-
-        if (end < 0) {
-          fieldLength = rowLength - start;
-        } else {
-          fieldLength = end - start;
-        }
-
-        if (projection.length > currentTarget && currentIndex == projection[currentTarget]) {
-          lineBuf.setIndex(start, start + fieldLength);
-          Datum datum = serde.deserialize(lineBuf, schema.getColumn(currentIndex), currentIndex, nullChars);
-          dst.put(currentIndex, datum);
-          currentTarget++;
-        }
-
-        if (projection.length == currentTarget) {
-          break;
-        }
-
-        start = end + 1;
-        currentIndex++;
-      }
-    }
-
     @Override
     public void reset() throws IOException {
       init();
@@ -416,10 +382,7 @@ public class DelimitedTextFile {
     @Override
     public void close() throws IOException {
       try {
-        if (nullChars != null) {
-          nullChars.release();
-          nullChars = null;
-        }
+        deserializer.release();
 
         if (tableStats != null && reader != null) {
           tableStats.setReadBytes(reader.getReadBytes());  //Actual Processed Bytes. (decompressed bytes + overhead)

http://git-wip-us.apache.org/repos/asf/tajo/blob/72dd29c5/tajo-storage/src/main/java/org/apache/tajo/storage/text/TextLineDeserializer.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/text/TextLineDeserializer.java b/tajo-storage/src/main/java/org/apache/tajo/storage/text/TextLineDeserializer.java
new file mode 100644
index 0000000..645d118
--- /dev/null
+++ b/tajo-storage/src/main/java/org/apache/tajo/storage/text/TextLineDeserializer.java
@@ -0,0 +1,60 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.storage.text;
+
+import io.netty.buffer.ByteBuf;
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.catalog.TableMeta;
+import org.apache.tajo.storage.Tuple;
+
+import java.io.IOException;
+
+/**
+ * Reads a text line and fills a Tuple with values
+ */
+public abstract class TextLineDeserializer {
+  protected Schema schema;
+  protected TableMeta meta;
+  protected int [] targetColumnIndexes;
+
+  public TextLineDeserializer(Schema schema, TableMeta meta, int [] targetColumnIndexes) {
+    this.schema = schema;
+    this.meta = meta;
+    this.targetColumnIndexes = targetColumnIndexes;
+  }
+
+  /**
+   * Initialize SerDe
+   */
+  public abstract void init();
+
+  /**
+   * It fills a tuple with a read fields in a given line.
+   *
+   * @param buf Read line
+   * @param tuple Tuple to be filled with read fields
+   * @throws java.io.IOException
+   */
+  public abstract void deserialize(final ByteBuf buf, Tuple tuple) throws IOException;
+
+  /**
+   * Release external resources
+   */
+  public abstract void release();
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/72dd29c5/tajo-storage/src/main/java/org/apache/tajo/storage/text/TextLineSerDe.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/text/TextLineSerDe.java b/tajo-storage/src/main/java/org/apache/tajo/storage/text/TextLineSerDe.java
new file mode 100644
index 0000000..e81e289
--- /dev/null
+++ b/tajo-storage/src/main/java/org/apache/tajo/storage/text/TextLineSerDe.java
@@ -0,0 +1,65 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.storage.text;
+
+import io.netty.buffer.ByteBuf;
+import org.apache.commons.lang.StringEscapeUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.catalog.TableMeta;
+import org.apache.tajo.datum.NullDatum;
+import org.apache.tajo.storage.BufferPool;
+import org.apache.tajo.storage.StorageConstants;
+
+/**
+ * Pluggable Text Line SerDe class
+ */
+public abstract class TextLineSerDe {
+
+  public TextLineSerDe() {
+  }
+
+  public abstract TextLineDeserializer createDeserializer(Schema schema, TableMeta meta, int [] targetColumnIndexes);
+
+  public abstract TextLineSerializer createSerializer(Schema schema, TableMeta meta);
+
+  public static ByteBuf getNullChars(TableMeta meta) {
+    byte[] nullCharByteArray = getNullCharsAsBytes(meta);
+
+    ByteBuf nullChars = BufferPool.directBuffer(nullCharByteArray.length, nullCharByteArray.length);
+    nullChars.writeBytes(nullCharByteArray);
+
+    return nullChars;
+  }
+
+  public static byte [] getNullCharsAsBytes(TableMeta meta) {
+    byte [] nullChars;
+
+    String nullCharacters = StringEscapeUtils.unescapeJava(meta.getOption(StorageConstants.TEXT_NULL,
+        NullDatum.DEFAULT_TEXT));
+    if (StringUtils.isEmpty(nullCharacters)) {
+      nullChars = NullDatum.get().asTextBytes();
+    } else {
+      nullChars = nullCharacters.getBytes();
+    }
+
+    return nullChars;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/72dd29c5/tajo-storage/src/main/java/org/apache/tajo/storage/text/TextLineSerializer.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/text/TextLineSerializer.java b/tajo-storage/src/main/java/org/apache/tajo/storage/text/TextLineSerializer.java
new file mode 100644
index 0000000..0c2761f
--- /dev/null
+++ b/tajo-storage/src/main/java/org/apache/tajo/storage/text/TextLineSerializer.java
@@ -0,0 +1,45 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.storage.text;
+
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.catalog.TableMeta;
+import org.apache.tajo.storage.Tuple;
+
+import java.io.IOException;
+import java.io.OutputStream;
+
+/**
+ * Write a Tuple into single text formatted line
+ */
+public abstract class TextLineSerializer {
+  protected Schema schema;
+  protected TableMeta meta;
+
+  public TextLineSerializer(Schema schema, TableMeta meta) {
+    this.schema = schema;
+    this.meta = meta;
+  }
+
+  public abstract void init();
+
+  public abstract int serialize(OutputStream out, Tuple input) throws IOException;
+
+  public abstract void release();
+}


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

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

Conflicts:
	CHANGES
	tajo-core/src/main/java/org/apache/tajo/master/querymaster/Query.java
	tajo-plan/src/main/java/org/apache/tajo/plan/verifier/PreLogicalPlanVerifier.java
	tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/json/JsonLineDeserializer.java
	tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/DelimitedTextFile.java


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

Branch: refs/heads/hbase_storage
Commit: 0073ef23a3873bbe20534f6a2a0c75a5dcbda167
Parents: 940546a 2a69bcc
Author: HyoungJun Kim <ba...@babokim-MacBook-Pro.local>
Authored: Fri Dec 5 16:55:06 2014 +0900
Committer: HyoungJun Kim <ba...@babokim-MacBook-Pro.local>
Committed: Fri Dec 5 16:55:06 2014 +0900

----------------------------------------------------------------------
 CHANGES                                         |  17 +
 .../java/org/apache/tajo/cli/tsql/TajoCli.java  |  13 +-
 .../java/org/apache/tajo/datum/DateDatum.java   |  11 +-
 .../org/apache/tajo/datum/IntervalDatum.java    |   1 -
 .../java/org/apache/tajo/datum/TimeDatum.java   |   4 -
 .../apache/tajo/storage/StorageConstants.java   |   2 +-
 .../apache/tajo/datum/TestIntervalDatum.java    |   8 +-
 .../apache/tajo/master/querymaster/Query.java   |   2 +-
 .../tajo/engine/eval/TestIntervalType.java      | 172 +++---
 .../engine/function/TestDateTimeFunctions.java  | 184 +++---
 .../tajo/engine/query/TestCaseByCases.java      |   8 +
 .../tajo/engine/query/TestInsertQuery.java      |  22 +
 .../tajo/engine/query/TestTablePartitions.java  |  21 +-
 .../TestCaseByCases/testTAJO1224Case1.sql       |   1 +
 .../TestInsertQuery/lineitem_year_month_ddl.sql |  18 +
 .../load_to_lineitem_year_month.sql             |   1 +
 .../testInsertOverwriteWithAsteriskAndMore.sql  |   1 +
 .../TestCaseByCases/testTAJO1224Case1.result    |   3 +
 ...estInsertOverwriteWithAsteriskAndMore.result |   7 +
 .../src/main/sphinx/table_management/csv.rst    |   5 +
 .../org/apache/tajo/jdbc/JdbcConnection.java    |  13 +-
 .../org/apache/tajo/jdbc/TajoStatement.java     |   7 +-
 .../plan/verifier/PreLogicalPlanVerifier.java   |  65 ++-
 .../storage/FieldSerializerDeserializer.java    |  35 --
 .../org/apache/tajo/storage/StorageManager.java |   9 +-
 .../apache/tajo/tuple/TestBaseTupleBuilder.java |  76 +++
 .../tajo/tuple/offheap/TestHeapTuple.java       |  45 ++
 .../tajo/tuple/offheap/TestOffHeapRowBlock.java | 577 +++++++++++++++++++
 .../tajo/tuple/offheap/TestResizableSpec.java   |  59 ++
 .../storage/FieldSerializerDeserializer.java    |  37 ++
 .../apache/tajo/storage/avro/AvroAppender.java  |   1 +
 .../tajo/storage/json/JsonLineDeserializer.java |  11 +-
 .../tajo/storage/text/DelimitedTextFile.java    |  20 +-
 .../tajo/storage/TestDelimitedTextFile.java     | 163 ++++++
 .../org/apache/tajo/storage/TestLineReader.java | 193 +++++++
 .../apache/tajo/storage/TestSplitProcessor.java |  72 +++
 .../apache/tajo/storage/json/TestJsonSerDe.java | 101 ++++
 .../tajo/storage/json/TestLineReader.java       | 197 -------
 .../testErrorTolerance1.json                    |   6 +
 .../testErrorTolerance2.json                    |   4 +
 40 files changed, 1701 insertions(+), 491 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tajo/blob/0073ef23/CHANGES
----------------------------------------------------------------------
diff --cc CHANGES
index 6769e9f,0c785ce..c4c7435
--- a/CHANGES
+++ b/CHANGES
@@@ -5,10 -5,9 +5,13 @@@ Release 0.9.1 - unrelease
  
    NEW FEATURES
  
+     TAJO-1222: DelimitedTextFile should be tolerant against parsing errors.
+     (hyunsik)
+ 
 +    TAJO-1131: Supports Inserting or Creating table into 
 +    the HBase mapped table.(Hyoungjun Kim)
 +
 +
      TAJO-1026: Implement Query history persistency manager.(Hyoungjun Kim)
  
      TAJO-233: Support PostgreSQL CatalogStore. (Jihun Kang via hyunsik)

http://git-wip-us.apache.org/repos/asf/tajo/blob/0073ef23/tajo-client/src/main/java/org/apache/tajo/cli/tsql/TajoCli.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/tajo/blob/0073ef23/tajo-core/src/main/java/org/apache/tajo/master/querymaster/Query.java
----------------------------------------------------------------------
diff --cc tajo-core/src/main/java/org/apache/tajo/master/querymaster/Query.java
index b68eb2e,6f80171..978480e
--- a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/Query.java
+++ b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/Query.java
@@@ -435,24 -428,20 +435,24 @@@ public class Query implements EventHand
      }
  
      private QueryState finalizeQuery(Query query, QueryCompletedEvent event) {
 -      MasterPlan masterPlan = query.getPlan();
 +      SubQuery lastStage = query.getSubQuery(event.getExecutionBlockId());
 +      StoreType storeType = lastStage.getTableMeta().getStoreType();
 +      try {
 +        LogicalRootNode rootNode = lastStage.getMasterPlan().getLogicalPlan().getRootBlock().getRoot();
 +        CatalogService catalog = lastStage.getContext().getQueryMasterContext().getWorkerContext().getCatalog();
 +        TableDesc tableDesc =  PlannerUtil.getTableDesc(catalog, rootNode.getChild());
  
 -      ExecutionBlock terminal = query.getPlan().getTerminalBlock();
 -      DataChannel finalChannel = masterPlan.getChannel(event.getExecutionBlockId(), terminal.getId());
 +        Path finalOutputDir = StorageManager.getStorageManager(query.systemConf, storeType)
 +            .commitOutputData(query.context.getQueryContext(),
 +                lastStage.getId(), lastStage.getMasterPlan().getLogicalPlan(), lastStage.getSchema(), tableDesc);
  
 -      QueryHookExecutor hookExecutor = new QueryHookExecutor(query.context.getQueryMasterContext());
 -      try {
 -        Path finalOutputDir = commitOutputData(query);
 +        QueryHookExecutor hookExecutor = new QueryHookExecutor(query.context.getQueryMasterContext());
          hookExecutor.execute(query.context.getQueryContext(), query, event.getExecutionBlockId(), finalOutputDir);
 -      } catch (Throwable t) {
 -        query.eventHandler.handle(new QueryDiagnosticsUpdateEvent(query.id, ExceptionUtils.getStackTrace(t)));
 +      } catch (Exception e) {
 +        query.eventHandler.handle(new QueryDiagnosticsUpdateEvent(query.id, ExceptionUtils.getStackTrace(e)));
          return QueryState.QUERY_ERROR;
        }
-       
+ 
        return QueryState.QUERY_SUCCEEDED;
      }
  

http://git-wip-us.apache.org/repos/asf/tajo/blob/0073ef23/tajo-core/src/test/java/org/apache/tajo/engine/eval/TestIntervalType.java
----------------------------------------------------------------------
diff --cc tajo-core/src/test/java/org/apache/tajo/engine/eval/TestIntervalType.java
index c054fd1,c054fd1..e40eced
--- a/tajo-core/src/test/java/org/apache/tajo/engine/eval/TestIntervalType.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/eval/TestIntervalType.java
@@@ -26,92 -26,92 +26,92 @@@ import java.io.IOException
  import static org.junit.Assert.fail;
  
  public class TestIntervalType extends ExprTestBase {
--  @Test
--  public void testIntervalPostgresqlCase() throws IOException {
--
--    // http://www.postgresql.org/docs/8.2/static/functions-datetime.html
--    testSimpleEval("select date '2001-09-28' + 7", new String[]{"2001-10-05"});
--    testSimpleEval("select date '2001-09-28' + interval '1 hour'",
--        new String[]{"2001-09-28 01:00:00" + getUserTimeZoneDisplay()});
--
--    testSimpleEval("select date '2001-09-28' + time '03:00'",
--        new String[]{"2001-09-28 03:00:00" + getUserTimeZoneDisplay()});
--    testSimpleEval("select time '03:00' + date '2001-09-28'",
--        new String[]{"2001-09-28 03:00:00" + getUserTimeZoneDisplay()});
--    testSimpleEval("select interval '1 day' + interval '1 hour'", new String[]{"1 day 01:00:00"});
--
--    testSimpleEval("select timestamp '2001-09-28 01:00' + interval '23 hours'",
--        new String[]{"2001-09-29 00:00:00" + getUserTimeZoneDisplay()});
--
--    testSimpleEval("select time '01:00' + interval '3 hours'", new String[]{"04:00:00" + getUserTimeZoneDisplay()});
--
--    testSimpleEval("select date '2001-10-01' - date '2001-09-28'", new String[]{"3"});
--    testSimpleEval("select date '2001-10-01' - 7", new String[]{"2001-09-24"});
--    testSimpleEval("select date '2001-09-28' - interval '1 hour'",
--        new String[]{"2001-09-27 23:00:00" + getUserTimeZoneDisplay()});
--
--    testSimpleEval("select time '05:00' - time '03:00'", new String[]{"02:00:00"});
--    testSimpleEval("select time '05:00' - interval '2 hours'", new String[]{"03:00:00" + getUserTimeZoneDisplay()});
--    testSimpleEval("select timestamp '2001-09-28 23:00' - interval '23 hours'",
--        new String[]{"2001-09-28 00:00:00" + getUserTimeZoneDisplay()});
--
--    testSimpleEval("select interval '1 day' - interval '1 hour'", new String[]{"23:00:00"});
--
--    testSimpleEval("select timestamp '2001-09-29 03:00' - timestamp '2001-09-27 12:00'", new String[]{"1 day 15:00:00"});
--    testSimpleEval("select 900 * interval '1 second'", new String[]{"00:15:00"});
--    testSimpleEval("select 21 * interval '1 day'", new String[]{"21 days"});
--    testSimpleEval("select 3.5 * interval '1 hour'", new String[]{"03:30:00"});
--    testSimpleEval("select interval '1 hour' / 1.5", new String[]{"00:40:00"});
--  }
--
--  @Test
--  public void testCaseByCase() throws Exception {
--    testSimpleEval("select date '2001-08-28' + interval '10 day 1 hour'",
--        new String[]{"2001-09-07 01:00:00" + getUserTimeZoneDisplay()});
--    testSimpleEval("select interval '10 day 01:00:00' + date '2001-08-28'",
--        new String[]{"2001-09-07 01:00:00" + getUserTimeZoneDisplay()});
--    testSimpleEval("select time '10:20:30' + interval '1 day 01:00:00'",
--        new String[]{"11:20:30" + getUserTimeZoneDisplay()});
--    testSimpleEval("select interval '1 day 01:00:00' + time '10:20:30'",
--        new String[]{"11:20:30" + getUserTimeZoneDisplay()});
--    testSimpleEval("select time '10:20:30' - interval '1 day 01:00:00'",
--        new String[]{"09:20:30" + getUserTimeZoneDisplay()});
--
--    testSimpleEval("select (interval '1 month 20 day' + interval '50 day')", new String[]{"1 month 70 days"});
--    testSimpleEval("select date '2013-01-01' + interval '1 month 70 day'",
--        new String[]{"2013-04-12 00:00:00" + getUserTimeZoneDisplay()});
--    testSimpleEval("select date '2013-01-01' + (interval '1 month 20 day' + interval '50 day')",
--        new String[]{"2013-04-12 00:00:00" + getUserTimeZoneDisplay()});
--    testSimpleEval("select interval '1 month 70 day' + date '2013-01-01'",
--        new String[]{"2013-04-12 00:00:00" + getUserTimeZoneDisplay()});
--    testSimpleEval("select date '2013-01-01' - interval '1 month 70 day'",
--        new String[]{"2012-09-22 00:00:00" + getUserTimeZoneDisplay()});
--
--    testSimpleEval("select timestamp '2001-09-28 23:00' - interval '1 month 2 day 10:20:30'",
--        new String[]{"2001-08-26 12:39:30" + getUserTimeZoneDisplay()});
--    testSimpleEval("select timestamp '2001-09-28 23:00' + interval '1 month 2 day 10:20:30'",
--        new String[]{"2001-10-31 09:20:30" + getUserTimeZoneDisplay()});
--    testSimpleEval("select interval '1 month 2 day 10:20:30' + timestamp '2001-09-28 23:00'",
--        new String[]{"2001-10-31 09:20:30" + getUserTimeZoneDisplay()});
--
--
--    testSimpleEval("select interval '5 month' / 3", new String[]{"1 month 20 days"});
--
--    // Notice: Different from postgresql result(13 days 01:02:36.4992) because of double type precision.
--    testSimpleEval("select interval '1 month' / 2.3", new String[]{"13 days 01:02:36.522"});
--
--    testSimpleEval("select interval '1 month' * 2.3", new String[]{"2 months 9 days"});
--    testSimpleEval("select interval '3 year 5 month 1 hour' / 1.5", new String[]{"2 years 3 months 10 days 00:40:00"});
--
--    testSimpleEval("select date '2001-09-28' - time '03:00'",
--        new String[]{"2001-09-27 21:00:00" + getUserTimeZoneDisplay()});
--
--    testSimpleEval("select date '2014-03-20' + interval '1 day'",
--        new String[]{"2014-03-21 00:00:00" + getUserTimeZoneDisplay()});
--
--    testSimpleEval("select date '2014-03-20' - interval '1 day'",
--        new String[]{"2014-03-19 00:00:00" + getUserTimeZoneDisplay()});
--  }
++//  @Test
++//  public void testIntervalPostgresqlCase() throws IOException {
++//
++//    // http://www.postgresql.org/docs/8.2/static/functions-datetime.html
++//    testSimpleEval("select date '2001-09-28' + 7", new String[]{"2001-10-05"});
++//    testSimpleEval("select date '2001-09-28' + interval '1 hour'",
++//        new String[]{"2001-09-28 01:00:00" + getUserTimeZoneDisplay()});
++//
++//    testSimpleEval("select date '2001-09-28' + time '03:00'",
++//        new String[]{"2001-09-28 03:00:00" + getUserTimeZoneDisplay()});
++//    testSimpleEval("select time '03:00' + date '2001-09-28'",
++//        new String[]{"2001-09-28 03:00:00" + getUserTimeZoneDisplay()});
++//    testSimpleEval("select interval '1 day' + interval '1 hour'", new String[]{"1 day 01:00:00"});
++//
++//    testSimpleEval("select timestamp '2001-09-28 01:00' + interval '23 hours'",
++//        new String[]{"2001-09-29 00:00:00" + getUserTimeZoneDisplay()});
++//
++//    testSimpleEval("select time '01:00' + interval '3 hours'", new String[]{"04:00:00" + getUserTimeZoneDisplay()});
++//
++//    testSimpleEval("select date '2001-10-01' - date '2001-09-28'", new String[]{"3"});
++//    testSimpleEval("select date '2001-10-01' - 7", new String[]{"2001-09-24"});
++//    testSimpleEval("select date '2001-09-28' - interval '1 hour'",
++//        new String[]{"2001-09-27 23:00:00" + getUserTimeZoneDisplay()});
++//
++//    testSimpleEval("select time '05:00' - time '03:00'", new String[]{"02:00:00"});
++//    testSimpleEval("select time '05:00' - interval '2 hours'", new String[]{"03:00:00" + getUserTimeZoneDisplay()});
++//    testSimpleEval("select timestamp '2001-09-28 23:00' - interval '23 hours'",
++//        new String[]{"2001-09-28 00:00:00" + getUserTimeZoneDisplay()});
++//
++//    testSimpleEval("select interval '1 day' - interval '1 hour'", new String[]{"23:00:00"});
++//
++//    testSimpleEval("select timestamp '2001-09-29 03:00' - timestamp '2001-09-27 12:00'", new String[]{"1 day 15:00:00"});
++//    testSimpleEval("select 900 * interval '1 second'", new String[]{"00:15:00"});
++//    testSimpleEval("select 21 * interval '1 day'", new String[]{"21 days"});
++//    testSimpleEval("select 3.5 * interval '1 hour'", new String[]{"03:30:00"});
++//    testSimpleEval("select interval '1 hour' / 1.5", new String[]{"00:40:00"});
++//  }
++//
++//  @Test
++//  public void testCaseByCase() throws Exception {
++//    testSimpleEval("select date '2001-08-28' + interval '10 day 1 hour'",
++//        new String[]{"2001-09-07 01:00:00" + getUserTimeZoneDisplay()});
++//    testSimpleEval("select interval '10 day 01:00:00' + date '2001-08-28'",
++//        new String[]{"2001-09-07 01:00:00" + getUserTimeZoneDisplay()});
++//    testSimpleEval("select time '10:20:30' + interval '1 day 01:00:00'",
++//        new String[]{"11:20:30" + getUserTimeZoneDisplay()});
++//    testSimpleEval("select interval '1 day 01:00:00' + time '10:20:30'",
++//        new String[]{"11:20:30" + getUserTimeZoneDisplay()});
++//    testSimpleEval("select time '10:20:30' - interval '1 day 01:00:00'",
++//        new String[]{"09:20:30" + getUserTimeZoneDisplay()});
++//
++//    testSimpleEval("select (interval '1 month 20 day' + interval '50 day')", new String[]{"1 month 70 days"});
++//    testSimpleEval("select date '2013-01-01' + interval '1 month 70 day'",
++//        new String[]{"2013-04-12 00:00:00" + getUserTimeZoneDisplay()});
++//    testSimpleEval("select date '2013-01-01' + (interval '1 month 20 day' + interval '50 day')",
++//        new String[]{"2013-04-12 00:00:00" + getUserTimeZoneDisplay()});
++//    testSimpleEval("select interval '1 month 70 day' + date '2013-01-01'",
++//        new String[]{"2013-04-12 00:00:00" + getUserTimeZoneDisplay()});
++//    testSimpleEval("select date '2013-01-01' - interval '1 month 70 day'",
++//        new String[]{"2012-09-22 00:00:00" + getUserTimeZoneDisplay()});
++//
++//    testSimpleEval("select timestamp '2001-09-28 23:00' - interval '1 month 2 day 10:20:30'",
++//        new String[]{"2001-08-26 12:39:30" + getUserTimeZoneDisplay()});
++//    testSimpleEval("select timestamp '2001-09-28 23:00' + interval '1 month 2 day 10:20:30'",
++//        new String[]{"2001-10-31 09:20:30" + getUserTimeZoneDisplay()});
++//    testSimpleEval("select interval '1 month 2 day 10:20:30' + timestamp '2001-09-28 23:00'",
++//        new String[]{"2001-10-31 09:20:30" + getUserTimeZoneDisplay()});
++//
++//
++//    testSimpleEval("select interval '5 month' / 3", new String[]{"1 month 20 days"});
++//
++//    // Notice: Different from postgresql result(13 days 01:02:36.4992) because of double type precision.
++//    testSimpleEval("select interval '1 month' / 2.3", new String[]{"13 days 01:02:36.522"});
++//
++//    testSimpleEval("select interval '1 month' * 2.3", new String[]{"2 months 9 days"});
++//    testSimpleEval("select interval '3 year 5 month 1 hour' / 1.5", new String[]{"2 years 3 months 10 days 00:40:00"});
++//
++//    testSimpleEval("select date '2001-09-28' - time '03:00'",
++//        new String[]{"2001-09-27 21:00:00" + getUserTimeZoneDisplay()});
++//
++//    testSimpleEval("select date '2014-03-20' + interval '1 day'",
++//        new String[]{"2014-03-21 00:00:00" + getUserTimeZoneDisplay()});
++//
++//    testSimpleEval("select date '2014-03-20' - interval '1 day'",
++//        new String[]{"2014-03-19 00:00:00" + getUserTimeZoneDisplay()});
++//  }
  
    @Test
    public void testWrongFormatLiteral() throws Exception {

http://git-wip-us.apache.org/repos/asf/tajo/blob/0073ef23/tajo-core/src/test/java/org/apache/tajo/engine/function/TestDateTimeFunctions.java
----------------------------------------------------------------------
diff --cc tajo-core/src/test/java/org/apache/tajo/engine/function/TestDateTimeFunctions.java
index 7cca13d,7cca13d..c48b4d8
--- a/tajo-core/src/test/java/org/apache/tajo/engine/function/TestDateTimeFunctions.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/function/TestDateTimeFunctions.java
@@@ -314,98 -314,98 +314,98 @@@ public class TestDateTimeFunctions exte
       testSimpleEval("select utc_usec_to('week' ,1207929480000000, 2);", new String[]{1207612800000000L+""});
    }
  
--  @Test
--  public void testToDate() throws IOException {
--    testSimpleEval("select to_date('2014-01-04', 'YYYY-MM-DD')", new String[]{"2014-01-04"});
--    testSimpleEval("select to_date('2014-01-04', 'YYYY-MM-DD') + interval '1 day'",
--        new String[]{"2014-01-05 00:00:00" + getUserTimeZoneDisplay()});
--
--    testSimpleEval("SELECT to_date('201404', 'yyyymm');", new String[]{"2014-04-01"});
--  }
--
--  @Test
--  public void testAddMonths() throws Exception {
--    testSimpleEval("SELECT add_months(date '2013-12-17', 2::INT2);",
--        new String[]{"2014-02-17 00:00:00" + getUserTimeZoneDisplay()});
--    testSimpleEval("SELECT add_months(date '2013-12-17', 2::INT4);",
--        new String[]{"2014-02-17 00:00:00" + getUserTimeZoneDisplay()});
--    testSimpleEval("SELECT add_months(date '2013-12-17', 2::INT8);",
--        new String[]{"2014-02-17 00:00:00" + getUserTimeZoneDisplay()});
--
--    testSimpleEval("SELECT add_months(timestamp '2013-12-17 12:10:20', 2::INT2);",
--        new String[]{"2014-02-17 12:10:20" + getUserTimeZoneDisplay()});
--    testSimpleEval("SELECT add_months(timestamp '2013-12-17 12:10:20', 2::INT4);",
--        new String[]{"2014-02-17 12:10:20" + getUserTimeZoneDisplay()});
--    testSimpleEval("SELECT add_months(timestamp '2013-12-17 12:10:20', 2::INT8);",
--        new String[]{"2014-02-17 12:10:20" + getUserTimeZoneDisplay()});
--
--    testSimpleEval("SELECT add_months(date '2014-02-05', -3::INT2);",
--        new String[]{"2013-11-05 00:00:00" + getUserTimeZoneDisplay()});
--    testSimpleEval("SELECT add_months(date '2014-02-05', -3::INT4);",
--        new String[]{"2013-11-05 00:00:00" + getUserTimeZoneDisplay()});
--    testSimpleEval("SELECT add_months(date '2014-02-05', -3::INT8);",
--        new String[]{"2013-11-05 00:00:00" + getUserTimeZoneDisplay()});
--
--    testSimpleEval("SELECT add_months(timestamp '2014-02-05 12:10:20', -3::INT2);",
--        new String[]{"2013-11-05 12:10:20" + getUserTimeZoneDisplay()});
--    testSimpleEval("SELECT add_months(timestamp '2014-02-05 12:10:20', -3::INT4);",
--        new String[]{"2013-11-05 12:10:20" + getUserTimeZoneDisplay()});
--    testSimpleEval("SELECT add_months(timestamp '2014-02-05 12:10:20', -3::INT8);",
--        new String[]{"2013-11-05 12:10:20" + getUserTimeZoneDisplay()});
--  }
--
--  @Test
--  public void testAddDays() throws IOException {
--    testSimpleEval("SELECT add_days(date '2013-12-30', 5::INT2);",
--        new String[]{"2014-01-04 00:00:00" + getUserTimeZoneDisplay()});
--    testSimpleEval("SELECT add_days(date '2013-12-30', 5::INT4);",
--        new String[]{"2014-01-04 00:00:00" + getUserTimeZoneDisplay()});
--    testSimpleEval("SELECT add_days(date '2013-12-30', 5::INT8);",
--        new String[]{"2014-01-04 00:00:00" + getUserTimeZoneDisplay()});
--
--    testSimpleEval("SELECT add_days(timestamp '2013-12-30 12:10:20', 5::INT2);",
--        new String[]{"2014-01-04 12:10:20" + getUserTimeZoneDisplay()});
--    testSimpleEval("SELECT add_days(timestamp '2013-12-30 12:10:20', 5::INT4);",
--        new String[]{"2014-01-04 12:10:20" + getUserTimeZoneDisplay()});
--    testSimpleEval("SELECT add_days(timestamp '2013-12-30 12:10:20', 5::INT8);",
--        new String[]{"2014-01-04 12:10:20" + getUserTimeZoneDisplay()});
--
--    testSimpleEval("SELECT add_days(date '2013-12-05', -7::INT2);",
--        new String[]{"2013-11-28 00:00:00" + getUserTimeZoneDisplay()});
--    testSimpleEval("SELECT add_days(date '2013-12-05', -7::INT4);",
--        new String[]{"2013-11-28 00:00:00" + getUserTimeZoneDisplay()});
--    testSimpleEval("SELECT add_days(date '2013-12-05', -7::INT8);",
--        new String[]{"2013-11-28 00:00:00" + getUserTimeZoneDisplay()});
--
--    testSimpleEval("SELECT add_days(timestamp '2013-12-05 12:10:20', -7::INT2);",
--        new String[]{"2013-11-28 12:10:20" + getUserTimeZoneDisplay()});
--    testSimpleEval("SELECT add_days(timestamp '2013-12-05 12:10:20', -7::INT4);",
--        new String[]{"2013-11-28 12:10:20" + getUserTimeZoneDisplay()});
--    testSimpleEval("SELECT add_days(timestamp '2013-12-05 12:10:20', -7::INT8);",
--        new String[]{"2013-11-28 12:10:20" + getUserTimeZoneDisplay()});
--  }
--
--  @Test
--  public void testDateTimeNow() throws IOException {
--    TimeZone originTimeZone = TajoConf.setCurrentTimeZone(TimeZone.getTimeZone("GMT-6"));
--    TimeZone systemOriginTimeZone = TimeZone.getDefault();
--    TimeZone.setDefault(TimeZone.getTimeZone("GMT-6"));
--    try {
--      Date expectedDate = new Date(System.currentTimeMillis());
--
--      testSimpleEval("select to_char(now(), 'yyyy-MM-dd');",
--          new String[]{dateFormat(expectedDate, "yyyy-MM-dd")});
--      testSimpleEval("select cast(extract(year from now()) as INT4);",
--          new String[]{dateFormat(expectedDate, "yyyy")});
--      testSimpleEval("select current_date();",
--          new String[]{dateFormat(expectedDate, "yyyy-MM-dd")});
--      testSimpleEval("select cast(extract(hour from current_time()) as INT4);",
--          new String[]{String.valueOf(Integer.parseInt(dateFormat(expectedDate, "HH")))});
--    } finally {
--      TajoConf.setCurrentTimeZone(originTimeZone);
--      TimeZone.setDefault(systemOriginTimeZone);
--    }
--  }
++//  @Test
++//  public void testToDate() throws IOException {
++//    testSimpleEval("select to_date('2014-01-04', 'YYYY-MM-DD')", new String[]{"2014-01-04"});
++//    testSimpleEval("select to_date('2014-01-04', 'YYYY-MM-DD') + interval '1 day'",
++//        new String[]{"2014-01-05 00:00:00" + getUserTimeZoneDisplay()});
++//
++//    testSimpleEval("SELECT to_date('201404', 'yyyymm');", new String[]{"2014-04-01"});
++//  }
++
++//  @Test
++//  public void testAddMonths() throws Exception {
++//    testSimpleEval("SELECT add_months(date '2013-12-17', 2::INT2);",
++//        new String[]{"2014-02-17 00:00:00" + getUserTimeZoneDisplay()});
++//    testSimpleEval("SELECT add_months(date '2013-12-17', 2::INT4);",
++//        new String[]{"2014-02-17 00:00:00" + getUserTimeZoneDisplay()});
++//    testSimpleEval("SELECT add_months(date '2013-12-17', 2::INT8);",
++//        new String[]{"2014-02-17 00:00:00" + getUserTimeZoneDisplay()});
++//
++//    testSimpleEval("SELECT add_months(timestamp '2013-12-17 12:10:20', 2::INT2);",
++//        new String[]{"2014-02-17 12:10:20" + getUserTimeZoneDisplay()});
++//    testSimpleEval("SELECT add_months(timestamp '2013-12-17 12:10:20', 2::INT4);",
++//        new String[]{"2014-02-17 12:10:20" + getUserTimeZoneDisplay()});
++//    testSimpleEval("SELECT add_months(timestamp '2013-12-17 12:10:20', 2::INT8);",
++//        new String[]{"2014-02-17 12:10:20" + getUserTimeZoneDisplay()});
++//
++//    testSimpleEval("SELECT add_months(date '2014-02-05', -3::INT2);",
++//        new String[]{"2013-11-05 00:00:00" + getUserTimeZoneDisplay()});
++//    testSimpleEval("SELECT add_months(date '2014-02-05', -3::INT4);",
++//        new String[]{"2013-11-05 00:00:00" + getUserTimeZoneDisplay()});
++//    testSimpleEval("SELECT add_months(date '2014-02-05', -3::INT8);",
++//        new String[]{"2013-11-05 00:00:00" + getUserTimeZoneDisplay()});
++//
++//    testSimpleEval("SELECT add_months(timestamp '2014-02-05 12:10:20', -3::INT2);",
++//        new String[]{"2013-11-05 12:10:20" + getUserTimeZoneDisplay()});
++//    testSimpleEval("SELECT add_months(timestamp '2014-02-05 12:10:20', -3::INT4);",
++//        new String[]{"2013-11-05 12:10:20" + getUserTimeZoneDisplay()});
++//    testSimpleEval("SELECT add_months(timestamp '2014-02-05 12:10:20', -3::INT8);",
++//        new String[]{"2013-11-05 12:10:20" + getUserTimeZoneDisplay()});
++//  }
++
++//  @Test
++//  public void testAddDays() throws IOException {
++//    testSimpleEval("SELECT add_days(date '2013-12-30', 5::INT2);",
++//        new String[]{"2014-01-04 00:00:00" + getUserTimeZoneDisplay()});
++//    testSimpleEval("SELECT add_days(date '2013-12-30', 5::INT4);",
++//        new String[]{"2014-01-04 00:00:00" + getUserTimeZoneDisplay()});
++//    testSimpleEval("SELECT add_days(date '2013-12-30', 5::INT8);",
++//        new String[]{"2014-01-04 00:00:00" + getUserTimeZoneDisplay()});
++//
++//    testSimpleEval("SELECT add_days(timestamp '2013-12-30 12:10:20', 5::INT2);",
++//        new String[]{"2014-01-04 12:10:20" + getUserTimeZoneDisplay()});
++//    testSimpleEval("SELECT add_days(timestamp '2013-12-30 12:10:20', 5::INT4);",
++//        new String[]{"2014-01-04 12:10:20" + getUserTimeZoneDisplay()});
++//    testSimpleEval("SELECT add_days(timestamp '2013-12-30 12:10:20', 5::INT8);",
++//        new String[]{"2014-01-04 12:10:20" + getUserTimeZoneDisplay()});
++//
++//    testSimpleEval("SELECT add_days(date '2013-12-05', -7::INT2);",
++//        new String[]{"2013-11-28 00:00:00" + getUserTimeZoneDisplay()});
++//    testSimpleEval("SELECT add_days(date '2013-12-05', -7::INT4);",
++//        new String[]{"2013-11-28 00:00:00" + getUserTimeZoneDisplay()});
++//    testSimpleEval("SELECT add_days(date '2013-12-05', -7::INT8);",
++//        new String[]{"2013-11-28 00:00:00" + getUserTimeZoneDisplay()});
++//
++//    testSimpleEval("SELECT add_days(timestamp '2013-12-05 12:10:20', -7::INT2);",
++//        new String[]{"2013-11-28 12:10:20" + getUserTimeZoneDisplay()});
++//    testSimpleEval("SELECT add_days(timestamp '2013-12-05 12:10:20', -7::INT4);",
++//        new String[]{"2013-11-28 12:10:20" + getUserTimeZoneDisplay()});
++//    testSimpleEval("SELECT add_days(timestamp '2013-12-05 12:10:20', -7::INT8);",
++//        new String[]{"2013-11-28 12:10:20" + getUserTimeZoneDisplay()});
++//  }
++//
++//  @Test
++//  public void testDateTimeNow() throws IOException {
++//    TimeZone originTimeZone = TajoConf.setCurrentTimeZone(TimeZone.getTimeZone("GMT-6"));
++//    TimeZone systemOriginTimeZone = TimeZone.getDefault();
++//    TimeZone.setDefault(TimeZone.getTimeZone("GMT-6"));
++//    try {
++//      Date expectedDate = new Date(System.currentTimeMillis());
++//
++//      testSimpleEval("select to_char(now(), 'yyyy-MM-dd');",
++//          new String[]{dateFormat(expectedDate, "yyyy-MM-dd")});
++//      testSimpleEval("select cast(extract(year from now()) as INT4);",
++//          new String[]{dateFormat(expectedDate, "yyyy")});
++//      testSimpleEval("select current_date();",
++//          new String[]{dateFormat(expectedDate, "yyyy-MM-dd")});
++//      testSimpleEval("select cast(extract(hour from current_time()) as INT4);",
++//          new String[]{String.valueOf(Integer.parseInt(dateFormat(expectedDate, "HH")))});
++//    } finally {
++//      TajoConf.setCurrentTimeZone(originTimeZone);
++//      TimeZone.setDefault(systemOriginTimeZone);
++//    }
++//  }
  
    @Test
    public void testTimeValueKeyword() throws IOException {

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

http://git-wip-us.apache.org/repos/asf/tajo/blob/0073ef23/tajo-storage/tajo-storage-common/src/test/java/org/apache/tajo/tuple/TestBaseTupleBuilder.java
----------------------------------------------------------------------
diff --cc tajo-storage/tajo-storage-common/src/test/java/org/apache/tajo/tuple/TestBaseTupleBuilder.java
index 0000000,0000000..b332364
new file mode 100644
--- /dev/null
+++ b/tajo-storage/tajo-storage-common/src/test/java/org/apache/tajo/tuple/TestBaseTupleBuilder.java
@@@ -1,0 -1,0 +1,76 @@@
++/**
++ * Licensed to the Apache Software Foundation (ASF) under one
++ * or more contributor license agreements.  See the NOTICE file
++ * distributed with this work for additional information
++ * regarding copyright ownership.  The ASF licenses this file
++ * to you under the Apache License, Version 2.0 (the
++ * "License"); you may not use this file except in compliance
++ * with the License.  You may obtain a copy of the License at
++ *
++ *      http://www.apache.org/licenses/LICENSE-2.0
++ *
++ * Unless required by applicable law or agreed to in writing, software
++ * distributed under the License is distributed on an "AS IS" BASIS,
++ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
++ * See the License for the specific language governing permissions and
++ * limitations under the License.
++ */
++
++package org.apache.tajo.tuple;
++
++import org.apache.tajo.storage.RowStoreUtil;
++import org.apache.tajo.tuple.offheap.*;
++import org.junit.Test;
++
++public class TestBaseTupleBuilder {
++
++  @Test
++  public void testBuild() {
++    BaseTupleBuilder builder = new BaseTupleBuilder(TestOffHeapRowBlock.schema);
++
++    OffHeapRowBlock rowBlock = TestOffHeapRowBlock.createRowBlock(10248);
++    OffHeapRowBlockReader reader = rowBlock.getReader();
++
++    ZeroCopyTuple inputTuple = new ZeroCopyTuple();
++
++    HeapTuple heapTuple = null;
++    ZeroCopyTuple zcTuple = null;
++    int i = 0;
++    while(reader.next(inputTuple)) {
++      RowStoreUtil.convert(inputTuple, builder);
++
++      heapTuple = builder.buildToHeapTuple();
++      TestOffHeapRowBlock.validateTupleResult(i, heapTuple);
++
++      zcTuple = builder.buildToZeroCopyTuple();
++      TestOffHeapRowBlock.validateTupleResult(i, zcTuple);
++
++      i++;
++    }
++  }
++
++  @Test
++  public void testBuildWithNull() {
++    BaseTupleBuilder builder = new BaseTupleBuilder(TestOffHeapRowBlock.schema);
++
++    OffHeapRowBlock rowBlock = TestOffHeapRowBlock.createRowBlockWithNull(10248);
++    OffHeapRowBlockReader reader = rowBlock.getReader();
++
++    ZeroCopyTuple inputTuple = new ZeroCopyTuple();
++
++    HeapTuple heapTuple = null;
++    ZeroCopyTuple zcTuple = null;
++    int i = 0;
++    while(reader.next(inputTuple)) {
++      RowStoreUtil.convert(inputTuple, builder);
++
++      heapTuple = builder.buildToHeapTuple();
++      TestOffHeapRowBlock.validateNullity(i, heapTuple);
++
++      zcTuple = builder.buildToZeroCopyTuple();
++      TestOffHeapRowBlock.validateNullity(i, zcTuple);
++
++      i++;
++    }
++  }
++}

http://git-wip-us.apache.org/repos/asf/tajo/blob/0073ef23/tajo-storage/tajo-storage-common/src/test/java/org/apache/tajo/tuple/offheap/TestHeapTuple.java
----------------------------------------------------------------------
diff --cc tajo-storage/tajo-storage-common/src/test/java/org/apache/tajo/tuple/offheap/TestHeapTuple.java
index 0000000,0000000..96f465a
new file mode 100644
--- /dev/null
+++ b/tajo-storage/tajo-storage-common/src/test/java/org/apache/tajo/tuple/offheap/TestHeapTuple.java
@@@ -1,0 -1,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.tuple.offheap;
++
++import org.apache.tajo.catalog.SchemaUtil;
++import org.junit.Test;
++
++public class TestHeapTuple {
++
++  @Test
++  public void testHeapTuple() {
++    OffHeapRowBlock rowBlock = TestOffHeapRowBlock.createRowBlock(1024);
++
++    OffHeapRowBlockReader reader = new OffHeapRowBlockReader(rowBlock);
++
++    ZeroCopyTuple zcTuple = new ZeroCopyTuple();
++    int i = 0;
++    while (reader.next(zcTuple)) {
++      byte [] bytes = new byte[zcTuple.nioBuffer().limit()];
++      zcTuple.nioBuffer().get(bytes);
++
++      HeapTuple heapTuple = new HeapTuple(bytes, SchemaUtil.toDataTypes(TestOffHeapRowBlock.schema));
++      TestOffHeapRowBlock.validateTupleResult(i, heapTuple);
++      i++;
++    }
++
++    rowBlock.release();
++  }
++}


[20/29] tajo git commit: TAJO-1194: 'INSERT OVERWRITE .. SELECT' does not remove existing data when result is empty. (missing code)

Posted by hj...@apache.org.
TAJO-1194: 'INSERT OVERWRITE .. SELECT' does not remove existing data when result is empty. (missing code)


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

Branch: refs/heads/hbase_storage
Commit: 2a69bcc218a53eced0cdb9217de1fdd1837c0827
Parents: f6da07b
Author: JaeHwa Jung <bl...@apache.org>
Authored: Thu Dec 4 18:54:03 2014 +0900
Committer: JaeHwa Jung <bl...@apache.org>
Committed: Thu Dec 4 18:54:03 2014 +0900

----------------------------------------------------------------------
 .../tajo/engine/query/TestTablePartitions.java  | 232 ++++++++++++-------
 1 file changed, 142 insertions(+), 90 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tajo/blob/2a69bcc2/tajo-core/src/test/java/org/apache/tajo/engine/query/TestTablePartitions.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestTablePartitions.java b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestTablePartitions.java
index 15cbde0..b1e1bec 100644
--- a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestTablePartitions.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestTablePartitions.java
@@ -19,10 +19,7 @@
 package org.apache.tajo.engine.query;
 
 import com.google.common.collect.Maps;
-import org.apache.hadoop.fs.ContentSummary;
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.*;
 import org.apache.hadoop.io.compress.CompressionCodec;
 import org.apache.hadoop.io.compress.CompressionCodecFactory;
 import org.apache.hadoop.io.compress.DeflateCodec;
@@ -44,6 +41,7 @@ import org.apache.tajo.jdbc.TajoResultSet;
 import org.apache.tajo.master.querymaster.QueryMasterTask;
 import org.apache.tajo.plan.logical.NodeType;
 import org.apache.tajo.storage.StorageConstants;
+import org.apache.tajo.util.CommonTestingUtil;
 import org.apache.tajo.util.KeyValueSet;
 import org.apache.tajo.worker.TajoWorker;
 import org.junit.Test;
@@ -70,7 +68,7 @@ public class TestTablePartitions extends QueryTestCaseBase {
   public final void testCreateColumnPartitionedTable() throws Exception {
     String tableName = CatalogUtil.normalizeIdentifier("testCreateColumnPartitionedTable");
     ResultSet res = executeString(
-      "create table " + tableName + " (col1 int4, col2 int4) partition by column(key float8) ");
+        "create table " + tableName + " (col1 int4, col2 int4) partition by column(key float8) ");
     res.close();
 
     assertTrue(catalog.existsTable(DEFAULT_DATABASE_NAME, tableName));
@@ -78,8 +76,8 @@ public class TestTablePartitions extends QueryTestCaseBase {
     assertEquals(3, catalog.getTableDesc(DEFAULT_DATABASE_NAME, tableName).getLogicalSchema().size());
 
     res = testBase.execute(
-      "insert overwrite into " + tableName + " select l_orderkey, l_partkey, " +
-        "l_quantity from lineitem");
+        "insert overwrite into " + tableName + " select l_orderkey, l_partkey, " +
+            "l_quantity from lineitem");
 
     MasterPlan plan = getQueryPlan(res);
     ExecutionBlock rootEB = plan.getRoot();
@@ -113,7 +111,7 @@ public class TestTablePartitions extends QueryTestCaseBase {
   public final void testCreateColumnPartitionedTableWithJoin() throws Exception {
     String tableName = CatalogUtil.normalizeIdentifier("testCreateColumnPartitionedTableWithJoin");
     ResultSet res = executeString(
-      "create table " + tableName + " (col1 int4, col2 int4) partition by column(key float8) ");
+        "create table " + tableName + " (col1 int4, col2 int4) partition by column(key float8) ");
     res.close();
 
     assertTrue(catalog.existsTable(DEFAULT_DATABASE_NAME, tableName));
@@ -121,8 +119,8 @@ public class TestTablePartitions extends QueryTestCaseBase {
     assertEquals(3, catalog.getTableDesc(DEFAULT_DATABASE_NAME, tableName).getLogicalSchema().size());
 
     res = testBase.execute(
-      "insert overwrite into " + tableName + " select l_orderkey, l_partkey, " +
-        "l_quantity from lineitem join orders on l_orderkey = o_orderkey");
+        "insert overwrite into " + tableName + " select l_orderkey, l_partkey, " +
+            "l_quantity from lineitem join orders on l_orderkey = o_orderkey");
 
     MasterPlan plan = getQueryPlan(res);
     ExecutionBlock rootEB = plan.getRoot();
@@ -158,7 +156,7 @@ public class TestTablePartitions extends QueryTestCaseBase {
   public final void testCreateColumnPartitionedTableWithSelectedColumns() throws Exception {
     String tableName = CatalogUtil.normalizeIdentifier("testCreateColumnPartitionedTableWithSelectedColumns");
     ResultSet res = executeString(
-      "create table " + tableName + " (col1 int4, col2 int4, null_col int4) partition by column(key float8) ");
+        "create table " + tableName + " (col1 int4, col2 int4, null_col int4) partition by column(key float8) ");
     res.close();
 
     assertTrue(catalog.existsTable(DEFAULT_DATABASE_NAME, tableName));
@@ -166,7 +164,7 @@ public class TestTablePartitions extends QueryTestCaseBase {
     assertEquals(4, catalog.getTableDesc(DEFAULT_DATABASE_NAME, tableName).getLogicalSchema().size());
 
     res = executeString("insert overwrite into " + tableName + " (col1, col2, key) select l_orderkey, " +
-      "l_partkey, l_quantity from lineitem");
+        "l_partkey, l_quantity from lineitem");
     res.close();
   }
 
@@ -174,20 +172,20 @@ public class TestTablePartitions extends QueryTestCaseBase {
   public final void testColumnPartitionedTableByOneColumn() throws Exception {
     String tableName = CatalogUtil.normalizeIdentifier("testColumnPartitionedTableByOneColumn");
     ResultSet res = executeString(
-      "create table " + tableName + " (col1 int4, col2 int4, null_col int4) partition by column(key float8) ");
+        "create table " + tableName + " (col1 int4, col2 int4, null_col int4) partition by column(key float8) ");
     res.close();
 
     assertTrue(catalog.existsTable(DEFAULT_DATABASE_NAME, tableName));
 
     res = executeString("insert overwrite into " + tableName
-      + " (col1, col2, key) select l_orderkey, l_partkey, l_quantity from lineitem");
+        + " (col1, col2, key) select l_orderkey, l_partkey, l_quantity from lineitem");
     res.close();
 
     TableDesc desc = catalog.getTableDesc(DEFAULT_DATABASE_NAME, tableName);
     assertPartitionDirectories(desc);
 
     res = executeString(
-      "select distinct * from " + tableName + " where (key = 45.0 or key = 38.0) and null_col is null");
+        "select distinct * from " + tableName + " where (key = 45.0 or key = 38.0) and null_col is null");
 
     Map<Double, int []> resultRows1 = Maps.newHashMap();
     resultRows1.put(45.0d, new int[]{3, 2});
@@ -219,14 +217,14 @@ public class TestTablePartitions extends QueryTestCaseBase {
   public final void testQueryCasesOnColumnPartitionedTable() throws Exception {
     String tableName = CatalogUtil.normalizeIdentifier("testQueryCasesOnColumnPartitionedTable");
     ResultSet res = executeString(
-      "create table " + tableName + " (col1 int4, col2 int4, null_col int4) partition by column(key float8) ");
+        "create table " + tableName + " (col1 int4, col2 int4, null_col int4) partition by column(key float8) ");
     res.close();
 
     assertTrue(catalog.existsTable(DEFAULT_DATABASE_NAME, tableName));
 
     res = executeString(
-      "insert overwrite into " + tableName
-        + " (col1, col2, key) select l_orderkey, l_partkey, l_quantity from lineitem");
+        "insert overwrite into " + tableName
+            + " (col1, col2, key) select l_orderkey, l_partkey, l_quantity from lineitem");
     res.close();
 
     TableDesc desc = catalog.getTableDesc(DEFAULT_DATABASE_NAME, tableName);
@@ -299,14 +297,14 @@ public class TestTablePartitions extends QueryTestCaseBase {
   public final void testColumnPartitionedTableByThreeColumns() throws Exception {
     String tableName = CatalogUtil.normalizeIdentifier("testColumnPartitionedTableByThreeColumns");
     ResultSet res = testBase.execute(
-      "create table " + tableName + " (col4 text) partition by column(col1 int4, col2 int4, col3 float8) ");
+        "create table " + tableName + " (col4 text) partition by column(col1 int4, col2 int4, col3 float8) ");
     res.close();
     TajoTestingCluster cluster = testBase.getTestingCluster();
     CatalogService catalog = cluster.getMaster().getCatalog();
     assertTrue(catalog.existsTable(DEFAULT_DATABASE_NAME, tableName));
 
     res = executeString("insert overwrite into " + tableName
-      + " select l_returnflag, l_orderkey, l_partkey, l_quantity from lineitem");
+        + " select l_returnflag, l_orderkey, l_partkey, l_quantity from lineitem");
     res.close();
 
     TableDesc desc = catalog.getTableDesc(DEFAULT_DATABASE_NAME, tableName);
@@ -363,14 +361,14 @@ public class TestTablePartitions extends QueryTestCaseBase {
   public final void testInsertIntoColumnPartitionedTableByThreeColumns() throws Exception {
     String tableName = CatalogUtil.normalizeIdentifier("testInsertIntoColumnPartitionedTableByThreeColumns");
     ResultSet res = testBase.execute(
-      "create table " + tableName + " (col4 text) partition by column(col1 int4, col2 int4, col3 float8) ");
+        "create table " + tableName + " (col4 text) partition by column(col1 int4, col2 int4, col3 float8) ");
     res.close();
     TajoTestingCluster cluster = testBase.getTestingCluster();
     CatalogService catalog = cluster.getMaster().getCatalog();
     assertTrue(catalog.existsTable(DEFAULT_DATABASE_NAME, tableName));
 
     res = executeString("insert into " + tableName
-      + " select l_returnflag, l_orderkey, l_partkey, l_quantity from lineitem");
+        + " select l_returnflag, l_orderkey, l_partkey, l_quantity from lineitem");
     res.close();
 
     TableDesc desc = catalog.getTableDesc(DEFAULT_DATABASE_NAME, tableName);
@@ -423,7 +421,7 @@ public class TestTablePartitions extends QueryTestCaseBase {
 
     // insert into already exists partitioned table
     res = executeString("insert into " + tableName
-      + " select l_returnflag, l_orderkey, l_partkey, l_quantity from lineitem");
+        + " select l_returnflag, l_orderkey, l_partkey, l_quantity from lineitem");
     res.close();
 
     desc = catalog.getTableDesc(DEFAULT_DATABASE_NAME, tableName);
@@ -446,15 +444,15 @@ public class TestTablePartitions extends QueryTestCaseBase {
       assertEquals(5, desc.getStats().getNumRows().intValue());
     }
     String expected = "N\n" +
-      "N\n" +
-      "N\n" +
-      "N\n" +
-      "N\n" +
-      "N\n" +
-      "R\n" +
-      "R\n" +
-      "R\n" +
-      "R\n";
+        "N\n" +
+        "N\n" +
+        "N\n" +
+        "N\n" +
+        "N\n" +
+        "R\n" +
+        "R\n" +
+        "R\n" +
+        "R\n";
 
     String tableData = getTableFileContents(new Path(desc.getPath()));
     assertEquals(expected, tableData);
@@ -463,30 +461,30 @@ public class TestTablePartitions extends QueryTestCaseBase {
     String resultSetData = resultSetToString(res);
     res.close();
     expected = "col4,col1,col2,col3\n" +
-      "-------------------------------\n" +
-      "N,2,2,38.0\n" +
-      "N,2,2,38.0\n" +
-      "R,3,2,45.0\n" +
-      "R,3,2,45.0\n";
+        "-------------------------------\n" +
+        "N,2,2,38.0\n" +
+        "N,2,2,38.0\n" +
+        "R,3,2,45.0\n" +
+        "R,3,2,45.0\n";
     assertEquals(expected, resultSetData);
 
     res = executeString("select * from " + tableName + " where (col1 = 2 or col1 = 3) and col2 >= 2");
     resultSetData = resultSetToString(res);
     res.close();
     expected = "col4,col1,col2,col3\n" +
-      "-------------------------------\n" +
-      "N,2,2,38.0\n" +
-      "N,2,2,38.0\n" +
-      "R,3,2,45.0\n" +
-      "R,3,2,45.0\n" +
-      "R,3,3,49.0\n" +
-      "R,3,3,49.0\n";
+        "-------------------------------\n" +
+        "N,2,2,38.0\n" +
+        "N,2,2,38.0\n" +
+        "R,3,2,45.0\n" +
+        "R,3,2,45.0\n" +
+        "R,3,3,49.0\n" +
+        "R,3,3,49.0\n";
     assertEquals(expected, resultSetData);
 
     // Check not to remove existing partition directories.
     res = executeString("insert overwrite into " + tableName
-      + " select l_returnflag, l_orderkey, l_partkey, 30.0 as l_quantity from lineitem "
-      + " where l_orderkey = 1 and l_partkey = 1 and  l_linenumber = 1");
+        + " select l_returnflag, l_orderkey, l_partkey, 30.0 as l_quantity from lineitem "
+        + " where l_orderkey = 1 and l_partkey = 1 and  l_linenumber = 1");
     res.close();
 
     desc = catalog.getTableDesc(DEFAULT_DATABASE_NAME, tableName);
@@ -513,12 +511,12 @@ public class TestTablePartitions extends QueryTestCaseBase {
     resultSetData = resultSetToString(res);
     res.close();
     expected = "col4,col1,col2,col3\n" +
-      "-------------------------------\n" +
-      "N,1,1,17.0\n" +
-      "N,1,1,17.0\n" +
-      "N,1,1,30.0\n" +
-      "N,1,1,36.0\n" +
-      "N,1,1,36.0\n";
+        "-------------------------------\n" +
+        "N,1,1,17.0\n" +
+        "N,1,1,17.0\n" +
+        "N,1,1,30.0\n" +
+        "N,1,1,36.0\n" +
+        "N,1,1,36.0\n";
 
     assertEquals(expected, resultSetData);
 
@@ -541,14 +539,14 @@ public class TestTablePartitions extends QueryTestCaseBase {
   public final void testColumnPartitionedTableByOneColumnsWithCompression() throws Exception {
     String tableName = CatalogUtil.normalizeIdentifier("testColumnPartitionedTableByOneColumnsWithCompression");
     ResultSet res = executeString(
-      "create table " + tableName + " (col2 int4, col3 float8) USING csv " +
-        "WITH ('csvfile.delimiter'='|','compression.codec'='org.apache.hadoop.io.compress.DeflateCodec') " +
-        "PARTITION BY column(col1 int4)");
+        "create table " + tableName + " (col2 int4, col3 float8) USING csv " +
+            "WITH ('csvfile.delimiter'='|','compression.codec'='org.apache.hadoop.io.compress.DeflateCodec') " +
+            "PARTITION BY column(col1 int4)");
     res.close();
     assertTrue(catalog.existsTable(DEFAULT_DATABASE_NAME, tableName));
 
     res = executeString(
-      "insert overwrite into " + tableName + " select l_partkey, l_quantity, l_orderkey from lineitem");
+        "insert overwrite into " + tableName + " select l_partkey, l_quantity, l_orderkey from lineitem");
     res.close();
     TableDesc desc = catalog.getTableDesc(DEFAULT_DATABASE_NAME, tableName);
     if (!testingCluster.isHCatalogStoreRunning()) {
@@ -577,15 +575,15 @@ public class TestTablePartitions extends QueryTestCaseBase {
   public final void testColumnPartitionedTableByTwoColumnsWithCompression() throws Exception {
     String tableName = CatalogUtil.normalizeIdentifier("testColumnPartitionedTableByTwoColumnsWithCompression");
     ResultSet res = executeString("create table " + tableName + " (col3 float8, col4 text) USING csv " +
-      "WITH ('csvfile.delimiter'='|','compression.codec'='org.apache.hadoop.io.compress.DeflateCodec') " +
-      "PARTITION by column(col1 int4, col2 int4)");
+        "WITH ('csvfile.delimiter'='|','compression.codec'='org.apache.hadoop.io.compress.DeflateCodec') " +
+        "PARTITION by column(col1 int4, col2 int4)");
     res.close();
 
     assertTrue(catalog.existsTable(DEFAULT_DATABASE_NAME, tableName));
 
     res = executeString(
-      "insert overwrite into " + tableName +
-        " select  l_quantity, l_returnflag, l_orderkey, l_partkey from lineitem");
+        "insert overwrite into " + tableName +
+            " select  l_quantity, l_returnflag, l_orderkey, l_partkey from lineitem");
     res.close();
     TableDesc desc = catalog.getTableDesc(DEFAULT_DATABASE_NAME, tableName);
     if (!testingCluster.isHCatalogStoreRunning()) {
@@ -621,16 +619,16 @@ public class TestTablePartitions extends QueryTestCaseBase {
   public final void testColumnPartitionedTableByThreeColumnsWithCompression() throws Exception {
     String tableName = CatalogUtil.normalizeIdentifier("testColumnPartitionedTableByThreeColumnsWithCompression");
     ResultSet res = executeString(
-      "create table " + tableName + " (col4 text) USING csv " +
-        "WITH ('csvfile.delimiter'='|','compression.codec'='org.apache.hadoop.io.compress.DeflateCodec') " +
-        "partition by column(col1 int4, col2 int4, col3 float8)");
+        "create table " + tableName + " (col4 text) USING csv " +
+            "WITH ('csvfile.delimiter'='|','compression.codec'='org.apache.hadoop.io.compress.DeflateCodec') " +
+            "partition by column(col1 int4, col2 int4, col3 float8)");
     res.close();
 
     assertTrue(catalog.existsTable(DEFAULT_DATABASE_NAME, tableName));
 
     res = executeString(
-      "insert overwrite into " + tableName +
-        " select l_returnflag, l_orderkey, l_partkey, l_quantity from lineitem");
+        "insert overwrite into " + tableName +
+            " select l_returnflag, l_orderkey, l_partkey, l_quantity from lineitem");
     res.close();
     TableDesc desc = catalog.getTableDesc(DEFAULT_DATABASE_NAME, tableName);
     if (!testingCluster.isHCatalogStoreRunning()) {
@@ -704,16 +702,16 @@ public class TestTablePartitions extends QueryTestCaseBase {
   public final void testColumnPartitionedTableNoMatchedPartition() throws Exception {
     String tableName = CatalogUtil.normalizeIdentifier("testColumnPartitionedTableNoMatchedPartition");
     ResultSet res = executeString(
-      "create table " + tableName + " (col4 text) USING csv " +
-        "WITH ('csvfile.delimiter'='|','compression.codec'='org.apache.hadoop.io.compress.DeflateCodec') " +
-        "partition by column(col1 int4, col2 int4, col3 float8)");
+        "create table " + tableName + " (col4 text) USING csv " +
+            "WITH ('csvfile.delimiter'='|','compression.codec'='org.apache.hadoop.io.compress.DeflateCodec') " +
+            "partition by column(col1 int4, col2 int4, col3 float8)");
     res.close();
 
     assertTrue(catalog.existsTable(DEFAULT_DATABASE_NAME, tableName));
 
     res = executeString(
-      "insert overwrite into " + tableName +
-        " select l_returnflag , l_orderkey, l_partkey, l_quantity from lineitem");
+        "insert overwrite into " + tableName +
+            " select l_returnflag , l_orderkey, l_partkey, l_quantity from lineitem");
     res.close();
     TableDesc desc = catalog.getTableDesc(DEFAULT_DATABASE_NAME, tableName);
     if (!testingCluster.isHCatalogStoreRunning()) {
@@ -760,13 +758,13 @@ public class TestTablePartitions extends QueryTestCaseBase {
   public final void testColumnPartitionedTableWithSmallerExpressions1() throws Exception {
     String tableName = CatalogUtil.normalizeIdentifier("testColumnPartitionedTableWithSmallerExpressions1");
     ResultSet res = executeString(
-      "create table " + tableName + " (col1 int4, col2 int4, null_col int4) partition by column(key float8) ");
+        "create table " + tableName + " (col1 int4, col2 int4, null_col int4) partition by column(key float8) ");
     res.close();
 
     assertTrue(catalog.existsTable(DEFAULT_DATABASE_NAME, tableName));
 
     ClientProtos.SubmitQueryResponse response = client.executeQuery("insert overwrite into " + tableName
-      + " select l_orderkey, l_partkey from lineitem");
+        + " select l_orderkey, l_partkey from lineitem");
 
     assertTrue(response.hasErrorMessage());
     assertEquals(response.getErrorMessage(), "INSERT has smaller expressions than target columns\n");
@@ -780,13 +778,13 @@ public class TestTablePartitions extends QueryTestCaseBase {
   public final void testColumnPartitionedTableWithSmallerExpressions2() throws Exception {
     String tableName = CatalogUtil.normalizeIdentifier("testColumnPartitionedTableWithSmallerExpressions2");
     ResultSet res = executeString(
-      "create table " + tableName + " (col1 int4, col2 int4, null_col int4) partition by column(key float8) ");
+        "create table " + tableName + " (col1 int4, col2 int4, null_col int4) partition by column(key float8) ");
     res.close();
 
     assertTrue(catalog.existsTable(DEFAULT_DATABASE_NAME, tableName));
 
     ClientProtos.SubmitQueryResponse response = client.executeQuery("insert overwrite into " + tableName
-      + " select l_returnflag , l_orderkey, l_partkey from lineitem");
+        + " select l_returnflag , l_orderkey, l_partkey from lineitem");
 
     assertTrue(response.hasErrorMessage());
     assertEquals(response.getErrorMessage(), "INSERT has smaller expressions than target columns\n");
@@ -805,11 +803,11 @@ public class TestTablePartitions extends QueryTestCaseBase {
     res.close();
 
     res = executeString("create table testinsertquery1.table1 " +
-      "(col1 int4, col2 int4, col3 float8)");
+        "(col1 int4, col2 int4, col3 float8)");
     res.close();
 
     res = executeString("create table testinsertquery2.table1 " +
-      "(col1 int4, col2 int4, col3 float8)");
+        "(col1 int4, col2 int4, col3 float8)");
     res.close();
 
     CatalogService catalog = testingCluster.getMaster().getCatalog();
@@ -817,7 +815,7 @@ public class TestTablePartitions extends QueryTestCaseBase {
     assertTrue(catalog.existsTable("testinsertquery2", "table1"));
 
     res = executeString("insert overwrite into testinsertquery1.table1 " +
-      "select l_orderkey, l_partkey, l_quantity from default.lineitem;");
+        "select l_orderkey, l_partkey, l_quantity from default.lineitem;");
     res.close();
 
     TableDesc desc = catalog.getTableDesc("testinsertquery1", "table1");
@@ -826,7 +824,7 @@ public class TestTablePartitions extends QueryTestCaseBase {
     }
 
     res = executeString("insert overwrite into testinsertquery2.table1 " +
-      "select col1, col2, col3 from testinsertquery1.table1;");
+        "select col1, col2, col3 from testinsertquery1.table1;");
     res.close();
 
     desc = catalog.getTableDesc("testinsertquery2", "table1");
@@ -835,6 +833,39 @@ public class TestTablePartitions extends QueryTestCaseBase {
     }
   }
 
+  @Test
+  public final void testColumnPartitionedTableWithSmallerExpressions5() throws Exception {
+    String tableName = CatalogUtil.normalizeIdentifier("testColumnPartitionedTableWithSmallerExpressions5");
+    ResultSet res = executeString(
+        "create table " + tableName + " (col1 text) partition by column(col2 text) ");
+    res.close();
+
+    assertTrue(catalog.existsTable(DEFAULT_DATABASE_NAME, tableName));
+
+    res = executeString("insert overwrite into " + tableName + "(col1) select l_returnflag from lineitem");
+    res.close();
+    res = executeString("select * from " + tableName);
+    assertResultSet(res);
+    res.close();
+  }
+
+  @Test
+  public final void testColumnPartitionedTableWithSmallerExpressions6() throws Exception {
+    String tableName = CatalogUtil.normalizeIdentifier("testColumnPartitionedTableWithSmallerExpressions6");
+    ResultSet res = executeString(
+        "create table " + tableName + " (col1 text) partition by column(col2 text) ");
+    res.close();
+
+    assertTrue(catalog.existsTable(DEFAULT_DATABASE_NAME, tableName));
+
+    res = executeString(
+        "insert overwrite into " + tableName + "(col1) select l_returnflag from lineitem where l_orderkey = 1");
+    res.close();
+    res = executeString("select * from " + tableName);
+    assertResultSet(res);
+    res.close();
+  }
+
   private MasterPlan getQueryPlan(ResultSet res) {
     QueryId queryId = ((TajoResultSet)res).getQueryId();
     for (TajoWorker eachWorker: testingCluster.getTajoWorkers()) {
@@ -865,8 +896,8 @@ public class TestTablePartitions extends QueryTestCaseBase {
       int totalBytes = 0;
       Random rand = new Random(System.currentTimeMillis());
       String col2Data = "Column-2Column-2Column-2Column-2Column-2Column-2Column-2Column-2Column-2Column-2Column-2" +
-        "Column-2Column-2Column-2Column-2Column-2Column-2Column-2Column-2Column-2Column-2Column-2" +
-        "Column-2Column-2Column-2Column-2Column-2Column-2Column-2Column-2Column-2Column-2Column-2";
+          "Column-2Column-2Column-2Column-2Column-2Column-2Column-2Column-2Column-2Column-2Column-2" +
+          "Column-2Column-2Column-2Column-2Column-2Column-2Column-2Column-2Column-2Column-2Column-2";
 
       int index = 0;
       while(true) {
@@ -901,9 +932,9 @@ public class TestTablePartitions extends QueryTestCaseBase {
 
     } finally {
       testingCluster.setAllTajoDaemonConfValue(TajoConf.ConfVars.$DIST_QUERY_TABLE_PARTITION_VOLUME.varname,
-        TajoConf.ConfVars.$DIST_QUERY_TABLE_PARTITION_VOLUME.defaultVal);
+          TajoConf.ConfVars.$DIST_QUERY_TABLE_PARTITION_VOLUME.defaultVal);
       testingCluster.setAllTajoDaemonConfValue(TajoConf.ConfVars.SHUFFLE_HASH_APPENDER_PAGE_VOLUME.varname,
-        TajoConf.ConfVars.SHUFFLE_HASH_APPENDER_PAGE_VOLUME.defaultVal);
+          TajoConf.ConfVars.SHUFFLE_HASH_APPENDER_PAGE_VOLUME.defaultVal);
       executeString("DROP TABLE test_partition PURGE").close();
       executeString("DROP TABLE testScatteredHashShuffle PURGE").close();
     }
@@ -916,17 +947,17 @@ public class TestTablePartitions extends QueryTestCaseBase {
     executeDDL("lineitemspecial_ddl.sql", "lineitemspecial.tbl");
 
     executeString("CREATE TABLE IF NOT EXISTS pTable947 (id int, name text) PARTITION BY COLUMN (type text)")
-      .close();
+        .close();
     executeString("INSERT OVERWRITE INTO pTable947 SELECT l_orderkey, l_shipinstruct, l_shipmode FROM lineitemspecial")
-      .close();
+        .close();
     ResultSet res = executeString("select * from pTable947 where type='RA:*?><I/L#%S' or type='AIR'");
 
     String resStr = resultSetToString(res);
     String expected =
-      "id,name,type\n" +
-        "-------------------------------\n"
-        + "3,NONE,AIR\n"
-        + "3,TEST SPECIAL CHARS,RA:*?><I/L#%S\n";
+        "id,name,type\n" +
+            "-------------------------------\n"
+            + "3,NONE,AIR\n"
+            + "3,TEST SPECIAL CHARS,RA:*?><I/L#%S\n";
 
     assertEquals(expected, resStr);
     cleanupQuery(res);
@@ -939,9 +970,9 @@ public class TestTablePartitions extends QueryTestCaseBase {
     executeDDL("lineitemspecial_ddl.sql", "lineitemspecial.tbl");
 
     executeString("CREATE TABLE IF NOT EXISTS pTable948 (id int, name text) PARTITION BY COLUMN (type text)")
-      .close();
+        .close();
     executeString("INSERT OVERWRITE INTO pTable948 SELECT l_orderkey, l_shipinstruct, l_shipmode FROM lineitemspecial")
-      .close();
+        .close();
 
     ResultSet res = executeString("select * from pTable948 where type='RA:*?><I/L#%S'");
     assertResultSet(res);
@@ -951,4 +982,25 @@ public class TestTablePartitions extends QueryTestCaseBase {
     assertResultSet(res);
     cleanupQuery(res);
   }
+
+  @Test
+  public final void testIgnoreFilesInIntermediateDir() throws Exception {
+    // See - TAJO-1219: Files located in intermediate directories of partitioned table should be ignored
+    // It verifies that Tajo ignores files located in intermediate directories of partitioned table.
+
+    Path testDir = CommonTestingUtil.getTestDir();
+
+    executeString(
+      "CREATE EXTERNAL TABLE testIgnoreFilesInIntermediateDir (col1 int) USING CSV PARTITION BY COLUMN (col2 text) " +
+        "LOCATION '" + testDir + "'");
+
+    FileSystem fs = testDir.getFileSystem(conf);
+    FSDataOutputStream fos = fs.create(new Path(testDir, "table1.data"));
+    fos.write("a|b|c".getBytes());
+    fos.close();
+
+    ResultSet res = executeString("select * from testIgnoreFilesInIntermediateDir;");
+    assertFalse(res.next());
+    res.close();
+  }
 }


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

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

Conflicts:
	CHANGES
	tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/CatalogUtil.java
	tajo-catalog/tajo-catalog-common/src/main/proto/CatalogProtos.proto
	tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/PhysicalPlanUtil.java
	tajo-core/src/main/java/org/apache/tajo/master/LazyTaskScheduler.java
	tajo-core/src/main/java/org/apache/tajo/master/querymaster/Query.java
	tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryMasterTask.java
	tajo-core/src/main/java/org/apache/tajo/master/querymaster/SubQuery.java
	tajo-storage/pom.xml
	tajo-storage/src/test/java/org/apache/tajo/storage/TestLineReader.java
	tajo-storage/tajo-storage-common/src/main/resources/storage-default.xml
	tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/DelimitedTextFile.java
	tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestFileSystems.java
	tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/s3/INode.java
	tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/s3/InMemoryFileSystemStore.java
	tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/s3/S3OutputStream.java
	tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/s3/SmallBlockS3FileSystem.java
	tajo-storage/tajo-storage-hdfs/src/test/resources/testVariousTypes.avsc


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

Branch: refs/heads/hbase_storage
Commit: 940546a06143d9ba508b4d82a921b98b52116d0e
Parents: dfd7f99 1cdbe46
Author: HyoungJun Kim <ba...@babokim-MacBook-Pro.local>
Authored: Fri Dec 5 14:38:36 2014 +0900
Committer: HyoungJun Kim <ba...@babokim-MacBook-Pro.local>
Committed: Fri Dec 5 14:38:36 2014 +0900

----------------------------------------------------------------------
 CHANGES                                         |  28 ++
 .../org/apache/tajo/algebra/CreateTable.java    |   2 +-
 .../main/java/org/apache/tajo/algebra/Expr.java |   2 +-
 .../org/apache/tajo/catalog/AlterTableDesc.java |   2 +-
 .../org/apache/tajo/catalog/CatalogUtil.java    |   9 +-
 .../java/org/apache/tajo/catalog/Column.java    |   2 +-
 .../org/apache/tajo/catalog/FunctionDesc.java   |   2 +-
 .../java/org/apache/tajo/catalog/Schema.java    |   3 +-
 .../org/apache/tajo/catalog/SchemaUtil.java     |   8 +
 .../java/org/apache/tajo/catalog/SortSpec.java  |   2 +-
 .../java/org/apache/tajo/catalog/TableDesc.java |   2 +-
 .../java/org/apache/tajo/catalog/TableMeta.java |   2 +-
 .../tajo/catalog/json/CatalogGsonHelper.java    |   2 +-
 .../tajo/catalog/json/FunctionAdapter.java      |   4 +-
 .../tajo/catalog/json/TableMetaAdapter.java     |   4 +-
 .../tajo/catalog/partition/PartitionDesc.java   |   2 +-
 .../catalog/partition/PartitionMethodDesc.java  |   2 +-
 .../tajo/catalog/statistics/ColumnStats.java    |   2 +-
 .../tajo/catalog/statistics/TableStats.java     |   2 +-
 .../java/org/apache/tajo/function/Function.java |   2 +-
 .../src/main/proto/CatalogProtos.proto          |   3 +-
 .../org/apache/tajo/client/ResultSetUtil.java   |   8 +
 .../java/org/apache/tajo/conf/TajoConf.java     |  12 +-
 .../main/java/org/apache/tajo/datum/Datum.java  |   4 +-
 .../apache/tajo/json/ClassNameSerializer.java   |  47 ---
 .../org/apache/tajo/json/CommonGsonHelper.java  |  92 ------
 .../org/apache/tajo/json/DataTypeAdapter.java   |  63 ----
 .../java/org/apache/tajo/json/DatumAdapter.java |  76 -----
 .../java/org/apache/tajo/json/GsonHelper.java   |  49 ---
 .../java/org/apache/tajo/json/GsonObject.java   |  23 --
 .../org/apache/tajo/json/GsonSerDerAdapter.java |  25 --
 .../org/apache/tajo/json/PathSerializer.java    |  41 ---
 .../apache/tajo/storage/StorageConstants.java   |  15 +-
 .../java/org/apache/tajo/storage/VTuple.java    |   4 +-
 .../tajo/storage/json/ClassNameSerializer.java  |  47 +++
 .../tajo/storage/json/CommonGsonHelper.java     |  92 ++++++
 .../tajo/storage/json/DataTypeAdapter.java      |  63 ++++
 .../apache/tajo/storage/json/DatumAdapter.java  |  76 +++++
 .../apache/tajo/storage/json/GsonHelper.java    |  49 +++
 .../apache/tajo/storage/json/GsonObject.java    |  23 ++
 .../tajo/storage/json/GsonSerDerAdapter.java    |  25 ++
 .../tajo/storage/json/PathSerializer.java       |  41 +++
 .../java/org/apache/tajo/util/KeyValueSet.java  |   4 +-
 .../org/apache/tajo/util/ReflectionUtil.java    |   4 +-
 .../org/apache/tajo/datum/TestBytesDatum.java   |   2 +-
 .../org/apache/tajo/datum/TestDateDatum.java    |   2 +-
 .../org/apache/tajo/datum/TestInet4Datum.java   |   2 +-
 .../org/apache/tajo/datum/TestTimeDatum.java    |   2 +-
 .../apache/tajo/datum/TestTimestampDatum.java   |   5 +-
 tajo-core/pom.xml                               |   1 +
 .../apache/tajo/engine/json/CoreGsonHelper.java |   2 +-
 .../planner/physical/PhysicalPlanUtil.java      |  63 +++-
 .../engine/planner/physical/SeqScanExec.java    |   7 +-
 .../org/apache/tajo/master/ContainerProxy.java  |  10 +-
 .../tajo/master/DefaultTaskScheduler.java       |  18 +-
 .../org/apache/tajo/master/GlobalEngine.java    |   7 +-
 .../tajo/master/LaunchTaskRunnersEvent.java     |   5 +-
 .../apache/tajo/master/LazyTaskScheduler.java   |  10 +-
 .../apache/tajo/master/TajoContainerProxy.java  |  22 +-
 .../java/org/apache/tajo/master/TajoMaster.java |   2 +-
 .../apache/tajo/master/TajoMasterService.java   |   5 +-
 .../tajo/master/TaskRunnerGroupEvent.java       |   8 +-
 .../apache/tajo/master/cluster/ServerName.java  | 123 --------
 .../tajo/master/container/TajoContainer.java    | 173 ++++++++++
 .../tajo/master/container/TajoContainerId.java  | 172 ++++++++++
 .../master/container/TajoConverterUtils.java    | 263 ++++++++++++++++
 .../master/container/TajoRecordFactory.java     |  31 ++
 .../container/TajoRecordFactoryPBImpl.java      | 104 ++++++
 .../container/TajoRecordFactoryProvider.java    |  70 +++++
 .../tajo/master/container/TajoRecords.java      |  39 +++
 .../impl/pb/TajoContainerIdPBImpl.java          | 100 ++++++
 .../tajo/master/event/LocalTaskEvent.java       |   9 +-
 .../event/QueryUnitAttemptScheduleEvent.java    |  10 +-
 .../event/SubQueryContainerAllocationEvent.java |   8 +-
 .../master/event/TaskAttemptAssignedEvent.java  |   8 +-
 .../tajo/master/event/TaskRequestEvent.java     |   8 +-
 .../apache/tajo/master/querymaster/Query.java   |  18 +-
 .../master/querymaster/QueryInProgress.java     |   3 +-
 .../tajo/master/querymaster/QueryInfo.java      |   2 +-
 .../querymaster/QueryMasterManagerService.java  |   4 +-
 .../master/querymaster/QueryMasterTask.java     |  29 +-
 .../master/querymaster/QueryUnitAttempt.java    |   6 +-
 .../tajo/master/querymaster/SubQuery.java       |  18 +-
 .../apache/tajo/master/rm/TajoRMContext.java    |   8 +-
 .../tajo/master/rm/TajoWorkerContainer.java     |  15 +-
 .../tajo/master/rm/TajoWorkerContainerId.java   |  53 ++--
 .../master/rm/TajoWorkerResourceManager.java    |  63 ++--
 .../tajo/master/rm/WorkerResourceManager.java   |   4 +-
 .../apache/tajo/util/history/QueryHistory.java  |   2 +-
 .../tajo/util/history/QueryUnitHistory.java     |   2 +-
 .../tajo/util/history/SubQueryHistory.java      |   2 +-
 .../tajo/webapp/QueryExecutorServlet.java       |  16 +
 .../tajo/worker/AbstractResourceAllocator.java  |  14 +-
 .../apache/tajo/worker/ResourceAllocator.java   |   6 +-
 .../tajo/worker/TajoResourceAllocator.java      | 101 +++---
 .../tajo/worker/TajoWorkerClientService.java    |  16 +-
 .../java/org/apache/tajo/worker/TaskRunner.java |  16 +-
 .../apache/tajo/worker/TaskRunnerHistory.java   |  14 +-
 .../src/main/proto/ContainerProtocol.proto      |  48 +++
 .../src/main/proto/QueryMasterProtocol.proto    |   3 +
 .../main/proto/ResourceTrackerProtocol.proto    |   3 +
 .../src/main/proto/TajoMasterProtocol.proto     |   7 +-
 .../src/main/proto/TajoWorkerProtocol.proto     |   5 +-
 .../src/main/resources/webapps/admin/index.jsp  |   2 +-
 .../resources/webapps/admin/query_executor.jsp  |   4 +-
 .../org/apache/tajo/cluster/TestServerName.java | 102 ------
 .../apache/tajo/engine/query/TestCTASQuery.java |   1 +
 .../tajo/engine/query/TestHBaseTable.java       |   9 +-
 .../tajo/engine/query/TestInsertQuery.java      |  14 +-
 .../tajo/engine/query/TestTablePartitions.java  |  56 ++++
 .../java/org/apache/tajo/jdbc/TestTajoJdbc.java | 181 ++++++++---
 .../tajo/master/rm/TestTajoResourceManager.java |  16 +-
 .../dataset/TestTajoJdbc/table1/table1.tbl      |   5 +
 .../TestTajoJdbc/create_table_with_date_ddl.sql |  10 +
 ...rtitionedTableWithSmallerExpressions5.result |   7 +
 ...rtitionedTableWithSmallerExpressions6.result |   4 +
 .../TestTajoJdbc/testSortWithDateTime.result    |   7 +
 .../src/main/sphinx/hcatalog_integration.rst    |  13 +
 .../org/apache/tajo/jdbc/MetaDataTuple.java     |   4 +-
 .../main/java/org/apache/tajo/plan/Target.java  |   2 +-
 .../org/apache/tajo/plan/expr/CaseWhenEval.java |   2 +-
 .../org/apache/tajo/plan/expr/EvalNode.java     |   2 +-
 .../tajo/plan/function/GeneralFunction.java     |   2 +-
 .../apache/tajo/plan/logical/BinaryNode.java    |   2 +-
 .../apache/tajo/plan/logical/LogicalNode.java   |   2 +-
 .../tajo/plan/serder/EvalNodeAdapter.java       |   4 +-
 .../tajo/plan/serder/LogicalNodeAdapter.java    |   4 +-
 .../apache/tajo/plan/serder/PlanGsonHelper.java |   2 +-
 .../apache/tajo/rpc/RemoteCallException.java    |   6 +-
 .../org/apache/tajo/storage/StorageManager.java | 237 ++++++++------
 .../src/main/resources/storage-default.xml      |  16 +-
 tajo-storage/tajo-storage-hdfs/pom.xml          |   7 +-
 .../apache/tajo/storage/FileStorageManager.java |  78 +++--
 .../tajo/storage/json/JsonLineDeserializer.java | 220 +++++++++++++
 .../apache/tajo/storage/json/JsonLineSerDe.java |  37 +++
 .../tajo/storage/json/JsonLineSerializer.java   | 130 ++++++++
 .../tajo/storage/text/ByteBufLineReader.java    |  22 +-
 .../tajo/storage/text/CSVLineDeserializer.java  |  96 ++++++
 .../apache/tajo/storage/text/CSVLineSerDe.java  |  41 +++
 .../tajo/storage/text/CSVLineSerializer.java    |  70 +++++
 .../tajo/storage/text/DelimitedTextFile.java    | 231 +++++++-------
 .../text/TextFieldSerializerDeserializer.java   |   2 +-
 .../tajo/storage/text/TextLineDeserializer.java |  60 ++++
 .../tajo/storage/text/TextLineParsingError.java |  31 ++
 .../apache/tajo/storage/text/TextLineSerDe.java |  65 ++++
 .../tajo/storage/text/TextLineSerializer.java   |  45 +++
 .../apache/tajo/storage/TestFileSystems.java    | 266 ++++++++--------
 .../org/apache/tajo/storage/TestStorages.java   | 107 ++++---
 .../apache/tajo/storage/avro/TestAvroUtil.java  |   2 +-
 .../tajo/storage/json/TestLineReader.java       | 197 ++++++++++++
 .../java/org/apache/tajo/storage/s3/INode.java  | 124 --------
 .../storage/s3/InMemoryFileSystemStore.java     | 176 -----------
 .../apache/tajo/storage/s3/S3OutputStream.java  | 234 --------------
 .../tajo/storage/s3/SmallBlockS3FileSystem.java | 314 -------------------
 .../dataset/TestJsonSerDe/testVariousType.json  |   1 +
 .../src/test/resources/dataset/testLineText.txt |   2 +
 .../resources/dataset/testVariousTypes.avsc     |  20 ++
 .../src/test/resources/storage-default.xml      |  16 +-
 .../src/test/resources/testVariousTypes.avsc    |  21 --
 159 files changed, 3760 insertions(+), 2310 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tajo/blob/940546a0/CHANGES
----------------------------------------------------------------------
diff --cc CHANGES
index 87ef9a6,188e024..6769e9f
--- a/CHANGES
+++ b/CHANGES
@@@ -15,10 -11,16 +15,19 @@@ Release 0.9.1 - unrelease
  
      TAJO-235: Support Oracle CatalogStore. (Jihun Kang via hyunsik)
  
+     TAJO-1095: Implement Json file scanner. (hyunsik)
+ 
    IMPROVEMENT
  
 +    TAJO-1122: Refactor the tajo-storage project structure.
 +    (Hyoungjun Kim)
++    
+     TAJO-1165: Needs to show error messages on query_executor.jsp. 
+     (Jihun Kang via jaehwa)
+ 
+     TAJO-1204: Remove unused ServerName class. (DaeMyung Kang via jaehwa)
+ 
+     TAJO-1053: ADD PARTITIONS for HCatalogStore. (jaehwa)
  
      TAJO-1195: Remove unused CachedDNSResolver Class. (DaeMyung Kang via jaehwa)
  

http://git-wip-us.apache.org/repos/asf/tajo/blob/940546a0/tajo-algebra/src/main/java/org/apache/tajo/algebra/CreateTable.java
----------------------------------------------------------------------
diff --cc tajo-algebra/src/main/java/org/apache/tajo/algebra/CreateTable.java
index 4056ed3,4056ed3..0216128
--- a/tajo-algebra/src/main/java/org/apache/tajo/algebra/CreateTable.java
+++ b/tajo-algebra/src/main/java/org/apache/tajo/algebra/CreateTable.java
@@@ -22,7 -22,7 +22,7 @@@ import com.google.common.base.Objects
  import com.google.gson.*;
  import com.google.gson.annotations.Expose;
  import com.google.gson.annotations.SerializedName;
--import org.apache.tajo.json.CommonGsonHelper;
++import org.apache.tajo.storage.json.CommonGsonHelper;
  import org.apache.tajo.util.TUtil;
  
  import java.util.ArrayList;

http://git-wip-us.apache.org/repos/asf/tajo/blob/940546a0/tajo-algebra/src/main/java/org/apache/tajo/algebra/Expr.java
----------------------------------------------------------------------
diff --cc tajo-algebra/src/main/java/org/apache/tajo/algebra/Expr.java
index b53a6db,b53a6db..6a41caf
--- a/tajo-algebra/src/main/java/org/apache/tajo/algebra/Expr.java
+++ b/tajo-algebra/src/main/java/org/apache/tajo/algebra/Expr.java
@@@ -22,7 -22,7 +22,7 @@@ import com.google.gson.*
  import com.google.gson.annotations.Expose;
  import com.google.gson.annotations.SerializedName;
  import org.apache.tajo.algebra.LiteralValue.LiteralType;
--import org.apache.tajo.json.CommonGsonHelper;
++import org.apache.tajo.storage.json.CommonGsonHelper;
  
  import java.lang.reflect.Type;
  

http://git-wip-us.apache.org/repos/asf/tajo/blob/940546a0/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/AlterTableDesc.java
----------------------------------------------------------------------
diff --cc tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/AlterTableDesc.java
index 69d5be4,69d5be4..3a3eb66
--- a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/AlterTableDesc.java
+++ b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/AlterTableDesc.java
@@@ -23,7 -23,7 +23,7 @@@ import com.google.gson.annotations.Expo
  import org.apache.tajo.catalog.json.CatalogGsonHelper;
  import org.apache.tajo.catalog.proto.CatalogProtos;
  import org.apache.tajo.common.ProtoObject;
--import org.apache.tajo.json.GsonObject;
++import org.apache.tajo.storage.json.GsonObject;
  
  import static org.apache.tajo.catalog.proto.CatalogProtos.AlterTableDescProto;
  

http://git-wip-us.apache.org/repos/asf/tajo/blob/940546a0/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/CatalogUtil.java
----------------------------------------------------------------------
diff --cc tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/CatalogUtil.java
index a6e1a31,f2d9b9c..aca2460
--- a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/CatalogUtil.java
+++ b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/CatalogUtil.java
@@@ -278,8 -280,8 +280,10 @@@ public class CatalogUtil 
        return StoreType.AVRO;
      } else if (typeStr.equalsIgnoreCase(StoreType.TEXTFILE.name())) {
        return StoreType.TEXTFILE;
+     } else if (typeStr.equalsIgnoreCase(StoreType.JSON.name())) {
+       return StoreType.JSON;
 +    } else if (typeStr.equalsIgnoreCase(StoreType.HBASE.name())) {
 +      return StoreType.HBASE;
      } else {
        return null;
      }

http://git-wip-us.apache.org/repos/asf/tajo/blob/940546a0/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/Column.java
----------------------------------------------------------------------
diff --cc tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/Column.java
index aceb6f1,aceb6f1..f6acfae
--- a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/Column.java
+++ b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/Column.java
@@@ -25,7 -25,7 +25,7 @@@ import org.apache.tajo.catalog.proto.Ca
  import org.apache.tajo.common.ProtoObject;
  import org.apache.tajo.common.TajoDataTypes;
  import org.apache.tajo.common.TajoDataTypes.DataType;
--import org.apache.tajo.json.GsonObject;
++import org.apache.tajo.storage.json.GsonObject;
  
  /**
   * Describes a column. It is an immutable object.

http://git-wip-us.apache.org/repos/asf/tajo/blob/940546a0/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/FunctionDesc.java
----------------------------------------------------------------------
diff --cc tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/FunctionDesc.java
index 23d39f2,23d39f2..5aea768
--- a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/FunctionDesc.java
+++ b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/FunctionDesc.java
@@@ -22,7 -22,7 +22,7 @@@ import com.google.common.base.Objects
  import com.google.gson.annotations.Expose;
  import org.apache.tajo.annotation.NotNull;
  import org.apache.tajo.function.*;
--import org.apache.tajo.json.GsonObject;
++import org.apache.tajo.storage.json.GsonObject;
  import org.apache.tajo.catalog.json.CatalogGsonHelper;
  import org.apache.tajo.catalog.proto.CatalogProtos.FunctionDescProto;
  import org.apache.tajo.catalog.proto.CatalogProtos.FunctionType;

http://git-wip-us.apache.org/repos/asf/tajo/blob/940546a0/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/Schema.java
----------------------------------------------------------------------
diff --cc tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/Schema.java
index 296230b,296230b..eae52e5
--- a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/Schema.java
+++ b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/Schema.java
@@@ -25,13 -25,13 +25,12 @@@ import org.apache.commons.logging.Log
  import org.apache.commons.logging.LogFactory;
  import org.apache.tajo.catalog.exception.AlreadyExistsFieldException;
  import org.apache.tajo.catalog.json.CatalogGsonHelper;
--import org.apache.tajo.catalog.proto.CatalogProtos;
  import org.apache.tajo.catalog.proto.CatalogProtos.ColumnProto;
  import org.apache.tajo.catalog.proto.CatalogProtos.SchemaProto;
  import org.apache.tajo.common.ProtoObject;
  import org.apache.tajo.common.TajoDataTypes.DataType;
  import org.apache.tajo.common.TajoDataTypes.Type;
--import org.apache.tajo.json.GsonObject;
++import org.apache.tajo.storage.json.GsonObject;
  import org.apache.tajo.util.TUtil;
  
  import java.util.*;

http://git-wip-us.apache.org/repos/asf/tajo/blob/940546a0/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/SortSpec.java
----------------------------------------------------------------------
diff --cc tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/SortSpec.java
index 5b10dfd,5b10dfd..3390146
--- a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/SortSpec.java
+++ b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/SortSpec.java
@@@ -22,7 -22,7 +22,7 @@@ import com.google.common.base.Objects
  import com.google.gson.annotations.Expose;
  import org.apache.tajo.catalog.json.CatalogGsonHelper;
  import org.apache.tajo.common.ProtoObject;
--import org.apache.tajo.json.GsonObject;
++import org.apache.tajo.storage.json.GsonObject;
  
  import static org.apache.tajo.catalog.proto.CatalogProtos.SortSpecProto;
  

http://git-wip-us.apache.org/repos/asf/tajo/blob/940546a0/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/TableDesc.java
----------------------------------------------------------------------
diff --cc tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/TableDesc.java
index ce167e1,ce167e1..001b71a
--- a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/TableDesc.java
+++ b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/TableDesc.java
@@@ -30,7 -30,7 +30,7 @@@ import org.apache.tajo.catalog.proto.Ca
  import org.apache.tajo.catalog.proto.CatalogProtos.TableDescProto;
  import org.apache.tajo.catalog.statistics.TableStats;
  import org.apache.tajo.common.ProtoObject;
--import org.apache.tajo.json.GsonObject;
++import org.apache.tajo.storage.json.GsonObject;
  import org.apache.tajo.util.KeyValueSet;
  import org.apache.tajo.util.TUtil;
  

http://git-wip-us.apache.org/repos/asf/tajo/blob/940546a0/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/TableMeta.java
----------------------------------------------------------------------
diff --cc tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/TableMeta.java
index 2d95e6b,2d95e6b..6378d42
--- a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/TableMeta.java
+++ b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/TableMeta.java
@@@ -28,7 -28,7 +28,7 @@@ import org.apache.tajo.catalog.proto.Ca
  import org.apache.tajo.catalog.proto.CatalogProtos.TableProto;
  import org.apache.tajo.catalog.proto.CatalogProtos.TableProtoOrBuilder;
  import org.apache.tajo.common.ProtoObject;
--import org.apache.tajo.json.GsonObject;
++import org.apache.tajo.storage.json.GsonObject;
  import org.apache.tajo.util.KeyValueSet;
  
  import java.util.Map;

http://git-wip-us.apache.org/repos/asf/tajo/blob/940546a0/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/json/CatalogGsonHelper.java
----------------------------------------------------------------------
diff --cc tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/json/CatalogGsonHelper.java
index 2ecbe98,2ecbe98..5090a7d
--- a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/json/CatalogGsonHelper.java
+++ b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/json/CatalogGsonHelper.java
@@@ -25,7 -25,7 +25,7 @@@ import org.apache.tajo.catalog.TableMet
  import org.apache.tajo.function.Function;
  import org.apache.tajo.common.TajoDataTypes.DataType;
  import org.apache.tajo.datum.Datum;
--import org.apache.tajo.json.*;
++import org.apache.tajo.storage.json.*;
  import org.apache.tajo.util.TUtil;
  
  import java.lang.reflect.Type;

http://git-wip-us.apache.org/repos/asf/tajo/blob/940546a0/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/json/FunctionAdapter.java
----------------------------------------------------------------------
diff --cc tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/json/FunctionAdapter.java
index 14722c3,14722c3..849d970
--- a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/json/FunctionAdapter.java
+++ b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/json/FunctionAdapter.java
@@@ -20,8 -20,8 +20,8 @@@ package org.apache.tajo.catalog.json
  
  import com.google.gson.*;
  import org.apache.tajo.function.Function;
--import org.apache.tajo.json.CommonGsonHelper;
--import org.apache.tajo.json.GsonSerDerAdapter;
++import org.apache.tajo.storage.json.CommonGsonHelper;
++import org.apache.tajo.storage.json.GsonSerDerAdapter;
  
  import java.lang.reflect.Type;
  

http://git-wip-us.apache.org/repos/asf/tajo/blob/940546a0/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/json/TableMetaAdapter.java
----------------------------------------------------------------------
diff --cc tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/json/TableMetaAdapter.java
index 3f4304a,3f4304a..33d504e
--- a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/json/TableMetaAdapter.java
+++ b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/json/TableMetaAdapter.java
@@@ -20,11 -20,11 +20,11 @@@ package org.apache.tajo.catalog.json
  
  import com.google.common.base.Preconditions;
  import com.google.gson.*;
--import org.apache.tajo.json.CommonGsonHelper;
++import org.apache.tajo.storage.json.CommonGsonHelper;
  import org.apache.tajo.util.KeyValueSet;
  import org.apache.tajo.catalog.TableMeta;
  import org.apache.tajo.catalog.proto.CatalogProtos;
--import org.apache.tajo.json.GsonSerDerAdapter;
++import org.apache.tajo.storage.json.GsonSerDerAdapter;
  
  import java.lang.reflect.Type;
  

http://git-wip-us.apache.org/repos/asf/tajo/blob/940546a0/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/partition/PartitionDesc.java
----------------------------------------------------------------------
diff --cc tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/partition/PartitionDesc.java
index d775ba8,d775ba8..9d520d4
--- a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/partition/PartitionDesc.java
+++ b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/partition/PartitionDesc.java
@@@ -23,7 -23,7 +23,7 @@@ import com.google.gson.annotations.Expo
  import org.apache.tajo.catalog.json.CatalogGsonHelper;
  import org.apache.tajo.catalog.proto.CatalogProtos;
  import org.apache.tajo.common.ProtoObject;
--import org.apache.tajo.json.GsonObject;
++import org.apache.tajo.storage.json.GsonObject;
  
  /**
   * <code>PartitionDesc</code> presents a table partition.

http://git-wip-us.apache.org/repos/asf/tajo/blob/940546a0/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/partition/PartitionMethodDesc.java
----------------------------------------------------------------------
diff --cc tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/partition/PartitionMethodDesc.java
index 721a7a0,721a7a0..adc5620
--- a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/partition/PartitionMethodDesc.java
+++ b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/partition/PartitionMethodDesc.java
@@@ -26,7 -26,7 +26,7 @@@ import org.apache.tajo.catalog.Schema
  import org.apache.tajo.catalog.json.CatalogGsonHelper;
  import org.apache.tajo.catalog.proto.CatalogProtos;
  import org.apache.tajo.common.ProtoObject;
--import org.apache.tajo.json.GsonObject;
++import org.apache.tajo.storage.json.GsonObject;
  import org.apache.tajo.util.TUtil;
  
  import static org.apache.tajo.catalog.proto.CatalogProtos.PartitionType;

http://git-wip-us.apache.org/repos/asf/tajo/blob/940546a0/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/statistics/ColumnStats.java
----------------------------------------------------------------------
diff --cc tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/statistics/ColumnStats.java
index 1bd0276,1bd0276..ef6e35a
--- a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/statistics/ColumnStats.java
+++ b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/statistics/ColumnStats.java
@@@ -30,7 -30,7 +30,7 @@@ import org.apache.tajo.catalog.proto.Ca
  import org.apache.tajo.common.ProtoObject;
  import org.apache.tajo.datum.Datum;
  import org.apache.tajo.datum.DatumFactory;
--import org.apache.tajo.json.GsonObject;
++import org.apache.tajo.storage.json.GsonObject;
  import org.apache.tajo.util.TUtil;
  
  public class ColumnStats implements ProtoObject<CatalogProtos.ColumnStatsProto>, Cloneable, GsonObject {

http://git-wip-us.apache.org/repos/asf/tajo/blob/940546a0/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/statistics/TableStats.java
----------------------------------------------------------------------
diff --cc tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/statistics/TableStats.java
index dd358ae,dd358ae..9df01da
--- a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/statistics/TableStats.java
+++ b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/statistics/TableStats.java
@@@ -26,7 -26,7 +26,7 @@@ import com.google.gson.Gson
  import com.google.gson.annotations.Expose;
  import org.apache.tajo.catalog.proto.CatalogProtos;
  import org.apache.tajo.common.TajoDataTypes;
--import org.apache.tajo.json.GsonObject;
++import org.apache.tajo.storage.json.GsonObject;
  import org.apache.tajo.catalog.json.CatalogGsonHelper;
  import org.apache.tajo.catalog.proto.CatalogProtos.TableStatsProto;
  import org.apache.tajo.common.ProtoObject;

http://git-wip-us.apache.org/repos/asf/tajo/blob/940546a0/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/function/Function.java
----------------------------------------------------------------------
diff --cc tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/function/Function.java
index 6a538b8,6a538b8..8842746
--- a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/function/Function.java
+++ b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/function/Function.java
@@@ -21,7 -21,7 +21,7 @@@ package org.apache.tajo.function
  import com.google.common.base.Objects;
  import com.google.gson.annotations.Expose;
  import org.apache.tajo.catalog.proto.CatalogProtos;
--import org.apache.tajo.json.GsonObject;
++import org.apache.tajo.storage.json.GsonObject;
  import org.apache.tajo.catalog.Column;
  import org.apache.tajo.catalog.json.CatalogGsonHelper;
  import org.apache.tajo.datum.Datum;

http://git-wip-us.apache.org/repos/asf/tajo/blob/940546a0/tajo-catalog/tajo-catalog-common/src/main/proto/CatalogProtos.proto
----------------------------------------------------------------------
diff --cc tajo-catalog/tajo-catalog-common/src/main/proto/CatalogProtos.proto
index 92437af,f29bc6c..26951dd
--- a/tajo-catalog/tajo-catalog-common/src/main/proto/CatalogProtos.proto
+++ b/tajo-catalog/tajo-catalog-common/src/main/proto/CatalogProtos.proto
@@@ -36,7 -36,7 +36,8 @@@ enum StoreType 
    SEQUENCEFILE = 8;
    AVRO = 9;
    TEXTFILE = 10;
-   HBASE = 11;
+   JSON = 11;
++  HBASE = 12;
  }
  
  enum OrderType {

http://git-wip-us.apache.org/repos/asf/tajo/blob/940546a0/tajo-common/src/main/java/org/apache/tajo/conf/TajoConf.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/tajo/blob/940546a0/tajo-common/src/main/java/org/apache/tajo/datum/Datum.java
----------------------------------------------------------------------
diff --cc tajo-common/src/main/java/org/apache/tajo/datum/Datum.java
index 442db71,442db71..dcf966c
--- a/tajo-common/src/main/java/org/apache/tajo/datum/Datum.java
+++ b/tajo-common/src/main/java/org/apache/tajo/datum/Datum.java
@@@ -23,8 -23,8 +23,8 @@@ import org.apache.tajo.conf.TajoConf
  import org.apache.tajo.conf.TajoConf.ConfVars;
  import org.apache.tajo.exception.InvalidCastException;
  import org.apache.tajo.exception.InvalidOperationException;
--import org.apache.tajo.json.CommonGsonHelper;
--import org.apache.tajo.json.GsonObject;
++import org.apache.tajo.storage.json.CommonGsonHelper;
++import org.apache.tajo.storage.json.GsonObject;
  
  import static org.apache.tajo.common.TajoDataTypes.Type;
  

http://git-wip-us.apache.org/repos/asf/tajo/blob/940546a0/tajo-common/src/main/java/org/apache/tajo/json/ClassNameSerializer.java
----------------------------------------------------------------------
diff --cc tajo-common/src/main/java/org/apache/tajo/json/ClassNameSerializer.java
index c7ec7ae,c7ec7ae..0000000
deleted file mode 100644,100644
--- a/tajo-common/src/main/java/org/apache/tajo/json/ClassNameSerializer.java
+++ /dev/null
@@@ -1,47 -1,47 +1,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.json;
--
--import com.google.gson.*;
--import org.apache.tajo.util.ClassUtil;
--
--import java.lang.reflect.Type;
--
--public class ClassNameSerializer implements GsonSerDerAdapter<Class> {
--
--	@Override
--	public JsonElement serialize(Class clazz, Type type,
--			JsonSerializationContext ctx) {
--		return new JsonPrimitive(clazz.getName());
--	}
--
--  @Override
--  public Class deserialize(JsonElement json, Type type,
--                           JsonDeserializationContext ctx) throws JsonParseException {
--    try {
--      return ClassUtil.forName(json.getAsString());
--    } catch (ClassNotFoundException e) {
--      e.printStackTrace();
--    }
--    return null;
--  }
--}

http://git-wip-us.apache.org/repos/asf/tajo/blob/940546a0/tajo-common/src/main/java/org/apache/tajo/json/CommonGsonHelper.java
----------------------------------------------------------------------
diff --cc tajo-common/src/main/java/org/apache/tajo/json/CommonGsonHelper.java
index 8994b11,8994b11..0000000
deleted file mode 100644,100644
--- a/tajo-common/src/main/java/org/apache/tajo/json/CommonGsonHelper.java
+++ /dev/null
@@@ -1,92 -1,92 +1,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.json;
--
--import com.google.gson.Gson;
--import com.google.gson.GsonBuilder;
--import com.google.gson.JsonElement;
--import com.google.gson.JsonNull;
--import com.google.gson.JsonObject;
--import com.google.gson.JsonParseException;
--import org.apache.tajo.datum.Datum;
--import org.apache.tajo.util.TUtil;
--
--import java.lang.reflect.Type;
--import java.util.Map;
--
--public class CommonGsonHelper {
--  private static Gson gson;
--  private static Gson gsonPretty;
--
--  private CommonGsonHelper() {
--  }
--
--	private static Map<Type, GsonSerDerAdapter> registerAdapters() {
--    Map<Type, GsonSerDerAdapter> adapters = TUtil.newHashMap();
--    adapters.put(Datum.class, new DatumAdapter());
--
--    return adapters;
--	}
--
--	public static Gson getInstance() {
--	  if (gson == null ) {
--      GsonHelper helper = new GsonHelper(registerAdapters());
--      gson = helper.getGson();
--	  }
--	  return gson;
--	}
--
--  public static Gson getPrettyInstance() {
--    if (gsonPretty == null) {
--      GsonBuilder prettyBuilder = new GsonBuilder()
--          .setPrettyPrinting()
--          .excludeFieldsWithoutExposeAnnotation();
--      GsonHelper.registerAdapters(prettyBuilder, registerAdapters());
--      gsonPretty = prettyBuilder.create();
--    }
--
--    return gsonPretty;
--  }
--
--  public static String toJson(GsonObject object, Class<? extends GsonObject> clazz) {
--    return getInstance().toJson(object, clazz);
--  }
--
--  public static <T extends GsonObject> T fromJson(String json, Class<T> clazz) {
--    return getInstance().fromJson(json, clazz);
--  }
--
--  /**
--   * A helper method that gets a JSON object member value after making sure it exists and has a valid value. Useful when
--   * a member value should present to proceed.
--   * @param object A JSON object to get a member value from
--   * @param memberName The name of a member to get value of
--   * @return {@link JsonElement} value read from the given member
--   * @throws JsonParseException When the specified member does not exist or have a value.
--   */
--  public static JsonElement getOrDie(JsonObject object, String memberName) throws JsonParseException {
--    if (object.has(memberName)) {
--      JsonElement element = object.get(memberName);
--      if (!JsonNull.INSTANCE.equals(element)) {
--        return element;
--      }
--    }
--    throw new JsonParseException("Field '" + memberName + "' not found in JSON object '" + object + "'");
--  }
--}

http://git-wip-us.apache.org/repos/asf/tajo/blob/940546a0/tajo-common/src/main/java/org/apache/tajo/json/DataTypeAdapter.java
----------------------------------------------------------------------
diff --cc tajo-common/src/main/java/org/apache/tajo/json/DataTypeAdapter.java
index fc15aea,fc15aea..0000000
deleted file mode 100644,100644
--- a/tajo-common/src/main/java/org/apache/tajo/json/DataTypeAdapter.java
+++ /dev/null
@@@ -1,63 -1,63 +1,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.json;
--
--import com.google.gson.*;
--import org.apache.tajo.common.TajoDataTypes;
--import org.apache.tajo.common.TajoDataTypes.DataType;
--
--import java.lang.reflect.Type;
--
--
--public class DataTypeAdapter implements GsonSerDerAdapter<DataType> {
--
--  @Override
--  public DataType deserialize(JsonElement json, Type typeOfT, JsonDeserializationContext context)
--      throws JsonParseException {
--
--    JsonObject obj = (JsonObject) json;
--    DataType.Builder builder = DataType.newBuilder();
--    TajoDataTypes.Type type = TajoDataTypes.Type.valueOf(CommonGsonHelper.getOrDie(obj, "type").getAsString());
--    builder.setType(type);
--
--    JsonElement len = obj.get("len");
--    if (len != null) {
--      builder.setLength(len.getAsInt());
--    }
--    JsonElement code = obj.get("code");
--    if (code != null) {
--      builder.setCode(code.getAsString());
--    }
--    return builder.build();
--  }
--
--  @Override
--  public JsonElement serialize(DataType src, Type typeOfSrc, JsonSerializationContext context) {
--    JsonObject json = new JsonObject();
--    json.addProperty("type", src.getType().name());
--    if (src.hasLength()) {
--      json.addProperty("len", src.getLength());
--    }
--    if (src.hasCode()) {
--      json.addProperty("code", src.getCode());
--    }
--
--    return json;
--  }
--}

http://git-wip-us.apache.org/repos/asf/tajo/blob/940546a0/tajo-common/src/main/java/org/apache/tajo/json/DatumAdapter.java
----------------------------------------------------------------------
diff --cc tajo-common/src/main/java/org/apache/tajo/json/DatumAdapter.java
index d65559d,d65559d..0000000
deleted file mode 100644,100644
--- a/tajo-common/src/main/java/org/apache/tajo/json/DatumAdapter.java
+++ /dev/null
@@@ -1,76 -1,76 +1,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.json;
--
--import com.google.gson.*;
--import org.apache.tajo.common.TajoDataTypes;
--import org.apache.tajo.datum.*;
--
--import java.lang.reflect.Type;
--
--public class DatumAdapter implements GsonSerDerAdapter<Datum> {
--
--	@Override
--	public Datum deserialize(JsonElement json, Type typeOfT,
--			JsonDeserializationContext context) throws JsonParseException {
--		JsonObject jsonObject = json.getAsJsonObject();
--		String typeName = CommonGsonHelper.getOrDie(jsonObject, "type").getAsString();
--    TajoDataTypes.Type type = TajoDataTypes.Type.valueOf(typeName);
--    switch (type) {
--    case DATE:
--      return new DateDatum(CommonGsonHelper.getOrDie(jsonObject, "value").getAsInt());
--    case TIME:
--      return new TimeDatum(CommonGsonHelper.getOrDie(jsonObject, "value").getAsLong());
--    case TIMESTAMP:
--      return new TimestampDatum(CommonGsonHelper.getOrDie(jsonObject, "value").getAsLong());
--    case INTERVAL:
--      String[] values = CommonGsonHelper.getOrDie(jsonObject, "value").getAsString().split(",");
--
--      return new IntervalDatum(Integer.parseInt(values[0]), Long.parseLong(values[1]));
--    default:
--      return context.deserialize(CommonGsonHelper.getOrDie(jsonObject, "body"),
--          DatumFactory.getDatumClass(TajoDataTypes.Type.valueOf(typeName)));
--    }
--	}
--
--	@Override
--	public JsonElement serialize(Datum src, Type typeOfSrc, JsonSerializationContext context) {
--		JsonObject jsonObj = new JsonObject();
--		jsonObj.addProperty("type", src.type().name());
--    switch (src.type()) {
--    case DATE:
--      jsonObj.addProperty("value", src.asInt4());
--      break;
--    case TIME:
--      jsonObj.addProperty("value", src.asInt8());
--      break;
--    case TIMESTAMP:
--      jsonObj.addProperty("value", src.asInt8());
--      break;
--    case INTERVAL:
--      IntervalDatum interval = (IntervalDatum)src;
--      jsonObj.addProperty("value", interval.getMonths() + "," + interval.getMilliSeconds());
--      break;
--    default:
--      jsonObj.add("body", context.serialize(src));
--    }
--
--		return jsonObj;
--	}
--}

http://git-wip-us.apache.org/repos/asf/tajo/blob/940546a0/tajo-common/src/main/java/org/apache/tajo/json/GsonHelper.java
----------------------------------------------------------------------
diff --cc tajo-common/src/main/java/org/apache/tajo/json/GsonHelper.java
index 8c6d891,8c6d891..0000000
deleted file mode 100644,100644
--- a/tajo-common/src/main/java/org/apache/tajo/json/GsonHelper.java
+++ /dev/null
@@@ -1,49 -1,49 +1,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.json;
--
--import com.google.gson.*;
--
--import java.lang.reflect.Type;
--import java.util.Map;
--
--public class GsonHelper {
--  private final GsonBuilder builder;
--  private final Gson gson;
--
--  public GsonHelper(Map<Type, GsonSerDerAdapter> adapters) {
--    builder = new GsonBuilder().excludeFieldsWithoutExposeAnnotation();
--    registerAdapters(builder, adapters);
--    gson = builder.create();
--  }
--
--  public static void registerAdapters(GsonBuilder builder, Map<Type, GsonSerDerAdapter> adapters) {
--    for (Map.Entry<Type, GsonSerDerAdapter> entry : adapters.entrySet()) {
--      try {
--        builder.registerTypeAdapter(entry.getKey(), entry.getValue());
--      } catch (Exception e) {
--        e.printStackTrace();
--      }
--    }
--  }
--
--  public Gson getGson() {
--    return gson;
--  }
--}

http://git-wip-us.apache.org/repos/asf/tajo/blob/940546a0/tajo-common/src/main/java/org/apache/tajo/json/GsonObject.java
----------------------------------------------------------------------
diff --cc tajo-common/src/main/java/org/apache/tajo/json/GsonObject.java
index 3c49a21,3c49a21..0000000
deleted file mode 100644,100644
--- a/tajo-common/src/main/java/org/apache/tajo/json/GsonObject.java
+++ /dev/null
@@@ -1,23 -1,23 +1,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.json;
--
--public interface GsonObject {
--  public String toJson();
--}

http://git-wip-us.apache.org/repos/asf/tajo/blob/940546a0/tajo-common/src/main/java/org/apache/tajo/json/GsonSerDerAdapter.java
----------------------------------------------------------------------
diff --cc tajo-common/src/main/java/org/apache/tajo/json/GsonSerDerAdapter.java
index 80a40f8,80a40f8..0000000
deleted file mode 100644,100644
--- a/tajo-common/src/main/java/org/apache/tajo/json/GsonSerDerAdapter.java
+++ /dev/null
@@@ -1,25 -1,25 +1,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.json;
--
--import com.google.gson.JsonDeserializer;
--import com.google.gson.JsonSerializer;
--
--public interface GsonSerDerAdapter<T> extends JsonSerializer<T>, JsonDeserializer<T> {
--}

http://git-wip-us.apache.org/repos/asf/tajo/blob/940546a0/tajo-common/src/main/java/org/apache/tajo/json/PathSerializer.java
----------------------------------------------------------------------
diff --cc tajo-common/src/main/java/org/apache/tajo/json/PathSerializer.java
index c286ae3,c286ae3..0000000
deleted file mode 100644,100644
--- a/tajo-common/src/main/java/org/apache/tajo/json/PathSerializer.java
+++ /dev/null
@@@ -1,41 -1,41 +1,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.json;
--
--import com.google.gson.*;
--import org.apache.hadoop.fs.Path;
--
--import java.lang.reflect.Type;
--
--public class PathSerializer implements GsonSerDerAdapter<Path> {
--
--	@Override
--	public JsonElement serialize(Path path, Type arg1,
--			JsonSerializationContext arg2) {
--		return new JsonPrimitive(path.toString());
--	}
--
--  @Override
--  public Path deserialize(JsonElement arg0, Type arg1, JsonDeserializationContext context) throws JsonParseException {
--    return new Path(arg0.getAsJsonPrimitive().getAsString());
--  }
--}

http://git-wip-us.apache.org/repos/asf/tajo/blob/940546a0/tajo-common/src/main/java/org/apache/tajo/storage/StorageConstants.java
----------------------------------------------------------------------
diff --cc tajo-common/src/main/java/org/apache/tajo/storage/StorageConstants.java
index 11ac9b7,a3d8de0..942718b
--- a/tajo-common/src/main/java/org/apache/tajo/storage/StorageConstants.java
+++ b/tajo-common/src/main/java/org/apache/tajo/storage/StorageConstants.java
@@@ -32,7 -32,8 +32,20 @@@ public class StorageConstants 
  
    public static final String TEXT_DELIMITER = "text.delimiter";
    public static final String TEXT_NULL = "text.null";
-   public static final String TEXTFILE_SERDE = "textfile.serde";
+   public static final String TEXT_SERDE_CLASS = "text.serde.class";
+   public static final String DEFAULT_TEXT_SERDE_CLASS = "org.apache.tajo.storage.text.CSVLineSerDe";
++  /**
++   * It's the maximum number of parsing error torrence.
++   *
++   * <ul>
++   *   <li>If it is -1, it is always torrent against any parsing error.</li>
++   *   <li>If it is 0, it does not permit any parsing error.</li>
++   *   <li>If it is some positive integer (i.e., > 0), the given number of parsing errors in each
++   *       task will be permissible</li>
++   * </ul>
++   **/
++  public static final String TEXT_ERROR_TOLERANCE_MAXNUM = "text.error-tolerance.max-num";
++  public static final String DEFAULT_TEXT_ERROR_TOLERANCE_MAXNUM = "0";
  
    @Deprecated
    public static final String SEQUENCEFILE_DELIMITER = "sequencefile.delimiter";

http://git-wip-us.apache.org/repos/asf/tajo/blob/940546a0/tajo-common/src/main/java/org/apache/tajo/storage/json/ClassNameSerializer.java
----------------------------------------------------------------------
diff --cc tajo-common/src/main/java/org/apache/tajo/storage/json/ClassNameSerializer.java
index 0000000,0000000..7069134
new file mode 100644
--- /dev/null
+++ b/tajo-common/src/main/java/org/apache/tajo/storage/json/ClassNameSerializer.java
@@@ -1,0 -1,0 +1,47 @@@
++/**
++ * Licensed to the Apache Software Foundation (ASF) under one
++ * or more contributor license agreements.  See the NOTICE file
++ * distributed with this work for additional information
++ * regarding copyright ownership.  The ASF licenses this file
++ * to you under the Apache License, Version 2.0 (the
++ * "License"); you may not use this file except in compliance
++ * with the License.  You may obtain a copy of the License at
++ *
++ *     http://www.apache.org/licenses/LICENSE-2.0
++ *
++ * Unless required by applicable law or agreed to in writing, software
++ * distributed under the License is distributed on an "AS IS" BASIS,
++ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
++ * See the License for the specific language governing permissions and
++ * limitations under the License.
++ */
++
++/**
++ * 
++ */
++package org.apache.tajo.storage.json;
++
++import com.google.gson.*;
++import org.apache.tajo.util.ClassUtil;
++
++import java.lang.reflect.Type;
++
++public class ClassNameSerializer implements GsonSerDerAdapter<Class> {
++
++	@Override
++	public JsonElement serialize(Class clazz, Type type,
++			JsonSerializationContext ctx) {
++		return new JsonPrimitive(clazz.getName());
++	}
++
++  @Override
++  public Class deserialize(JsonElement json, Type type,
++                           JsonDeserializationContext ctx) throws JsonParseException {
++    try {
++      return ClassUtil.forName(json.getAsString());
++    } catch (ClassNotFoundException e) {
++      e.printStackTrace();
++    }
++    return null;
++  }
++}

http://git-wip-us.apache.org/repos/asf/tajo/blob/940546a0/tajo-common/src/main/java/org/apache/tajo/storage/json/CommonGsonHelper.java
----------------------------------------------------------------------
diff --cc tajo-common/src/main/java/org/apache/tajo/storage/json/CommonGsonHelper.java
index 0000000,0000000..e2d20cf
new file mode 100644
--- /dev/null
+++ b/tajo-common/src/main/java/org/apache/tajo/storage/json/CommonGsonHelper.java
@@@ -1,0 -1,0 +1,92 @@@
++/**
++ * Licensed to the Apache Software Foundation (ASF) under one
++ * or more contributor license agreements.  See the NOTICE file
++ * distributed with this work for additional information
++ * regarding copyright ownership.  The ASF licenses this file
++ * to you under the Apache License, Version 2.0 (the
++ * "License"); you may not use this file except in compliance
++ * with the License.  You may obtain a copy of the License at
++ *
++ *     http://www.apache.org/licenses/LICENSE-2.0
++ *
++ * Unless required by applicable law or agreed to in writing, software
++ * distributed under the License is distributed on an "AS IS" BASIS,
++ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
++ * See the License for the specific language governing permissions and
++ * limitations under the License.
++ */
++
++package org.apache.tajo.storage.json;
++
++import com.google.gson.Gson;
++import com.google.gson.GsonBuilder;
++import com.google.gson.JsonElement;
++import com.google.gson.JsonNull;
++import com.google.gson.JsonObject;
++import com.google.gson.JsonParseException;
++import org.apache.tajo.datum.Datum;
++import org.apache.tajo.util.TUtil;
++
++import java.lang.reflect.Type;
++import java.util.Map;
++
++public class CommonGsonHelper {
++  private static Gson gson;
++  private static Gson gsonPretty;
++
++  private CommonGsonHelper() {
++  }
++
++	private static Map<Type, GsonSerDerAdapter> registerAdapters() {
++    Map<Type, GsonSerDerAdapter> adapters = TUtil.newHashMap();
++    adapters.put(Datum.class, new DatumAdapter());
++
++    return adapters;
++	}
++
++	public static Gson getInstance() {
++	  if (gson == null ) {
++      GsonHelper helper = new GsonHelper(registerAdapters());
++      gson = helper.getGson();
++	  }
++	  return gson;
++	}
++
++  public static Gson getPrettyInstance() {
++    if (gsonPretty == null) {
++      GsonBuilder prettyBuilder = new GsonBuilder()
++          .setPrettyPrinting()
++          .excludeFieldsWithoutExposeAnnotation();
++      GsonHelper.registerAdapters(prettyBuilder, registerAdapters());
++      gsonPretty = prettyBuilder.create();
++    }
++
++    return gsonPretty;
++  }
++
++  public static String toJson(GsonObject object, Class<? extends GsonObject> clazz) {
++    return getInstance().toJson(object, clazz);
++  }
++
++  public static <T extends GsonObject> T fromJson(String json, Class<T> clazz) {
++    return getInstance().fromJson(json, clazz);
++  }
++
++  /**
++   * A helper method that gets a JSON object member value after making sure it exists and has a valid value. Useful when
++   * a member value should present to proceed.
++   * @param object A JSON object to get a member value from
++   * @param memberName The name of a member to get value of
++   * @return {@link JsonElement} value read from the given member
++   * @throws JsonParseException When the specified member does not exist or have a value.
++   */
++  public static JsonElement getOrDie(JsonObject object, String memberName) throws JsonParseException {
++    if (object.has(memberName)) {
++      JsonElement element = object.get(memberName);
++      if (!JsonNull.INSTANCE.equals(element)) {
++        return element;
++      }
++    }
++    throw new JsonParseException("Field '" + memberName + "' not found in JSON object '" + object + "'");
++  }
++}

http://git-wip-us.apache.org/repos/asf/tajo/blob/940546a0/tajo-common/src/main/java/org/apache/tajo/storage/json/DataTypeAdapter.java
----------------------------------------------------------------------
diff --cc tajo-common/src/main/java/org/apache/tajo/storage/json/DataTypeAdapter.java
index 0000000,0000000..e8765c9
new file mode 100644
--- /dev/null
+++ b/tajo-common/src/main/java/org/apache/tajo/storage/json/DataTypeAdapter.java
@@@ -1,0 -1,0 +1,63 @@@
++/**
++ * Licensed to the Apache Software Foundation (ASF) under one
++ * or more contributor license agreements.  See the NOTICE file
++ * distributed with this work for additional information
++ * regarding copyright ownership.  The ASF licenses this file
++ * to you under the Apache License, Version 2.0 (the
++ * "License"); you may not use this file except in compliance
++ * with the License.  You may obtain a copy of the License at
++ *
++ *      http://www.apache.org/licenses/LICENSE-2.0
++ *
++ * Unless required by applicable law or agreed to in writing, software
++ * distributed under the License is distributed on an "AS IS" BASIS,
++ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
++ * See the License for the specific language governing permissions and
++ * limitations under the License.
++ */
++
++package org.apache.tajo.storage.json;
++
++import com.google.gson.*;
++import org.apache.tajo.common.TajoDataTypes;
++import org.apache.tajo.common.TajoDataTypes.DataType;
++
++import java.lang.reflect.Type;
++
++
++public class DataTypeAdapter implements GsonSerDerAdapter<DataType> {
++
++  @Override
++  public DataType deserialize(JsonElement json, Type typeOfT, JsonDeserializationContext context)
++      throws JsonParseException {
++
++    JsonObject obj = (JsonObject) json;
++    DataType.Builder builder = DataType.newBuilder();
++    TajoDataTypes.Type type = TajoDataTypes.Type.valueOf(CommonGsonHelper.getOrDie(obj, "type").getAsString());
++    builder.setType(type);
++
++    JsonElement len = obj.get("len");
++    if (len != null) {
++      builder.setLength(len.getAsInt());
++    }
++    JsonElement code = obj.get("code");
++    if (code != null) {
++      builder.setCode(code.getAsString());
++    }
++    return builder.build();
++  }
++
++  @Override
++  public JsonElement serialize(DataType src, Type typeOfSrc, JsonSerializationContext context) {
++    JsonObject json = new JsonObject();
++    json.addProperty("type", src.getType().name());
++    if (src.hasLength()) {
++      json.addProperty("len", src.getLength());
++    }
++    if (src.hasCode()) {
++      json.addProperty("code", src.getCode());
++    }
++
++    return json;
++  }
++}

http://git-wip-us.apache.org/repos/asf/tajo/blob/940546a0/tajo-common/src/main/java/org/apache/tajo/storage/json/DatumAdapter.java
----------------------------------------------------------------------
diff --cc tajo-common/src/main/java/org/apache/tajo/storage/json/DatumAdapter.java
index 0000000,0000000..b176720
new file mode 100644
--- /dev/null
+++ b/tajo-common/src/main/java/org/apache/tajo/storage/json/DatumAdapter.java
@@@ -1,0 -1,0 +1,76 @@@
++/**
++ * Licensed to the Apache Software Foundation (ASF) under one
++ * or more contributor license agreements.  See the NOTICE file
++ * distributed with this work for additional information
++ * regarding copyright ownership.  The ASF licenses this file
++ * to you under the Apache License, Version 2.0 (the
++ * "License"); you may not use this file except in compliance
++ * with the License.  You may obtain a copy of the License at
++ *
++ *     http://www.apache.org/licenses/LICENSE-2.0
++ *
++ * Unless required by applicable law or agreed to in writing, software
++ * distributed under the License is distributed on an "AS IS" BASIS,
++ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
++ * See the License for the specific language governing permissions and
++ * limitations under the License.
++ */
++
++package org.apache.tajo.storage.json;
++
++import com.google.gson.*;
++import org.apache.tajo.common.TajoDataTypes;
++import org.apache.tajo.datum.*;
++
++import java.lang.reflect.Type;
++
++public class DatumAdapter implements GsonSerDerAdapter<Datum> {
++
++	@Override
++	public Datum deserialize(JsonElement json, Type typeOfT,
++			JsonDeserializationContext context) throws JsonParseException {
++		JsonObject jsonObject = json.getAsJsonObject();
++		String typeName = CommonGsonHelper.getOrDie(jsonObject, "type").getAsString();
++    TajoDataTypes.Type type = TajoDataTypes.Type.valueOf(typeName);
++    switch (type) {
++    case DATE:
++      return new DateDatum(CommonGsonHelper.getOrDie(jsonObject, "value").getAsInt());
++    case TIME:
++      return new TimeDatum(CommonGsonHelper.getOrDie(jsonObject, "value").getAsLong());
++    case TIMESTAMP:
++      return new TimestampDatum(CommonGsonHelper.getOrDie(jsonObject, "value").getAsLong());
++    case INTERVAL:
++      String[] values = CommonGsonHelper.getOrDie(jsonObject, "value").getAsString().split(",");
++
++      return new IntervalDatum(Integer.parseInt(values[0]), Long.parseLong(values[1]));
++    default:
++      return context.deserialize(CommonGsonHelper.getOrDie(jsonObject, "body"),
++          DatumFactory.getDatumClass(TajoDataTypes.Type.valueOf(typeName)));
++    }
++	}
++
++	@Override
++	public JsonElement serialize(Datum src, Type typeOfSrc, JsonSerializationContext context) {
++		JsonObject jsonObj = new JsonObject();
++		jsonObj.addProperty("type", src.type().name());
++    switch (src.type()) {
++    case DATE:
++      jsonObj.addProperty("value", src.asInt4());
++      break;
++    case TIME:
++      jsonObj.addProperty("value", src.asInt8());
++      break;
++    case TIMESTAMP:
++      jsonObj.addProperty("value", src.asInt8());
++      break;
++    case INTERVAL:
++      IntervalDatum interval = (IntervalDatum)src;
++      jsonObj.addProperty("value", interval.getMonths() + "," + interval.getMilliSeconds());
++      break;
++    default:
++      jsonObj.add("body", context.serialize(src));
++    }
++
++		return jsonObj;
++	}
++}

http://git-wip-us.apache.org/repos/asf/tajo/blob/940546a0/tajo-common/src/main/java/org/apache/tajo/storage/json/GsonHelper.java
----------------------------------------------------------------------
diff --cc tajo-common/src/main/java/org/apache/tajo/storage/json/GsonHelper.java
index 0000000,0000000..2e193ba
new file mode 100644
--- /dev/null
+++ b/tajo-common/src/main/java/org/apache/tajo/storage/json/GsonHelper.java
@@@ -1,0 -1,0 +1,49 @@@
++/**
++ * Licensed to the Apache Software Foundation (ASF) under one
++ * or more contributor license agreements.  See the NOTICE file
++ * distributed with this work for additional information
++ * regarding copyright ownership.  The ASF licenses this file
++ * to you under the Apache License, Version 2.0 (the
++ * "License"); you may not use this file except in compliance
++ * with the License.  You may obtain a copy of the License at
++ *
++ *     http://www.apache.org/licenses/LICENSE-2.0
++ *
++ * Unless required by applicable law or agreed to in writing, software
++ * distributed under the License is distributed on an "AS IS" BASIS,
++ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
++ * See the License for the specific language governing permissions and
++ * limitations under the License.
++ */
++
++package org.apache.tajo.storage.json;
++
++import com.google.gson.*;
++
++import java.lang.reflect.Type;
++import java.util.Map;
++
++public class GsonHelper {
++  private final GsonBuilder builder;
++  private final Gson gson;
++
++  public GsonHelper(Map<Type, GsonSerDerAdapter> adapters) {
++    builder = new GsonBuilder().excludeFieldsWithoutExposeAnnotation();
++    registerAdapters(builder, adapters);
++    gson = builder.create();
++  }
++
++  public static void registerAdapters(GsonBuilder builder, Map<Type, GsonSerDerAdapter> adapters) {
++    for (Map.Entry<Type, GsonSerDerAdapter> entry : adapters.entrySet()) {
++      try {
++        builder.registerTypeAdapter(entry.getKey(), entry.getValue());
++      } catch (Exception e) {
++        e.printStackTrace();
++      }
++    }
++  }
++
++  public Gson getGson() {
++    return gson;
++  }
++}

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

http://git-wip-us.apache.org/repos/asf/tajo/blob/940546a0/tajo-common/src/main/java/org/apache/tajo/storage/json/GsonSerDerAdapter.java
----------------------------------------------------------------------
diff --cc tajo-common/src/main/java/org/apache/tajo/storage/json/GsonSerDerAdapter.java
index 0000000,0000000..493a997
new file mode 100644
--- /dev/null
+++ b/tajo-common/src/main/java/org/apache/tajo/storage/json/GsonSerDerAdapter.java
@@@ -1,0 -1,0 +1,25 @@@
++/**
++ * Licensed to the Apache Software Foundation (ASF) under one
++ * or more contributor license agreements.  See the NOTICE file
++ * distributed with this work for additional information
++ * regarding copyright ownership.  The ASF licenses this file
++ * to you under the Apache License, Version 2.0 (the
++ * "License"); you may not use this file except in compliance
++ * with the License.  You may obtain a copy of the License at
++ *
++ *     http://www.apache.org/licenses/LICENSE-2.0
++ *
++ * Unless required by applicable law or agreed to in writing, software
++ * distributed under the License is distributed on an "AS IS" BASIS,
++ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
++ * See the License for the specific language governing permissions and
++ * limitations under the License.
++ */
++
++package org.apache.tajo.storage.json;
++
++import com.google.gson.JsonDeserializer;
++import com.google.gson.JsonSerializer;
++
++public interface GsonSerDerAdapter<T> extends JsonSerializer<T>, JsonDeserializer<T> {
++}

http://git-wip-us.apache.org/repos/asf/tajo/blob/940546a0/tajo-common/src/main/java/org/apache/tajo/storage/json/PathSerializer.java
----------------------------------------------------------------------
diff --cc tajo-common/src/main/java/org/apache/tajo/storage/json/PathSerializer.java
index 0000000,0000000..6d9d32f
new file mode 100644
--- /dev/null
+++ b/tajo-common/src/main/java/org/apache/tajo/storage/json/PathSerializer.java
@@@ -1,0 -1,0 +1,41 @@@
++/**
++ * Licensed to the Apache Software Foundation (ASF) under one
++ * or more contributor license agreements.  See the NOTICE file
++ * distributed with this work for additional information
++ * regarding copyright ownership.  The ASF licenses this file
++ * to you under the Apache License, Version 2.0 (the
++ * "License"); you may not use this file except in compliance
++ * with the License.  You may obtain a copy of the License at
++ *
++ *     http://www.apache.org/licenses/LICENSE-2.0
++ *
++ * Unless required by applicable law or agreed to in writing, software
++ * distributed under the License is distributed on an "AS IS" BASIS,
++ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
++ * See the License for the specific language governing permissions and
++ * limitations under the License.
++ */
++
++/**
++ * 
++ */
++package org.apache.tajo.storage.json;
++
++import com.google.gson.*;
++import org.apache.hadoop.fs.Path;
++
++import java.lang.reflect.Type;
++
++public class PathSerializer implements GsonSerDerAdapter<Path> {
++
++	@Override
++	public JsonElement serialize(Path path, Type arg1,
++			JsonSerializationContext arg2) {
++		return new JsonPrimitive(path.toString());
++	}
++
++  @Override
++  public Path deserialize(JsonElement arg0, Type arg1, JsonDeserializationContext context) throws JsonParseException {
++    return new Path(arg0.getAsJsonPrimitive().getAsString());
++  }
++}

http://git-wip-us.apache.org/repos/asf/tajo/blob/940546a0/tajo-common/src/main/java/org/apache/tajo/util/KeyValueSet.java
----------------------------------------------------------------------
diff --cc tajo-common/src/main/java/org/apache/tajo/util/KeyValueSet.java
index 8e3eb2a,8e3eb2a..a52651b
--- a/tajo-common/src/main/java/org/apache/tajo/util/KeyValueSet.java
+++ b/tajo-common/src/main/java/org/apache/tajo/util/KeyValueSet.java
@@@ -22,8 -22,8 +22,8 @@@ import com.google.common.base.Objects
  import com.google.common.base.Preconditions;
  import com.google.gson.annotations.Expose;
  import org.apache.tajo.common.ProtoObject;
--import org.apache.tajo.json.CommonGsonHelper;
--import org.apache.tajo.json.GsonObject;
++import org.apache.tajo.storage.json.CommonGsonHelper;
++import org.apache.tajo.storage.json.GsonObject;
  
  import java.util.HashMap;
  import java.util.Map;

http://git-wip-us.apache.org/repos/asf/tajo/blob/940546a0/tajo-common/src/test/java/org/apache/tajo/datum/TestBytesDatum.java
----------------------------------------------------------------------
diff --cc tajo-common/src/test/java/org/apache/tajo/datum/TestBytesDatum.java
index 4dcbbee,4dcbbee..d42b496
--- a/tajo-common/src/test/java/org/apache/tajo/datum/TestBytesDatum.java
+++ b/tajo-common/src/test/java/org/apache/tajo/datum/TestBytesDatum.java
@@@ -19,7 -19,7 +19,7 @@@
  package org.apache.tajo.datum;
  
  import org.apache.tajo.common.TajoDataTypes.Type;
--import org.apache.tajo.json.CommonGsonHelper;
++import org.apache.tajo.storage.json.CommonGsonHelper;
  import org.apache.tajo.util.Bytes;
  import org.junit.Test;
  

http://git-wip-us.apache.org/repos/asf/tajo/blob/940546a0/tajo-common/src/test/java/org/apache/tajo/datum/TestDateDatum.java
----------------------------------------------------------------------
diff --cc tajo-common/src/test/java/org/apache/tajo/datum/TestDateDatum.java
index 41b4dca,41b4dca..56673f2
--- a/tajo-common/src/test/java/org/apache/tajo/datum/TestDateDatum.java
+++ b/tajo-common/src/test/java/org/apache/tajo/datum/TestDateDatum.java
@@@ -20,7 -20,7 +20,7 @@@ package org.apache.tajo.datum
  
  import org.apache.tajo.common.TajoDataTypes.Type;
  import org.apache.tajo.exception.InvalidCastException;
--import org.apache.tajo.json.CommonGsonHelper;
++import org.apache.tajo.storage.json.CommonGsonHelper;
  import org.junit.Test;
  
  import static org.junit.Assert.*;

http://git-wip-us.apache.org/repos/asf/tajo/blob/940546a0/tajo-common/src/test/java/org/apache/tajo/datum/TestInet4Datum.java
----------------------------------------------------------------------
diff --cc tajo-common/src/test/java/org/apache/tajo/datum/TestInet4Datum.java
index 14ebc7c,14ebc7c..0ef905d
--- a/tajo-common/src/test/java/org/apache/tajo/datum/TestInet4Datum.java
+++ b/tajo-common/src/test/java/org/apache/tajo/datum/TestInet4Datum.java
@@@ -18,7 -18,7 +18,7 @@@
  
  package org.apache.tajo.datum;
  
--import org.apache.tajo.json.CommonGsonHelper;
++import org.apache.tajo.storage.json.CommonGsonHelper;
  import org.junit.Before;
  import org.junit.Test;
  

http://git-wip-us.apache.org/repos/asf/tajo/blob/940546a0/tajo-common/src/test/java/org/apache/tajo/datum/TestTimeDatum.java
----------------------------------------------------------------------
diff --cc tajo-common/src/test/java/org/apache/tajo/datum/TestTimeDatum.java
index ea641ec,ea641ec..a130c0b
--- a/tajo-common/src/test/java/org/apache/tajo/datum/TestTimeDatum.java
+++ b/tajo-common/src/test/java/org/apache/tajo/datum/TestTimeDatum.java
@@@ -21,7 -21,7 +21,7 @@@ package org.apache.tajo.datum
  import org.apache.tajo.common.TajoDataTypes.Type;
  import org.apache.tajo.common.TajoDataTypes.DataType;
  import org.apache.tajo.exception.InvalidCastException;
--import org.apache.tajo.json.CommonGsonHelper;
++import org.apache.tajo.storage.json.CommonGsonHelper;
  import org.junit.Test;
  
  import static org.junit.Assert.assertArrayEquals;

http://git-wip-us.apache.org/repos/asf/tajo/blob/940546a0/tajo-common/src/test/java/org/apache/tajo/datum/TestTimestampDatum.java
----------------------------------------------------------------------
diff --cc tajo-common/src/test/java/org/apache/tajo/datum/TestTimestampDatum.java
index 5f27cfa,5886083..f4b5c97
--- a/tajo-common/src/test/java/org/apache/tajo/datum/TestTimestampDatum.java
+++ b/tajo-common/src/test/java/org/apache/tajo/datum/TestTimestampDatum.java
@@@ -21,7 -21,7 +21,7 @@@ package org.apache.tajo.datum
  import org.apache.tajo.common.TajoDataTypes.Type;
  import org.apache.tajo.conf.TajoConf;
  import org.apache.tajo.exception.InvalidCastException;
--import org.apache.tajo.json.CommonGsonHelper;
++import org.apache.tajo.storage.json.CommonGsonHelper;
  import org.apache.tajo.util.datetime.DateTimeUtil;
  import org.junit.BeforeClass;
  import org.junit.Test;

http://git-wip-us.apache.org/repos/asf/tajo/blob/940546a0/tajo-core/pom.xml
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/tajo/blob/940546a0/tajo-core/src/main/java/org/apache/tajo/engine/json/CoreGsonHelper.java
----------------------------------------------------------------------
diff --cc tajo-core/src/main/java/org/apache/tajo/engine/json/CoreGsonHelper.java
index 57f2536,57f2536..1d10e67
--- a/tajo-core/src/main/java/org/apache/tajo/engine/json/CoreGsonHelper.java
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/json/CoreGsonHelper.java
@@@ -31,7 -31,7 +31,7 @@@ import org.apache.tajo.plan.function.Ag
  import org.apache.tajo.plan.function.GeneralFunction;
  import org.apache.tajo.plan.logical.LogicalNode;
  import org.apache.tajo.function.Function;
--import org.apache.tajo.json.*;
++import org.apache.tajo.storage.json.*;
  import org.apache.tajo.plan.serder.EvalNodeAdapter;
  import org.apache.tajo.plan.serder.LogicalNodeAdapter;
  import org.apache.tajo.util.TUtil;

http://git-wip-us.apache.org/repos/asf/tajo/blob/940546a0/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/PhysicalPlanUtil.java
----------------------------------------------------------------------
diff --cc tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/PhysicalPlanUtil.java
index f38ae3c,a63b838..247b373
--- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/PhysicalPlanUtil.java
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/PhysicalPlanUtil.java
@@@ -117,13 -122,33 +119,31 @@@ public class PhysicalPlanUtil 
      return FragmentConvertor.toFragmentProtoArray(fragments.toArray(new FileFragment[]{}));
    }
  
+   /**
+    *
+    * @param fs
+    * @param path The table path
+    * @param result The final result files to be used
+    * @param startFileIndex
+    * @param numResultFiles
+    * @param currentFileIndex
+    * @param partitioned A flag to indicate if this table is partitioned
+    * @param currentDepth Current visiting depth of partition directories
+    * @param maxDepth The partition depth of this table
+    * @throws IOException
+    */
    private static void getNonZeroLengthDataFiles(FileSystem fs, Path path, List<FileStatus> result,
                                           int startFileIndex, int numResultFiles,
-                                          AtomicInteger currentFileIndex) throws IOException {
+                                          AtomicInteger currentFileIndex, boolean partitioned,
+                                          int currentDepth, int maxDepth) throws IOException {
+     // Intermediate directory
      if (fs.isDirectory(path)) {
 -
 -      FileStatus[] files = fs.listStatus(path, StorageManager.hiddenFileFilter);
 -
 +      FileStatus[] files = fs.listStatus(path, FileStorageManager.hiddenFileFilter);
        if (files != null && files.length > 0) {
+ 
          for (FileStatus eachFile : files) {
+ 
+           // checking if the enough number of files are found
            if (result.size() >= numResultFiles) {
              return;
            }

http://git-wip-us.apache.org/repos/asf/tajo/blob/940546a0/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/SeqScanExec.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/tajo/blob/940546a0/tajo-core/src/main/java/org/apache/tajo/master/DefaultTaskScheduler.java
----------------------------------------------------------------------
diff --cc tajo-core/src/main/java/org/apache/tajo/master/DefaultTaskScheduler.java
index b20c5d8,77e3257..01137aa
--- a/tajo-core/src/main/java/org/apache/tajo/master/DefaultTaskScheduler.java
+++ b/tajo-core/src/main/java/org/apache/tajo/master/DefaultTaskScheduler.java
@@@ -41,9 -41,9 +41,10 @@@ import org.apache.tajo.master.event.Tas
  import org.apache.tajo.master.querymaster.QueryUnit;
  import org.apache.tajo.master.querymaster.QueryUnitAttempt;
  import org.apache.tajo.master.querymaster.SubQuery;
+ import org.apache.tajo.master.container.TajoContainerId;
  import org.apache.tajo.storage.DataLocation;
  import org.apache.tajo.storage.fragment.FileFragment;
 +import org.apache.tajo.storage.fragment.Fragment;
  import org.apache.tajo.util.NetUtils;
  import org.apache.tajo.worker.FetchImpl;
  

http://git-wip-us.apache.org/repos/asf/tajo/blob/940546a0/tajo-core/src/main/java/org/apache/tajo/master/GlobalEngine.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/tajo/blob/940546a0/tajo-core/src/main/java/org/apache/tajo/master/LazyTaskScheduler.java
----------------------------------------------------------------------
diff --cc tajo-core/src/main/java/org/apache/tajo/master/LazyTaskScheduler.java
index 038c334,b2883cc..c691a98
--- a/tajo-core/src/main/java/org/apache/tajo/master/LazyTaskScheduler.java
+++ b/tajo-core/src/main/java/org/apache/tajo/master/LazyTaskScheduler.java
@@@ -40,10 -37,8 +39,10 @@@ import org.apache.tajo.master.event.Tas
  import org.apache.tajo.master.querymaster.QueryUnit;
  import org.apache.tajo.master.querymaster.QueryUnitAttempt;
  import org.apache.tajo.master.querymaster.SubQuery;
+ import org.apache.tajo.master.container.TajoContainerId;
 +import org.apache.tajo.storage.StorageManager;
  import org.apache.tajo.storage.fragment.FileFragment;
- import org.apache.tajo.storage.StorageManager;
 +import org.apache.tajo.storage.fragment.Fragment;
  import org.apache.tajo.util.NetUtils;
  import org.apache.tajo.worker.FetchImpl;
  

http://git-wip-us.apache.org/repos/asf/tajo/blob/940546a0/tajo-core/src/main/java/org/apache/tajo/master/TajoMaster.java
----------------------------------------------------------------------


[03/29] tajo git commit: TAJO-1166: S3 related storage causes compilation error in Hadoop 2.6.0-SNAPSHOT. (jaehwa)

Posted by hj...@apache.org.
TAJO-1166: S3 related storage causes compilation error in Hadoop 2.6.0-SNAPSHOT. (jaehwa)

Closes #262


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

Branch: refs/heads/hbase_storage
Commit: 3d485ecb0112af12258a5a2bdc4a400b8df4fae8
Parents: 0c97fc0
Author: JaeHwa Jung <bl...@apache.org>
Authored: Thu Nov 27 18:42:24 2014 +0900
Committer: JaeHwa Jung <bl...@apache.org>
Committed: Thu Nov 27 18:45:18 2014 +0900

----------------------------------------------------------------------
 CHANGES                                         |   2 +
 tajo-core/pom.xml                               |   1 +
 .../org/apache/tajo/master/ContainerProxy.java  |  10 +-
 .../tajo/master/DefaultTaskScheduler.java       |  18 +-
 .../tajo/master/LaunchTaskRunnersEvent.java     |   5 +-
 .../apache/tajo/master/LazyTaskScheduler.java   |   9 +-
 .../apache/tajo/master/TajoContainerProxy.java  |  22 +-
 .../apache/tajo/master/TajoMasterService.java   |   5 +-
 .../tajo/master/TaskRunnerGroupEvent.java       |   8 +-
 .../tajo/master/container/TajoContainer.java    | 173 ++++++++++
 .../tajo/master/container/TajoContainerId.java  | 172 ++++++++++
 .../master/container/TajoConverterUtils.java    | 263 ++++++++++++++++
 .../master/container/TajoRecordFactory.java     |  31 ++
 .../container/TajoRecordFactoryPBImpl.java      | 104 ++++++
 .../container/TajoRecordFactoryProvider.java    |  70 +++++
 .../tajo/master/container/TajoRecords.java      |  39 +++
 .../impl/pb/TajoContainerIdPBImpl.java          | 100 ++++++
 .../tajo/master/event/LocalTaskEvent.java       |   9 +-
 .../event/QueryUnitAttemptScheduleEvent.java    |  10 +-
 .../event/SubQueryContainerAllocationEvent.java |   8 +-
 .../master/event/TaskAttemptAssignedEvent.java  |   8 +-
 .../tajo/master/event/TaskRequestEvent.java     |   8 +-
 .../master/querymaster/QueryInProgress.java     |   3 +-
 .../querymaster/QueryMasterManagerService.java  |   4 +-
 .../master/querymaster/QueryUnitAttempt.java    |   6 +-
 .../tajo/master/querymaster/SubQuery.java       |  17 +-
 .../apache/tajo/master/rm/TajoRMContext.java    |   8 +-
 .../tajo/master/rm/TajoWorkerContainer.java     |  15 +-
 .../tajo/master/rm/TajoWorkerContainerId.java   |  53 ++--
 .../master/rm/TajoWorkerResourceManager.java    |  63 ++--
 .../tajo/master/rm/WorkerResourceManager.java   |   4 +-
 .../tajo/worker/AbstractResourceAllocator.java  |  14 +-
 .../apache/tajo/worker/ResourceAllocator.java   |   6 +-
 .../tajo/worker/TajoResourceAllocator.java      | 101 +++---
 .../java/org/apache/tajo/worker/TaskRunner.java |  16 +-
 .../apache/tajo/worker/TaskRunnerHistory.java   |  14 +-
 .../src/main/proto/ContainerProtocol.proto      |  48 +++
 .../src/main/proto/QueryMasterProtocol.proto    |   3 +
 .../main/proto/ResourceTrackerProtocol.proto    |   3 +
 .../src/main/proto/TajoMasterProtocol.proto     |   7 +-
 .../src/main/proto/TajoWorkerProtocol.proto     |   5 +-
 .../tajo/master/rm/TestTajoResourceManager.java |  16 +-
 .../org/apache/tajo/storage/StorageManager.java |   1 +
 .../apache/tajo/storage/TestFileSystems.java    | 267 ++++++++--------
 .../java/org/apache/tajo/storage/s3/INode.java  | 124 --------
 .../storage/s3/InMemoryFileSystemStore.java     | 175 -----------
 .../apache/tajo/storage/s3/S3OutputStream.java  | 234 --------------
 .../tajo/storage/s3/SmallBlockS3FileSystem.java | 314 -------------------
 48 files changed, 1392 insertions(+), 1204 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tajo/blob/3d485ecb/CHANGES
----------------------------------------------------------------------
diff --git a/CHANGES b/CHANGES
index 6521c1c..1657c20 100644
--- a/CHANGES
+++ b/CHANGES
@@ -68,6 +68,8 @@ Release 0.9.1 - unreleased
 
   BUG FIXES
 
+    TAJO-1166: S3 related storage causes compilation error in Hadoop 2.6.0-SNAPSHOT. (jaehwa)
+
     TAJO-1208: Failure of create table using textfile on hivemeta.
     (jinho)
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/3d485ecb/tajo-core/pom.xml
----------------------------------------------------------------------
diff --git a/tajo-core/pom.xml b/tajo-core/pom.xml
index fce96e4..060ac1b 100644
--- a/tajo-core/pom.xml
+++ b/tajo-core/pom.xml
@@ -162,6 +162,7 @@
                 <argument>--proto_path=../tajo-client/src/main/proto</argument>
                 <argument>--proto_path=../tajo-plan/src/main/proto</argument>
                 <argument>--java_out=target/generated-sources/proto</argument>
+                <argument>src/main/proto/ContainerProtocol.proto</argument>
                 <argument>src/main/proto/ResourceTrackerProtocol.proto</argument>
                 <argument>src/main/proto/QueryMasterProtocol.proto</argument>
                 <argument>src/main/proto/TajoMasterProtocol.proto</argument>

http://git-wip-us.apache.org/repos/asf/tajo/blob/3d485ecb/tajo-core/src/main/java/org/apache/tajo/master/ContainerProxy.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/ContainerProxy.java b/tajo-core/src/main/java/org/apache/tajo/master/ContainerProxy.java
index 59b071a..462de91 100644
--- a/tajo-core/src/main/java/org/apache/tajo/master/ContainerProxy.java
+++ b/tajo-core/src/main/java/org/apache/tajo/master/ContainerProxy.java
@@ -22,11 +22,11 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.permission.FsPermission;
-import org.apache.hadoop.yarn.api.records.Container;
-import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
 import org.apache.tajo.ExecutionBlockId;
 import org.apache.tajo.master.querymaster.QueryMasterTask;
+import org.apache.tajo.master.container.TajoContainer;
+import org.apache.tajo.master.container.TajoContainerId;
 
 public abstract class ContainerProxy {
   protected static final Log LOG = LogFactory.getLog(ContainerProxy.class);
@@ -45,8 +45,8 @@ public abstract class ContainerProxy {
 
   protected ContainerState state;
   // store enough information to be able to cleanup the container
-  protected Container container;
-  protected ContainerId containerID;
+  protected TajoContainer container;
+  protected TajoContainerId containerID;
   protected String hostName;
   protected int port = -1;
 
@@ -54,7 +54,7 @@ public abstract class ContainerProxy {
   public abstract void stopContainer();
 
   public ContainerProxy(QueryMasterTask.QueryMasterTaskContext context, Configuration conf,
-                        ExecutionBlockId executionBlockId, Container container) {
+                        ExecutionBlockId executionBlockId, TajoContainer container) {
     this.context = context;
     this.conf = conf;
     this.state = ContainerState.PREP;

http://git-wip-us.apache.org/repos/asf/tajo/blob/3d485ecb/tajo-core/src/main/java/org/apache/tajo/master/DefaultTaskScheduler.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/DefaultTaskScheduler.java b/tajo-core/src/main/java/org/apache/tajo/master/DefaultTaskScheduler.java
index 62d4892..77e3257 100644
--- a/tajo-core/src/main/java/org/apache/tajo/master/DefaultTaskScheduler.java
+++ b/tajo-core/src/main/java/org/apache/tajo/master/DefaultTaskScheduler.java
@@ -41,6 +41,7 @@ import org.apache.tajo.master.event.TaskSchedulerEvent.EventType;
 import org.apache.tajo.master.querymaster.QueryUnit;
 import org.apache.tajo.master.querymaster.QueryUnitAttempt;
 import org.apache.tajo.master.querymaster.SubQuery;
+import org.apache.tajo.master.container.TajoContainerId;
 import org.apache.tajo.storage.DataLocation;
 import org.apache.tajo.storage.fragment.FileFragment;
 import org.apache.tajo.util.NetUtils;
@@ -338,7 +339,8 @@ public class DefaultTaskScheduler extends AbstractTaskScheduler {
     private Map<Integer, LinkedHashSet<QueryUnitAttempt>> unassignedTaskForEachVolume =
         Collections.synchronizedMap(new HashMap<Integer, LinkedHashSet<QueryUnitAttempt>>());
     /** A value is last assigned volume id for each task runner */
-    private HashMap<ContainerId, Integer> lastAssignedVolumeId = new HashMap<ContainerId, Integer>();
+    private HashMap<TajoContainerId, Integer> lastAssignedVolumeId = new HashMap<TajoContainerId,
+      Integer>();
     /**
      * A key is disk volume id, and a value is the load of this volume.
      * This load is measured by counting how many number of tasks are running.
@@ -378,7 +380,7 @@ public class DefaultTaskScheduler extends AbstractTaskScheduler {
      *  2. unknown block or Non-splittable task in host
      *  3. remote tasks. unassignedTaskForEachVolume is only contained local task. so it will be null
      */
-    public synchronized QueryUnitAttemptId getLocalTask(ContainerId containerId) {
+    public synchronized QueryUnitAttemptId getLocalTask(TajoContainerId containerId) {
       int volumeId;
       QueryUnitAttemptId queryUnitAttemptId = null;
 
@@ -489,7 +491,7 @@ public class DefaultTaskScheduler extends AbstractTaskScheduler {
      * @param volumeId Volume identifier
      * @return the volume load (i.e., how many running tasks use this volume)
      */
-    private synchronized int increaseConcurrency(ContainerId containerId, int volumeId) {
+    private synchronized int increaseConcurrency(TajoContainerId containerId, int volumeId) {
 
       int concurrency = 1;
       if (diskVolumeLoads.containsKey(volumeId)) {
@@ -514,7 +516,7 @@ public class DefaultTaskScheduler extends AbstractTaskScheduler {
     /**
      * Decrease the count of running tasks of a certain task runner
      */
-    private synchronized void decreaseConcurrency(ContainerId containerId){
+    private synchronized void decreaseConcurrency(TajoContainerId containerId){
       Integer volumeId = lastAssignedVolumeId.get(containerId);
       if(volumeId != null && diskVolumeLoads.containsKey(volumeId)){
         Integer concurrency = diskVolumeLoads.get(volumeId);
@@ -552,11 +554,11 @@ public class DefaultTaskScheduler extends AbstractTaskScheduler {
       }
     }
 
-    public boolean isAssigned(ContainerId containerId){
+    public boolean isAssigned(TajoContainerId containerId){
       return lastAssignedVolumeId.containsKey(containerId);
     }
 
-    public boolean isRemote(ContainerId containerId){
+    public boolean isRemote(TajoContainerId containerId){
       Integer volumeId = lastAssignedVolumeId.get(containerId);
       if(volumeId == null || volumeId > REMOTE){
         return false;
@@ -647,7 +649,7 @@ public class DefaultTaskScheduler extends AbstractTaskScheduler {
 
     public Set<QueryUnitAttemptId> assignedRequest = new HashSet<QueryUnitAttemptId>();
 
-    private QueryUnitAttemptId allocateLocalTask(String host, ContainerId containerId){
+    private QueryUnitAttemptId allocateLocalTask(String host, TajoContainerId containerId){
       HostVolumeMapping hostVolumeMapping = leafTaskHostMapping.get(host);
 
       if (hostVolumeMapping != null) { //tajo host is located in hadoop datanode
@@ -778,7 +780,7 @@ public class DefaultTaskScheduler extends AbstractTaskScheduler {
           }
         }
 
-        ContainerId containerId = taskRequest.getContainerId();
+        TajoContainerId containerId = taskRequest.getContainerId();
         LOG.debug("assignToLeafTasks: " + taskRequest.getExecutionBlockId() + "," +
             "containerId=" + containerId);
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/3d485ecb/tajo-core/src/main/java/org/apache/tajo/master/LaunchTaskRunnersEvent.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/LaunchTaskRunnersEvent.java b/tajo-core/src/main/java/org/apache/tajo/master/LaunchTaskRunnersEvent.java
index 9a4a01d..e620afa 100644
--- a/tajo-core/src/main/java/org/apache/tajo/master/LaunchTaskRunnersEvent.java
+++ b/tajo-core/src/main/java/org/apache/tajo/master/LaunchTaskRunnersEvent.java
@@ -18,9 +18,9 @@
 
 package org.apache.tajo.master;
 
-import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.tajo.ExecutionBlockId;
 import org.apache.tajo.engine.query.QueryContext;
+import org.apache.tajo.master.container.TajoContainer;
 
 import java.util.Collection;
 
@@ -29,7 +29,8 @@ public class LaunchTaskRunnersEvent extends TaskRunnerGroupEvent {
   private final String planJson;
 
   public LaunchTaskRunnersEvent(ExecutionBlockId executionBlockId,
-                                Collection<Container> containers, QueryContext queryContext, String planJson) {
+                                Collection<TajoContainer> containers, QueryContext queryContext,
+                                String planJson) {
     super(EventType.CONTAINER_REMOTE_LAUNCH, executionBlockId, containers);
     this.queryContext = queryContext;
     this.planJson = planJson;

http://git-wip-us.apache.org/repos/asf/tajo/blob/3d485ecb/tajo-core/src/main/java/org/apache/tajo/master/LazyTaskScheduler.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/LazyTaskScheduler.java b/tajo-core/src/main/java/org/apache/tajo/master/LazyTaskScheduler.java
index f7953e0..b2883cc 100644
--- a/tajo-core/src/main/java/org/apache/tajo/master/LazyTaskScheduler.java
+++ b/tajo-core/src/main/java/org/apache/tajo/master/LazyTaskScheduler.java
@@ -21,7 +21,6 @@ package org.apache.tajo.master;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.tajo.ExecutionBlockId;
 import org.apache.tajo.QueryIdFactory;
@@ -38,6 +37,7 @@ import org.apache.tajo.master.event.TaskSchedulerEvent.EventType;
 import org.apache.tajo.master.querymaster.QueryUnit;
 import org.apache.tajo.master.querymaster.QueryUnitAttempt;
 import org.apache.tajo.master.querymaster.SubQuery;
+import org.apache.tajo.master.container.TajoContainerId;
 import org.apache.tajo.storage.fragment.FileFragment;
 import org.apache.tajo.util.NetUtils;
 import org.apache.tajo.worker.FetchImpl;
@@ -246,7 +246,8 @@ public class LazyTaskScheduler extends AbstractTaskScheduler {
   }
 
   private static class DiskBalancer {
-    private HashMap<ContainerId, Integer> containerDiskMap = new HashMap<ContainerId, Integer>();
+    private HashMap<TajoContainerId, Integer> containerDiskMap = new HashMap<TajoContainerId,
+      Integer>();
     private HashMap<Integer, Integer> diskReferMap = new HashMap<Integer, Integer>();
     private String host;
 
@@ -260,7 +261,7 @@ public class LazyTaskScheduler extends AbstractTaskScheduler {
       }
     }
 
-    public Integer getDiskId(ContainerId containerId) {
+    public Integer getDiskId(TajoContainerId containerId) {
       if (!containerDiskMap.containsKey(containerId)) {
         assignVolumeId(containerId);
       }
@@ -268,7 +269,7 @@ public class LazyTaskScheduler extends AbstractTaskScheduler {
       return containerDiskMap.get(containerId);
     }
 
-    public void assignVolumeId(ContainerId containerId){
+    public void assignVolumeId(TajoContainerId containerId){
       Map.Entry<Integer, Integer> volumeEntry = null;
 
       for (Map.Entry<Integer, Integer> entry : diskReferMap.entrySet()) {

http://git-wip-us.apache.org/repos/asf/tajo/blob/3d485ecb/tajo-core/src/main/java/org/apache/tajo/master/TajoContainerProxy.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/TajoContainerProxy.java b/tajo-core/src/main/java/org/apache/tajo/master/TajoContainerProxy.java
index c236c20..158316e 100644
--- a/tajo-core/src/main/java/org/apache/tajo/master/TajoContainerProxy.java
+++ b/tajo-core/src/main/java/org/apache/tajo/master/TajoContainerProxy.java
@@ -20,17 +20,17 @@ package org.apache.tajo.master;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.util.StringUtils;
-import org.apache.hadoop.yarn.api.records.Container;
-import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
-import org.apache.hadoop.yarn.proto.YarnProtos;
 import org.apache.tajo.ExecutionBlockId;
 import org.apache.tajo.QueryUnitAttemptId;
 import org.apache.tajo.conf.TajoConf;
 import org.apache.tajo.engine.query.QueryContext;
+import org.apache.tajo.ipc.ContainerProtocol;
 import org.apache.tajo.ipc.TajoMasterProtocol;
 import org.apache.tajo.ipc.TajoWorkerProtocol;
 import org.apache.tajo.master.querymaster.QueryMasterTask;
+import org.apache.tajo.master.container.TajoContainer;
+import org.apache.tajo.master.container.TajoContainerId;
 import org.apache.tajo.master.rm.TajoWorkerContainer;
 import org.apache.tajo.master.rm.TajoWorkerContainerId;
 import org.apache.tajo.rpc.NettyClientBase;
@@ -47,7 +47,7 @@ public class TajoContainerProxy extends ContainerProxy {
   private final String planJson;
 
   public TajoContainerProxy(QueryMasterTask.QueryMasterTaskContext context,
-                            Configuration conf, Container container,
+                            Configuration conf, TajoContainer container,
                             QueryContext queryContext, ExecutionBlockId executionBlockId, String planJson) {
     super(context, conf, executionBlockId, container);
     this.queryContext = queryContext;
@@ -89,7 +89,7 @@ public class TajoContainerProxy extends ContainerProxy {
     }
   }
 
-  private void assignExecutionBlock(ExecutionBlockId executionBlockId, Container container) {
+  private void assignExecutionBlock(ExecutionBlockId executionBlockId, TajoContainer container) {
     NettyClientBase tajoWorkerRpc = null;
     try {
       InetSocketAddress myAddr= context.getQueryMasterContext().getWorkerContext()
@@ -149,8 +149,8 @@ public class TajoContainerProxy extends ContainerProxy {
 
   public static void releaseWorkerResource(QueryMasterTask.QueryMasterTaskContext context,
                                            ExecutionBlockId executionBlockId,
-                                           ContainerId containerId) throws Exception {
-    List<ContainerId> containerIds = new ArrayList<ContainerId>();
+                                           TajoContainerId containerId) throws Exception {
+    List<TajoContainerId> containerIds = new ArrayList<TajoContainerId>();
     containerIds.add(containerId);
 
     releaseWorkerResource(context, executionBlockId, containerIds);
@@ -158,11 +158,11 @@ public class TajoContainerProxy extends ContainerProxy {
 
   public static void releaseWorkerResource(QueryMasterTask.QueryMasterTaskContext context,
                                            ExecutionBlockId executionBlockId,
-                                           List<ContainerId> containerIds) throws Exception {
-    List<YarnProtos.ContainerIdProto> containerIdProtos =
-        new ArrayList<YarnProtos.ContainerIdProto>();
+                                           List<TajoContainerId> containerIds) throws Exception {
+    List<ContainerProtocol.TajoContainerIdProto> containerIdProtos =
+        new ArrayList<ContainerProtocol.TajoContainerIdProto>();
 
-    for(ContainerId eachContainerId: containerIds) {
+    for(TajoContainerId eachContainerId: containerIds) {
       containerIdProtos.add(TajoWorkerContainerId.getContainerIdProto(eachContainerId));
     }
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/3d485ecb/tajo-core/src/main/java/org/apache/tajo/master/TajoMasterService.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/TajoMasterService.java b/tajo-core/src/main/java/org/apache/tajo/master/TajoMasterService.java
index ddf24d3..1e3501c 100644
--- a/tajo-core/src/main/java/org/apache/tajo/master/TajoMasterService.java
+++ b/tajo-core/src/main/java/org/apache/tajo/master/TajoMasterService.java
@@ -27,6 +27,7 @@ import org.apache.hadoop.yarn.proto.YarnProtos;
 import org.apache.tajo.QueryId;
 import org.apache.tajo.TajoIdProtos;
 import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.ipc.ContainerProtocol;
 import org.apache.tajo.ipc.TajoMasterProtocol;
 import org.apache.tajo.master.cluster.WorkerConnectionInfo;
 import org.apache.tajo.master.querymaster.QueryJobManager;
@@ -128,9 +129,9 @@ public class TajoMasterService extends AbstractService {
     public void releaseWorkerResource(RpcController controller,
                                            TajoMasterProtocol.WorkerResourceReleaseRequest request,
                                            RpcCallback<PrimitiveProtos.BoolProto> done) {
-      List<YarnProtos.ContainerIdProto> containerIds = request.getContainerIdsList();
+      List<ContainerProtocol.TajoContainerIdProto> containerIds = request.getContainerIdsList();
 
-      for(YarnProtos.ContainerIdProto eachContainer: containerIds) {
+      for(ContainerProtocol.TajoContainerIdProto eachContainer: containerIds) {
         context.getResourceManager().releaseWorkerResource(eachContainer);
       }
       done.run(BOOL_TRUE);

http://git-wip-us.apache.org/repos/asf/tajo/blob/3d485ecb/tajo-core/src/main/java/org/apache/tajo/master/TaskRunnerGroupEvent.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/TaskRunnerGroupEvent.java b/tajo-core/src/main/java/org/apache/tajo/master/TaskRunnerGroupEvent.java
index 1e6655c..c1c6522 100644
--- a/tajo-core/src/main/java/org/apache/tajo/master/TaskRunnerGroupEvent.java
+++ b/tajo-core/src/main/java/org/apache/tajo/master/TaskRunnerGroupEvent.java
@@ -18,10 +18,10 @@
 
 package org.apache.tajo.master;
 
-import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.event.AbstractEvent;
 import org.apache.tajo.ExecutionBlockId;
 import org.apache.tajo.master.TaskRunnerGroupEvent.EventType;
+import org.apache.tajo.master.container.TajoContainer;
 
 import java.util.Collection;
 
@@ -32,16 +32,16 @@ public class TaskRunnerGroupEvent extends AbstractEvent<EventType> {
   }
 
   protected final ExecutionBlockId executionBlockId;
-  protected final Collection<Container> containers;
+  protected final Collection<TajoContainer> containers;
   public TaskRunnerGroupEvent(EventType eventType,
                               ExecutionBlockId executionBlockId,
-                              Collection<Container> containers) {
+                              Collection<TajoContainer> containers) {
     super(eventType);
     this.executionBlockId = executionBlockId;
     this.containers = containers;
   }
 
-  public Collection<Container> getContainers() {
+  public Collection<TajoContainer> getContainers() {
     return containers;
   }
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/3d485ecb/tajo-core/src/main/java/org/apache/tajo/master/container/TajoContainer.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/container/TajoContainer.java b/tajo-core/src/main/java/org/apache/tajo/master/container/TajoContainer.java
new file mode 100644
index 0000000..77562b5
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/master/container/TajoContainer.java
@@ -0,0 +1,173 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.master.container;
+
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Stable;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.api.ApplicationMasterProtocol;
+import org.apache.hadoop.yarn.api.records.*;
+import org.apache.hadoop.yarn.util.Records;
+
+/**
+ * This class is borrowed from the following source code :
+ * ${hadoop-yarn-api}/src/main/java/org/apache/hadoop/yarn/api/records/Container.java
+ *
+ * <p><code>TajoContainer</code> represents an allocated resource in the cluster.
+ * </p>
+ *
+ * <p>The <code>ResourceManager</code> is the sole authority to allocate any
+ * <code>TajoContainer</code> to applications. The allocated <code>TajoContainer</code>
+ * is always on a single node and has a unique {@link org.apache.tajo.master.container.TajoContainerId}. It has
+ * a specific amount of {@link org.apache.hadoop.yarn.api.records.Resource} allocated.</p>
+ *
+ * <p>It includes details such as:
+ *   <ul>
+ *     <li>{@link org.apache.tajo.master.container.TajoContainerId} for the container, which is globally unique.</li>
+ *     <li>
+ *       {@link org.apache.hadoop.yarn.api.records.NodeId} of the node on which it is allocated.
+ *     </li>
+ *     <li>HTTP uri of the node.</li>
+ *     <li>{@link org.apache.hadoop.yarn.api.records.Resource} allocated to the container.</li>
+ *     <li>{@link org.apache.hadoop.yarn.api.records.Priority} at which the container was allocated.</li>
+ *     <li>
+ *       TajoContainer {@link org.apache.hadoop.yarn.api.records.Token} of the container, used to securely verify
+ *       authenticity of the allocation. 
+ *     </li>
+ *   </ul>
+ * </p>
+ *
+ * <p>Typically, an <code>ApplicationMaster</code> receives the 
+ * <code>TajoContainer</code> from the <code>ResourceManager</code> during
+ * resource-negotiation and then talks to the <code>NodeManager</code> to 
+ * start/stop containers.</p>
+ *
+ * @see ApplicationMasterProtocol#allocate(org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest)
+ * @see org.apache.hadoop.yarn.api.ContainerManagementProtocol#startContainers(org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest)
+ * @see org.apache.hadoop.yarn.api.ContainerManagementProtocol#stopContainers(org.apache.hadoop.yarn.api.protocolrecords.StopContainersRequest)
+ */
+@Public
+@Stable
+public abstract class TajoContainer implements Comparable<TajoContainer> {
+
+  @Private
+  @Unstable
+  public static TajoContainer newInstance(TajoContainerId containerId, NodeId nodeId,
+                                      String nodeHttpAddress, Resource resource, Priority priority,
+                                      Token containerToken) {
+    TajoContainer container = Records.newRecord(TajoContainer.class);
+    container.setId(containerId);
+    container.setNodeId(nodeId);
+    container.setNodeHttpAddress(nodeHttpAddress);
+    container.setResource(resource);
+    container.setPriority(priority);
+    container.setContainerToken(containerToken);
+    return container;
+  }
+
+  /**
+   * Get the globally unique identifier for the container.
+   * @return globally unique identifier for the container
+   */
+  @Public
+  @Stable
+  public abstract TajoContainerId getId();
+
+  @Private
+  @Unstable
+  public abstract void setId(TajoContainerId id);
+
+  /**
+   * Get the identifier of the node on which the container is allocated.
+   * @return identifier of the node on which the container is allocated
+   */
+  @Public
+  @Stable
+  public abstract NodeId getNodeId();
+
+  @Private
+  @Unstable
+  public abstract void setNodeId(NodeId nodeId);
+
+  /**
+   * Get the http uri of the node on which the container is allocated.
+   * @return http uri of the node on which the container is allocated
+   */
+  @Public
+  @Stable
+  public abstract String getNodeHttpAddress();
+
+  @Private
+  @Unstable
+  public abstract void setNodeHttpAddress(String nodeHttpAddress);
+
+  /**
+   * Get the <code>Resource</code> allocated to the container.
+   * @return <code>Resource</code> allocated to the container
+   */
+  @Public
+  @Stable
+  public abstract Resource getResource();
+
+  @Private
+  @Unstable
+  public abstract void setResource(Resource resource);
+
+  /**
+   * Get the <code>Priority</code> at which the <code>TajoContainer</code> was
+   * allocated.
+   * @return <code>Priority</code> at which the <code>TajoContainer</code> was
+   *         allocated
+   */
+  @Public
+  @Stable
+  public abstract Priority getPriority();
+
+  @Private
+  @Unstable
+  public abstract void setPriority(Priority priority);
+
+  /**
+   * Get the <code>TajoContainerToken</code> for the container.
+   * <p><code>TajoContainerToken</code> is the security token used by the framework
+   * to verify authenticity of any <code>TajoContainer</code>.</p>
+   *
+   * <p>The <code>ResourceManager</code>, on container allocation provides a
+   * secure token which is verified by the <code>NodeManager</code> on
+   * container launch.</p>
+   *
+   * <p>Applications do not need to care about <code>TajoContainerToken</code>, they
+   * are transparently handled by the framework - the allocated
+   * <code>TajoContainer</code> includes the <code>TajoContainerToken</code>.</p>
+   *
+   * @see ApplicationMasterProtocol#allocate(org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest)
+   * @see org.apache.hadoop.yarn.api.ContainerManagementProtocol#startContainers(org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest)
+   *
+   * @return <code>TajoContainerToken</code> for the container
+   */
+  @Public
+  @Stable
+  public abstract Token getContainerToken();
+
+  @Private
+  @Unstable
+  public abstract void setContainerToken(Token containerToken);
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/3d485ecb/tajo-core/src/main/java/org/apache/tajo/master/container/TajoContainerId.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/container/TajoContainerId.java b/tajo-core/src/main/java/org/apache/tajo/master/container/TajoContainerId.java
new file mode 100644
index 0000000..0de5fe0
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/master/container/TajoContainerId.java
@@ -0,0 +1,172 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.master.container;
+
+import java.text.NumberFormat;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Stable;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.util.Records;
+
+/**
+ * This class is borrowed from the following source code :
+ * ${hadoop-yarn-api}/src/main/java/org/apache/hadoop/yarn/api/records/ContainerId.java
+ *
+ * <p><code>TajoContainerId</code> represents a globally unique identifier
+ * for a {@link org.apache.tajo.master.container.TajoContainer} in the cluster.</p>
+ */
+@Public
+@Stable
+public abstract class TajoContainerId implements Comparable<TajoContainerId>{
+
+  @Private
+  @Unstable
+  public static TajoContainerId newInstance(ApplicationAttemptId appAttemptId,
+                                        int containerId) {
+    TajoContainerId id = TajoRecords.newRecord(TajoContainerId.class);
+    id.setId(containerId);
+    id.setApplicationAttemptId(appAttemptId);
+    id.build();
+    return id;
+  }
+
+  /**
+   * Get the <code>ApplicationAttemptId</code> of the application to which the
+   * <code>Container</code> was assigned.
+   * <p>
+   * Note: If containers are kept alive across application attempts via
+   * {@link org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext#setKeepContainersAcrossApplicationAttempts(boolean)}
+   * the <code>TajoContainerId</code> does not necessarily contain the current
+   * running application attempt's <code>ApplicationAttemptId</code> This
+   * container can be allocated by previously exited application attempt and
+   * managed by the current running attempt thus have the previous application
+   * attempt's <code>ApplicationAttemptId</code>.
+   * </p>
+   *
+   * @return <code>ApplicationAttemptId</code> of the application to which the
+   *         <code>Container</code> was assigned
+   */
+  @Public
+  @Stable
+  public abstract ApplicationAttemptId getApplicationAttemptId();
+
+  @Private
+  @Unstable
+  protected abstract void setApplicationAttemptId(ApplicationAttemptId atId);
+
+  /**
+   * Get the identifier of the <code>TajoContainerId</code>.
+   * @return identifier of the <code>TajoContainerId</code>
+   */
+  @Public
+  @Stable
+  public abstract int getId();
+
+  @Private
+  @Unstable
+  protected abstract void setId(int id);
+
+
+  // TODO: fail the app submission if attempts are more than 10 or something
+  private static final ThreadLocal<NumberFormat> appAttemptIdFormat =
+    new ThreadLocal<NumberFormat>() {
+      @Override
+      public NumberFormat initialValue() {
+        NumberFormat fmt = NumberFormat.getInstance();
+        fmt.setGroupingUsed(false);
+        fmt.setMinimumIntegerDigits(2);
+        return fmt;
+      }
+    };
+  // TODO: Why thread local?
+  // ^ NumberFormat instances are not threadsafe
+  private static final ThreadLocal<NumberFormat> containerIdFormat =
+    new ThreadLocal<NumberFormat>() {
+      @Override
+      public NumberFormat initialValue() {
+        NumberFormat fmt = NumberFormat.getInstance();
+        fmt.setGroupingUsed(false);
+        fmt.setMinimumIntegerDigits(6);
+        return fmt;
+      }
+    };
+
+  @Override
+  public int hashCode() {
+    // Generated by eclipse.
+    final int prime = 435569;
+    int result = 7507;
+    result = prime * result + getId();
+    result = prime * result + getApplicationAttemptId().hashCode();
+    return result;
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (this == obj)
+      return true;
+    if (obj == null)
+      return false;
+    if (getClass() != obj.getClass())
+      return false;
+    TajoContainerId other = (TajoContainerId) obj;
+    if (!this.getApplicationAttemptId().equals(other.getApplicationAttemptId()))
+      return false;
+    if (this.getId() != other.getId())
+      return false;
+    return true;
+  }
+
+  @Override
+  public int compareTo(TajoContainerId other) {
+    if (this.getApplicationAttemptId().compareTo(
+      other.getApplicationAttemptId()) == 0) {
+      return this.getId() - other.getId();
+    } else {
+      return this.getApplicationAttemptId().compareTo(
+        other.getApplicationAttemptId());
+    }
+
+  }
+
+  @Override
+  public String toString() {
+    NumberFormat fmt = NumberFormat.getInstance();
+    fmt.setGroupingUsed(false);
+    fmt.setMinimumIntegerDigits(4);
+
+    StringBuilder sb = new StringBuilder();
+    sb.append("container_");
+    ApplicationId appId = getApplicationAttemptId().getApplicationId();
+    sb.append(appId.getClusterTimestamp()).append("_");
+    sb.append(fmt.format(appId.getId()))
+      .append("_");
+    sb.append(
+      appAttemptIdFormat.get().format(
+        getApplicationAttemptId().getAttemptId())).append("_");
+    sb.append(containerIdFormat.get().format(getId()));
+    return sb.toString();
+  }
+
+  protected abstract void build();
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/3d485ecb/tajo-core/src/main/java/org/apache/tajo/master/container/TajoConverterUtils.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/container/TajoConverterUtils.java b/tajo-core/src/main/java/org/apache/tajo/master/container/TajoConverterUtils.java
new file mode 100644
index 0000000..a6db654
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/master/container/TajoConverterUtils.java
@@ -0,0 +1,263 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.tajo.master.container;
+
+
+import static org.apache.hadoop.yarn.util.StringHelper._split;
+
+import java.net.InetSocketAddress;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Map.Entry;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.security.SecurityUtil;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.TokenIdentifier;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.hadoop.yarn.api.records.URL;
+import org.apache.hadoop.yarn.factories.RecordFactory;
+import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
+
+
+/**
+ * This class is borrowed from the following source code :
+ * ${hadoop-yarn-common}/src/main/java/org/apache/hadoop/yarn/util/ConverterUtils.java
+ *
+ * This class contains a set of utilities which help converting data structures
+ * from/to 'serializableFormat' to/from hadoop/nativejava data structures.
+ *
+ */
+@Private
+public class TajoConverterUtils {
+
+  public static final String APPLICATION_PREFIX = "application";
+  public static final String CONTAINER_PREFIX = "container";
+  public static final String APPLICATION_ATTEMPT_PREFIX = "appattempt";
+
+  /**
+   * return a hadoop path from a given url
+   *
+   * @param url
+   *          url to convert
+   * @return path from {@link URL}
+   * @throws URISyntaxException
+   */
+  public static Path getPathFromYarnURL(URL url) throws URISyntaxException {
+    String scheme = url.getScheme() == null ? "" : url.getScheme();
+
+    String authority = "";
+    if (url.getHost() != null) {
+      authority = url.getHost();
+      if (url.getUserInfo() != null) {
+        authority = url.getUserInfo() + "@" + authority;
+      }
+      if (url.getPort() > 0) {
+        authority += ":" + url.getPort();
+      }
+    }
+
+    return new Path(
+      (new URI(scheme, authority, url.getFile(), null, null)).normalize());
+  }
+
+  /**
+   * change from CharSequence to string for map key and value
+   * @param env map for converting
+   * @return string,string map
+   */
+  public static Map<String, String> convertToString(
+    Map<CharSequence, CharSequence> env) {
+
+    Map<String, String> stringMap = new HashMap<String, String>();
+    for (Entry<CharSequence, CharSequence> entry: env.entrySet()) {
+      stringMap.put(entry.getKey().toString(), entry.getValue().toString());
+    }
+    return stringMap;
+  }
+
+  public static URL getYarnUrlFromPath(Path path) {
+    return getYarnUrlFromURI(path.toUri());
+  }
+
+  public static URL getYarnUrlFromURI(URI uri) {
+    URL url = RecordFactoryProvider.getRecordFactory(null).newRecordInstance(URL.class);
+    if (uri.getHost() != null) {
+      url.setHost(uri.getHost());
+    }
+    if (uri.getUserInfo() != null) {
+      url.setUserInfo(uri.getUserInfo());
+    }
+    url.setPort(uri.getPort());
+    url.setScheme(uri.getScheme());
+    url.setFile(uri.getPath());
+    return url;
+  }
+
+  public static String toString(ApplicationId appId) {
+    return appId.toString();
+  }
+
+  public static ApplicationId toApplicationId(RecordFactory recordFactory,
+                                              String appIdStr) {
+    Iterator<String> it = _split(appIdStr).iterator();
+    it.next(); // prefix. TODO: Validate application prefix
+    return toApplicationId(recordFactory, it);
+  }
+
+  private static ApplicationId toApplicationId(RecordFactory recordFactory,
+                                               Iterator<String> it) {
+    ApplicationId appId = ApplicationId.newInstance(Long.parseLong(it.next()),
+      Integer.parseInt(it.next()));
+    return appId;
+  }
+
+  private static ApplicationAttemptId toApplicationAttemptId(
+    Iterator<String> it) throws NumberFormatException {
+    ApplicationId appId = ApplicationId.newInstance(Long.parseLong(it.next()),
+      Integer.parseInt(it.next()));
+    ApplicationAttemptId appAttemptId =
+      ApplicationAttemptId.newInstance(appId, Integer.parseInt(it.next()));
+    return appAttemptId;
+  }
+
+  private static ApplicationId toApplicationId(
+    Iterator<String> it) throws NumberFormatException {
+    ApplicationId appId = ApplicationId.newInstance(Long.parseLong(it.next()),
+      Integer.parseInt(it.next()));
+    return appId;
+  }
+
+  public static String toString(TajoContainerId cId) {
+    return cId == null ? null : cId.toString();
+  }
+
+  public static NodeId toNodeId(String nodeIdStr) {
+    String[] parts = nodeIdStr.split(":");
+    if (parts.length != 2) {
+      throw new IllegalArgumentException("Invalid NodeId [" + nodeIdStr
+        + "]. Expected host:port");
+    }
+    try {
+      NodeId nodeId =
+        NodeId.newInstance(parts[0], Integer.parseInt(parts[1]));
+      return nodeId;
+    } catch (NumberFormatException e) {
+      throw new IllegalArgumentException("Invalid port: " + parts[1], e);
+    }
+  }
+
+  public static TajoContainerId toTajoContainerId(String containerIdStr) {
+    Iterator<String> it = _split(containerIdStr).iterator();
+    if (!it.next().equals(CONTAINER_PREFIX)) {
+      throw new IllegalArgumentException("Invalid TajoContainerId prefix: "
+        + containerIdStr);
+    }
+    try {
+      ApplicationAttemptId appAttemptID = toApplicationAttemptId(it);
+      TajoContainerId containerId =
+        TajoContainerId.newInstance(appAttemptID, Integer.parseInt(it.next()));
+      return containerId;
+    } catch (NumberFormatException n) {
+      throw new IllegalArgumentException("Invalid TajoContainerId: "
+        + containerIdStr, n);
+    }
+  }
+
+  public static ApplicationAttemptId toApplicationAttemptId(
+    String applicationAttmeptIdStr) {
+    Iterator<String> it = _split(applicationAttmeptIdStr).iterator();
+    if (!it.next().equals(APPLICATION_ATTEMPT_PREFIX)) {
+      throw new IllegalArgumentException("Invalid AppAttemptId prefix: "
+        + applicationAttmeptIdStr);
+    }
+    try {
+      return toApplicationAttemptId(it);
+    } catch (NumberFormatException n) {
+      throw new IllegalArgumentException("Invalid AppAttemptId: "
+        + applicationAttmeptIdStr, n);
+    }
+  }
+
+  public static ApplicationId toApplicationId(
+    String appIdStr) {
+    Iterator<String> it = _split(appIdStr).iterator();
+    if (!it.next().equals(APPLICATION_PREFIX)) {
+      throw new IllegalArgumentException("Invalid ApplicationId prefix: "
+        + appIdStr + ". The valid ApplicationId should start with prefix "
+        + APPLICATION_PREFIX);
+    }
+    try {
+      return toApplicationId(it);
+    } catch (NumberFormatException n) {
+      throw new IllegalArgumentException("Invalid AppAttemptId: "
+        + appIdStr, n);
+    }
+  }
+
+  /**
+   * Convert a protobuf token into a rpc token and set its service. Supposed
+   * to be used for tokens other than RMDelegationToken. For
+   * RMDelegationToken, use
+   * {@link #convertFromYarn(org.apache.hadoop.yarn.api.records.Token,
+   * org.apache.hadoop.io.Text)} instead.
+   *
+   * @param protoToken the yarn token
+   * @param serviceAddr the connect address for the service
+   * @return rpc token
+   */
+  public static <T extends TokenIdentifier> Token<T> convertFromYarn(
+    org.apache.hadoop.yarn.api.records.Token protoToken,
+    InetSocketAddress serviceAddr) {
+    Token<T> token = new Token<T>(protoToken.getIdentifier().array(),
+      protoToken.getPassword().array(),
+      new Text(protoToken.getKind()),
+      new Text(protoToken.getService()));
+    if (serviceAddr != null) {
+      SecurityUtil.setTokenService(token, serviceAddr);
+    }
+    return token;
+  }
+
+  /**
+   * Convert a protobuf token into a rpc token and set its service.
+   *
+   * @param protoToken the yarn token
+   * @param service the service for the token
+   */
+  public static <T extends TokenIdentifier> Token<T> convertFromYarn(
+    org.apache.hadoop.yarn.api.records.Token protoToken,
+    Text service) {
+    Token<T> token = new Token<T>(protoToken.getIdentifier().array(),
+      protoToken.getPassword().array(),
+      new Text(protoToken.getKind()),
+      new Text(protoToken.getService()));
+
+    if (service != null) {
+      token.setService(service);
+    }
+    return token;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/3d485ecb/tajo-core/src/main/java/org/apache/tajo/master/container/TajoRecordFactory.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/container/TajoRecordFactory.java b/tajo-core/src/main/java/org/apache/tajo/master/container/TajoRecordFactory.java
new file mode 100644
index 0000000..2fd8697
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/master/container/TajoRecordFactory.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.master.container;
+
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+
+/**
+ * This class is borrowed from the following source code :
+ * ${hadoop-yarn-api}/src/main/java/org/apache/hadoop/yarn/factories/RecordFactory.java
+ *
+ */
+@Unstable
+public interface TajoRecordFactory {
+  public <T> T newRecordInstance(Class<T> clazz);
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/3d485ecb/tajo-core/src/main/java/org/apache/tajo/master/container/TajoRecordFactoryPBImpl.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/container/TajoRecordFactoryPBImpl.java b/tajo-core/src/main/java/org/apache/tajo/master/container/TajoRecordFactoryPBImpl.java
new file mode 100644
index 0000000..c352a28
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/master/container/TajoRecordFactoryPBImpl.java
@@ -0,0 +1,104 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+
+package org.apache.tajo.master.container;
+
+
+import java.lang.reflect.Constructor;
+import java.lang.reflect.InvocationTargetException;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
+import org.apache.hadoop.yarn.factories.RecordFactory;
+import org.apache.tajo.master.container.TajoRecordFactory;
+
+/**
+ * This class is borrowed from the following source code :
+ * ${hadoop-yarn-common}/src/main/java/org/apache/hadoop/yarn/factories/impl/pb/RecordFactoryPBImpl.java
+ */
+@Private
+public class TajoRecordFactoryPBImpl implements TajoRecordFactory {
+
+  private static final String PB_IMPL_PACKAGE_SUFFIX = "impl.pb";
+  private static final String PB_IMPL_CLASS_SUFFIX = "PBImpl";
+
+  private static final TajoRecordFactoryPBImpl self = new TajoRecordFactoryPBImpl();
+  private Configuration localConf = new Configuration();
+  private ConcurrentMap<Class<?>, Constructor<?>> cache = new ConcurrentHashMap<Class<?>, Constructor<?>>();
+
+  private TajoRecordFactoryPBImpl() {
+  }
+
+  public static TajoRecordFactory get() {
+    return self;
+  }
+
+  @SuppressWarnings("unchecked")
+  @Override
+  public <T> T newRecordInstance(Class<T> clazz) {
+
+    Constructor<?> constructor = cache.get(clazz);
+    if (constructor == null) {
+      Class<?> pbClazz = null;
+      try {
+        pbClazz = localConf.getClassByName(getPBImplClassName(clazz));
+      } catch (ClassNotFoundException e) {
+        throw new YarnRuntimeException("Failed to load class: ["
+          + getPBImplClassName(clazz) + "]", e);
+      }
+      try {
+        constructor = pbClazz.getConstructor();
+        constructor.setAccessible(true);
+        cache.putIfAbsent(clazz, constructor);
+      } catch (NoSuchMethodException e) {
+        throw new YarnRuntimeException("Could not find 0 argument constructor", e);
+      }
+    }
+    try {
+      Object retObject = constructor.newInstance();
+      return (T)retObject;
+    } catch (InvocationTargetException e) {
+      throw new YarnRuntimeException(e);
+    } catch (IllegalAccessException e) {
+      throw new YarnRuntimeException(e);
+    } catch (InstantiationException e) {
+      throw new YarnRuntimeException(e);
+    }
+  }
+
+  private String getPBImplClassName(Class<?> clazz) {
+    String srcPackagePart = getPackageName(clazz);
+    String srcClassName = getClassName(clazz);
+    String destPackagePart = srcPackagePart + "." + PB_IMPL_PACKAGE_SUFFIX;
+    String destClassPart = srcClassName + PB_IMPL_CLASS_SUFFIX;
+    return destPackagePart + "." + destClassPart;
+  }
+
+  private String getClassName(Class<?> clazz) {
+    String fqName = clazz.getName();
+    return (fqName.substring(fqName.lastIndexOf(".") + 1, fqName.length()));
+  }
+
+  private String getPackageName(Class<?> clazz) {
+    return clazz.getPackage().getName();
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/3d485ecb/tajo-core/src/main/java/org/apache/tajo/master/container/TajoRecordFactoryProvider.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/container/TajoRecordFactoryProvider.java b/tajo-core/src/main/java/org/apache/tajo/master/container/TajoRecordFactoryProvider.java
new file mode 100644
index 0000000..c260e85
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/master/container/TajoRecordFactoryProvider.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.master.container;
+
+
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
+
+/**
+ * This class is borrowed from the following source code :
+ * ${hadoop-yarn-api}/src/main/java/org/apache/hadoop/yarn/factory/providers/RecordFactoryProvider.java
+ */
+@Unstable
+public class TajoRecordFactoryProvider {
+  private static Configuration defaultConf;
+
+  static {
+    defaultConf = new Configuration();
+  }
+
+  private TajoRecordFactoryProvider() {
+  }
+
+  public static TajoRecordFactory getRecordFactory(Configuration conf) {
+    if (conf == null) {
+      //Assuming the default configuration has the correct factories set.
+      //Users can specify a particular factory by providing a configuration.
+      conf = defaultConf;
+    }
+    return (TajoRecordFactory) getFactoryClassInstance(TajoRecordFactoryPBImpl.class.getCanonicalName());
+  }
+
+  private static Object getFactoryClassInstance(String factoryClassName) {
+    try {
+      Class<?> clazz = Class.forName(factoryClassName);
+      Method method = clazz.getMethod("get", null);
+      method.setAccessible(true);
+      return method.invoke(null, null);
+    } catch (ClassNotFoundException e) {
+      throw new YarnRuntimeException(e);
+    } catch (NoSuchMethodException e) {
+      throw new YarnRuntimeException(e);
+    } catch (InvocationTargetException e) {
+      throw new YarnRuntimeException(e);
+    } catch (IllegalAccessException e) {
+      throw new YarnRuntimeException(e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/3d485ecb/tajo-core/src/main/java/org/apache/tajo/master/container/TajoRecords.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/container/TajoRecords.java b/tajo-core/src/main/java/org/apache/tajo/master/container/TajoRecords.java
new file mode 100644
index 0000000..e85edf9
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/master/container/TajoRecords.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.master.container;
+
+
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+
+/**
+ * This class is borrowed from the following source code :
+ * ${hadoop-yarn-api}/src/main/java/org/apache/hadoop/yarn/util/Records.java
+ *
+ * Convenient API record utils
+ */
+@Unstable
+public class TajoRecords {
+  // The default record factory
+  private static final TajoRecordFactory factory =
+    TajoRecordFactoryProvider.getRecordFactory(null);
+
+  public static <T> T newRecord(Class<T> cls) {
+    return factory.newRecordInstance(cls);
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/3d485ecb/tajo-core/src/main/java/org/apache/tajo/master/container/impl/pb/TajoContainerIdPBImpl.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/container/impl/pb/TajoContainerIdPBImpl.java b/tajo-core/src/main/java/org/apache/tajo/master/container/impl/pb/TajoContainerIdPBImpl.java
new file mode 100644
index 0000000..9d31050
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/master/container/impl/pb/TajoContainerIdPBImpl.java
@@ -0,0 +1,100 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.master.container.impl.pb;
+
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationAttemptIdPBImpl;
+import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationAttemptIdProto;
+
+import com.google.common.base.Preconditions;
+import org.apache.tajo.ipc.ContainerProtocol;
+import org.apache.tajo.master.container.TajoContainerId;
+
+/**
+ * This class is borrowed from the following source code :
+ * ${hadoop-yarn-common}/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerIdPBImpl.java
+ *
+ */
+@Private
+@Unstable
+public class TajoContainerIdPBImpl extends TajoContainerId {
+  ContainerProtocol.TajoContainerIdProto proto = null;
+  ContainerProtocol.TajoContainerIdProto.Builder builder = null;
+  private ApplicationAttemptId applicationAttemptId = null;
+
+  public TajoContainerIdPBImpl() {
+    builder = ContainerProtocol.TajoContainerIdProto.newBuilder();
+  }
+
+  public TajoContainerIdPBImpl(ContainerProtocol.TajoContainerIdProto proto) {
+    this.proto = proto;
+    this.applicationAttemptId = convertFromProtoFormat(proto.getAppAttemptId());
+  }
+
+  public ContainerProtocol.TajoContainerIdProto getProto() {
+    return proto;
+  }
+
+  @Override
+  public int getId() {
+    Preconditions.checkNotNull(proto);
+    return proto.getId();
+  }
+
+  @Override
+  protected void setId(int id) {
+    Preconditions.checkNotNull(builder);
+    builder.setId((id));
+  }
+
+
+  @Override
+  public ApplicationAttemptId getApplicationAttemptId() {
+    return this.applicationAttemptId;
+  }
+
+  @Override
+  protected void setApplicationAttemptId(ApplicationAttemptId atId) {
+    if (atId != null) {
+      Preconditions.checkNotNull(builder);
+      builder.setAppAttemptId(convertToProtoFormat(atId));
+    }
+    this.applicationAttemptId = atId;
+  }
+
+  private ApplicationAttemptIdPBImpl convertFromProtoFormat(
+    ApplicationAttemptIdProto p) {
+    return new ApplicationAttemptIdPBImpl(p);
+  }
+
+  private ApplicationAttemptIdProto convertToProtoFormat(
+    ApplicationAttemptId t) {
+    return ((ApplicationAttemptIdPBImpl)t).getProto();
+  }
+
+  @Override
+  protected void build() {
+    proto = builder.build();
+    builder = null;
+  }
+}  
+

http://git-wip-us.apache.org/repos/asf/tajo/blob/3d485ecb/tajo-core/src/main/java/org/apache/tajo/master/event/LocalTaskEvent.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/event/LocalTaskEvent.java b/tajo-core/src/main/java/org/apache/tajo/master/event/LocalTaskEvent.java
index 92e6695..cab2202 100644
--- a/tajo-core/src/main/java/org/apache/tajo/master/event/LocalTaskEvent.java
+++ b/tajo-core/src/main/java/org/apache/tajo/master/event/LocalTaskEvent.java
@@ -18,18 +18,19 @@
 
 package org.apache.tajo.master.event;
 
-import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.event.AbstractEvent;
 import org.apache.tajo.QueryUnitAttemptId;
+import org.apache.tajo.master.container.TajoContainerId;
 
 /**
  * This event is sent to a running TaskAttempt on a worker.
  */
 public class LocalTaskEvent extends AbstractEvent<LocalTaskEventType> {
   private final QueryUnitAttemptId taskAttemptId;
-  private final ContainerId containerId;
+  private final TajoContainerId containerId;
 
-  public LocalTaskEvent(QueryUnitAttemptId taskAttemptId, ContainerId containerId, LocalTaskEventType eventType) {
+  public LocalTaskEvent(QueryUnitAttemptId taskAttemptId, TajoContainerId containerId,
+                        LocalTaskEventType eventType) {
     super(eventType);
     this.taskAttemptId = taskAttemptId;
     this.containerId = containerId;
@@ -39,7 +40,7 @@ public class LocalTaskEvent extends AbstractEvent<LocalTaskEventType> {
     return taskAttemptId;
   }
 
-  public ContainerId getContainerId() {
+  public TajoContainerId getContainerId() {
     return containerId;
   }
 }

http://git-wip-us.apache.org/repos/asf/tajo/blob/3d485ecb/tajo-core/src/main/java/org/apache/tajo/master/event/QueryUnitAttemptScheduleEvent.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/event/QueryUnitAttemptScheduleEvent.java b/tajo-core/src/main/java/org/apache/tajo/master/event/QueryUnitAttemptScheduleEvent.java
index a2acc7e..6e0d9fd 100644
--- a/tajo-core/src/main/java/org/apache/tajo/master/event/QueryUnitAttemptScheduleEvent.java
+++ b/tajo-core/src/main/java/org/apache/tajo/master/event/QueryUnitAttemptScheduleEvent.java
@@ -19,10 +19,10 @@
 package org.apache.tajo.master.event;
 
 import com.google.protobuf.RpcCallback;
-import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.tajo.ExecutionBlockId;
 import org.apache.tajo.ipc.TajoWorkerProtocol.QueryUnitRequestProto;
 import org.apache.tajo.master.querymaster.QueryUnitAttempt;
+import org.apache.tajo.master.container.TajoContainerId;
 
 public class QueryUnitAttemptScheduleEvent extends TaskSchedulerEvent {
   private final QueryUnitAttemptScheduleContext context;
@@ -44,7 +44,7 @@ public class QueryUnitAttemptScheduleEvent extends TaskSchedulerEvent {
   }
 
   public static class QueryUnitAttemptScheduleContext {
-    private ContainerId containerId;
+    private TajoContainerId containerId;
     private String host;
     private RpcCallback<QueryUnitRequestProto> callback;
 
@@ -52,7 +52,7 @@ public class QueryUnitAttemptScheduleEvent extends TaskSchedulerEvent {
 
     }
 
-    public QueryUnitAttemptScheduleContext(ContainerId containerId,
+    public QueryUnitAttemptScheduleContext(TajoContainerId containerId,
                                            String host,
                                            RpcCallback<QueryUnitRequestProto> callback) {
       this.containerId = containerId;
@@ -60,11 +60,11 @@ public class QueryUnitAttemptScheduleEvent extends TaskSchedulerEvent {
       this.callback = callback;
     }
 
-    public ContainerId getContainerId() {
+    public TajoContainerId getContainerId() {
       return containerId;
     }
 
-    public void setContainerId(ContainerId containerId) {
+    public void setContainerId(TajoContainerId containerId) {
       this.containerId = containerId;
     }
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/3d485ecb/tajo-core/src/main/java/org/apache/tajo/master/event/SubQueryContainerAllocationEvent.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/event/SubQueryContainerAllocationEvent.java b/tajo-core/src/main/java/org/apache/tajo/master/event/SubQueryContainerAllocationEvent.java
index a8f4800..e617d53 100644
--- a/tajo-core/src/main/java/org/apache/tajo/master/event/SubQueryContainerAllocationEvent.java
+++ b/tajo-core/src/main/java/org/apache/tajo/master/event/SubQueryContainerAllocationEvent.java
@@ -18,21 +18,21 @@
 
 package org.apache.tajo.master.event;
 
-import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.tajo.ExecutionBlockId;
+import org.apache.tajo.master.container.TajoContainer;
 
 import java.util.List;
 
 public class SubQueryContainerAllocationEvent extends SubQueryEvent {
-  private List<Container> allocatedContainer;
+  private List<TajoContainer> allocatedContainer;
 
   public SubQueryContainerAllocationEvent(final ExecutionBlockId id,
-                                          List<Container> allocatedContainer) {
+                                          List<TajoContainer> allocatedContainer) {
     super(id, SubQueryEventType.SQ_CONTAINER_ALLOCATED);
     this.allocatedContainer = allocatedContainer;
   }
 
-  public List<Container> getAllocatedContainer() {
+  public List<TajoContainer> getAllocatedContainer() {
     return this.allocatedContainer;
   }
 }

http://git-wip-us.apache.org/repos/asf/tajo/blob/3d485ecb/tajo-core/src/main/java/org/apache/tajo/master/event/TaskAttemptAssignedEvent.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/event/TaskAttemptAssignedEvent.java b/tajo-core/src/main/java/org/apache/tajo/master/event/TaskAttemptAssignedEvent.java
index e0928c5..3b9edcb 100644
--- a/tajo-core/src/main/java/org/apache/tajo/master/event/TaskAttemptAssignedEvent.java
+++ b/tajo-core/src/main/java/org/apache/tajo/master/event/TaskAttemptAssignedEvent.java
@@ -18,22 +18,22 @@
 
 package org.apache.tajo.master.event;
 
-import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.tajo.QueryUnitAttemptId;
 import org.apache.tajo.master.cluster.WorkerConnectionInfo;
+import org.apache.tajo.master.container.TajoContainerId;
 
 public class TaskAttemptAssignedEvent extends TaskAttemptEvent {
-  private final ContainerId cId;
+  private final TajoContainerId cId;
   private final WorkerConnectionInfo workerConnectionInfo;
 
-  public TaskAttemptAssignedEvent(QueryUnitAttemptId id, ContainerId cId,
+  public TaskAttemptAssignedEvent(QueryUnitAttemptId id, TajoContainerId cId,
                                   WorkerConnectionInfo connectionInfo) {
     super(id, TaskAttemptEventType.TA_ASSIGNED);
     this.cId = cId;
     this.workerConnectionInfo = connectionInfo;
   }
 
-  public ContainerId getContainerId() {
+  public TajoContainerId getContainerId() {
     return cId;
   }
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/3d485ecb/tajo-core/src/main/java/org/apache/tajo/master/event/TaskRequestEvent.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/event/TaskRequestEvent.java b/tajo-core/src/main/java/org/apache/tajo/master/event/TaskRequestEvent.java
index 2197c33..9e8e3dd 100644
--- a/tajo-core/src/main/java/org/apache/tajo/master/event/TaskRequestEvent.java
+++ b/tajo-core/src/main/java/org/apache/tajo/master/event/TaskRequestEvent.java
@@ -19,11 +19,11 @@
 package org.apache.tajo.master.event;
 
 import com.google.protobuf.RpcCallback;
-import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.event.AbstractEvent;
 import org.apache.tajo.ExecutionBlockId;
 import org.apache.tajo.ipc.TajoWorkerProtocol.QueryUnitRequestProto;
 import org.apache.tajo.master.event.TaskRequestEvent.TaskRequestEventType;
+import org.apache.tajo.master.container.TajoContainerId;
 
 public class TaskRequestEvent extends AbstractEvent<TaskRequestEventType> {
 
@@ -32,13 +32,13 @@ public class TaskRequestEvent extends AbstractEvent<TaskRequestEventType> {
   }
 
   private final int workerId;
-  private final ContainerId containerId;
+  private final TajoContainerId containerId;
   private final ExecutionBlockId executionBlockId;
 
   private final RpcCallback<QueryUnitRequestProto> callback;
 
   public TaskRequestEvent(int workerId,
-                          ContainerId containerId,
+                          TajoContainerId containerId,
                           ExecutionBlockId executionBlockId,
                           RpcCallback<QueryUnitRequestProto> callback) {
     super(TaskRequestEventType.TASK_REQ);
@@ -52,7 +52,7 @@ public class TaskRequestEvent extends AbstractEvent<TaskRequestEventType> {
     return this.workerId;
   }
 
-  public ContainerId getContainerId() {
+  public TajoContainerId getContainerId() {
     return this.containerId;
   }
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/3d485ecb/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryInProgress.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryInProgress.java b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryInProgress.java
index d949ca4..e361c7f 100644
--- a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryInProgress.java
+++ b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryInProgress.java
@@ -28,6 +28,7 @@ import org.apache.hadoop.yarn.proto.YarnProtos;
 import org.apache.tajo.QueryId;
 import org.apache.tajo.TajoProtos;
 import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.ipc.ContainerProtocol;
 import org.apache.tajo.plan.logical.LogicalRootNode;
 import org.apache.tajo.engine.query.QueryContext;
 import org.apache.tajo.ipc.QueryMasterProtocol;
@@ -74,7 +75,7 @@ public class QueryInProgress extends CompositeService {
 
   private QueryMasterProtocolService queryMasterRpcClient;
 
-  private YarnProtos.ContainerIdProto qmContainerId;
+  private ContainerProtocol.TajoContainerIdProto qmContainerId;
 
   public QueryInProgress(
       TajoMaster.MasterContext masterContext,

http://git-wip-us.apache.org/repos/asf/tajo/blob/3d485ecb/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryMasterManagerService.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryMasterManagerService.java b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryMasterManagerService.java
index f953995..f4bd8a3 100644
--- a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryMasterManagerService.java
+++ b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryMasterManagerService.java
@@ -25,7 +25,6 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.service.CompositeService;
-import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.tajo.*;
 import org.apache.tajo.conf.TajoConf;
 import org.apache.tajo.engine.query.QueryContext;
@@ -33,6 +32,7 @@ import org.apache.tajo.ipc.QueryMasterProtocol;
 import org.apache.tajo.ipc.TajoWorkerProtocol;
 import org.apache.tajo.master.LazyTaskScheduler;
 import org.apache.tajo.master.event.*;
+import org.apache.tajo.master.container.TajoContainerId;
 import org.apache.tajo.master.session.Session;
 import org.apache.tajo.rpc.AsyncRpcServer;
 import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos;
@@ -130,7 +130,7 @@ public class QueryMasterManagerService extends CompositeService
       if(queryMasterTask == null || queryMasterTask.isStopped()) {
         done.run(LazyTaskScheduler.stopTaskRunnerReq);
       } else {
-        ContainerId cid =
+        TajoContainerId cid =
             queryMasterTask.getQueryTaskContext().getResourceAllocator().makeContainerId(request.getContainerId());
         LOG.debug("getTask:" + cid + ", ebId:" + ebId);
         queryMasterTask.handleTaskRequestEvent(new TaskRequestEvent(request.getWorkerId(), cid, ebId, done));

http://git-wip-us.apache.org/repos/asf/tajo/blob/3d485ecb/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryUnitAttempt.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryUnitAttempt.java b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryUnitAttempt.java
index db6f130..d88173f 100644
--- a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryUnitAttempt.java
+++ b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryUnitAttempt.java
@@ -21,7 +21,6 @@ package org.apache.tajo.master.querymaster;
 import org.apache.commons.lang.exception.ExceptionUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.state.*;
 import org.apache.tajo.QueryUnitAttemptId;
@@ -35,6 +34,7 @@ import org.apache.tajo.master.event.QueryUnitAttemptScheduleEvent.QueryUnitAttem
 import org.apache.tajo.master.event.TaskSchedulerEvent.EventType;
 import org.apache.tajo.master.querymaster.QueryUnit.IntermediateEntry;
 import org.apache.tajo.master.querymaster.QueryUnit.PullHost;
+import org.apache.tajo.master.container.TajoContainerId;
 
 import java.util.ArrayList;
 import java.util.EnumSet;
@@ -55,7 +55,7 @@ public class QueryUnitAttempt implements EventHandler<TaskAttemptEvent> {
   private final QueryUnit queryUnit;
   final EventHandler eventHandler;
 
-  private ContainerId containerId;
+  private TajoContainerId containerId;
   private WorkerConnectionInfo workerConnectionInfo;
   private int expire;
 
@@ -214,7 +214,7 @@ public class QueryUnitAttempt implements EventHandler<TaskAttemptEvent> {
     return this.workerConnectionInfo;
   }
 
-  public void setContainerId(ContainerId containerId) {
+  public void setContainerId(TajoContainerId containerId) {
     this.containerId = containerId;
   }
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/3d485ecb/tajo-core/src/main/java/org/apache/tajo/master/querymaster/SubQuery.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/SubQuery.java b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/SubQuery.java
index 476da04..39bb7ed 100644
--- a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/SubQuery.java
+++ b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/SubQuery.java
@@ -24,7 +24,6 @@ import org.apache.commons.lang.exception.ExceptionUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Resource;
@@ -58,6 +57,8 @@ import org.apache.tajo.master.TaskRunnerGroupEvent.EventType;
 import org.apache.tajo.master.event.*;
 import org.apache.tajo.master.event.QueryUnitAttemptScheduleEvent.QueryUnitAttemptScheduleContext;
 import org.apache.tajo.master.querymaster.QueryUnit.IntermediateEntry;
+import org.apache.tajo.master.container.TajoContainer;
+import org.apache.tajo.master.container.TajoContainerId;
 import org.apache.tajo.plan.util.PlannerUtil;
 import org.apache.tajo.plan.logical.*;
 import org.apache.tajo.storage.StorageManager;
@@ -105,7 +106,8 @@ public class SubQuery implements EventHandler<SubQueryEvent> {
   private long finishTime;
 
   volatile Map<QueryUnitId, QueryUnit> tasks = new ConcurrentHashMap<QueryUnitId, QueryUnit>();
-  volatile Map<ContainerId, Container> containers = new ConcurrentHashMap<ContainerId, Container>();
+  volatile Map<TajoContainerId, TajoContainer> containers = new ConcurrentHashMap<TajoContainerId,
+    TajoContainer>();
 
   private static final DiagnosticsUpdateTransition DIAGNOSTIC_UPDATE_TRANSITION = new DiagnosticsUpdateTransition();
   private static final InternalErrorTransition INTERNAL_ERROR_TRANSITION = new InternalErrorTransition();
@@ -663,13 +665,6 @@ public class SubQuery implements EventHandler<SubQueryEvent> {
     eventHandler.handle(new TaskRunnerGroupEvent(EventType.CONTAINER_REMOTE_CLEANUP, getId(), containers.values()));
   }
 
-  public void releaseContainer(ContainerId containerId) {
-    // try to kill the container.
-    ArrayList<Container> list = new ArrayList<Container>();
-    list.add(containers.get(containerId));
-    eventHandler.handle(new TaskRunnerGroupEvent(EventType.CONTAINER_REMOTE_CLEANUP, getId(), list));
-  }
-
   /**
    * It computes all stats and sets the intermediate result.
    */
@@ -1129,8 +1124,8 @@ public class SubQuery implements EventHandler<SubQueryEvent> {
       try {
         SubQueryContainerAllocationEvent allocationEvent =
             (SubQueryContainerAllocationEvent) event;
-        for (Container container : allocationEvent.getAllocatedContainer()) {
-          ContainerId cId = container.getId();
+        for (TajoContainer container : allocationEvent.getAllocatedContainer()) {
+          TajoContainerId cId = container.getId();
           if (subQuery.containers.containsKey(cId)) {
             subQuery.eventHandler.handle(new SubQueryDiagnosticsUpdateEvent(subQuery.getId(),
                 "Duplicated containers are allocated: " + cId.toString()));

http://git-wip-us.apache.org/repos/asf/tajo/blob/3d485ecb/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoRMContext.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoRMContext.java b/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoRMContext.java
index 5d07ff2..bb8cc12 100644
--- a/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoRMContext.java
+++ b/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoRMContext.java
@@ -21,14 +21,13 @@ package org.apache.tajo.master.rm;
 import com.google.common.collect.Maps;
 import org.apache.hadoop.yarn.event.Dispatcher;
 import org.apache.tajo.QueryId;
+import org.apache.tajo.ipc.ContainerProtocol;
 
 import java.util.Collections;
 import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 
-import static org.apache.hadoop.yarn.proto.YarnProtos.ContainerIdProto;
-
 /**
  * It's a worker resource manager context. It contains all context data about TajoWorkerResourceManager.
  */
@@ -43,7 +42,8 @@ public class TajoRMContext {
   private final ConcurrentMap<Integer, Worker> inactiveWorkers = Maps.newConcurrentMap();
 
   /** map between queryIds and query master ContainerId */
-  private final ConcurrentMap<QueryId, ContainerIdProto> qmContainerMap = Maps.newConcurrentMap();
+  private final ConcurrentMap<QueryId, ContainerProtocol.TajoContainerIdProto> qmContainerMap = Maps
+    .newConcurrentMap();
 
   private final Set<Integer> liveQueryMasterWorkerResources =
       Collections.newSetFromMap(new ConcurrentHashMap<Integer, Boolean>());
@@ -77,7 +77,7 @@ public class TajoRMContext {
    *
    * @return The Map for query master containers
    */
-  public ConcurrentMap<QueryId, ContainerIdProto> getQueryMasterContainer() {
+  public ConcurrentMap<QueryId, ContainerProtocol.TajoContainerIdProto> getQueryMasterContainer() {
     return qmContainerMap;
   }
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/3d485ecb/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoWorkerContainer.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoWorkerContainer.java b/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoWorkerContainer.java
index 4d6cbd2..3d28d85 100644
--- a/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoWorkerContainer.java
+++ b/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoWorkerContainer.java
@@ -19,9 +19,12 @@
 package org.apache.tajo.master.rm;
 
 import org.apache.hadoop.yarn.api.records.*;
+import org.apache.tajo.master.container.TajoContainer;
+import org.apache.tajo.master.container.TajoContainerId;
 
-public class TajoWorkerContainer extends Container {
-  ContainerId id;
+
+public class TajoWorkerContainer extends TajoContainer {
+  TajoContainerId id;
   NodeId nodeId;
   Worker worker;
 
@@ -34,12 +37,12 @@ public class TajoWorkerContainer extends Container {
   }
 
   @Override
-  public ContainerId getId() {
+  public TajoContainerId getId() {
     return id;
   }
 
   @Override
-  public void setId(ContainerId id) {
+  public void setId(TajoContainerId id) {
     this.id = id;
   }
 
@@ -94,7 +97,7 @@ public class TajoWorkerContainer extends Container {
   }
 
   @Override
-  public int compareTo(Container container) {
-    return 0;  //To change body of implemented methods use File | Settings | File Templates.
+  public int compareTo(TajoContainer container) {
+    return getId().compareTo(container.getId());
   }
 }

http://git-wip-us.apache.org/repos/asf/tajo/blob/3d485ecb/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoWorkerContainerId.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoWorkerContainerId.java b/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoWorkerContainerId.java
index 634ad2b..184de71 100644
--- a/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoWorkerContainerId.java
+++ b/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoWorkerContainerId.java
@@ -19,10 +19,11 @@
 package org.apache.tajo.master.rm;
 
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
-import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.proto.YarnProtos;
+import org.apache.tajo.ipc.ContainerProtocol;
+import org.apache.tajo.master.container.TajoContainerId;
 
-public class TajoWorkerContainerId extends ContainerId {
+public class TajoWorkerContainerId extends TajoContainerId {
   ApplicationAttemptId applicationAttemptId;
   int id;
 
@@ -46,43 +47,43 @@ public class TajoWorkerContainerId extends ContainerId {
     this.id = id;
   }
 
-  public YarnProtos.ContainerIdProto getProto() {
+  public ContainerProtocol.TajoContainerIdProto getProto() {
     YarnProtos.ApplicationIdProto appIdProto = YarnProtos.ApplicationIdProto.newBuilder()
-        .setClusterTimestamp(applicationAttemptId.getApplicationId().getClusterTimestamp())
-        .setId(applicationAttemptId.getApplicationId().getId())
-        .build();
+      .setClusterTimestamp(applicationAttemptId.getApplicationId().getClusterTimestamp())
+      .setId(applicationAttemptId.getApplicationId().getId())
+      .build();
 
     YarnProtos.ApplicationAttemptIdProto attemptIdProto = YarnProtos.ApplicationAttemptIdProto.newBuilder()
-        .setAttemptId(applicationAttemptId.getAttemptId())
-        .setApplicationId(appIdProto)
-        .build();
+      .setAttemptId(applicationAttemptId.getAttemptId())
+      .setApplicationId(appIdProto)
+      .build();
 
-    return YarnProtos.ContainerIdProto.newBuilder()
-        .setAppAttemptId(attemptIdProto)
-        .setAppId(appIdProto)
-        .setId(id)
-        .build();
+    return ContainerProtocol.TajoContainerIdProto.newBuilder()
+      .setAppAttemptId(attemptIdProto)
+      .setAppId(appIdProto)
+      .setId(id)
+      .build();
   }
 
-  public static YarnProtos.ContainerIdProto getContainerIdProto(ContainerId containerId) {
+  public static ContainerProtocol.TajoContainerIdProto getContainerIdProto(TajoContainerId containerId) {
     if(containerId instanceof TajoWorkerContainerId) {
       return ((TajoWorkerContainerId)containerId).getProto();
     } else {
       YarnProtos.ApplicationIdProto appIdProto = YarnProtos.ApplicationIdProto.newBuilder()
-          .setClusterTimestamp(containerId.getApplicationAttemptId().getApplicationId().getClusterTimestamp())
-          .setId(containerId.getApplicationAttemptId().getApplicationId().getId())
-          .build();
+        .setClusterTimestamp(containerId.getApplicationAttemptId().getApplicationId().getClusterTimestamp())
+        .setId(containerId.getApplicationAttemptId().getApplicationId().getId())
+        .build();
 
       YarnProtos.ApplicationAttemptIdProto attemptIdProto = YarnProtos.ApplicationAttemptIdProto.newBuilder()
-          .setAttemptId(containerId.getApplicationAttemptId().getAttemptId())
-          .setApplicationId(appIdProto)
-          .build();
+        .setAttemptId(containerId.getApplicationAttemptId().getAttemptId())
+        .setApplicationId(appIdProto)
+        .build();
 
-      return YarnProtos.ContainerIdProto.newBuilder()
-          .setAppAttemptId(attemptIdProto)
-          .setAppId(appIdProto)
-          .setId(containerId.getId())
-          .build();
+      return ContainerProtocol.TajoContainerIdProto.newBuilder()
+        .setAppAttemptId(attemptIdProto)
+        .setAppId(appIdProto)
+        .setId(containerId.getId())
+        .build();
     }
   }
 


[17/29] tajo git commit: TAJO-1224: When there is no projected columns, json scan can be hang.

Posted by hj...@apache.org.
TAJO-1224: When there is no projected columns, json scan can be hang.

Closes #281


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

Branch: refs/heads/hbase_storage
Commit: 9f8be1a695298e2e9fe0d881ddfcb310b5a7460b
Parents: 20d1f01
Author: Hyunsik Choi <hy...@apache.org>
Authored: Thu Dec 4 10:55:19 2014 +0900
Committer: Hyunsik Choi <hy...@apache.org>
Committed: Thu Dec 4 11:28:44 2014 +0900

----------------------------------------------------------------------
 CHANGES                                          |  3 +++
 .../tajo/engine/query/TestCaseByCases.java       |  8 ++++++++
 .../TestCaseByCases/testTAJO1224Case1.sql        |  1 +
 .../TestCaseByCases/testTAJO1224Case1.result     |  3 +++
 .../tajo/storage/text/DelimitedTextFile.java     | 19 ++++++++++++-------
 5 files changed, 27 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tajo/blob/9f8be1a6/CHANGES
----------------------------------------------------------------------
diff --git a/CHANGES b/CHANGES
index ea8e1ca..60aa3e0 100644
--- a/CHANGES
+++ b/CHANGES
@@ -83,6 +83,9 @@ Release 0.9.1 - unreleased
 
   BUG FIXES
 
+    TAJO-1224: When there is no projected column, json scan can be hang. 
+    (hyunsik) 
+
     TAJO-1220: Implement createStatement() and setEscapeProcessing() in 
     JdbcConnection. (YeonSu Han via hyunsik)
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/9f8be1a6/tajo-core/src/test/java/org/apache/tajo/engine/query/TestCaseByCases.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestCaseByCases.java b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestCaseByCases.java
index 846c290..bcf00f8 100644
--- a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestCaseByCases.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestCaseByCases.java
@@ -172,4 +172,12 @@ public class TestCaseByCases extends QueryTestCaseBase {
     assertResultSet(res);
     cleanupQuery(res);
   }
+
+  @Test
+  public final void testTAJO1224Case1() throws Exception {
+    executeString("CREATE TABLE TAJO1224 USING JSON AS SELECT * FROM LINEITEM").close();
+    ResultSet res = executeQuery();
+    assertResultSet(res);
+    cleanupQuery(res);
+  }
 }

http://git-wip-us.apache.org/repos/asf/tajo/blob/9f8be1a6/tajo-core/src/test/resources/queries/TestCaseByCases/testTAJO1224Case1.sql
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/resources/queries/TestCaseByCases/testTAJO1224Case1.sql b/tajo-core/src/test/resources/queries/TestCaseByCases/testTAJO1224Case1.sql
new file mode 100644
index 0000000..d05a563
--- /dev/null
+++ b/tajo-core/src/test/resources/queries/TestCaseByCases/testTAJO1224Case1.sql
@@ -0,0 +1 @@
+select count(*) from tajo1224;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/9f8be1a6/tajo-core/src/test/resources/results/TestCaseByCases/testTAJO1224Case1.result
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/resources/results/TestCaseByCases/testTAJO1224Case1.result b/tajo-core/src/test/resources/results/TestCaseByCases/testTAJO1224Case1.result
new file mode 100644
index 0000000..19336a7
--- /dev/null
+++ b/tajo-core/src/test/resources/results/TestCaseByCases/testTAJO1224Case1.result
@@ -0,0 +1,3 @@
+?count
+-------------------------------
+5
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/9f8be1a6/tajo-storage/src/main/java/org/apache/tajo/storage/text/DelimitedTextFile.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/text/DelimitedTextFile.java b/tajo-storage/src/main/java/org/apache/tajo/storage/text/DelimitedTextFile.java
index c54131b..ab8a0b5 100644
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/text/DelimitedTextFile.java
+++ b/tajo-storage/src/main/java/org/apache/tajo/storage/text/DelimitedTextFile.java
@@ -358,29 +358,34 @@ public class DelimitedTextFile {
 
     @Override
     public Tuple next() throws IOException {
+      VTuple tuple;
 
       if (!reader.isReadable()) {
         return null;
       }
 
-      if (targets.length == 0) {
-        return EmptyTuple.get();
-      }
-
-      VTuple tuple = new VTuple(schema.size());
-
       try {
 
         // this loop will continue until one tuple is build or EOS (end of stream).
         do {
 
           ByteBuf buf = reader.readLine();
+
+          // if no more line, then return EOT (end of tuple)
           if (buf == null) {
             return null;
           }
 
-          try {
+          // If there is no required column, we just read each line
+          // and then return an empty tuple without parsing line.
+          if (targets.length == 0) {
+            recordCount++;
+            return EmptyTuple.get();
+          }
 
+          tuple = new VTuple(schema.size());
+
+          try {
             deserializer.deserialize(buf, tuple);
             // if a line is read normaly, it exists this loop.
             break;


[09/29] tajo git commit: TAJO-1095: Implement Json file scanner.

Posted by hj...@apache.org.
TAJO-1095: Implement Json file scanner.

Closes #181


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

Branch: refs/heads/hbase_storage
Commit: cd38dffb908a3959472f5ddb705db71d0e48ad89
Parents: 7d41c67
Author: Hyunsik Choi <hy...@apache.org>
Authored: Fri Nov 28 17:26:38 2014 +0900
Committer: Hyunsik Choi <hy...@apache.org>
Committed: Fri Nov 28 17:26:38 2014 +0900

----------------------------------------------------------------------
 CHANGES                                         |   2 +
 .../org/apache/tajo/catalog/CatalogUtil.java    |   9 +-
 .../org/apache/tajo/catalog/SchemaUtil.java     |   8 +
 .../src/main/proto/CatalogProtos.proto          |   1 +
 .../apache/tajo/storage/StorageConstants.java   |   2 +-
 .../java/org/apache/tajo/storage/VTuple.java    |   4 +-
 tajo-storage/pom.xml                            |   6 +
 .../tajo/storage/json/JsonLineDeserializer.java | 220 +++++++++++++++++++
 .../apache/tajo/storage/json/JsonLineSerDe.java |  37 ++++
 .../tajo/storage/json/JsonLineSerializer.java   | 134 +++++++++++
 .../tajo/storage/text/CSVLineDeserializer.java  |   4 +-
 .../apache/tajo/storage/text/CSVLineSerDe.java  |   4 -
 .../tajo/storage/text/CSVLineSerializer.java    |  15 +-
 .../tajo/storage/text/DelimitedTextFile.java    |   8 +-
 .../text/TextFieldSerializerDeserializer.java   |   2 +-
 .../tajo/storage/text/TextLineDeserializer.java |   4 +-
 .../src/main/resources/storage-default.xml      |  16 +-
 .../org/apache/tajo/storage/TestStorages.java   | 105 +++++----
 .../apache/tajo/storage/json/TestJsonSerDe.java | 101 +++++++++
 .../dataset/TestJsonSerDe/testVariousType.json  |   1 +
 .../src/test/resources/storage-default.xml      |  16 +-
 .../src/test/resources/testVariousTypes.avsc    |  19 +-
 22 files changed, 637 insertions(+), 81 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tajo/blob/cd38dffb/CHANGES
----------------------------------------------------------------------
diff --git a/CHANGES b/CHANGES
index 0d0677c..025ae88 100644
--- a/CHANGES
+++ b/CHANGES
@@ -11,6 +11,8 @@ Release 0.9.1 - unreleased
 
     TAJO-235: Support Oracle CatalogStore. (Jihun Kang via hyunsik)
 
+    TAJO-1095: Implement Json file scanner. (hyunsik)
+
   IMPROVEMENT
 
     TAJO-1165: Needs to show error messages on query_executor.jsp. (Jihun Kang via jaehwa)

http://git-wip-us.apache.org/repos/asf/tajo/blob/cd38dffb/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/CatalogUtil.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/CatalogUtil.java b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/CatalogUtil.java
index 737c9ae..f2d9b9c 100644
--- a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/CatalogUtil.java
+++ b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/CatalogUtil.java
@@ -31,9 +31,11 @@ import org.apache.tajo.common.TajoDataTypes;
 import org.apache.tajo.common.TajoDataTypes.DataType;
 import org.apache.tajo.exception.InvalidOperationException;
 import org.apache.tajo.storage.StorageConstants;
+import org.apache.tajo.unit.StorageUnit;
 import org.apache.tajo.util.KeyValueSet;
 import org.apache.tajo.util.StringUtils;
 import org.apache.tajo.util.TUtil;
+import org.mortbay.util.ajax.JSON;
 
 import java.sql.Connection;
 import java.sql.ResultSet;
@@ -278,13 +280,16 @@ public class CatalogUtil {
       return StoreType.AVRO;
     } else if (typeStr.equalsIgnoreCase(StoreType.TEXTFILE.name())) {
       return StoreType.TEXTFILE;
+    } else if (typeStr.equalsIgnoreCase(StoreType.JSON.name())) {
+      return StoreType.JSON;
     } else {
       return null;
     }
   }
 
   public static TableMeta newTableMeta(StoreType type) {
-    return new TableMeta(type, new KeyValueSet());
+    KeyValueSet defaultProperties = CatalogUtil.newPhysicalProperties(type);
+    return new TableMeta(type, defaultProperties);
   }
 
   public static TableMeta newTableMeta(StoreType type, KeyValueSet options) {
@@ -821,6 +826,8 @@ public class CatalogUtil {
     KeyValueSet options = new KeyValueSet();
     if (StoreType.CSV == type || StoreType.TEXTFILE == type) {
       options.set(StorageConstants.TEXT_DELIMITER, StorageConstants.DEFAULT_FIELD_DELIMITER);
+    } else if (StoreType.JSON == type) {
+      options.set(StorageConstants.TEXT_SERDE_CLASS, "org.apache.tajo.storage.json.JsonLineSerDe");
     } else if (StoreType.RCFILE == type) {
       options.set(StorageConstants.RCFILE_SERDE, StorageConstants.DEFAULT_BINARY_SERDE);
     } else if (StoreType.SEQUENCEFILE == type) {

http://git-wip-us.apache.org/repos/asf/tajo/blob/cd38dffb/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/SchemaUtil.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/SchemaUtil.java b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/SchemaUtil.java
index ee670ef..23ebe1b 100644
--- a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/SchemaUtil.java
+++ b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/SchemaUtil.java
@@ -100,4 +100,12 @@ public class SchemaUtil {
     }
     return types;
   }
+
+  public static String [] toSimpleNames(Schema schema) {
+    String [] names = new String[schema.size()];
+    for (int i = 0; i < schema.size(); i++) {
+      names[i] = schema.getColumn(i).getSimpleName();
+    }
+    return names;
+  }
 }

http://git-wip-us.apache.org/repos/asf/tajo/blob/cd38dffb/tajo-catalog/tajo-catalog-common/src/main/proto/CatalogProtos.proto
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-common/src/main/proto/CatalogProtos.proto b/tajo-catalog/tajo-catalog-common/src/main/proto/CatalogProtos.proto
index 99f594a..f29bc6c 100644
--- a/tajo-catalog/tajo-catalog-common/src/main/proto/CatalogProtos.proto
+++ b/tajo-catalog/tajo-catalog-common/src/main/proto/CatalogProtos.proto
@@ -36,6 +36,7 @@ enum StoreType {
   SEQUENCEFILE = 8;
   AVRO = 9;
   TEXTFILE = 10;
+  JSON = 11;
 }
 
 enum OrderType {

http://git-wip-us.apache.org/repos/asf/tajo/blob/cd38dffb/tajo-common/src/main/java/org/apache/tajo/storage/StorageConstants.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/storage/StorageConstants.java b/tajo-common/src/main/java/org/apache/tajo/storage/StorageConstants.java
index 3065d31..a3d8de0 100644
--- a/tajo-common/src/main/java/org/apache/tajo/storage/StorageConstants.java
+++ b/tajo-common/src/main/java/org/apache/tajo/storage/StorageConstants.java
@@ -33,7 +33,7 @@ public class StorageConstants {
   public static final String TEXT_DELIMITER = "text.delimiter";
   public static final String TEXT_NULL = "text.null";
   public static final String TEXT_SERDE_CLASS = "text.serde.class";
-  public static final String DEFAULT_TEXT_SERDE_CLASS = "org.apache.tajo.storage.text.CSVLineSerde";
+  public static final String DEFAULT_TEXT_SERDE_CLASS = "org.apache.tajo.storage.text.CSVLineSerDe";
 
   @Deprecated
   public static final String SEQUENCEFILE_DELIMITER = "sequencefile.delimiter";

http://git-wip-us.apache.org/repos/asf/tajo/blob/cd38dffb/tajo-common/src/main/java/org/apache/tajo/storage/VTuple.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/storage/VTuple.java b/tajo-common/src/main/java/org/apache/tajo/storage/VTuple.java
index 6304734..5e839b7 100644
--- a/tajo-common/src/main/java/org/apache/tajo/storage/VTuple.java
+++ b/tajo-common/src/main/java/org/apache/tajo/storage/VTuple.java
@@ -56,7 +56,7 @@ public class VTuple implements Tuple, Cloneable {
 
   @Override
   public boolean isNull(int fieldid) {
-    return values[fieldid].isNull();
+    return values[fieldid] == null || values[fieldid].isNull();
   }
 
   @Override
@@ -93,7 +93,7 @@ public class VTuple implements Tuple, Cloneable {
   }
 
   public void put(Datum [] values) {
-    System.arraycopy(values, 0, this.values, 0, size());
+    System.arraycopy(values, 0, this.values, 0, values.length);
 	}
 	
 	//////////////////////////////////////////////////////

http://git-wip-us.apache.org/repos/asf/tajo/blob/cd38dffb/tajo-storage/pom.xml
----------------------------------------------------------------------
diff --git a/tajo-storage/pom.xml b/tajo-storage/pom.xml
index c6877c4..ef26a32 100644
--- a/tajo-storage/pom.xml
+++ b/tajo-storage/pom.xml
@@ -72,6 +72,7 @@
         <configuration>
           <excludes>
             <exclude>src/test/resources/testVariousTypes.avsc</exclude>
+            <exclude>src/test/resources/dataset/TestJsonSerDe/*.json</exclude>
           </excludes>
         </configuration>
       </plugin>
@@ -313,6 +314,11 @@
       <groupId>io.netty</groupId>
       <artifactId>netty-buffer</artifactId>
     </dependency>
+    <dependency>
+      <groupId>net.minidev</groupId>
+      <artifactId>json-smart</artifactId>
+      <version>2.0</version>
+    </dependency>
   </dependencies>
 
   <profiles>

http://git-wip-us.apache.org/repos/asf/tajo/blob/cd38dffb/tajo-storage/src/main/java/org/apache/tajo/storage/json/JsonLineDeserializer.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/json/JsonLineDeserializer.java b/tajo-storage/src/main/java/org/apache/tajo/storage/json/JsonLineDeserializer.java
new file mode 100644
index 0000000..37cd9f3
--- /dev/null
+++ b/tajo-storage/src/main/java/org/apache/tajo/storage/json/JsonLineDeserializer.java
@@ -0,0 +1,220 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.storage.json;
+
+
+import io.netty.buffer.ByteBuf;
+import net.minidev.json.JSONArray;
+import net.minidev.json.JSONObject;
+import net.minidev.json.parser.JSONParser;
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.catalog.SchemaUtil;
+import org.apache.tajo.catalog.TableMeta;
+import org.apache.tajo.common.TajoDataTypes.Type;
+import org.apache.tajo.common.exception.NotImplementedException;
+import org.apache.tajo.datum.DatumFactory;
+import org.apache.tajo.datum.NullDatum;
+import org.apache.tajo.datum.protobuf.ProtobufJsonFormat;
+import org.apache.tajo.storage.Tuple;
+import org.apache.tajo.storage.text.TextLineDeserializer;
+
+import java.io.IOException;
+import java.util.Iterator;
+
+public class JsonLineDeserializer extends TextLineDeserializer {
+  private JSONParser parser;
+  private Type [] types;
+  private String [] columnNames;
+
+  public JsonLineDeserializer(Schema schema, TableMeta meta, int[] targetColumnIndexes) {
+    super(schema, meta, targetColumnIndexes);
+  }
+
+  @Override
+  public void init() {
+    types = SchemaUtil.toTypes(schema);
+    columnNames = SchemaUtil.toSimpleNames(schema);
+
+    parser = new JSONParser(JSONParser.MODE_JSON_SIMPLE);
+  }
+
+  @Override
+  public void deserialize(ByteBuf buf, Tuple output) throws IOException {
+    byte [] line = new byte[buf.readableBytes()];
+    buf.readBytes(line);
+
+    try {
+      JSONObject object = (JSONObject) parser.parse(line);
+
+      for (int i = 0; i < targetColumnIndexes.length; i++) {
+        int actualIdx = targetColumnIndexes[i];
+        String fieldName = columnNames[actualIdx];
+
+        if (!object.containsKey(fieldName)) {
+          output.put(actualIdx, NullDatum.get());
+          continue;
+        }
+
+        switch (types[actualIdx]) {
+        case BOOLEAN:
+          String boolStr = object.getAsString(fieldName);
+          if (boolStr != null) {
+            output.put(actualIdx, DatumFactory.createBool(boolStr.equals("true")));
+          } else {
+            output.put(actualIdx, NullDatum.get());
+          }
+          break;
+        case CHAR:
+          String charStr = object.getAsString(fieldName);
+          if (charStr != null) {
+            output.put(actualIdx, DatumFactory.createChar(charStr));
+          } else {
+            output.put(actualIdx, NullDatum.get());
+          }
+          break;
+        case INT1:
+        case INT2:
+          Number int2Num = object.getAsNumber(fieldName);
+          if (int2Num != null) {
+            output.put(actualIdx, DatumFactory.createInt2(int2Num.shortValue()));
+          } else {
+            output.put(actualIdx, NullDatum.get());
+          }
+          break;
+        case INT4:
+          Number int4Num = object.getAsNumber(fieldName);
+          if (int4Num != null) {
+            output.put(actualIdx, DatumFactory.createInt4(int4Num.intValue()));
+          } else {
+            output.put(actualIdx, NullDatum.get());
+          }
+          break;
+        case INT8:
+          Number int8Num = object.getAsNumber(fieldName);
+          if (int8Num != null) {
+            output.put(actualIdx, DatumFactory.createInt8(int8Num.longValue()));
+          } else {
+            output.put(actualIdx, NullDatum.get());
+          }
+          break;
+        case FLOAT4:
+          Number float4Num = object.getAsNumber(fieldName);
+          if (float4Num != null) {
+            output.put(actualIdx, DatumFactory.createFloat4(float4Num.floatValue()));
+          } else {
+            output.put(actualIdx, NullDatum.get());
+          }
+          break;
+        case FLOAT8:
+          Number float8Num = object.getAsNumber(fieldName);
+          if (float8Num != null) {
+            output.put(actualIdx, DatumFactory.createFloat8(float8Num.doubleValue()));
+          } else {
+            output.put(actualIdx, NullDatum.get());
+          }
+          break;
+        case TEXT:
+          String textStr = object.getAsString(fieldName);
+          if (textStr != null) {
+            output.put(actualIdx, DatumFactory.createText(textStr));
+          } else {
+            output.put(actualIdx, NullDatum.get());
+          }
+          break;
+        case TIMESTAMP:
+          String timestampStr = object.getAsString(fieldName);
+          if (timestampStr != null) {
+            output.put(actualIdx, DatumFactory.createTimestamp(timestampStr));
+          } else {
+            output.put(actualIdx, NullDatum.get());
+          }
+          break;
+        case TIME:
+          String timeStr = object.getAsString(fieldName);
+          if (timeStr != null) {
+            output.put(actualIdx, DatumFactory.createTime(timeStr));
+          } else {
+            output.put(actualIdx, NullDatum.get());
+          }
+          break;
+        case DATE:
+          String dateStr = object.getAsString(fieldName);
+          if (dateStr != null) {
+            output.put(actualIdx, DatumFactory.createDate(dateStr));
+          } else {
+            output.put(actualIdx, NullDatum.get());
+          }
+          break;
+        case BIT:
+        case BINARY:
+        case VARBINARY:
+        case BLOB: {
+          Object jsonObject = object.get(fieldName);
+
+          if (jsonObject == null) {
+            output.put(actualIdx, NullDatum.get());
+            break;
+          } if (jsonObject instanceof String) {
+            output.put(actualIdx, DatumFactory.createBlob((String)jsonObject));
+          } else if (jsonObject instanceof JSONArray) {
+            JSONArray jsonArray = (JSONArray) jsonObject;
+            byte[] bytes = new byte[jsonArray.size()];
+            Iterator<Object> it = jsonArray.iterator();
+            int arrayIdx = 0;
+            while (it.hasNext()) {
+              bytes[arrayIdx++] = ((Long) it.next()).byteValue();
+            }
+            if (bytes.length > 0) {
+              output.put(actualIdx, DatumFactory.createBlob(bytes));
+            } else {
+              output.put(actualIdx, NullDatum.get());
+            }
+            break;
+          } else {
+            throw new IOException("Unknown json object: " + object.getClass().getSimpleName());
+          }
+          break;
+        }
+        case INET4:
+          String inetStr = object.getAsString(fieldName);
+          if (inetStr != null) {
+            output.put(actualIdx, DatumFactory.createInet4(inetStr));
+          } else {
+            output.put(actualIdx, NullDatum.get());
+          }
+          break;
+
+        case NULL_TYPE:
+          output.put(actualIdx, NullDatum.get());
+          break;
+
+        default:
+          throw new NotImplementedException(types[actualIdx].name() + " is not supported.");
+        }
+      }
+
+    } catch (Throwable e) {
+      throw new IOException(e);
+    }
+  }
+
+  @Override
+  public void release() {
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/cd38dffb/tajo-storage/src/main/java/org/apache/tajo/storage/json/JsonLineSerDe.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/json/JsonLineSerDe.java b/tajo-storage/src/main/java/org/apache/tajo/storage/json/JsonLineSerDe.java
new file mode 100644
index 0000000..6db2c29
--- /dev/null
+++ b/tajo-storage/src/main/java/org/apache/tajo/storage/json/JsonLineSerDe.java
@@ -0,0 +1,37 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.storage.json;
+
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.catalog.TableMeta;
+import org.apache.tajo.storage.text.TextLineDeserializer;
+import org.apache.tajo.storage.text.TextLineSerDe;
+import org.apache.tajo.storage.text.TextLineSerializer;
+
+public class JsonLineSerDe extends TextLineSerDe {
+  @Override
+  public TextLineDeserializer createDeserializer(Schema schema, TableMeta meta, int[] targetColumnIndexes) {
+    return new JsonLineDeserializer(schema, meta, targetColumnIndexes);
+  }
+
+  @Override
+  public TextLineSerializer createSerializer(Schema schema, TableMeta meta) {
+    return new JsonLineSerializer(schema, meta);
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/cd38dffb/tajo-storage/src/main/java/org/apache/tajo/storage/json/JsonLineSerializer.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/json/JsonLineSerializer.java b/tajo-storage/src/main/java/org/apache/tajo/storage/json/JsonLineSerializer.java
new file mode 100644
index 0000000..c7007d8
--- /dev/null
+++ b/tajo-storage/src/main/java/org/apache/tajo/storage/json/JsonLineSerializer.java
@@ -0,0 +1,134 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.storage.json;
+
+
+import net.minidev.json.JSONObject;
+import org.apache.commons.lang.CharSet;
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.catalog.SchemaUtil;
+import org.apache.tajo.catalog.TableMeta;
+import org.apache.tajo.common.TajoDataTypes.Type;
+import org.apache.tajo.common.exception.NotImplementedException;
+import org.apache.tajo.datum.ProtobufDatum;
+import org.apache.tajo.datum.TextDatum;
+import org.apache.tajo.datum.protobuf.ProtobufJsonFormat;
+import org.apache.tajo.storage.Tuple;
+import org.apache.tajo.storage.text.TextLineSerDe;
+import org.apache.tajo.storage.text.TextLineSerializer;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.nio.charset.Charset;
+
+public class JsonLineSerializer extends TextLineSerializer {
+  private static ProtobufJsonFormat protobufJsonFormat = ProtobufJsonFormat.getInstance();
+
+  private Type [] types;
+  private String [] simpleNames;
+  private int columnNum;
+
+
+  public JsonLineSerializer(Schema schema, TableMeta meta) {
+    super(schema, meta);
+  }
+
+  @Override
+  public void init() {
+    types = SchemaUtil.toTypes(schema);
+    simpleNames = SchemaUtil.toSimpleNames(schema);
+    columnNum = schema.size();
+  }
+
+  @Override
+  public int serialize(OutputStream out, Tuple input) throws IOException {
+    JSONObject jsonObject = new JSONObject();
+
+    for (int i = 0; i < columnNum; i++) {
+      if (input.isNull(i)) {
+        continue;
+      }
+
+      String fieldName = simpleNames[i];
+      Type type = types[i];
+
+      switch (type) {
+
+      case BOOLEAN:
+        jsonObject.put(fieldName, input.getBool(i));
+        break;
+
+      case INT1:
+      case INT2:
+        jsonObject.put(fieldName, input.getInt2(i));
+        break;
+
+      case INT4:
+        jsonObject.put(fieldName, input.getInt4(i));
+        break;
+
+      case INT8:
+        jsonObject.put(fieldName, input.getInt8(i));
+        break;
+
+      case FLOAT4:
+        jsonObject.put(fieldName, input.getFloat4(i));
+        break;
+
+      case FLOAT8:
+        jsonObject.put(fieldName, input.getFloat8(i));
+        break;
+
+      case CHAR:
+      case TEXT:
+      case VARCHAR:
+      case INET4:
+      case TIMESTAMP:
+      case DATE:
+      case TIME:
+      case INTERVAL:
+        jsonObject.put(fieldName, input.getText(i));
+        break;
+
+      case BIT:
+      case BINARY:
+      case BLOB:
+      case VARBINARY:
+        jsonObject.put(fieldName, input.getBytes(i));
+        break;
+
+      case NULL_TYPE:
+        break;
+
+      default:
+        throw new NotImplementedException(types[i].name() + " is not supported.");
+      }
+    }
+
+    String jsonStr = jsonObject.toJSONString();
+    byte [] jsonBytes = jsonStr.getBytes(TextDatum.DEFAULT_CHARSET);
+    out.write(jsonBytes);
+    return jsonBytes.length;
+  }
+
+  @Override
+  public void release() {
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/cd38dffb/tajo-storage/src/main/java/org/apache/tajo/storage/text/CSVLineDeserializer.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/text/CSVLineDeserializer.java b/tajo-storage/src/main/java/org/apache/tajo/storage/text/CSVLineDeserializer.java
index f580da1..0e2dfb0 100644
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/text/CSVLineDeserializer.java
+++ b/tajo-storage/src/main/java/org/apache/tajo/storage/text/CSVLineDeserializer.java
@@ -48,7 +48,7 @@ public class CSVLineDeserializer extends TextLineDeserializer {
     fieldSerDer = new TextFieldSerializerDeserializer();
   }
 
-  public void deserialize(final ByteBuf lineBuf, Tuple tuple) throws IOException {
+  public void deserialize(final ByteBuf lineBuf, Tuple output) throws IOException {
     int[] projection = targetColumnIndexes;
     if (lineBuf == null || targetColumnIndexes == null || targetColumnIndexes.length == 0) {
       return;
@@ -73,7 +73,7 @@ public class CSVLineDeserializer extends TextLineDeserializer {
       if (projection.length > currentTarget && currentIndex == projection[currentTarget]) {
         lineBuf.setIndex(start, start + fieldLength);
         Datum datum = fieldSerDer.deserialize(lineBuf, schema.getColumn(currentIndex), currentIndex, nullChars);
-        tuple.put(currentIndex, datum);
+        output.put(currentIndex, datum);
         currentTarget++;
       }
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/cd38dffb/tajo-storage/src/main/java/org/apache/tajo/storage/text/CSVLineSerDe.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/text/CSVLineSerDe.java b/tajo-storage/src/main/java/org/apache/tajo/storage/text/CSVLineSerDe.java
index e2686a6..2fe7f23 100644
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/text/CSVLineSerDe.java
+++ b/tajo-storage/src/main/java/org/apache/tajo/storage/text/CSVLineSerDe.java
@@ -24,10 +24,6 @@ import org.apache.tajo.catalog.TableMeta;
 import org.apache.tajo.storage.StorageConstants;
 
 public class CSVLineSerDe extends TextLineSerDe {
-
-  public CSVLineSerDe() {
-  }
-
   @Override
   public TextLineDeserializer createDeserializer(Schema schema, TableMeta meta, int[] targetColumnIndexes) {
     return new CSVLineDeserializer(schema, meta, targetColumnIndexes);

http://git-wip-us.apache.org/repos/asf/tajo/blob/cd38dffb/tajo-storage/src/main/java/org/apache/tajo/storage/text/CSVLineSerializer.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/text/CSVLineSerializer.java b/tajo-storage/src/main/java/org/apache/tajo/storage/text/CSVLineSerializer.java
index 684519c..7397000 100644
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/text/CSVLineSerializer.java
+++ b/tajo-storage/src/main/java/org/apache/tajo/storage/text/CSVLineSerializer.java
@@ -18,6 +18,7 @@
 
 package org.apache.tajo.storage.text;
 
+import org.apache.tajo.catalog.Column;
 import org.apache.tajo.catalog.Schema;
 import org.apache.tajo.catalog.TableMeta;
 import org.apache.tajo.datum.Datum;
@@ -32,6 +33,7 @@ public class CSVLineSerializer extends TextLineSerializer {
 
   private byte [] nullChars;
   private char delimiter;
+  private int columnNum;
 
   public CSVLineSerializer(Schema schema, TableMeta meta) {
     super(schema, meta);
@@ -41,25 +43,26 @@ public class CSVLineSerializer extends TextLineSerializer {
   public void init() {
     nullChars = TextLineSerDe.getNullCharsAsBytes(meta);
     delimiter = CSVLineSerDe.getFieldDelimiter(meta);
+    columnNum = schema.size();
 
     serde = new TextFieldSerializerDeserializer();
   }
 
   @Override
   public int serialize(OutputStream out, Tuple input) throws IOException {
-    int rowBytes = 0;
+    int writtenBytes = 0;
 
-    for (int i = 0; i < schema.size(); i++) {
+    for (int i = 0; i < columnNum; i++) {
       Datum datum = input.get(i);
-      rowBytes += serde.serialize(out, datum, schema.getColumn(i), i, nullChars);
+      writtenBytes += serde.serialize(out, datum, schema.getColumn(i), i, nullChars);
 
-      if (schema.size() - 1 > i) {
+      if (columnNum - 1 > i) {
         out.write((byte) delimiter);
-        rowBytes += 1;
+        writtenBytes += 1;
       }
     }
 
-    return rowBytes;
+    return writtenBytes;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/tajo/blob/cd38dffb/tajo-storage/src/main/java/org/apache/tajo/storage/text/DelimitedTextFile.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/text/DelimitedTextFile.java b/tajo-storage/src/main/java/org/apache/tajo/storage/text/DelimitedTextFile.java
index d15f394..2218fae 100644
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/text/DelimitedTextFile.java
+++ b/tajo-storage/src/main/java/org/apache/tajo/storage/text/DelimitedTextFile.java
@@ -79,12 +79,12 @@ public class DelimitedTextFile {
       if (serdeClassCache.containsKey(serDeClassName)) {
         serdeClass = serdeClassCache.get(serDeClassName);
       } else {
-        serdeClass = (Class<? extends TextLineSerDe>) Class.forName(CSVLineSerDe.class.getName());
+        serdeClass = (Class<? extends TextLineSerDe>) Class.forName(serDeClassName);
         serdeClassCache.put(serDeClassName, serdeClass);
       }
       lineSerder = (TextLineSerDe) ReflectionUtil.newInstance(serdeClass);
     } catch (Throwable e) {
-      throw new RuntimeException("TextLineSerde class cannot be initialized");
+      throw new RuntimeException("TextLineSerde class cannot be initialized.", e);
     }
 
     return lineSerder;
@@ -382,7 +382,9 @@ public class DelimitedTextFile {
     @Override
     public void close() throws IOException {
       try {
-        deserializer.release();
+        if (deserializer != null) {
+          deserializer.release();
+        }
 
         if (tableStats != null && reader != null) {
           tableStats.setReadBytes(reader.getReadBytes());  //Actual Processed Bytes. (decompressed bytes + overhead)

http://git-wip-us.apache.org/repos/asf/tajo/blob/cd38dffb/tajo-storage/src/main/java/org/apache/tajo/storage/text/TextFieldSerializerDeserializer.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/text/TextFieldSerializerDeserializer.java b/tajo-storage/src/main/java/org/apache/tajo/storage/text/TextFieldSerializerDeserializer.java
index 9722959..95d0407 100644
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/text/TextFieldSerializerDeserializer.java
+++ b/tajo-storage/src/main/java/org/apache/tajo/storage/text/TextFieldSerializerDeserializer.java
@@ -38,7 +38,7 @@ import java.nio.charset.CharsetDecoder;
 public class TextFieldSerializerDeserializer implements FieldSerializerDeserializer {
   public static final byte[] trueBytes = "true".getBytes();
   public static final byte[] falseBytes = "false".getBytes();
-  private ProtobufJsonFormat protobufJsonFormat = ProtobufJsonFormat.getInstance();
+  private static ProtobufJsonFormat protobufJsonFormat = ProtobufJsonFormat.getInstance();
   private final CharsetDecoder decoder = CharsetUtil.getDecoder(CharsetUtil.UTF_8);
 
   private static boolean isNull(ByteBuf val, ByteBuf nullBytes) {

http://git-wip-us.apache.org/repos/asf/tajo/blob/cd38dffb/tajo-storage/src/main/java/org/apache/tajo/storage/text/TextLineDeserializer.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/text/TextLineDeserializer.java b/tajo-storage/src/main/java/org/apache/tajo/storage/text/TextLineDeserializer.java
index 645d118..b0d3c3a 100644
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/text/TextLineDeserializer.java
+++ b/tajo-storage/src/main/java/org/apache/tajo/storage/text/TextLineDeserializer.java
@@ -48,10 +48,10 @@ public abstract class TextLineDeserializer {
    * It fills a tuple with a read fields in a given line.
    *
    * @param buf Read line
-   * @param tuple Tuple to be filled with read fields
+   * @param output Tuple to be filled with read fields
    * @throws java.io.IOException
    */
-  public abstract void deserialize(final ByteBuf buf, Tuple tuple) throws IOException;
+  public abstract void deserialize(final ByteBuf buf, Tuple output) throws IOException;
 
   /**
    * Release external resources

http://git-wip-us.apache.org/repos/asf/tajo/blob/cd38dffb/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
index 064f250..e861b7d 100644
--- a/tajo-storage/src/main/resources/storage-default.xml
+++ b/tajo-storage/src/main/resources/storage-default.xml
@@ -35,7 +35,7 @@
   <!--- Registered Scanner Handler -->
   <property>
     <name>tajo.storage.scanner-handler</name>
-    <value>textfile,csv,raw,rcfile,row,parquet,sequencefile,avro</value>
+    <value>textfile,csv,json,raw,rcfile,row,parquet,sequencefile,avro</value>
   </property>
 
   <!--- Fragment Class Configurations -->
@@ -48,6 +48,10 @@
     <value>org.apache.tajo.storage.fragment.FileFragment</value>
   </property>
   <property>
+    <name>tajo.storage.fragment.json.class</name>
+    <value>org.apache.tajo.storage.fragment.FileFragment</value>
+  </property>
+  <property>
     <name>tajo.storage.fragment.raw.class</name>
     <value>org.apache.tajo.storage.fragment.FileFragment</value>
   </property>
@@ -84,6 +88,11 @@
   </property>
 
   <property>
+    <name>tajo.storage.scanner-handler.json.class</name>
+    <value>org.apache.tajo.storage.text.DelimitedTextFile$DelimitedTextFileScanner</value>
+  </property>
+
+  <property>
     <name>tajo.storage.scanner-handler.raw.class</name>
     <value>org.apache.tajo.storage.RawFile$RawFileScanner</value>
   </property>
@@ -130,6 +139,11 @@
   </property>
 
   <property>
+    <name>tajo.storage.appender-handler.json.class</name>
+    <value>org.apache.tajo.storage.text.DelimitedTextFile$DelimitedTextFileAppender</value>
+  </property>
+
+  <property>
     <name>tajo.storage.appender-handler.raw.class</name>
     <value>org.apache.tajo.storage.RawFile$RawFileAppender</value>
   </property>

http://git-wip-us.apache.org/repos/asf/tajo/blob/cd38dffb/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
index 6e2bc35..c581926 100644
--- a/tajo-storage/src/test/java/org/apache/tajo/storage/TestStorages.java
+++ b/tajo-storage/src/test/java/org/apache/tajo/storage/TestStorages.java
@@ -80,18 +80,17 @@ public class TestStorages {
       "  \"name\": \"testNullHandlingTypes\",\n" +
       "  \"fields\": [\n" +
       "    { \"name\": \"col1\", \"type\": [\"null\", \"boolean\"] },\n" +
-      "    { \"name\": \"col2\", \"type\": [\"null\", \"int\"] },\n" +
-      "    { \"name\": \"col3\", \"type\": [\"null\", \"string\"] },\n" +
+      "    { \"name\": \"col2\", \"type\": [\"null\", \"string\"] },\n" +
+      "    { \"name\": \"col3\", \"type\": [\"null\", \"int\"] },\n" +
       "    { \"name\": \"col4\", \"type\": [\"null\", \"int\"] },\n" +
-      "    { \"name\": \"col5\", \"type\": [\"null\", \"int\"] },\n" +
-      "    { \"name\": \"col6\", \"type\": [\"null\", \"long\"] },\n" +
-      "    { \"name\": \"col7\", \"type\": [\"null\", \"float\"] },\n" +
-      "    { \"name\": \"col8\", \"type\": [\"null\", \"double\"] },\n" +
-      "    { \"name\": \"col9\", \"type\": [\"null\", \"string\"] },\n" +
+      "    { \"name\": \"col5\", \"type\": [\"null\", \"long\"] },\n" +
+      "    { \"name\": \"col6\", \"type\": [\"null\", \"float\"] },\n" +
+      "    { \"name\": \"col7\", \"type\": [\"null\", \"double\"] },\n" +
+      "    { \"name\": \"col8\", \"type\": [\"null\", \"string\"] },\n" +
+      "    { \"name\": \"col9\", \"type\": [\"null\", \"bytes\"] },\n" +
       "    { \"name\": \"col10\", \"type\": [\"null\", \"bytes\"] },\n" +
-      "    { \"name\": \"col11\", \"type\": [\"null\", \"bytes\"] },\n" +
-      "    { \"name\": \"col12\", \"type\": \"null\" },\n" +
-      "    { \"name\": \"col13\", \"type\": [\"null\", \"bytes\"] }\n" +
+      "    { \"name\": \"col11\", \"type\": \"null\" },\n" +
+      "    { \"name\": \"col12\", \"type\": [\"null\", \"bytes\"] }\n" +
       "  ]\n" +
       "}\n";
 
@@ -129,6 +128,7 @@ public class TestStorages {
         {StoreType.SEQUENCEFILE, true, true, false},
         {StoreType.AVRO, false, false, false},
         {StoreType.TEXTFILE, true, true, false},
+        {StoreType.JSON, true, true, false},
     });
   }
 
@@ -298,20 +298,23 @@ public class TestStorages {
 
   @Test
   public void testVariousTypes() throws IOException {
+    boolean handleProtobuf = storeType != StoreType.JSON;
+
     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()));
+    schema.addColumn("col2", Type.CHAR, 7);
+    schema.addColumn("col3", Type.INT2);
+    schema.addColumn("col4", Type.INT4);
+    schema.addColumn("col5", Type.INT8);
+    schema.addColumn("col6", Type.FLOAT4);
+    schema.addColumn("col7", Type.FLOAT8);
+    schema.addColumn("col8", Type.TEXT);
+    schema.addColumn("col9", Type.BLOB);
+    schema.addColumn("col10", Type.INET4);
+    schema.addColumn("col11", Type.NULL_TYPE);
+    if (handleProtobuf) {
+      schema.addColumn("col12", CatalogUtil.newDataType(Type.PROTOBUF, TajoIdProtos.QueryIdProto.class.getName()));
+    }
 
     KeyValueSet options = new KeyValueSet();
     TableMeta meta = CatalogUtil.newTableMeta(storeType, options);
@@ -328,10 +331,9 @@ public class TestStorages {
     QueryId queryid = new QueryId("12345", 5);
     ProtobufDatumFactory factory = ProtobufDatumFactory.get(TajoIdProtos.QueryIdProto.class.getName());
 
-    Tuple tuple = new VTuple(13);
+    Tuple tuple = new VTuple(11 + (handleProtobuf ? 1 : 0));
     tuple.put(new Datum[] {
         DatumFactory.createBool(true),
-        DatumFactory.createBit((byte) 0x99),
         DatumFactory.createChar("hyunsik"),
         DatumFactory.createInt2((short) 17),
         DatumFactory.createInt4(59),
@@ -341,9 +343,12 @@ public class TestStorages {
         DatumFactory.createText("hyunsik"),
         DatumFactory.createBlob("hyunsik".getBytes()),
         DatumFactory.createInet4("192.168.0.1"),
-        NullDatum.get(),
-        factory.createDatum(queryid.getProto())
+        NullDatum.get()
     });
+    if (handleProtobuf) {
+      tuple.put(11, factory.createDatum(queryid.getProto()));
+    }
+
     appender.addTuple(tuple);
     appender.flush();
     appender.close();
@@ -364,20 +369,24 @@ public class TestStorages {
 
   @Test
   public void testNullHandlingTypes() throws IOException {
+    boolean handleProtobuf = storeType != StoreType.JSON;
+
     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()));
+    schema.addColumn("col2", Type.CHAR, 7);
+    schema.addColumn("col3", Type.INT2);
+    schema.addColumn("col4", Type.INT4);
+    schema.addColumn("col5", Type.INT8);
+    schema.addColumn("col6", Type.FLOAT4);
+    schema.addColumn("col7", Type.FLOAT8);
+    schema.addColumn("col8", Type.TEXT);
+    schema.addColumn("col9", Type.BLOB);
+    schema.addColumn("col10", Type.INET4);
+    schema.addColumn("col11", Type.NULL_TYPE);
+
+    if (handleProtobuf) {
+      schema.addColumn("col12", CatalogUtil.newDataType(Type.PROTOBUF, TajoIdProtos.QueryIdProto.class.getName()));
+    }
 
     KeyValueSet options = new KeyValueSet();
     TableMeta meta = CatalogUtil.newTableMeta(storeType, options);
@@ -397,11 +406,10 @@ public class TestStorages {
 
     QueryId queryid = new QueryId("12345", 5);
     ProtobufDatumFactory factory = ProtobufDatumFactory.get(TajoIdProtos.QueryIdProto.class.getName());
-
-    Tuple seedTuple = new VTuple(13);
+    int columnNum = 11 + (handleProtobuf ? 1 : 0);
+    Tuple seedTuple = new VTuple(columnNum);
     seedTuple.put(new Datum[]{
         DatumFactory.createBool(true),                // 0
-        DatumFactory.createBit((byte) 0x99),          // 1
         DatumFactory.createChar("hyunsik"),           // 2
         DatumFactory.createInt2((short) 17),          // 3
         DatumFactory.createInt4(59),                  // 4
@@ -412,14 +420,17 @@ public class TestStorages {
         DatumFactory.createBlob("hyunsik".getBytes()),// 9
         DatumFactory.createInet4("192.168.0.1"),      // 10
         NullDatum.get(),                              // 11
-        factory.createDatum(queryid.getProto())       // 12
     });
 
+    if (handleProtobuf) {
+      seedTuple.put(11, factory.createDatum(queryid.getProto()));       // 12
+    }
+
     // Making tuples with different null column positions
     Tuple tuple;
-    for (int i = 0; i < 13; i++) {
-      tuple = new VTuple(13);
-      for (int j = 0; j < 13; j++) {
+    for (int i = 0; i < columnNum; i++) {
+      tuple = new VTuple(columnNum);
+      for (int j = 0; j < columnNum; j++) {
         if (i == j) { // i'th column will have NULL value
           tuple.put(j, NullDatum.get());
         } else {
@@ -439,8 +450,8 @@ public class TestStorages {
     Tuple retrieved;
     int i = 0;
     while ((retrieved = scanner.next()) != null) {
-      assertEquals(13, retrieved.size());
-      for (int j = 0; j < 13; j++) {
+      assertEquals(columnNum, retrieved.size());
+      for (int j = 0; j < columnNum; j++) {
         if (i == j) {
           assertEquals(NullDatum.get(), retrieved.get(j));
         } else {

http://git-wip-us.apache.org/repos/asf/tajo/blob/cd38dffb/tajo-storage/src/test/java/org/apache/tajo/storage/json/TestJsonSerDe.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/test/java/org/apache/tajo/storage/json/TestJsonSerDe.java b/tajo-storage/src/test/java/org/apache/tajo/storage/json/TestJsonSerDe.java
new file mode 100644
index 0000000..038bc17
--- /dev/null
+++ b/tajo-storage/src/test/java/org/apache/tajo/storage/json/TestJsonSerDe.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.json;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.tajo.catalog.CatalogUtil;
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.catalog.TableMeta;
+import org.apache.tajo.catalog.proto.CatalogProtos;
+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.datum.NullDatum;
+import org.apache.tajo.storage.Scanner;
+import org.apache.tajo.storage.StorageManager;
+import org.apache.tajo.storage.Tuple;
+import org.apache.tajo.storage.VTuple;
+import org.apache.tajo.storage.fragment.FileFragment;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.net.URL;
+
+import static org.junit.Assert.*;
+
+public class TestJsonSerDe {
+  private static Schema schema = new Schema();
+
+  static {
+    schema.addColumn("col1", TajoDataTypes.Type.BOOLEAN);
+    schema.addColumn("col2", TajoDataTypes.Type.CHAR, 7);
+    schema.addColumn("col3", TajoDataTypes.Type.INT2);
+    schema.addColumn("col4", TajoDataTypes.Type.INT4);
+    schema.addColumn("col5", TajoDataTypes.Type.INT8);
+    schema.addColumn("col6", TajoDataTypes.Type.FLOAT4);
+    schema.addColumn("col7", TajoDataTypes.Type.FLOAT8);
+    schema.addColumn("col8", TajoDataTypes.Type.TEXT);
+    schema.addColumn("col9", TajoDataTypes.Type.BLOB);
+    schema.addColumn("col10", TajoDataTypes.Type.INET4);
+    schema.addColumn("col11", TajoDataTypes.Type.NULL_TYPE);
+  }
+
+  public static Path getResourcePath(String path, String suffix) {
+    URL resultBaseURL = ClassLoader.getSystemResource(path);
+    return new Path(resultBaseURL.toString(), suffix);
+  }
+
+  @Test
+  public void testVarioutType() throws IOException {
+    TajoConf conf = new TajoConf();
+
+    TableMeta meta = CatalogUtil.newTableMeta(CatalogProtos.StoreType.JSON);
+    Path tablePath = new Path(getResourcePath("dataset", "TestJsonSerDe"), "testVariousType.json");
+    FileSystem fs = FileSystem.getLocal(conf);
+    FileStatus status = fs.getFileStatus(tablePath);
+    FileFragment fragment = new FileFragment("table", tablePath, 0, status.getLen());
+    Scanner scanner =  StorageManager.getStorageManager(conf).getScanner(meta, schema, fragment);
+    scanner.init();
+
+    Tuple tuple = scanner.next();
+    assertNotNull(tuple);
+    assertNull(scanner.next());
+    scanner.close();
+
+    Tuple baseTuple = new VTuple(11);
+    baseTuple.put(new Datum[] {
+        DatumFactory.createBool(true),                  // 0
+        DatumFactory.createChar("hyunsik"),             // 1
+        DatumFactory.createInt2((short) 17),            // 2
+        DatumFactory.createInt4(59),                    // 3
+        DatumFactory.createInt8(23l),                   // 4
+        DatumFactory.createFloat4(77.9f),               // 5
+        DatumFactory.createFloat8(271.9d),              // 6
+        DatumFactory.createText("hyunsik"),             // 7
+        DatumFactory.createBlob("hyunsik".getBytes()),  // 8
+        DatumFactory.createInet4("192.168.0.1"),        // 9
+        NullDatum.get(),                                // 10
+    });
+
+    assertEquals(baseTuple, tuple);
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/cd38dffb/tajo-storage/src/test/resources/dataset/TestJsonSerDe/testVariousType.json
----------------------------------------------------------------------
diff --git a/tajo-storage/src/test/resources/dataset/TestJsonSerDe/testVariousType.json b/tajo-storage/src/test/resources/dataset/TestJsonSerDe/testVariousType.json
new file mode 100644
index 0000000..8ee3408
--- /dev/null
+++ b/tajo-storage/src/test/resources/dataset/TestJsonSerDe/testVariousType.json
@@ -0,0 +1 @@
+{"col1": "true", "col2": "hyunsik", "col3": 17, "col4": 59, "col5": 23, "col6": 77.9, "col7": 271.9, "col8": "hyunsik", "col9": "hyunsik", "col10": "192.168.0.1"}

http://git-wip-us.apache.org/repos/asf/tajo/blob/cd38dffb/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
index 790d5a8..f4c81c7 100644
--- a/tajo-storage/src/test/resources/storage-default.xml
+++ b/tajo-storage/src/test/resources/storage-default.xml
@@ -28,7 +28,7 @@
   <!--- Registered Scanner Handler -->
   <property>
     <name>tajo.storage.scanner-handler</name>
-    <value>textfile,csv,raw,rcfile,row,parquet,sequencefile,avro</value>
+    <value>textfile,csv,json,raw,rcfile,row,parquet,sequencefile,avro</value>
   </property>
 
   <!--- Fragment Class Configurations -->
@@ -41,6 +41,10 @@
     <value>org.apache.tajo.storage.fragment.FileFragment</value>
   </property>
   <property>
+    <name>tajo.storage.fragment.json.class</name>
+    <value>org.apache.tajo.storage.fragment.FileFragment</value>
+  </property>
+  <property>
     <name>tajo.storage.fragment.raw.class</name>
     <value>org.apache.tajo.storage.fragment.FileFragment</value>
   </property>
@@ -77,6 +81,11 @@
   </property>
 
   <property>
+    <name>tajo.storage.scanner-handler.json.class</name>
+    <value>org.apache.tajo.storage.text.DelimitedTextFile$DelimitedTextFileScanner</value>
+  </property>
+
+  <property>
     <name>tajo.storage.scanner-handler.raw.class</name>
     <value>org.apache.tajo.storage.RawFile$RawFileScanner</value>
   </property>
@@ -123,6 +132,11 @@
   </property>
 
   <property>
+    <name>tajo.storage.appender-handler.json.class</name>
+    <value>org.apache.tajo.storage.text.DelimitedTextFile$DelimitedTextFileAppender</value>
+  </property>
+
+  <property>
     <name>tajo.storage.appender-handler.raw.class</name>
     <value>org.apache.tajo.storage.RawFile$RawFileAppender</value>
   </property>

http://git-wip-us.apache.org/repos/asf/tajo/blob/cd38dffb/tajo-storage/src/test/resources/testVariousTypes.avsc
----------------------------------------------------------------------
diff --git a/tajo-storage/src/test/resources/testVariousTypes.avsc b/tajo-storage/src/test/resources/testVariousTypes.avsc
index 611b97f..d4250a9 100644
--- a/tajo-storage/src/test/resources/testVariousTypes.avsc
+++ b/tajo-storage/src/test/resources/testVariousTypes.avsc
@@ -4,18 +4,17 @@
   "name": "testVariousTypes",
   "fields": [
     { "name": "col1", "type": "boolean" },
-    { "name": "col2", "type": "int" },
-    { "name": "col3", "type": "string" },
+    { "name": "col2", "type": "string" },
+    { "name": "col3", "type": "int" },
     { "name": "col4", "type": "int" },
-    { "name": "col5", "type": "int" },
-    { "name": "col6", "type": "long" },
-    { "name": "col7", "type": "float" },
-    { "name": "col8", "type": "double" },
-    { "name": "col9", "type": "string" },
+    { "name": "col5", "type": "long" },
+    { "name": "col6", "type": "float" },
+    { "name": "col7", "type": "double" },
+    { "name": "col8", "type": "string" },
+    { "name": "col9", "type": "bytes" },
     { "name": "col10", "type": "bytes" },
-    { "name": "col11", "type": "bytes" },
-    { "name": "col12", "type": "null" },
-    { "name": "col13", "type": "bytes" }
+    { "name": "col11", "type": "null" },
+    { "name": "col12", "type": "bytes" }
   ]
 }
 


[12/29] tajo git commit: TAJO-1190: INSERT INTO to partition tables may cause NPE.

Posted by hj...@apache.org.
TAJO-1190: INSERT INTO to partition tables may cause NPE.

Closes #250


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

Branch: refs/heads/hbase_storage
Commit: 1cdbe467e3dc25d7af59afc116ff9e8e6273a1ac
Parents: b4adc18
Author: Hyunsik Choi <hy...@apache.org>
Authored: Wed Dec 3 02:25:34 2014 +0900
Committer: Hyunsik Choi <hy...@apache.org>
Committed: Wed Dec 3 02:25:34 2014 +0900

----------------------------------------------------------------------
 CHANGES                                         |  8 ++-
 .../planner/physical/PhysicalPlanUtil.java      | 65 +++++++++++++++-----
 .../engine/planner/physical/SeqScanExec.java    |  7 ++-
 .../tajo/worker/TajoWorkerClientService.java    | 16 ++---
 .../tajo/engine/query/TestTablePartitions.java  | 56 +++++++++++++++++
 ...rtitionedTableWithSmallerExpressions5.result |  7 +++
 ...rtitionedTableWithSmallerExpressions6.result |  4 ++
 .../apache/tajo/rpc/RemoteCallException.java    |  6 +-
 8 files changed, 144 insertions(+), 25 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tajo/blob/1cdbe467/CHANGES
----------------------------------------------------------------------
diff --git a/CHANGES b/CHANGES
index 8d51d44..188e024 100644
--- a/CHANGES
+++ b/CHANGES
@@ -15,7 +15,8 @@ Release 0.9.1 - unreleased
 
   IMPROVEMENT
 
-    TAJO-1165: Needs to show error messages on query_executor.jsp. (Jihun Kang via jaehwa)
+    TAJO-1165: Needs to show error messages on query_executor.jsp. 
+    (Jihun Kang via jaehwa)
 
     TAJO-1204: Remove unused ServerName class. (DaeMyung Kang via jaehwa)
 
@@ -79,6 +80,8 @@ Release 0.9.1 - unreleased
 
   BUG FIXES
 
+    TAJO-1190: INSERT INTO to partition tables may cause NPE. (hyunsik)
+
     TAJO-1211: Staging directory for CTAS and INSERT should be in 
     the output dir. (hyunsik)
 
@@ -87,7 +90,8 @@ Release 0.9.1 - unreleased
 
     TAJO-1119: JDBC driver should support TIMESTAMP type. (jaehwa)
 
-    TAJO-1166: S3 related storage causes compilation error in Hadoop 2.6.0-SNAPSHOT. (jaehwa)
+    TAJO-1166: S3 related storage causes compilation error in Hadoop 
+    2.6.0-SNAPSHOT. (jaehwa)
 
     TAJO-1208: Failure of create table using textfile on hivemeta.
     (jinho)

http://git-wip-us.apache.org/repos/asf/tajo/blob/1cdbe467/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/PhysicalPlanUtil.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/PhysicalPlanUtil.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/PhysicalPlanUtil.java
index fe1f795..a63b838 100644
--- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/PhysicalPlanUtil.java
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/PhysicalPlanUtil.java
@@ -77,31 +77,33 @@ public class PhysicalPlanUtil {
     Path path = new Path(tableDesc.getPath());
     FileSystem fs = path.getFileSystem(tajoConf);
 
+    //In the case of partitioned table, we should return same partition key data files.
+    int partitionDepth = 0;
+    if (tableDesc.hasPartition()) {
+      partitionDepth = tableDesc.getPartitionMethod().getExpressionSchema().getColumns().size();
+    }
+
     List<FileStatus> nonZeroLengthFiles = new ArrayList<FileStatus>();
     if (fs.exists(path)) {
       getNonZeroLengthDataFiles(fs, path, nonZeroLengthFiles, fileIndex, numResultFiles,
-          new AtomicInteger(0));
+          new AtomicInteger(0), tableDesc.hasPartition(), 0, partitionDepth);
     }
 
     List<FileFragment> fragments = new ArrayList<FileFragment>();
 
-    //In the case of partitioned table, return same partition key data files.
-    int numPartitionColumns = 0;
-    if (tableDesc.hasPartition()) {
-      numPartitionColumns = tableDesc.getPartitionMethod().getExpressionSchema().getColumns().size();
-    }
+
     String[] previousPartitionPathNames = null;
     for (FileStatus eachFile: nonZeroLengthFiles) {
       FileFragment fileFragment = new FileFragment(tableDesc.getName(), eachFile.getPath(), 0, eachFile.getLen(), null);
 
-      if (numPartitionColumns > 0) {
+      if (partitionDepth > 0) {
         // finding partition key;
         Path filePath = fileFragment.getPath();
         Path parentPath = filePath;
-        String[] parentPathNames = new String[numPartitionColumns];
-        for (int i = 0; i < numPartitionColumns; i++) {
+        String[] parentPathNames = new String[partitionDepth];
+        for (int i = 0; i < partitionDepth; i++) {
           parentPath = parentPath.getParent();
-          parentPathNames[numPartitionColumns - i - 1] = parentPath.getName();
+          parentPathNames[partitionDepth - i - 1] = parentPath.getName();
         }
 
         // If current partitionKey == previousPartitionKey, add to result.
@@ -120,20 +122,53 @@ public class PhysicalPlanUtil {
     return FragmentConvertor.toFragmentProtoArray(fragments.toArray(new FileFragment[]{}));
   }
 
+  /**
+   *
+   * @param fs
+   * @param path The table path
+   * @param result The final result files to be used
+   * @param startFileIndex
+   * @param numResultFiles
+   * @param currentFileIndex
+   * @param partitioned A flag to indicate if this table is partitioned
+   * @param currentDepth Current visiting depth of partition directories
+   * @param maxDepth The partition depth of this table
+   * @throws IOException
+   */
   private static void getNonZeroLengthDataFiles(FileSystem fs, Path path, List<FileStatus> result,
                                          int startFileIndex, int numResultFiles,
-                                         AtomicInteger currentFileIndex) throws IOException {
+                                         AtomicInteger currentFileIndex, boolean partitioned,
+                                         int currentDepth, int maxDepth) throws IOException {
+    // Intermediate directory
     if (fs.isDirectory(path)) {
+
       FileStatus[] files = fs.listStatus(path, StorageManager.hiddenFileFilter);
+
       if (files != null && files.length > 0) {
+
         for (FileStatus eachFile : files) {
+
+          // checking if the enough number of files are found
           if (result.size() >= numResultFiles) {
             return;
           }
+
           if (eachFile.isDirectory()) {
-            getNonZeroLengthDataFiles(fs, eachFile.getPath(), result, startFileIndex, numResultFiles,
-                currentFileIndex);
-          } else if (eachFile.isFile() && eachFile.getLen() > 0) {
+            getNonZeroLengthDataFiles(
+                fs,
+                eachFile.getPath(),
+                result,
+                startFileIndex,
+                numResultFiles,
+                currentFileIndex,
+                partitioned,
+                currentDepth + 1, // increment a visiting depth
+                maxDepth);
+
+
+            // if partitioned table, we should ignore files located in the intermediate directory.
+            // we can ensure that this file is in leaf directory if currentDepth == maxDepth.
+          } else if (eachFile.isFile() && eachFile.getLen() > 0 && (!partitioned || currentDepth == maxDepth)) {
             if (currentFileIndex.get() >= startFileIndex) {
               result.add(eachFile);
             }
@@ -141,6 +176,8 @@ public class PhysicalPlanUtil {
           }
         }
       }
+
+      // Files located in leaf directory
     } else {
       FileStatus fileStatus = fs.getFileStatus(path);
       if (fileStatus != null && fileStatus.getLen() > 0) {

http://git-wip-us.apache.org/repos/asf/tajo/blob/1cdbe467/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/SeqScanExec.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/SeqScanExec.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/SeqScanExec.java
index 3cbb7c9..759b19c 100644
--- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/SeqScanExec.java
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/SeqScanExec.java
@@ -332,7 +332,12 @@ public class SeqScanExec extends PhysicalExec {
     if (scanner != null) {
       return scanner.getInputStats();
     } else {
-      return inputStats;
+      if (inputStats != null) {
+        return inputStats;
+      } else {
+        // If no fragment, there is no scanner. So, we need to create a dummy table stat.
+        return new TableStats();
+      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/1cdbe467/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorkerClientService.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorkerClientService.java b/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorkerClientService.java
index a41ffce..0f4a60c 100644
--- a/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorkerClientService.java
+++ b/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorkerClientService.java
@@ -32,6 +32,7 @@ import org.apache.tajo.QueryIdFactory;
 import org.apache.tajo.TajoIdProtos;
 import org.apache.tajo.TajoProtos;
 import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.engine.query.QueryContext;
 import org.apache.tajo.ipc.ClientProtos;
 import org.apache.tajo.ipc.ClientProtos.GetQueryHistoryResponse;
 import org.apache.tajo.ipc.ClientProtos.QueryIdRequest;
@@ -39,8 +40,6 @@ import org.apache.tajo.ipc.ClientProtos.ResultCode;
 import org.apache.tajo.ipc.QueryMasterClientProtocol;
 import org.apache.tajo.ipc.TajoWorkerProtocol;
 import org.apache.tajo.master.querymaster.Query;
-import org.apache.tajo.master.querymaster.QueryInProgress;
-import org.apache.tajo.master.querymaster.QueryJobManager;
 import org.apache.tajo.master.querymaster.QueryMasterTask;
 import org.apache.tajo.rpc.BlockingRpcServer;
 import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos;
@@ -132,6 +131,10 @@ public class TajoWorkerClientService extends AbstractService {
       return null;
     }
 
+    private boolean hasResultTableDesc(QueryContext queryContext) {
+      return !(queryContext.isCreateTable() || queryContext.isInsert());
+    }
+
     @Override
     public ClientProtos.GetQueryResultResponse getQueryResult(
             RpcController controller,
@@ -151,7 +154,9 @@ public class TajoWorkerClientService extends AbstractService {
       } else {
         switch (queryMasterTask.getState()) {
           case QUERY_SUCCEEDED:
-            builder.setTableDesc(queryMasterTask.getQuery().getResultDesc().getProto());
+//            if (hasResultTableDesc(queryMasterTask.getQueryTaskContext().getQueryContext())) {
+              builder.setTableDesc(queryMasterTask.getQuery().getResultDesc().getProto());
+            //}
             break;
           case QUERY_FAILED:
           case QUERY_ERROR:
@@ -191,10 +196,7 @@ public class TajoWorkerClientService extends AbstractService {
           return builder.build();
         }
 
-        builder.setHasResult(
-            !(queryMasterTask.getQueryTaskContext().getQueryContext().isCreateTable() ||
-                queryMasterTask.getQueryTaskContext().getQueryContext().isInsert())
-        );
+        builder.setHasResult(hasResultTableDesc(queryMasterTask.getQueryTaskContext().getQueryContext()));
 
         queryMasterTask.touchSessionTime();
         Query query = queryMasterTask.getQuery();

http://git-wip-us.apache.org/repos/asf/tajo/blob/1cdbe467/tajo-core/src/test/java/org/apache/tajo/engine/query/TestTablePartitions.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestTablePartitions.java b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestTablePartitions.java
index 0e9ec7d..cff5bfb 100644
--- a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestTablePartitions.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestTablePartitions.java
@@ -19,6 +19,7 @@
 package org.apache.tajo.engine.query;
 
 import com.google.common.collect.Maps;
+import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -43,6 +44,7 @@ import org.apache.tajo.jdbc.TajoResultSet;
 import org.apache.tajo.master.querymaster.QueryMasterTask;
 import org.apache.tajo.plan.logical.NodeType;
 import org.apache.tajo.storage.StorageConstants;
+import org.apache.tajo.util.CommonTestingUtil;
 import org.apache.tajo.util.KeyValueSet;
 import org.apache.tajo.worker.TajoWorker;
 import org.junit.Test;
@@ -820,6 +822,39 @@ public class TestTablePartitions extends QueryTestCaseBase {
     }
   }
 
+  @Test
+  public final void testColumnPartitionedTableWithSmallerExpressions5() throws Exception {
+    String tableName = CatalogUtil.normalizeIdentifier("testColumnPartitionedTableWithSmallerExpressions5");
+    ResultSet res = executeString(
+        "create table " + tableName + " (col1 text) partition by column(col2 text) ");
+    res.close();
+
+    assertTrue(catalog.existsTable(DEFAULT_DATABASE_NAME, tableName));
+
+    res = executeString("insert overwrite into " + tableName + "(col1) select l_returnflag from lineitem");
+    res.close();
+    res = executeString("select * from " + tableName);
+    assertResultSet(res);
+    res.close();
+  }
+
+  @Test
+  public final void testColumnPartitionedTableWithSmallerExpressions6() throws Exception {
+    String tableName = CatalogUtil.normalizeIdentifier("testColumnPartitionedTableWithSmallerExpressions6");
+    ResultSet res = executeString(
+        "create table " + tableName + " (col1 text) partition by column(col2 text) ");
+    res.close();
+
+    assertTrue(catalog.existsTable(DEFAULT_DATABASE_NAME, tableName));
+
+    res = executeString(
+        "insert overwrite into " + tableName + "(col1) select l_returnflag from lineitem where l_orderkey = 1");
+    res.close();
+    res = executeString("select * from " + tableName);
+    assertResultSet(res);
+    res.close();
+  }
+
   private MasterPlan getQueryPlan(ResultSet res) {
     QueryId queryId = ((TajoResultSet)res).getQueryId();
     for (TajoWorker eachWorker: testingCluster.getTajoWorkers()) {
@@ -936,4 +971,25 @@ public class TestTablePartitions extends QueryTestCaseBase {
     assertResultSet(res);
     cleanupQuery(res);
   }
+
+  @Test
+  public final void testIgnoreFilesInIntermediateDir() throws Exception {
+    // See - TAJO-1219: Files located in intermediate directories of partitioned table should be ignored
+    // It verifies that Tajo ignores files located in intermediate directories of partitioned table.
+
+    Path testDir = CommonTestingUtil.getTestDir();
+
+    executeString(
+        "CREATE EXTERNAL TABLE testIgnoreFilesInIntermediateDir (col1 int) USING CSV PARTITION BY COLUMN (col2 text) " +
+        "LOCATION '" + testDir + "'");
+
+    FileSystem fs = testDir.getFileSystem(conf);
+    FSDataOutputStream fos = fs.create(new Path(testDir, "table1.data"));
+    fos.write("a|b|c".getBytes());
+    fos.close();
+
+    ResultSet res = executeString("select * from testIgnoreFilesInIntermediateDir;");
+    assertFalse(res.next());
+    res.close();
+  }
 }

http://git-wip-us.apache.org/repos/asf/tajo/blob/1cdbe467/tajo-core/src/test/resources/results/TestTablePartitions/testColumnPartitionedTableWithSmallerExpressions5.result
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/resources/results/TestTablePartitions/testColumnPartitionedTableWithSmallerExpressions5.result b/tajo-core/src/test/resources/results/TestTablePartitions/testColumnPartitionedTableWithSmallerExpressions5.result
new file mode 100644
index 0000000..f972753
--- /dev/null
+++ b/tajo-core/src/test/resources/results/TestTablePartitions/testColumnPartitionedTableWithSmallerExpressions5.result
@@ -0,0 +1,7 @@
+col1,col2
+-------------------------------
+N,__TAJO_DEFAULT_PARTITION__
+N,__TAJO_DEFAULT_PARTITION__
+N,__TAJO_DEFAULT_PARTITION__
+R,__TAJO_DEFAULT_PARTITION__
+R,__TAJO_DEFAULT_PARTITION__
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/1cdbe467/tajo-core/src/test/resources/results/TestTablePartitions/testColumnPartitionedTableWithSmallerExpressions6.result
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/resources/results/TestTablePartitions/testColumnPartitionedTableWithSmallerExpressions6.result b/tajo-core/src/test/resources/results/TestTablePartitions/testColumnPartitionedTableWithSmallerExpressions6.result
new file mode 100644
index 0000000..6b8e2f1
--- /dev/null
+++ b/tajo-core/src/test/resources/results/TestTablePartitions/testColumnPartitionedTableWithSmallerExpressions6.result
@@ -0,0 +1,4 @@
+col1,col2
+-------------------------------
+N,__TAJO_DEFAULT_PARTITION__
+N,__TAJO_DEFAULT_PARTITION__
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/1cdbe467/tajo-rpc/src/main/java/org/apache/tajo/rpc/RemoteCallException.java
----------------------------------------------------------------------
diff --git a/tajo-rpc/src/main/java/org/apache/tajo/rpc/RemoteCallException.java b/tajo-rpc/src/main/java/org/apache/tajo/rpc/RemoteCallException.java
index 90ee58a..52ef31a 100644
--- a/tajo-rpc/src/main/java/org/apache/tajo/rpc/RemoteCallException.java
+++ b/tajo-rpc/src/main/java/org/apache/tajo/rpc/RemoteCallException.java
@@ -49,7 +49,11 @@ public class RemoteCallException extends RemoteException {
   public RpcResponse getResponse() {
     RpcResponse.Builder builder = RpcResponse.newBuilder();
     builder.setId(seqId);
-    builder.setErrorMessage(getCause().getMessage());
+    if (getCause().getMessage() == null) {
+      builder.setErrorMessage(getCause().getClass().getName());
+    } else {
+      builder.setErrorMessage(getCause().getMessage());
+    }
     builder.setErrorTrace(getStackTraceString(getCause()));
     builder.setErrorClass(originExceptionClass);
 


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

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

http://git-wip-us.apache.org/repos/asf/tajo/blob/940546a0/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/TextLineDeserializer.java
----------------------------------------------------------------------
diff --cc tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/TextLineDeserializer.java
index 0000000,0000000..7ebfa79
new file mode 100644
--- /dev/null
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/TextLineDeserializer.java
@@@ -1,0 -1,0 +1,60 @@@
++/**
++ * Licensed to the Apache Software Foundation (ASF) under one
++ * or more contributor license agreements.  See the NOTICE file
++ * distributed with this work for additional information
++ * regarding copyright ownership.  The ASF licenses this file
++ * to you under the Apache License, Version 2.0 (the
++ * "License"); you may not use this file except in compliance
++ * with the License.  You may obtain a copy of the License at
++ *
++ *     http://www.apache.org/licenses/LICENSE-2.0
++ *
++ * Unless required by applicable law or agreed to in writing, software
++ * distributed under the License is distributed on an "AS IS" BASIS,
++ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
++ * See the License for the specific language governing permissions and
++ * limitations under the License.
++ */
++
++package org.apache.tajo.storage.text;
++
++import io.netty.buffer.ByteBuf;
++import org.apache.tajo.catalog.Schema;
++import org.apache.tajo.catalog.TableMeta;
++import org.apache.tajo.storage.Tuple;
++
++import java.io.IOException;
++
++/**
++ * Reads a text line and fills a Tuple with values
++ */
++public abstract class TextLineDeserializer {
++  protected Schema schema;
++  protected TableMeta meta;
++  protected int [] targetColumnIndexes;
++
++  public TextLineDeserializer(Schema schema, TableMeta meta, int [] targetColumnIndexes) {
++    this.schema = schema;
++    this.meta = meta;
++    this.targetColumnIndexes = targetColumnIndexes;
++  }
++
++  /**
++   * Initialize SerDe
++   */
++  public abstract void init();
++
++  /**
++   * It fills a tuple with a read fields in a given line.
++   *
++   * @param buf Read line
++   * @param output Tuple to be filled with read fields
++   * @throws java.io.IOException
++   */
++  public abstract void deserialize(final ByteBuf buf, Tuple output) throws IOException, TextLineParsingError;
++
++  /**
++   * Release external resources
++   */
++  public abstract void release();
++}

http://git-wip-us.apache.org/repos/asf/tajo/blob/940546a0/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/TextLineParsingError.java
----------------------------------------------------------------------
diff --cc tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/TextLineParsingError.java
index 0000000,0000000..f0bae5e
new file mode 100644
--- /dev/null
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/TextLineParsingError.java
@@@ -1,0 -1,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.text;
++
++public class TextLineParsingError extends Exception {
++
++  public TextLineParsingError(Throwable t) {
++    super(t);
++  }
++
++  public TextLineParsingError(String message, Throwable t) {
++    super(t.getMessage() + ", Error line: " + message);
++  }
++
++}

http://git-wip-us.apache.org/repos/asf/tajo/blob/940546a0/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/TextLineSerDe.java
----------------------------------------------------------------------
diff --cc tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/TextLineSerDe.java
index 0000000,0000000..e81e289
new file mode 100644
--- /dev/null
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/TextLineSerDe.java
@@@ -1,0 -1,0 +1,65 @@@
++/**
++ * Licensed to the Apache Software Foundation (ASF) under one
++ * or more contributor license agreements.  See the NOTICE file
++ * distributed with this work for additional information
++ * regarding copyright ownership.  The ASF licenses this file
++ * to you under the Apache License, Version 2.0 (the
++ * "License"); you may not use this file except in compliance
++ * with the License.  You may obtain a copy of the License at
++ *
++ *     http://www.apache.org/licenses/LICENSE-2.0
++ *
++ * Unless required by applicable law or agreed to in writing, software
++ * distributed under the License is distributed on an "AS IS" BASIS,
++ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
++ * See the License for the specific language governing permissions and
++ * limitations under the License.
++ */
++
++package org.apache.tajo.storage.text;
++
++import io.netty.buffer.ByteBuf;
++import org.apache.commons.lang.StringEscapeUtils;
++import org.apache.commons.lang.StringUtils;
++import org.apache.tajo.catalog.Schema;
++import org.apache.tajo.catalog.TableMeta;
++import org.apache.tajo.datum.NullDatum;
++import org.apache.tajo.storage.BufferPool;
++import org.apache.tajo.storage.StorageConstants;
++
++/**
++ * Pluggable Text Line SerDe class
++ */
++public abstract class TextLineSerDe {
++
++  public TextLineSerDe() {
++  }
++
++  public abstract TextLineDeserializer createDeserializer(Schema schema, TableMeta meta, int [] targetColumnIndexes);
++
++  public abstract TextLineSerializer createSerializer(Schema schema, TableMeta meta);
++
++  public static ByteBuf getNullChars(TableMeta meta) {
++    byte[] nullCharByteArray = getNullCharsAsBytes(meta);
++
++    ByteBuf nullChars = BufferPool.directBuffer(nullCharByteArray.length, nullCharByteArray.length);
++    nullChars.writeBytes(nullCharByteArray);
++
++    return nullChars;
++  }
++
++  public static byte [] getNullCharsAsBytes(TableMeta meta) {
++    byte [] nullChars;
++
++    String nullCharacters = StringEscapeUtils.unescapeJava(meta.getOption(StorageConstants.TEXT_NULL,
++        NullDatum.DEFAULT_TEXT));
++    if (StringUtils.isEmpty(nullCharacters)) {
++      nullChars = NullDatum.get().asTextBytes();
++    } else {
++      nullChars = nullCharacters.getBytes();
++    }
++
++    return nullChars;
++  }
++
++}

http://git-wip-us.apache.org/repos/asf/tajo/blob/940546a0/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/TextLineSerializer.java
----------------------------------------------------------------------
diff --cc tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/TextLineSerializer.java
index 0000000,0000000..0c2761f
new file mode 100644
--- /dev/null
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/TextLineSerializer.java
@@@ -1,0 -1,0 +1,45 @@@
++/**
++ * Licensed to the Apache Software Foundation (ASF) under one
++ * or more contributor license agreements.  See the NOTICE file
++ * distributed with this work for additional information
++ * regarding copyright ownership.  The ASF licenses this file
++ * to you under the Apache License, Version 2.0 (the
++ * "License"); you may not use this file except in compliance
++ * with the License.  You may obtain a copy of the License at
++ *
++ *     http://www.apache.org/licenses/LICENSE-2.0
++ *
++ * Unless required by applicable law or agreed to in writing, software
++ * distributed under the License is distributed on an "AS IS" BASIS,
++ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
++ * See the License for the specific language governing permissions and
++ * limitations under the License.
++ */
++
++package org.apache.tajo.storage.text;
++
++import org.apache.tajo.catalog.Schema;
++import org.apache.tajo.catalog.TableMeta;
++import org.apache.tajo.storage.Tuple;
++
++import java.io.IOException;
++import java.io.OutputStream;
++
++/**
++ * Write a Tuple into single text formatted line
++ */
++public abstract class TextLineSerializer {
++  protected Schema schema;
++  protected TableMeta meta;
++
++  public TextLineSerializer(Schema schema, TableMeta meta) {
++    this.schema = schema;
++    this.meta = meta;
++  }
++
++  public abstract void init();
++
++  public abstract int serialize(OutputStream out, Tuple input) throws IOException;
++
++  public abstract void release();
++}

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

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

http://git-wip-us.apache.org/repos/asf/tajo/blob/940546a0/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/avro/TestAvroUtil.java
----------------------------------------------------------------------
diff --cc tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/avro/TestAvroUtil.java
index 4f7ea1c,0000000..7b83894
mode 100644,000000..100644
--- a/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/avro/TestAvroUtil.java
+++ b/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/avro/TestAvroUtil.java
@@@ -1,106 -1,0 +1,106 @@@
 +/**
 + * Licensed to the Apache Software Foundation (ASF) under one
 + * or more contributor license agreements.  See the NOTICE file
 + * distributed with this work for additional information
 + * regarding copyright ownership.  The ASF licenses this file
 + * to you under the Apache License, Version 2.0 (the
 + * "License"); you may not use this file except in compliance
 + * with the License.  You may obtain a copy of the License at
 + *
 + *     http://www.apache.org/licenses/LICENSE-2.0
 + *
 + * Unless required by applicable law or agreed to in writing, software
 + * distributed under the License is distributed on an "AS IS" BASIS,
 + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 + * See the License for the specific language governing permissions and
 + * limitations under the License.
 + */
 +
 +package org.apache.tajo.storage.avro;
 +
 +import org.apache.avro.Schema;
 +import org.apache.tajo.HttpFileServer;
 +import org.apache.tajo.catalog.CatalogUtil;
 +import org.apache.tajo.catalog.TableMeta;
 +import org.apache.tajo.catalog.proto.CatalogProtos;
 +import org.apache.tajo.conf.TajoConf;
 +import org.apache.tajo.storage.StorageConstants;
 +import org.apache.tajo.util.FileUtil;
 +import org.apache.tajo.util.NetUtils;
 +import org.junit.Before;
 +import org.junit.Test;
 +
 +import java.io.File;
 +import java.io.IOException;
 +import java.net.InetSocketAddress;
 +import java.net.URISyntaxException;
 +import java.net.URL;
 +
 +import static org.junit.Assert.*;
 +
 +/**
 + * Tests for {@link org.apache.tajo.storage.avro.AvroUtil}.
 + */
 +public class TestAvroUtil {
 +  private Schema expected;
 +  private URL schemaUrl;
 +
 +  @Before
 +  public void setUp() throws Exception {
-     schemaUrl = FileUtil.getResourcePath("testVariousTypes.avsc");
++    schemaUrl = FileUtil.getResourcePath("dataset/testVariousTypes.avsc");
 +    assertNotNull(schemaUrl);
 +
 +    File file = new File(schemaUrl.getPath());
 +    assertTrue(file.exists());
 +
 +    expected = new Schema.Parser().parse(file);
 +  }
 +
 +  @Test
 +  public void testGetSchema() throws IOException, URISyntaxException {
 +    TableMeta meta = CatalogUtil.newTableMeta(CatalogProtos.StoreType.AVRO);
 +    meta.putOption(StorageConstants.AVRO_SCHEMA_LITERAL, FileUtil.readTextFile(new File(schemaUrl.getPath())));
 +    Schema schema = AvroUtil.getAvroSchema(meta, new TajoConf());
 +    assertEquals(expected, schema);
 +
 +    meta = CatalogUtil.newTableMeta(CatalogProtos.StoreType.AVRO);
 +    meta.putOption(StorageConstants.AVRO_SCHEMA_URL, schemaUrl.getPath());
 +    schema = AvroUtil.getAvroSchema(meta, new TajoConf());
 +    assertEquals(expected, schema);
 +
 +    HttpFileServer server = new HttpFileServer(NetUtils.createSocketAddr("127.0.0.1:0"));
 +    try {
 +      server.start();
 +      InetSocketAddress addr = server.getBindAddress();
 +
 +      String url = "http://127.0.0.1:" + addr.getPort() + schemaUrl.getPath();
 +      meta = CatalogUtil.newTableMeta(CatalogProtos.StoreType.AVRO);
 +      meta.putOption(StorageConstants.AVRO_SCHEMA_URL, url);
 +      schema = AvroUtil.getAvroSchema(meta, new TajoConf());
 +    } finally {
 +      server.stop();
 +    }
 +    assertEquals(expected, schema);
 +  }
 +
 +  @Test
 +  public void testGetSchemaFromHttp() throws IOException, URISyntaxException {
 +    HttpFileServer server = new HttpFileServer(NetUtils.createSocketAddr("127.0.0.1:0"));
 +    try {
 +      server.start();
 +      InetSocketAddress addr = server.getBindAddress();
 +
 +      Schema schema = AvroUtil.getAvroSchemaFromHttp("http://127.0.0.1:" + addr.getPort() + schemaUrl.getPath());
 +      assertEquals(expected, schema);
 +    } finally {
 +      server.stop();
 +    }
 +  }
 +
 +  @Test
 +  public void testGetSchemaFromFileSystem() throws IOException, URISyntaxException {
 +    Schema schema = AvroUtil.getAvroSchemaFromFileSystem(schemaUrl.toString(), new TajoConf());
 +
 +    assertEquals(expected, schema);
 +  }
 +}

http://git-wip-us.apache.org/repos/asf/tajo/blob/940546a0/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/json/TestLineReader.java
----------------------------------------------------------------------
diff --cc tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/json/TestLineReader.java
index 0000000,0000000..bf7516f
new file mode 100644
--- /dev/null
+++ b/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/json/TestLineReader.java
@@@ -1,0 -1,0 +1,197 @@@
++/**
++ * Licensed to the Apache Software Foundation (ASF) under one
++ * or more contributor license agreements.  See the NOTICE file
++ * distributed with this work for additional information
++ * regarding copyright ownership.  The ASF licenses this file
++ * to you under the Apache License, Version 2.0 (the
++ * "License"); you may not use this file except in compliance
++ * with the License.  You may obtain a copy of the License at
++ *
++ *     http://www.apache.org/licenses/LICENSE-2.0
++ *
++ * Unless required by applicable law or agreed to in writing, software
++ * distributed under the License is distributed on an "AS IS" BASIS,
++ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
++ * See the License for the specific language governing permissions and
++ * limitations under the License.
++ */
++
++package org.apache.tajo.storage.json;
++
++import io.netty.buffer.ByteBuf;
++import org.apache.hadoop.fs.FileStatus;
++import org.apache.hadoop.fs.FileSystem;
++import org.apache.hadoop.fs.Path;
++import org.apache.hadoop.io.IOUtils;
++import org.apache.hadoop.io.compress.DeflateCodec;
++import org.apache.tajo.catalog.CatalogUtil;
++import org.apache.tajo.catalog.Schema;
++import org.apache.tajo.catalog.TableMeta;
++import org.apache.tajo.catalog.proto.CatalogProtos.StoreType;
++import org.apache.tajo.common.TajoDataTypes.Type;
++import org.apache.tajo.conf.TajoConf;
++import org.apache.tajo.datum.DatumFactory;
++import org.apache.tajo.datum.NullDatum;
++import org.apache.tajo.storage.ByteBufInputChannel;
++import org.apache.tajo.storage.FileAppender;
++import org.apache.tajo.storage.StorageManager;
++import org.apache.tajo.storage.VTuple;
++import org.apache.tajo.storage.fragment.FileFragment;
++import org.apache.tajo.storage.text.ByteBufLineReader;
++import org.apache.tajo.storage.text.DelimitedLineReader;
++import org.apache.tajo.storage.text.DelimitedTextFile;
++import org.apache.tajo.util.CommonTestingUtil;
++import org.apache.tajo.util.FileUtil;
++import org.junit.Test;
++
++import java.io.File;
++import java.io.FileInputStream;
++import java.io.IOException;
++import java.util.concurrent.atomic.AtomicInteger;
++
++import static org.junit.Assert.*;
++
++public class TestLineReader {
++	private static String TEST_PATH = "target/test-data/TestLineReader";
++
++  @Test
++  public void testByteBufLineReader() throws IOException {
++    TajoConf conf = new TajoConf();
++    Path testDir = CommonTestingUtil.getTestDir(TEST_PATH);
++    FileSystem fs = testDir.getFileSystem(conf);
++
++    Schema schema = new Schema();
++    schema.addColumn("id", Type.INT4);
++    schema.addColumn("age", Type.INT8);
++    schema.addColumn("comment", Type.TEXT);
++    schema.addColumn("comment2", Type.TEXT);
++
++    TableMeta meta = CatalogUtil.newTableMeta(StoreType.TEXTFILE);
++    Path tablePath = new Path(testDir, "line.data");
++    FileAppender appender = (FileAppender) StorageManager.getFileStorageManager(conf).getAppender(
++        null, null, meta, schema, tablePath);
++    appender.enableStats();
++    appender.init();
++    int tupleNum = 10000;
++    VTuple vTuple;
++
++    for (int i = 0; i < tupleNum; i++) {
++      vTuple = new VTuple(4);
++      vTuple.put(0, DatumFactory.createInt4(i + 1));
++      vTuple.put(1, DatumFactory.createInt8(25l));
++      vTuple.put(2, DatumFactory.createText("emiya muljomdao"));
++      vTuple.put(3, NullDatum.get());
++      appender.addTuple(vTuple);
++    }
++    appender.close();
++
++    FileStatus status = fs.getFileStatus(tablePath);
++
++    ByteBufInputChannel channel = new ByteBufInputChannel(fs.open(tablePath));
++    assertEquals(status.getLen(), channel.available());
++    ByteBufLineReader reader = new ByteBufLineReader(channel);
++    assertEquals(status.getLen(), reader.available());
++
++    long totalRead = 0;
++    int i = 0;
++    AtomicInteger bytes = new AtomicInteger();
++    for(;;){
++      ByteBuf buf = reader.readLineBuf(bytes);
++      if(buf == null) break;
++
++      totalRead += bytes.get();
++      i++;
++    }
++    IOUtils.cleanup(null, reader, channel, fs);
++    assertEquals(tupleNum, i);
++    assertEquals(status.getLen(), totalRead);
++    assertEquals(status.getLen(), reader.readBytes());
++  }
++
++  @Test
++  public void testLineDelimitedReader() throws IOException {
++    TajoConf conf = new TajoConf();
++    Path testDir = CommonTestingUtil.getTestDir(TEST_PATH);
++    FileSystem fs = testDir.getFileSystem(conf);
++
++    Schema schema = new Schema();
++    schema.addColumn("id", Type.INT4);
++    schema.addColumn("age", Type.INT8);
++    schema.addColumn("comment", Type.TEXT);
++    schema.addColumn("comment2", Type.TEXT);
++
++    TableMeta meta = CatalogUtil.newTableMeta(StoreType.TEXTFILE);
++    meta.putOption("compression.codec", DeflateCodec.class.getCanonicalName());
++
++    Path tablePath = new Path(testDir, "line1." + DeflateCodec.class.getSimpleName());
++    FileAppender appender = (FileAppender) StorageManager.getFileStorageManager(conf).getAppender(
++        null, null, meta, schema, tablePath);
++    appender.enableStats();
++    appender.init();
++    int tupleNum = 10000;
++    VTuple vTuple;
++
++    long splitOffset = 0;
++    for (int i = 0; i < tupleNum; i++) {
++      vTuple = new VTuple(4);
++      vTuple.put(0, DatumFactory.createInt4(i + 1));
++      vTuple.put(1, DatumFactory.createInt8(25l));
++      vTuple.put(2, DatumFactory.createText("emiya muljomdao"));
++      vTuple.put(3, NullDatum.get());
++      appender.addTuple(vTuple);
++
++      if(i == (tupleNum / 2)){
++        splitOffset = appender.getOffset();
++      }
++    }
++    String extension = ((DelimitedTextFile.DelimitedTextFileAppender) appender).getExtension();
++    appender.close();
++
++    tablePath = tablePath.suffix(extension);
++    FileFragment fragment = new FileFragment("table", tablePath, 0, splitOffset);
++    DelimitedLineReader reader = new DelimitedLineReader(conf, fragment); // if file is compressed, will read to EOF
++    assertTrue(reader.isCompressed());
++    assertFalse(reader.isReadable());
++    reader.init();
++    assertTrue(reader.isReadable());
++
++
++    int i = 0;
++    while(reader.isReadable()){
++      ByteBuf buf = reader.readLine();
++      if(buf == null) break;
++      i++;
++    }
++
++    IOUtils.cleanup(null, reader, fs);
++    assertEquals(tupleNum, i);
++
++  }
++
++  @Test
++  public void testByteBufLineReaderWithoutTerminating() throws IOException {
++    String path = FileUtil.getResourcePath("dataset/testLineText.txt").getFile();
++    File file = new File(path);
++    String data = FileUtil.readTextFile(file);
++
++    ByteBufInputChannel channel = new ByteBufInputChannel(new FileInputStream(file));
++
++    assertEquals(file.length(), channel.available());
++    ByteBufLineReader reader = new ByteBufLineReader(channel);
++    assertEquals(file.length(), reader.available());
++
++    long totalRead = 0;
++    int i = 0;
++    AtomicInteger bytes = new AtomicInteger();
++    for(;;){
++      ByteBuf buf = reader.readLineBuf(bytes);
++      if(buf == null) break;
++      totalRead += bytes.get();
++      i++;
++    }
++    IOUtils.cleanup(null, reader);
++    assertEquals(file.length(), totalRead);
++    assertEquals(file.length(), reader.readBytes());
++    assertEquals(data.split("\n").length, i);
++  }
++}

http://git-wip-us.apache.org/repos/asf/tajo/blob/940546a0/tajo-storage/tajo-storage-hdfs/src/test/resources/dataset/TestJsonSerDe/testVariousType.json
----------------------------------------------------------------------
diff --cc tajo-storage/tajo-storage-hdfs/src/test/resources/dataset/TestJsonSerDe/testVariousType.json
index 0000000,0000000..8ee3408
new file mode 100644
--- /dev/null
+++ b/tajo-storage/tajo-storage-hdfs/src/test/resources/dataset/TestJsonSerDe/testVariousType.json
@@@ -1,0 -1,0 +1,1 @@@
++{"col1": "true", "col2": "hyunsik", "col3": 17, "col4": 59, "col5": 23, "col6": 77.9, "col7": 271.9, "col8": "hyunsik", "col9": "hyunsik", "col10": "192.168.0.1"}

http://git-wip-us.apache.org/repos/asf/tajo/blob/940546a0/tajo-storage/tajo-storage-hdfs/src/test/resources/dataset/testLineText.txt
----------------------------------------------------------------------
diff --cc tajo-storage/tajo-storage-hdfs/src/test/resources/dataset/testLineText.txt
index 0000000,0000000..7403c26
new file mode 100644
--- /dev/null
+++ b/tajo-storage/tajo-storage-hdfs/src/test/resources/dataset/testLineText.txt
@@@ -1,0 -1,0 +1,2 @@@
++1|25|emiya muljomdao
++2|25|emiya muljomdao

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

http://git-wip-us.apache.org/repos/asf/tajo/blob/940546a0/tajo-storage/tajo-storage-hdfs/src/test/resources/storage-default.xml
----------------------------------------------------------------------
diff --cc tajo-storage/tajo-storage-hdfs/src/test/resources/storage-default.xml
index 6190d1a,0000000..737284b
mode 100644,000000..100644
--- a/tajo-storage/tajo-storage-hdfs/src/test/resources/storage-default.xml
+++ b/tajo-storage/tajo-storage-hdfs/src/test/resources/storage-default.xml
@@@ -1,164 -1,0 +1,178 @@@
 +<?xml version="1.0"?>
 +<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
 +
 +<!--
 +  Licensed to the Apache Software Foundation (ASF) under one
 +  or more contributor license agreements.  See the NOTICE file
 +  distributed with this work for additional information
 +  regarding copyright ownership.  The ASF licenses this file
 +  to you under the Apache License, Version 2.0 (the
 +  "License"); you may not use this file except in compliance
 +  with the License.  You may obtain a copy of the License at
 +
 +      http://www.apache.org/licenses/LICENSE-2.0
 +
 +  Unless required by applicable law or agreed to in writing, software
 +  distributed under the License is distributed on an "AS IS" BASIS,
 +  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 +  See the License for the specific language governing permissions and
 +  limitations under the License.
 +  -->
 +
 +<configuration>
 +  <property>
 +    <name>fs.s3.impl</name>
 +    <value>org.apache.tajo.storage.s3.SmallBlockS3FileSystem</value>
 +  </property>
 +
 +  <!-- Storage Manager Configuration -->
 +  <property>
 +    <name>tajo.storage.manager.hdfs.class</name>
 +    <value>org.apache.tajo.storage.FileStorageManager</value>
 +  </property>
 +  <property>
 +    <name>tajo.storage.manager.hbase.class</name>
 +    <value>org.apache.tajo.storage.hbase.HBaseStorageManager</value>
 +  </property>
 +
 +  <!--- Registered Scanner Handler -->
 +  <property>
 +    <name>tajo.storage.scanner-handler</name>
-     <value>textfile,csv,raw,rcfile,row,parquet,sequencefile,avro</value>
++    <value>textfile,csv,json,raw,rcfile,row,parquet,sequencefile,avro</value>
 +  </property>
 +
 +  <!--- Fragment Class Configurations -->
 +  <property>
 +    <name>tajo.storage.fragment.textfile.class</name>
 +    <value>org.apache.tajo.storage.fragment.FileFragment</value>
 +  </property>
 +  <property>
 +    <name>tajo.storage.fragment.csv.class</name>
 +    <value>org.apache.tajo.storage.fragment.FileFragment</value>
 +  </property>
 +  <property>
++    <name>tajo.storage.fragment.json.class</name>
++    <value>org.apache.tajo.storage.fragment.FileFragment</value>
++  </property>
++  <property>
 +    <name>tajo.storage.fragment.raw.class</name>
 +    <value>org.apache.tajo.storage.fragment.FileFragment</value>
 +  </property>
 +  <property>
 +    <name>tajo.storage.fragment.rcfile.class</name>
 +    <value>org.apache.tajo.storage.fragment.FileFragment</value>
 +  </property>
 +  <property>
 +    <name>tajo.storage.fragment.row.class</name>
 +    <value>org.apache.tajo.storage.fragment.FileFragment</value>
 +  </property>
 +  <property>
 +    <name>tajo.storage.fragment.parquet.class</name>
 +    <value>org.apache.tajo.storage.fragment.FileFragment</value>
 +  </property>
 +  <property>
 +    <name>tajo.storage.fragment.sequencefile.class</name>
 +    <value>org.apache.tajo.storage.fragment.FileFragment</value>
 +  </property>
 +  <property>
 +    <name>tajo.storage.fragment.avro.class</name>
 +    <value>org.apache.tajo.storage.fragment.FileFragment</value>
 +  </property>
 +
 +  <!--- Scanner Handler -->
 +  <property>
 +    <name>tajo.storage.scanner-handler.textfile.class</name>
 +    <value>org.apache.tajo.storage.text.DelimitedTextFile$DelimitedTextFileScanner</value>
 +  </property>
 +
 +  <property>
 +    <name>tajo.storage.scanner-handler.csv.class</name>
 +    <value>org.apache.tajo.storage.CSVFile$CSVScanner</value>
 +  </property>
 +
 +  <property>
++    <name>tajo.storage.scanner-handler.json.class</name>
++    <value>org.apache.tajo.storage.text.DelimitedTextFile$DelimitedTextFileScanner</value>
++  </property>
++
++  <property>
 +    <name>tajo.storage.scanner-handler.raw.class</name>
 +    <value>org.apache.tajo.storage.RawFile$RawFileScanner</value>
 +  </property>
 +
 +  <property>
 +    <name>tajo.storage.scanner-handler.rcfile.class</name>
 +    <value>org.apache.tajo.storage.rcfile.RCFile$RCFileScanner</value>
 +  </property>
 +
 +  <property>
 +    <name>tajo.storage.scanner-handler.rowfile.class</name>
 +    <value>org.apache.tajo.storage.RowFile$RowFileScanner</value>
 +  </property>
 +
 +  <property>
 +    <name>tajo.storage.scanner-handler.parquet.class</name>
 +    <value>org.apache.tajo.storage.parquet.ParquetScanner</value>
 +  </property>
 +
 +  <property>
 +    <name>tajo.storage.scanner-handler.sequencefile.class</name>
 +    <value>org.apache.tajo.storage.sequencefile.SequenceFileScanner</value>
 +  </property>
 +
 +  <property>
 +    <name>tajo.storage.scanner-handler.avro.class</name>
 +    <value>org.apache.tajo.storage.avro.AvroScanner</value>
 +  </property>
 +
 +  <!--- Appender Handler -->
 +  <property>
 +    <name>tajo.storage.appender-handler</name>
 +    <value>textfile,csv,raw,rcfile,row,parquet,sequencefile,avro</value>
 +  </property>
 +
 +  <property>
 +    <name>tajo.storage.appender-handler.textfile.class</name>
 +    <value>org.apache.tajo.storage.text.DelimitedTextFile$DelimitedTextFileAppender</value>
 +  </property>
 +
 +  <property>
 +    <name>tajo.storage.appender-handler.csv.class</name>
 +    <value>org.apache.tajo.storage.CSVFile$CSVAppender</value>
 +  </property>
 +
 +  <property>
++    <name>tajo.storage.appender-handler.json.class</name>
++    <value>org.apache.tajo.storage.text.DelimitedTextFile$DelimitedTextFileAppender</value>
++  </property>
++
++  <property>
 +    <name>tajo.storage.appender-handler.raw.class</name>
 +    <value>org.apache.tajo.storage.RawFile$RawFileAppender</value>
 +  </property>
 +
 +  <property>
 +    <name>tajo.storage.appender-handler.rcfile.class</name>
 +    <value>org.apache.tajo.storage.rcfile.RCFile$RCFileAppender</value>
 +  </property>
 +
 +  <property>
 +    <name>tajo.storage.appender-handler.rowfile.class</name>
 +    <value>org.apache.tajo.storage.RowFile$RowFileAppender</value>
 +  </property>
 +
 +  <property>
 +    <name>tajo.storage.appender-handler.parquet.class</name>
 +    <value>org.apache.tajo.storage.parquet.ParquetAppender</value>
 +  </property>
 +
 +  <property>
 +    <name>tajo.storage.appender-handler.sequencefile.class</name>
 +    <value>org.apache.tajo.storage.sequencefile.SequenceFileAppender</value>
 +  </property>
 +
 +  <property>
 +    <name>tajo.storage.appender-handler.avro.class</name>
 +    <value>org.apache.tajo.storage.avro.AvroAppender</value>
 +  </property>
 +</configuration>


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

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

http://git-wip-us.apache.org/repos/asf/tajo/blob/940546a0/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/json/JsonLineDeserializer.java
----------------------------------------------------------------------
diff --cc tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/json/JsonLineDeserializer.java
index 0000000,0000000..c1f63a8
new file mode 100644
--- /dev/null
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/json/JsonLineDeserializer.java
@@@ -1,0 -1,0 +1,220 @@@
++/**
++ * Licensed to the Apache Software Foundation (ASF) under one
++ * or more contributor license agreements.  See the NOTICE file
++ * distributed with this work for additional information
++ * regarding copyright ownership.  The ASF licenses this file
++ * to you under the Apache License, Version 2.0 (the
++ * "License"); you may not use this file except in compliance
++ * with the License.  You may obtain a copy of the License at
++ *
++ *     http://www.apache.org/licenses/LICENSE-2.0
++ *
++ * Unless required by applicable law or agreed to in writing, software
++ * distributed under the License is distributed on an "AS IS" BASIS,
++ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
++ * See the License for the specific language governing permissions and
++ * limitations under the License.
++ */
++
++package org.apache.tajo.storage.json;
++
++
++import io.netty.buffer.ByteBuf;
++import net.minidev.json.JSONArray;
++import net.minidev.json.JSONObject;
++import net.minidev.json.parser.JSONParser;
++import org.apache.tajo.catalog.Schema;
++import org.apache.tajo.catalog.SchemaUtil;
++import org.apache.tajo.catalog.TableMeta;
++import org.apache.tajo.common.TajoDataTypes.Type;
++import org.apache.tajo.common.exception.NotImplementedException;
++import org.apache.tajo.datum.DatumFactory;
++import org.apache.tajo.datum.NullDatum;
++import org.apache.tajo.storage.Tuple;
++import org.apache.tajo.storage.text.TextLineDeserializer;
++import org.apache.tajo.storage.text.TextLineParsingError;
++
++import java.io.IOException;
++import java.util.Iterator;
++
++public class JsonLineDeserializer extends TextLineDeserializer {
++  private JSONParser parser;
++  private Type [] types;
++  private String [] columnNames;
++
++  public JsonLineDeserializer(Schema schema, TableMeta meta, int[] targetColumnIndexes) {
++    super(schema, meta, targetColumnIndexes);
++  }
++
++  @Override
++  public void init() {
++    types = SchemaUtil.toTypes(schema);
++    columnNames = SchemaUtil.toSimpleNames(schema);
++
++    parser = new JSONParser(JSONParser.MODE_JSON_SIMPLE);
++  }
++
++  @Override
++  public void deserialize(ByteBuf buf, Tuple output) throws IOException, TextLineParsingError {
++    byte [] line = new byte[buf.readableBytes()];
++    buf.readBytes(line);
++
++    try {
++      JSONObject object = (JSONObject) parser.parse(line);
++
++      for (int i = 0; i < targetColumnIndexes.length; i++) {
++        int actualIdx = targetColumnIndexes[i];
++        String fieldName = columnNames[actualIdx];
++
++        if (!object.containsKey(fieldName)) {
++          output.put(actualIdx, NullDatum.get());
++          continue;
++        }
++
++        switch (types[actualIdx]) {
++        case BOOLEAN:
++          String boolStr = object.getAsString(fieldName);
++          if (boolStr != null) {
++            output.put(actualIdx, DatumFactory.createBool(boolStr.equals("true")));
++          } else {
++            output.put(actualIdx, NullDatum.get());
++          }
++          break;
++        case CHAR:
++          String charStr = object.getAsString(fieldName);
++          if (charStr != null) {
++            output.put(actualIdx, DatumFactory.createChar(charStr));
++          } else {
++            output.put(actualIdx, NullDatum.get());
++          }
++          break;
++        case INT1:
++        case INT2:
++          Number int2Num = object.getAsNumber(fieldName);
++          if (int2Num != null) {
++            output.put(actualIdx, DatumFactory.createInt2(int2Num.shortValue()));
++          } else {
++            output.put(actualIdx, NullDatum.get());
++          }
++          break;
++        case INT4:
++          Number int4Num = object.getAsNumber(fieldName);
++          if (int4Num != null) {
++            output.put(actualIdx, DatumFactory.createInt4(int4Num.intValue()));
++          } else {
++            output.put(actualIdx, NullDatum.get());
++          }
++          break;
++        case INT8:
++          Number int8Num = object.getAsNumber(fieldName);
++          if (int8Num != null) {
++            output.put(actualIdx, DatumFactory.createInt8(int8Num.longValue()));
++          } else {
++            output.put(actualIdx, NullDatum.get());
++          }
++          break;
++        case FLOAT4:
++          Number float4Num = object.getAsNumber(fieldName);
++          if (float4Num != null) {
++            output.put(actualIdx, DatumFactory.createFloat4(float4Num.floatValue()));
++          } else {
++            output.put(actualIdx, NullDatum.get());
++          }
++          break;
++        case FLOAT8:
++          Number float8Num = object.getAsNumber(fieldName);
++          if (float8Num != null) {
++            output.put(actualIdx, DatumFactory.createFloat8(float8Num.doubleValue()));
++          } else {
++            output.put(actualIdx, NullDatum.get());
++          }
++          break;
++        case TEXT:
++          String textStr = object.getAsString(fieldName);
++          if (textStr != null) {
++            output.put(actualIdx, DatumFactory.createText(textStr));
++          } else {
++            output.put(actualIdx, NullDatum.get());
++          }
++          break;
++        case TIMESTAMP:
++          String timestampStr = object.getAsString(fieldName);
++          if (timestampStr != null) {
++            output.put(actualIdx, DatumFactory.createTimestamp(timestampStr));
++          } else {
++            output.put(actualIdx, NullDatum.get());
++          }
++          break;
++        case TIME:
++          String timeStr = object.getAsString(fieldName);
++          if (timeStr != null) {
++            output.put(actualIdx, DatumFactory.createTime(timeStr));
++          } else {
++            output.put(actualIdx, NullDatum.get());
++          }
++          break;
++        case DATE:
++          String dateStr = object.getAsString(fieldName);
++          if (dateStr != null) {
++            output.put(actualIdx, DatumFactory.createDate(dateStr));
++          } else {
++            output.put(actualIdx, NullDatum.get());
++          }
++          break;
++        case BIT:
++        case BINARY:
++        case VARBINARY:
++        case BLOB: {
++          Object jsonObject = object.get(fieldName);
++
++          if (jsonObject == null) {
++            output.put(actualIdx, NullDatum.get());
++            break;
++          } if (jsonObject instanceof String) {
++            output.put(actualIdx, DatumFactory.createBlob((String)jsonObject));
++          } else if (jsonObject instanceof JSONArray) {
++            JSONArray jsonArray = (JSONArray) jsonObject;
++            byte[] bytes = new byte[jsonArray.size()];
++            Iterator<Object> it = jsonArray.iterator();
++            int arrayIdx = 0;
++            while (it.hasNext()) {
++              bytes[arrayIdx++] = ((Long) it.next()).byteValue();
++            }
++            if (bytes.length > 0) {
++              output.put(actualIdx, DatumFactory.createBlob(bytes));
++            } else {
++              output.put(actualIdx, NullDatum.get());
++            }
++            break;
++          } else {
++            throw new IOException("Unknown json object: " + object.getClass().getSimpleName());
++          }
++          break;
++        }
++        case INET4:
++          String inetStr = object.getAsString(fieldName);
++          if (inetStr != null) {
++            output.put(actualIdx, DatumFactory.createInet4(inetStr));
++          } else {
++            output.put(actualIdx, NullDatum.get());
++          }
++          break;
++
++        case NULL_TYPE:
++          output.put(actualIdx, NullDatum.get());
++          break;
++
++        default:
++          throw new NotImplementedException(types[actualIdx].name() + " is not supported.");
++        }
++      }
++
++    } catch (Throwable e) {
++      throw new IOException(e);
++    }
++  }
++
++  @Override
++  public void release() {
++  }
++}

http://git-wip-us.apache.org/repos/asf/tajo/blob/940546a0/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/json/JsonLineSerDe.java
----------------------------------------------------------------------
diff --cc tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/json/JsonLineSerDe.java
index 0000000,0000000..6db2c29
new file mode 100644
--- /dev/null
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/json/JsonLineSerDe.java
@@@ -1,0 -1,0 +1,37 @@@
++/**
++ * Licensed to the Apache Software Foundation (ASF) under one
++ * or more contributor license agreements.  See the NOTICE file
++ * distributed with this work for additional information
++ * regarding copyright ownership.  The ASF licenses this file
++ * to you under the Apache License, Version 2.0 (the
++ * "License"); you may not use this file except in compliance
++ * with the License.  You may obtain a copy of the License at
++ *
++ *     http://www.apache.org/licenses/LICENSE-2.0
++ *
++ * Unless required by applicable law or agreed to in writing, software
++ * distributed under the License is distributed on an "AS IS" BASIS,
++ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
++ * See the License for the specific language governing permissions and
++ * limitations under the License.
++ */
++
++package org.apache.tajo.storage.json;
++
++import org.apache.tajo.catalog.Schema;
++import org.apache.tajo.catalog.TableMeta;
++import org.apache.tajo.storage.text.TextLineDeserializer;
++import org.apache.tajo.storage.text.TextLineSerDe;
++import org.apache.tajo.storage.text.TextLineSerializer;
++
++public class JsonLineSerDe extends TextLineSerDe {
++  @Override
++  public TextLineDeserializer createDeserializer(Schema schema, TableMeta meta, int[] targetColumnIndexes) {
++    return new JsonLineDeserializer(schema, meta, targetColumnIndexes);
++  }
++
++  @Override
++  public TextLineSerializer createSerializer(Schema schema, TableMeta meta) {
++    return new JsonLineSerializer(schema, meta);
++  }
++}

http://git-wip-us.apache.org/repos/asf/tajo/blob/940546a0/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/json/JsonLineSerializer.java
----------------------------------------------------------------------
diff --cc tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/json/JsonLineSerializer.java
index 0000000,0000000..cd31ada
new file mode 100644
--- /dev/null
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/json/JsonLineSerializer.java
@@@ -1,0 -1,0 +1,130 @@@
++/**
++ * Licensed to the Apache Software Foundation (ASF) under one
++ * or more contributor license agreements.  See the NOTICE file
++ * distributed with this work for additional information
++ * regarding copyright ownership.  The ASF licenses this file
++ * to you under the Apache License, Version 2.0 (the
++ * "License"); you may not use this file except in compliance
++ * with the License.  You may obtain a copy of the License at
++ *
++ *     http://www.apache.org/licenses/LICENSE-2.0
++ *
++ * Unless required by applicable law or agreed to in writing, software
++ * distributed under the License is distributed on an "AS IS" BASIS,
++ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
++ * See the License for the specific language governing permissions and
++ * limitations under the License.
++ */
++
++package org.apache.tajo.storage.json;
++
++
++import net.minidev.json.JSONObject;
++import org.apache.tajo.catalog.Schema;
++import org.apache.tajo.catalog.SchemaUtil;
++import org.apache.tajo.catalog.TableMeta;
++import org.apache.tajo.common.TajoDataTypes.Type;
++import org.apache.tajo.common.exception.NotImplementedException;
++import org.apache.tajo.datum.TextDatum;
++import org.apache.tajo.datum.protobuf.ProtobufJsonFormat;
++import org.apache.tajo.storage.Tuple;
++import org.apache.tajo.storage.text.TextLineSerializer;
++
++import java.io.IOException;
++import java.io.OutputStream;
++
++public class JsonLineSerializer extends TextLineSerializer {
++  private static ProtobufJsonFormat protobufJsonFormat = ProtobufJsonFormat.getInstance();
++
++  private Type [] types;
++  private String [] simpleNames;
++  private int columnNum;
++
++
++  public JsonLineSerializer(Schema schema, TableMeta meta) {
++    super(schema, meta);
++  }
++
++  @Override
++  public void init() {
++    types = SchemaUtil.toTypes(schema);
++    simpleNames = SchemaUtil.toSimpleNames(schema);
++    columnNum = schema.size();
++  }
++
++  @Override
++  public int serialize(OutputStream out, Tuple input) throws IOException {
++    JSONObject jsonObject = new JSONObject();
++
++    for (int i = 0; i < columnNum; i++) {
++      if (input.isNull(i)) {
++        continue;
++      }
++
++      String fieldName = simpleNames[i];
++      Type type = types[i];
++
++      switch (type) {
++
++      case BOOLEAN:
++        jsonObject.put(fieldName, input.getBool(i));
++        break;
++
++      case INT1:
++      case INT2:
++        jsonObject.put(fieldName, input.getInt2(i));
++        break;
++
++      case INT4:
++        jsonObject.put(fieldName, input.getInt4(i));
++        break;
++
++      case INT8:
++        jsonObject.put(fieldName, input.getInt8(i));
++        break;
++
++      case FLOAT4:
++        jsonObject.put(fieldName, input.getFloat4(i));
++        break;
++
++      case FLOAT8:
++        jsonObject.put(fieldName, input.getFloat8(i));
++        break;
++
++      case CHAR:
++      case TEXT:
++      case VARCHAR:
++      case INET4:
++      case TIMESTAMP:
++      case DATE:
++      case TIME:
++      case INTERVAL:
++        jsonObject.put(fieldName, input.getText(i));
++        break;
++
++      case BIT:
++      case BINARY:
++      case BLOB:
++      case VARBINARY:
++        jsonObject.put(fieldName, input.getBytes(i));
++        break;
++
++      case NULL_TYPE:
++        break;
++
++      default:
++        throw new NotImplementedException(types[i].name() + " is not supported.");
++      }
++    }
++
++    String jsonStr = jsonObject.toJSONString();
++    byte [] jsonBytes = jsonStr.getBytes(TextDatum.DEFAULT_CHARSET);
++    out.write(jsonBytes);
++    return jsonBytes.length;
++  }
++
++  @Override
++  public void release() {
++
++  }
++}

http://git-wip-us.apache.org/repos/asf/tajo/blob/940546a0/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/ByteBufLineReader.java
----------------------------------------------------------------------
diff --cc tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/ByteBufLineReader.java
index 1448885,0000000..86319e1
mode 100644,000000..100644
--- a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/ByteBufLineReader.java
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/ByteBufLineReader.java
@@@ -1,154 -1,0 +1,170 @@@
 +/**
 + * Licensed to the Apache Software Foundation (ASF) under one
 + * or more contributor license agreements.  See the NOTICE file
 + * distributed with this work for additional information
 + * regarding copyright ownership.  The ASF licenses this file
 + * to you under the Apache License, Version 2.0 (the
 + * "License"); you may not use this file except in compliance
 + * with the License.  You may obtain a copy of the License at
 + *
 + *     http://www.apache.org/licenses/LICENSE-2.0
 + *
 + * Unless required by applicable law or agreed to in writing, software
 + * distributed under the License is distributed on an "AS IS" BASIS,
 + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 + * See the License for the specific language governing permissions and
 + * limitations under the License.
 + */
 +
 +package org.apache.tajo.storage.text;
 +
 +import io.netty.buffer.ByteBuf;
 +import io.netty.util.CharsetUtil;
 +import org.apache.tajo.storage.BufferPool;
 +import org.apache.tajo.storage.ByteBufInputChannel;
 +
 +import java.io.Closeable;
 +import java.io.IOException;
 +import java.util.concurrent.atomic.AtomicInteger;
 +
 +public class ByteBufLineReader implements Closeable {
 +  private static int DEFAULT_BUFFER = 64 * 1024;
 +
 +  private int bufferSize;
 +  private long readBytes;
++  private boolean eof = false;
 +  private ByteBuf buffer;
 +  private final ByteBufInputChannel channel;
 +  private final AtomicInteger tempReadBytes = new AtomicInteger();
 +  private final LineSplitProcessor processor = new LineSplitProcessor();
 +
 +  public ByteBufLineReader(ByteBufInputChannel channel) {
 +    this(channel, BufferPool.directBuffer(DEFAULT_BUFFER));
 +  }
 +
 +  public ByteBufLineReader(ByteBufInputChannel channel, ByteBuf buf) {
 +    this.readBytes = 0;
 +    this.channel = channel;
 +    this.buffer = buf;
 +    this.bufferSize = buf.capacity();
 +  }
 +
 +  public long readBytes() {
 +    return readBytes - buffer.readableBytes();
 +  }
 +
 +  public long available() throws IOException {
 +    return channel.available() + buffer.readableBytes();
 +  }
 +
 +  @Override
 +  public void close() throws IOException {
 +    if (this.buffer.refCnt() > 0) {
 +      this.buffer.release();
 +    }
 +    this.channel.close();
 +  }
 +
 +  public String readLine() throws IOException {
 +    ByteBuf buf = readLineBuf(tempReadBytes);
 +    if (buf != null) {
 +      return buf.toString(CharsetUtil.UTF_8);
 +    }
 +    return null;
 +  }
 +
 +  private void fillBuffer() throws IOException {
 +
 +    int tailBytes = 0;
 +    if (this.readBytes > 0) {
 +      this.buffer.markReaderIndex();
 +      this.buffer.discardSomeReadBytes();  // compact the buffer
 +      tailBytes = this.buffer.writerIndex();
 +      if (!this.buffer.isWritable()) {
 +        // a line bytes is large than the buffer
 +        BufferPool.ensureWritable(buffer, bufferSize);
 +        this.bufferSize = buffer.capacity();
 +      }
 +    }
 +
 +    boolean release = true;
 +    try {
 +      int readBytes = tailBytes;
 +      for (; ; ) {
 +        int localReadBytes = buffer.writeBytes(channel, bufferSize - readBytes);
 +        if (localReadBytes < 0) {
++          if (tailBytes == readBytes) {
++            // no more bytes are in the channel
++            eof = true;
++          }
 +          break;
 +        }
 +        readBytes += localReadBytes;
 +        if (readBytes == bufferSize) {
 +          break;
 +        }
 +      }
 +      this.readBytes += (readBytes - tailBytes);
 +      release = false;
-       this.buffer.readerIndex(this.buffer.readerIndex() + tailBytes); //skip past buffer (tail)
++      if (!eof) {
++        this.buffer.readerIndex(this.buffer.readerIndex() + tailBytes); //skip past buffer (tail)
++      }
 +    } finally {
 +      if (release) {
 +        buffer.release();
 +      }
 +    }
 +  }
 +
 +  /**
 +   * Read a line terminated by one of CR, LF, or CRLF.
 +   */
 +  public ByteBuf readLineBuf(AtomicInteger reads) throws IOException {
++    if(eof) return null;
++
 +    int startIndex = buffer.readerIndex();
 +    int readBytes;
 +    int readable;
 +    int newlineLength; //length of terminating newline
 +
 +    loop:
 +    while (true) {
 +      readable = buffer.readableBytes();
 +      if (readable <= 0) {
 +        buffer.readerIndex(startIndex);
 +        fillBuffer(); //compact and fill buffer
 +        if (!buffer.isReadable()) {
 +          return null;
 +        } else {
-           startIndex = 0; // reset the line start position
++          if (!eof) startIndex = 0; // reset the line start position
++          else startIndex = buffer.readerIndex();
 +        }
 +        readable = buffer.readableBytes();
 +      }
 +
 +      int endIndex = buffer.forEachByte(buffer.readerIndex(), readable, processor);
 +      if (endIndex < 0) {
-         buffer.readerIndex(buffer.writerIndex());
++        //does not appeared terminating newline
++        buffer.readerIndex(buffer.writerIndex()); // set to end buffer
++        if(eof){
++          readBytes = buffer.readerIndex() - startIndex;
++          newlineLength = 0;
++          break loop;
++        }
 +      } else {
 +        buffer.readerIndex(endIndex + 1);
 +        readBytes = buffer.readerIndex() - startIndex;
 +        if (processor.isPrevCharCR() && buffer.isReadable()
 +            && buffer.getByte(buffer.readerIndex()) == LineSplitProcessor.LF) {
 +          buffer.skipBytes(1);
 +          newlineLength = 2;
 +        } else {
 +          newlineLength = 1;
 +        }
 +        break loop;
 +      }
 +    }
 +    reads.set(readBytes);
 +    return buffer.slice(startIndex, readBytes - newlineLength);
 +  }
 +}

http://git-wip-us.apache.org/repos/asf/tajo/blob/940546a0/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/CSVLineDeserializer.java
----------------------------------------------------------------------
diff --cc tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/CSVLineDeserializer.java
index 0000000,0000000..f2eebc6
new file mode 100644
--- /dev/null
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/CSVLineDeserializer.java
@@@ -1,0 -1,0 +1,96 @@@
++/**
++ * Licensed to the Apache Software Foundation (ASF) under one
++ * or more contributor license agreements.  See the NOTICE file
++ * distributed with this work for additional information
++ * regarding copyright ownership.  The ASF licenses this file
++ * to you under the Apache License, Version 2.0 (the
++ * "License"); you may not use this file except in compliance
++ * with the License.  You may obtain a copy of the License at
++ *
++ *     http://www.apache.org/licenses/LICENSE-2.0
++ *
++ * Unless required by applicable law or agreed to in writing, software
++ * distributed under the License is distributed on an "AS IS" BASIS,
++ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
++ * See the License for the specific language governing permissions and
++ * limitations under the License.
++ */
++
++package org.apache.tajo.storage.text;
++
++import io.netty.buffer.ByteBuf;
++import org.apache.tajo.catalog.Schema;
++import org.apache.tajo.catalog.TableMeta;
++import org.apache.tajo.datum.Datum;
++import org.apache.tajo.storage.FieldSerializerDeserializer;
++import org.apache.tajo.storage.Tuple;
++
++import java.io.IOException;
++
++public class CSVLineDeserializer extends TextLineDeserializer {
++  private FieldSplitProcessor processor;
++  private FieldSerializerDeserializer fieldSerDer;
++  private ByteBuf nullChars;
++
++  public CSVLineDeserializer(Schema schema, TableMeta meta, int[] targetColumnIndexes) {
++    super(schema, meta, targetColumnIndexes);
++  }
++
++  @Override
++  public void init() {
++    this.processor = new FieldSplitProcessor(CSVLineSerDe.getFieldDelimiter(meta));
++
++    if (nullChars != null) {
++      nullChars.release();
++    }
++    nullChars = TextLineSerDe.getNullChars(meta);
++
++    fieldSerDer = new TextFieldSerializerDeserializer();
++  }
++
++  public void deserialize(final ByteBuf lineBuf, Tuple output) throws IOException, TextLineParsingError {
++    int[] projection = targetColumnIndexes;
++    if (lineBuf == null || targetColumnIndexes == null || targetColumnIndexes.length == 0) {
++      return;
++    }
++
++    final int rowLength = lineBuf.readableBytes();
++    int start = 0, fieldLength = 0, end = 0;
++
++    //Projection
++    int currentTarget = 0;
++    int currentIndex = 0;
++
++    while (end != -1) {
++      end = lineBuf.forEachByte(start, rowLength - start, processor);
++
++      if (end < 0) {
++        fieldLength = rowLength - start;
++      } else {
++        fieldLength = end - start;
++      }
++
++      if (projection.length > currentTarget && currentIndex == projection[currentTarget]) {
++        lineBuf.setIndex(start, start + fieldLength);
++        Datum datum = fieldSerDer.deserialize(lineBuf, schema.getColumn(currentIndex), currentIndex, nullChars);
++        output.put(currentIndex, datum);
++        currentTarget++;
++      }
++
++      if (projection.length == currentTarget) {
++        break;
++      }
++
++      start = end + 1;
++      currentIndex++;
++    }
++  }
++
++  @Override
++  public void release() {
++    if (nullChars != null) {
++      nullChars.release();
++      nullChars = null;
++    }
++  }
++}

http://git-wip-us.apache.org/repos/asf/tajo/blob/940546a0/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/CSVLineSerDe.java
----------------------------------------------------------------------
diff --cc tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/CSVLineSerDe.java
index 0000000,0000000..2fe7f23
new file mode 100644
--- /dev/null
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/CSVLineSerDe.java
@@@ -1,0 -1,0 +1,41 @@@
++/**
++ * Licensed to the Apache Software Foundation (ASF) under one
++ * or more contributor license agreements.  See the NOTICE file
++ * distributed with this work for additional information
++ * regarding copyright ownership.  The ASF licenses this file
++ * to you under the Apache License, Version 2.0 (the
++ * "License"); you may not use this file except in compliance
++ * with the License.  You may obtain a copy of the License at
++ *
++ *     http://www.apache.org/licenses/LICENSE-2.0
++ *
++ * Unless required by applicable law or agreed to in writing, software
++ * distributed under the License is distributed on an "AS IS" BASIS,
++ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
++ * See the License for the specific language governing permissions and
++ * limitations under the License.
++ */
++
++package org.apache.tajo.storage.text;
++
++import org.apache.commons.lang.StringEscapeUtils;
++import org.apache.tajo.catalog.Schema;
++import org.apache.tajo.catalog.TableMeta;
++import org.apache.tajo.storage.StorageConstants;
++
++public class CSVLineSerDe extends TextLineSerDe {
++  @Override
++  public TextLineDeserializer createDeserializer(Schema schema, TableMeta meta, int[] targetColumnIndexes) {
++    return new CSVLineDeserializer(schema, meta, targetColumnIndexes);
++  }
++
++  @Override
++  public TextLineSerializer createSerializer(Schema schema, TableMeta meta) {
++    return new CSVLineSerializer(schema, meta);
++  }
++
++  public static char getFieldDelimiter(TableMeta meta) {
++    return StringEscapeUtils.unescapeJava(meta.getOption(StorageConstants.TEXT_DELIMITER,
++        StorageConstants.DEFAULT_FIELD_DELIMITER)).charAt(0);
++  }
++}

http://git-wip-us.apache.org/repos/asf/tajo/blob/940546a0/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/CSVLineSerializer.java
----------------------------------------------------------------------
diff --cc tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/CSVLineSerializer.java
index 0000000,0000000..48154eb
new file mode 100644
--- /dev/null
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/CSVLineSerializer.java
@@@ -1,0 -1,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.text;
++
++import org.apache.tajo.catalog.Schema;
++import org.apache.tajo.catalog.TableMeta;
++import org.apache.tajo.datum.Datum;
++import org.apache.tajo.storage.FieldSerializerDeserializer;
++import org.apache.tajo.storage.Tuple;
++
++import java.io.IOException;
++import java.io.OutputStream;
++
++public class CSVLineSerializer extends TextLineSerializer {
++  private FieldSerializerDeserializer serde;
++
++  private byte [] nullChars;
++  private char delimiter;
++  private int columnNum;
++
++  public CSVLineSerializer(Schema schema, TableMeta meta) {
++    super(schema, meta);
++  }
++
++  @Override
++  public void init() {
++    nullChars = TextLineSerDe.getNullCharsAsBytes(meta);
++    delimiter = CSVLineSerDe.getFieldDelimiter(meta);
++    columnNum = schema.size();
++
++    serde = new TextFieldSerializerDeserializer();
++  }
++
++  @Override
++  public int serialize(OutputStream out, Tuple input) throws IOException {
++    int writtenBytes = 0;
++
++    for (int i = 0; i < columnNum; i++) {
++      Datum datum = input.get(i);
++      writtenBytes += serde.serialize(out, datum, schema.getColumn(i), i, nullChars);
++
++      if (columnNum - 1 > i) {
++        out.write((byte) delimiter);
++        writtenBytes += 1;
++      }
++    }
++
++    return writtenBytes;
++  }
++
++  @Override
++  public void release() {
++  }
++}

http://git-wip-us.apache.org/repos/asf/tajo/blob/940546a0/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/DelimitedTextFile.java
----------------------------------------------------------------------
diff --cc tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/DelimitedTextFile.java
index a337509,0000000..7848198
mode 100644,000000..100644
--- a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/DelimitedTextFile.java
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/DelimitedTextFile.java
@@@ -1,468 -1,0 +1,475 @@@
 +/**
 + * Licensed to the Apache Software Foundation (ASF) under one
 + * or more contributor license agreements.  See the NOTICE file
 + * distributed with this work for additional information
 + * regarding copyright ownership.  The ASF licenses this file
 + * to you under the Apache License, Version 2.0 (the
 + * "License"); you may not use this file except in compliance
 + * with the License.  You may obtain a copy of the License at
 + *
 + *     http://www.apache.org/licenses/LICENSE-2.0
 + *
 + * Unless required by applicable law or agreed to in writing, software
 + * distributed under the License is distributed on an "AS IS" BASIS,
 + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 + * See the License for the specific language governing permissions and
 + * limitations under the License.
 + */
 +
 +package org.apache.tajo.storage.text;
 +
 +import io.netty.buffer.ByteBuf;
- import org.apache.commons.lang.StringEscapeUtils;
- import org.apache.commons.lang.StringUtils;
 +import org.apache.commons.logging.Log;
 +import org.apache.commons.logging.LogFactory;
 +import org.apache.hadoop.conf.Configuration;
 +import org.apache.hadoop.fs.FSDataOutputStream;
 +import org.apache.hadoop.fs.FileSystem;
 +import org.apache.hadoop.fs.Path;
 +import org.apache.hadoop.io.IOUtils;
 +import org.apache.hadoop.io.compress.CompressionCodec;
 +import org.apache.hadoop.io.compress.CompressionCodecFactory;
 +import org.apache.hadoop.io.compress.CompressionOutputStream;
 +import org.apache.hadoop.io.compress.Compressor;
 +import org.apache.tajo.QueryUnitAttemptId;
 +import org.apache.tajo.catalog.Schema;
 +import org.apache.tajo.catalog.TableMeta;
 +import org.apache.tajo.catalog.statistics.TableStats;
- import org.apache.tajo.datum.Datum;
- import org.apache.tajo.datum.NullDatum;
 +import org.apache.tajo.storage.*;
 +import org.apache.tajo.storage.compress.CodecPool;
 +import org.apache.tajo.storage.exception.AlreadyExistsStorageException;
++import org.apache.tajo.storage.fragment.FileFragment;
 +import org.apache.tajo.storage.fragment.Fragment;
 +import org.apache.tajo.storage.rcfile.NonSyncByteArrayOutputStream;
++import org.apache.tajo.util.ReflectionUtil;
 +
 +import java.io.BufferedOutputStream;
 +import java.io.DataOutputStream;
 +import java.io.FileNotFoundException;
 +import java.io.IOException;
 +import java.util.Arrays;
++import java.util.Map;
++import java.util.concurrent.ConcurrentHashMap;
++
++import static org.apache.tajo.storage.StorageConstants.DEFAULT_TEXT_ERROR_TOLERANCE_MAXNUM;
++import static org.apache.tajo.storage.StorageConstants.TEXT_ERROR_TOLERANCE_MAXNUM;
 +
 +public class DelimitedTextFile {
 +
 +  public static final byte LF = '\n';
-   public static int EOF = -1;
 +
 +  private static final Log LOG = LogFactory.getLog(DelimitedTextFile.class);
 +
++  /** it caches line serde classes. */
++  private static final Map<String, Class<? extends TextLineSerDe>> serdeClassCache =
++      new ConcurrentHashMap<String, Class<? extends TextLineSerDe>>();
++
++  /**
++   * By default, DelimitedTextFileScanner uses CSVLineSerder. If a table property 'text.serde.class' is given,
++   * it will use the specified serder class.
++   *
++   * @return TextLineSerder
++   */
++  public static TextLineSerDe getLineSerde(TableMeta meta) {
++    TextLineSerDe lineSerder;
++
++    String serDeClassName;
++
++    // if there is no given serde class, it will use CSV line serder.
++    serDeClassName = meta.getOption(StorageConstants.TEXT_SERDE_CLASS, StorageConstants.DEFAULT_TEXT_SERDE_CLASS);
++
++    try {
++      Class<? extends TextLineSerDe> serdeClass;
++
++      if (serdeClassCache.containsKey(serDeClassName)) {
++        serdeClass = serdeClassCache.get(serDeClassName);
++      } else {
++        serdeClass = (Class<? extends TextLineSerDe>) Class.forName(serDeClassName);
++        serdeClassCache.put(serDeClassName, serdeClass);
++      }
++      lineSerder = (TextLineSerDe) ReflectionUtil.newInstance(serdeClass);
++    } catch (Throwable e) {
++      throw new RuntimeException("TextLineSerde class cannot be initialized.", e);
++    }
++
++    return lineSerder;
++  }
++
 +  public static class DelimitedTextFileAppender extends FileAppender {
 +    private final TableMeta meta;
 +    private final Schema schema;
-     private final int columnNum;
 +    private final FileSystem fs;
 +    private FSDataOutputStream fos;
 +    private DataOutputStream outputStream;
 +    private CompressionOutputStream deflateFilter;
-     private char delimiter;
 +    private TableStatistics stats = null;
 +    private Compressor compressor;
 +    private CompressionCodecFactory codecFactory;
 +    private CompressionCodec codec;
 +    private Path compressedPath;
 +    private byte[] nullChars;
 +    private int BUFFER_SIZE = 128 * 1024;
 +    private int bufferedBytes = 0;
 +    private long pos = 0;
 +
 +    private NonSyncByteArrayOutputStream os;
-     private FieldSerializerDeserializer serde;
++    private TextLineSerializer serializer;
 +
 +    public DelimitedTextFileAppender(Configuration conf, QueryUnitAttemptId taskAttemptId,
 +                                     final Schema schema, final TableMeta meta, final Path path)
 +        throws IOException {
 +      super(conf, taskAttemptId, schema, meta, path);
 +      this.fs = path.getFileSystem(conf);
 +      this.meta = meta;
 +      this.schema = schema;
-       this.delimiter = StringEscapeUtils.unescapeJava(this.meta.getOption(StorageConstants.TEXT_DELIMITER,
-           StorageConstants.DEFAULT_FIELD_DELIMITER)).charAt(0);
-       this.columnNum = schema.size();
- 
-       String nullCharacters = StringEscapeUtils.unescapeJava(this.meta.getOption(StorageConstants.TEXT_NULL,
-           NullDatum.DEFAULT_TEXT));
-       if (StringUtils.isEmpty(nullCharacters)) {
-         nullChars = NullDatum.get().asTextBytes();
-       } else {
-         nullChars = nullCharacters.getBytes();
-       }
++    }
++
++    public TextLineSerDe getLineSerde() {
++      return DelimitedTextFile.getLineSerde(meta);
 +    }
 +
 +    @Override
 +    public void init() throws IOException {
 +      if (!fs.exists(path.getParent())) {
 +        throw new FileNotFoundException(path.toString());
 +      }
 +
 +      if (this.meta.containsOption(StorageConstants.COMPRESSION_CODEC)) {
 +        String codecName = this.meta.getOption(StorageConstants.COMPRESSION_CODEC);
 +        codecFactory = new CompressionCodecFactory(conf);
 +        codec = codecFactory.getCodecByClassName(codecName);
 +        compressor = CodecPool.getCompressor(codec);
 +        if (compressor != null) compressor.reset();  //builtin gzip is null
 +
 +        String extension = codec.getDefaultExtension();
 +        compressedPath = path.suffix(extension);
 +
 +        if (fs.exists(compressedPath)) {
 +          throw new AlreadyExistsStorageException(compressedPath);
 +        }
 +
 +        fos = fs.create(compressedPath);
 +        deflateFilter = codec.createOutputStream(fos, compressor);
 +        outputStream = new DataOutputStream(deflateFilter);
 +
 +      } else {
 +        if (fs.exists(path)) {
 +          throw new AlreadyExistsStorageException(path);
 +        }
 +        fos = fs.create(path);
 +        outputStream = new DataOutputStream(new BufferedOutputStream(fos));
 +      }
 +
 +      if (enabledStats) {
 +        this.stats = new TableStatistics(this.schema);
 +      }
 +
-       serde = new TextFieldSerializerDeserializer();
++      serializer = getLineSerde().createSerializer(schema, meta);
++      serializer.init();
 +
 +      if (os == null) {
 +        os = new NonSyncByteArrayOutputStream(BUFFER_SIZE);
 +      }
 +
 +      os.reset();
 +      pos = fos.getPos();
 +      bufferedBytes = 0;
 +      super.init();
 +    }
 +
- 
 +    @Override
 +    public void addTuple(Tuple tuple) throws IOException {
-       Datum datum;
-       int rowBytes = 0;
- 
-       for (int i = 0; i < columnNum; i++) {
-         datum = tuple.get(i);
-         rowBytes += serde.serialize(os, datum, schema.getColumn(i), i, nullChars);
++      // write
++      int rowBytes = serializer.serialize(os, tuple);
 +
-         if (columnNum - 1 > i) {
-           os.write((byte) delimiter);
-           rowBytes += 1;
-         }
-       }
++      // new line
 +      os.write(LF);
 +      rowBytes += 1;
 +
++      // update positions
 +      pos += rowBytes;
 +      bufferedBytes += rowBytes;
++
++      // refill buffer if necessary
 +      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 {
++        serializer.release();
++
 +        if(outputStream != null){
 +          flush();
 +        }
 +
 +        // Statistical section
 +        if (enabledStats) {
 +          stats.setNumBytes(getOffset());
 +        }
 +
 +        if (deflateFilter != null) {
 +          deflateFilter.finish();
 +          deflateFilter.resetState();
 +          deflateFilter = null;
 +        }
 +
 +        os.close();
 +      } finally {
 +        IOUtils.cleanup(LOG, fos);
 +        if (compressor != null) {
 +          CodecPool.returnCompressor(compressor);
 +          compressor = null;
 +        }
 +      }
 +    }
 +
 +    @Override
 +    public TableStats getStats() {
 +      if (enabledStats) {
 +        return stats.getTableStat();
 +      } else {
 +        return null;
 +      }
 +    }
 +
 +    public boolean isCompress() {
 +      return compressor != null;
 +    }
 +
 +    public String getExtension() {
 +      return codec != null ? codec.getDefaultExtension() : "";
 +    }
 +  }
 +
 +  public static class DelimitedTextFileScanner extends FileScanner {
- 
 +    private boolean splittable = false;
 +    private final long startOffset;
-     private final long endOffset;
 +
++    private final long endOffset;
++    /** The number of actual read records */
 +    private int recordCount = 0;
 +    private int[] targetColumnIndexes;
 +
-     private ByteBuf nullChars;
-     private FieldSerializerDeserializer serde;
 +    private DelimitedLineReader reader;
-     private FieldSplitProcessor processor;
++    private TextLineDeserializer deserializer;
++
++    private int errorPrintOutMaxNum = 5;
++    /** Maximum number of permissible errors */
++    private int errorTorrenceMaxNum;
++    /** How many errors have occurred? */
++    private int errorNum;
 +
 +    public DelimitedTextFileScanner(Configuration conf, final Schema schema, final TableMeta meta,
 +                                    final Fragment fragment)
 +        throws IOException {
 +      super(conf, schema, meta, fragment);
 +      reader = new DelimitedLineReader(conf, this.fragment);
 +      if (!reader.isCompressed()) {
 +        splittable = true;
 +      }
 +
 +      startOffset = this.fragment.getStartKey();
 +      endOffset = startOffset + fragment.getLength();
 +
-       //Delimiter
-       String delim = meta.getOption(StorageConstants.TEXT_DELIMITER, StorageConstants.DEFAULT_FIELD_DELIMITER);
-       this.processor = new FieldSplitProcessor(StringEscapeUtils.unescapeJava(delim).charAt(0));
++      errorTorrenceMaxNum =
++          Integer.parseInt(meta.getOption(TEXT_ERROR_TOLERANCE_MAXNUM, DEFAULT_TEXT_ERROR_TOLERANCE_MAXNUM));
 +    }
 +
++
 +    @Override
 +    public void init() throws IOException {
-       if (nullChars != null) {
-         nullChars.release();
-       }
- 
-       String nullCharacters = StringEscapeUtils.unescapeJava(meta.getOption(StorageConstants.TEXT_NULL,
-           NullDatum.DEFAULT_TEXT));
-       byte[] bytes;
-       if (StringUtils.isEmpty(nullCharacters)) {
-         bytes = NullDatum.get().asTextBytes();
-       } else {
-         bytes = nullCharacters.getBytes();
-       }
- 
-       nullChars = BufferPool.directBuffer(bytes.length, bytes.length);
-       nullChars.writeBytes(bytes);
- 
 +      if (reader != null) {
 +        reader.close();
 +      }
 +      reader = new DelimitedLineReader(conf, fragment);
 +      reader.init();
 +      recordCount = 0;
 +
 +      if (targets == null) {
 +        targets = schema.toArray();
 +      }
 +
 +      targetColumnIndexes = new int[targets.length];
 +      for (int i = 0; i < targets.length; i++) {
 +        targetColumnIndexes[i] = schema.getColumnId(targets[i].getQualifiedName());
 +      }
 +
-       serde = new TextFieldSerializerDeserializer();
- 
 +      super.init();
 +      Arrays.sort(targetColumnIndexes);
 +      if (LOG.isDebugEnabled()) {
 +        LOG.debug("DelimitedTextFileScanner open:" + fragment.getPath() + "," + startOffset + "," + endOffset);
 +      }
 +
 +      if (startOffset > 0) {
 +        reader.readLine();  // skip first line;
 +      }
-     }
 +
-     public ByteBuf readLine() throws IOException {
-       ByteBuf buf = reader.readLine();
-       if (buf == null) {
-         return null;
-       } else {
-         recordCount++;
-       }
++      deserializer = getLineSerde().createDeserializer(schema, meta, targetColumnIndexes);
++      deserializer.init();
++    }
 +
-       return buf;
++    public TextLineSerDe getLineSerde() {
++      return DelimitedTextFile.getLineSerde(meta);
 +    }
 +
 +    @Override
 +    public float getProgress() {
 +      try {
 +        if (!reader.isReadable()) {
 +          return 1.0f;
 +        }
 +        long filePos = reader.getCompressedPosition();
 +        if (startOffset == filePos) {
 +          return 0.0f;
 +        } else {
 +          long readBytes = filePos - startOffset;
 +          long remainingBytes = Math.max(endOffset - filePos, 0);
 +          return Math.min(1.0f, (float) (readBytes) / (float) (readBytes + remainingBytes));
 +        }
 +      } catch (IOException e) {
 +        LOG.error(e.getMessage(), e);
 +        return 0.0f;
 +      }
 +    }
 +
 +    @Override
 +    public Tuple next() throws IOException {
++
++      if (!reader.isReadable()) {
++        return null;
++      }
++
++      if (targets.length == 0) {
++        return EmptyTuple.get();
++      }
++
++      VTuple tuple = new VTuple(schema.size());
++
 +      try {
-         if (!reader.isReadable()) return null;
 +
-         ByteBuf buf = readLine();
-         if (buf == null) return null;
++        // this loop will continue until one tuple is build or EOS (end of stream).
++        do {
 +
-         if (targets.length == 0) {
-           return EmptyTuple.get();
-         }
++          ByteBuf buf = reader.readLine();
++          if (buf == null) {
++            return null;
++          }
 +
-         VTuple tuple = new VTuple(schema.size());
-         fillTuple(schema, tuple, buf, targetColumnIndexes);
-         return tuple;
-       } catch (Throwable t) {
-         LOG.error("Tuple list current index: " + recordCount + " file offset:" + reader.getCompressedPosition(), t);
-         throw new IOException(t);
-       }
-     }
++          try {
 +
-     private void fillTuple(Schema schema, Tuple dst, ByteBuf lineBuf, int[] target) throws IOException {
-       int[] projection = target;
-       if (lineBuf == null || target == null || target.length == 0) {
-         return;
-       }
++            deserializer.deserialize(buf, tuple);
++            // if a line is read normaly, it exists this loop.
++            break;
 +
-       final int rowLength = lineBuf.readableBytes();
-       int start = 0, fieldLength = 0, end = 0;
++          } catch (TextLineParsingError tae) {
 +
-       //Projection
-       int currentTarget = 0;
-       int currentIndex = 0;
++            errorNum++;
 +
-       while (end != -1) {
-         end = lineBuf.forEachByte(start, rowLength - start, processor);
++            // suppress too many log prints, which probably cause performance degradation
++            if (errorNum < errorPrintOutMaxNum) {
++              LOG.warn("Ignore JSON Parse Error (" + errorNum + "): ", tae);
++            }
 +
-         if (end < 0) {
-           fieldLength = rowLength - start;
-         } else {
-           fieldLength = end - start;
-         }
++            // Only when the maximum error torrence limit is set (i.e., errorTorrenceMaxNum >= 0),
++            // it checks if the number of parsing error exceeds the max limit.
++            // Otherwise, it will ignore all parsing errors.
++            if (errorTorrenceMaxNum >= 0 && errorNum > errorTorrenceMaxNum) {
++              throw tae;
++            }
++            continue;
++          }
 +
-         if (projection.length > currentTarget && currentIndex == projection[currentTarget]) {
-           lineBuf.setIndex(start, start + fieldLength);
-           Datum datum = serde.deserialize(lineBuf, schema.getColumn(currentIndex), currentIndex, nullChars);
-           dst.put(currentIndex, datum);
-           currentTarget++;
-         }
++        } while (reader.isReadable()); // continue until EOS
 +
-         if (projection.length == currentTarget) {
-           break;
-         }
++        // recordCount means the number of actual read records. We increment the count here.
++        recordCount++;
 +
-         start = end + 1;
-         currentIndex++;
++        return tuple;
++
++      } catch (Throwable t) {
++        LOG.error(t);
++        throw new IOException(t);
 +      }
 +    }
 +
 +    @Override
 +    public void reset() throws IOException {
 +      init();
 +    }
 +
 +    @Override
 +    public void close() throws IOException {
 +      try {
-         if (nullChars != null) {
-           nullChars.release();
-           nullChars = null;
++        if (deserializer != null) {
++          deserializer.release();
 +        }
 +
 +        if (tableStats != null && reader != null) {
 +          tableStats.setReadBytes(reader.getReadBytes());  //Actual Processed Bytes. (decompressed bytes + overhead)
 +          tableStats.setNumRows(recordCount);
 +        }
 +        if (LOG.isDebugEnabled()) {
 +          LOG.debug("DelimitedTextFileScanner processed record:" + recordCount);
 +        }
 +      } finally {
 +        IOUtils.cleanup(LOG, reader);
 +        reader = null;
 +      }
 +    }
 +
 +    @Override
 +    public boolean isProjectable() {
 +      return true;
 +    }
 +
 +    @Override
 +    public boolean isSelectable() {
 +      return false;
 +    }
 +
 +    @Override
 +    public void setSearchCondition(Object expr) {
 +    }
 +
 +    @Override
 +    public boolean isSplittable() {
 +      return splittable;
 +    }
 +
 +    @Override
 +    public TableStats getInputStats() {
 +      if (tableStats != null && reader != null) {
 +        tableStats.setReadBytes(reader.getReadBytes());  //Actual Processed Bytes. (decompressed bytes + overhead)
 +        tableStats.setNumRows(recordCount);
 +        tableStats.setNumBytes(fragment.getLength());
 +      }
 +      return tableStats;
 +    }
 +  }
 +}


[08/29] tajo git commit: TAJO-1119: JDBC driver should support TIMESTAMP type. (jaehwa)

Posted by hj...@apache.org.
TAJO-1119: JDBC driver should support TIMESTAMP type. (jaehwa)


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

Branch: refs/heads/hbase_storage
Commit: 7d41c67ca94493e38c67e62663dc097358a18539
Parents: 4637f7f
Author: JaeHwa Jung <bl...@apache.org>
Authored: Thu Nov 27 23:19:31 2014 +0900
Committer: JaeHwa Jung <bl...@apache.org>
Committed: Thu Nov 27 23:19:31 2014 +0900

----------------------------------------------------------------------
 CHANGES                                         |   2 +
 .../org/apache/tajo/client/ResultSetUtil.java   |   8 +
 .../java/org/apache/tajo/jdbc/TestTajoJdbc.java | 181 ++++++++++++++-----
 .../dataset/TestTajoJdbc/table1/table1.tbl      |   5 +
 .../TestTajoJdbc/create_table_with_date_ddl.sql |  10 +
 .../TestTajoJdbc/testSortWithDateTime.result    |   7 +
 .../org/apache/tajo/jdbc/MetaDataTuple.java     |   4 +-
 7 files changed, 174 insertions(+), 43 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tajo/blob/7d41c67c/CHANGES
----------------------------------------------------------------------
diff --git a/CHANGES b/CHANGES
index 8bf0c1c..0d0677c 100644
--- a/CHANGES
+++ b/CHANGES
@@ -77,6 +77,8 @@ Release 0.9.1 - unreleased
 
   BUG FIXES
 
+    TAJO-1119: JDBC driver should support TIMESTAMP type. (jaehwa)
+
     TAJO-1166: S3 related storage causes compilation error in Hadoop 2.6.0-SNAPSHOT. (jaehwa)
 
     TAJO-1208: Failure of create table using textfile on hivemeta.

http://git-wip-us.apache.org/repos/asf/tajo/blob/7d41c67c/tajo-client/src/main/java/org/apache/tajo/client/ResultSetUtil.java
----------------------------------------------------------------------
diff --git a/tajo-client/src/main/java/org/apache/tajo/client/ResultSetUtil.java b/tajo-client/src/main/java/org/apache/tajo/client/ResultSetUtil.java
index 056eb2c..9211a1b 100644
--- a/tajo-client/src/main/java/org/apache/tajo/client/ResultSetUtil.java
+++ b/tajo-client/src/main/java/org/apache/tajo/client/ResultSetUtil.java
@@ -75,6 +75,10 @@ public class ResultSetUtil {
       return "character";
     case DATE:
       return "date";
+    case TIMESTAMP:
+      return "timestamp";
+    case TIME:
+      return "time";
     case VARCHAR:
       return "varchar";
     case TEXT:
@@ -103,7 +107,11 @@ public class ResultSetUtil {
     case NUMERIC:
       return Types.NUMERIC;
     case DATE:
+      return Types.DATE;
+    case TIMESTAMP:
       return Types.TIMESTAMP;
+    case TIME:
+      return Types.TIME;
     case VARCHAR:
       return Types.VARCHAR;
     case TEXT:

http://git-wip-us.apache.org/repos/asf/tajo/blob/7d41c67c/tajo-core/src/test/java/org/apache/tajo/jdbc/TestTajoJdbc.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/jdbc/TestTajoJdbc.java b/tajo-core/src/test/java/org/apache/tajo/jdbc/TestTajoJdbc.java
index 08535ef..a004baa 100644
--- a/tajo-core/src/test/java/org/apache/tajo/jdbc/TestTajoJdbc.java
+++ b/tajo-core/src/test/java/org/apache/tajo/jdbc/TestTajoJdbc.java
@@ -26,6 +26,7 @@ import org.apache.tajo.catalog.CatalogUtil;
 import org.apache.tajo.catalog.Column;
 import org.apache.tajo.catalog.TableDesc;
 import org.apache.tajo.client.QueryClient;
+import org.apache.tajo.conf.TajoConf;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -33,13 +34,11 @@ import org.junit.experimental.categories.Category;
 
 import java.net.InetSocketAddress;
 import java.sql.*;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
+import java.util.*;
 
 import static org.apache.tajo.TajoConstants.DEFAULT_DATABASE_NAME;
 import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
 
 @Category(IntegrationTest.class)
 public class TestTajoJdbc extends QueryTestCaseBase {
@@ -69,13 +68,13 @@ public class TestTajoJdbc extends QueryTestCaseBase {
   @Test(expected = SQLException.class)
   public void testGetConnection() throws SQLException {
     DriverManager.getConnection("jdbc:taju://" + tajoMasterAddress.getHostName() + ":" + tajoMasterAddress.getPort()
-        + "/default");
+      + "/default");
   }
 
   @Test
   public void testStatement() throws Exception {
     String connUri = buildConnectionUri(tajoMasterAddress.getHostName(), tajoMasterAddress.getPort(),
-        DEFAULT_DATABASE_NAME);
+      DEFAULT_DATABASE_NAME);
     Connection conn = DriverManager.getConnection(connUri);
     assertTrue(conn.isValid(100));
 
@@ -85,10 +84,10 @@ public class TestTajoJdbc extends QueryTestCaseBase {
       stmt = conn.createStatement();
 
       res = stmt.executeQuery("select l_returnflag, l_linestatus, count(*) as count_order from lineitem " +
-          "group by l_returnflag, l_linestatus order by l_returnflag, l_linestatus");
+        "group by l_returnflag, l_linestatus order by l_returnflag, l_linestatus");
 
       try {
-        Map<String,Integer> result = Maps.newHashMap();
+        Map<String, Integer> result = Maps.newHashMap();
         result.put("NO", 3);
         result.put("RF", 2);
 
@@ -108,10 +107,10 @@ public class TestTajoJdbc extends QueryTestCaseBase {
         res.close();
       }
     } finally {
-      if(res != null) {
+      if (res != null) {
         res.close();
       }
-      if(stmt != null) {
+      if (stmt != null) {
         stmt.close();
       }
     }
@@ -120,7 +119,7 @@ public class TestTajoJdbc extends QueryTestCaseBase {
   @Test
   public void testPreparedStatement() throws Exception {
     String connUri = buildConnectionUri(tajoMasterAddress.getHostName(), tajoMasterAddress.getPort(),
-        TajoConstants.DEFAULT_DATABASE_NAME);
+      TajoConstants.DEFAULT_DATABASE_NAME);
     Connection conn = DriverManager.getConnection(connUri);
     assertTrue(conn.isValid(100));
 
@@ -137,7 +136,7 @@ public class TestTajoJdbc extends QueryTestCaseBase {
       */
 
       String sql =
-          "select l_orderkey, l_quantity, l_returnflag from lineitem where l_quantity > ? and l_returnflag = ?";
+        "select l_orderkey, l_quantity, l_returnflag from lineitem where l_quantity > ? and l_returnflag = ?";
 
       stmt = conn.prepareStatement(sql);
 
@@ -155,9 +154,9 @@ public class TestTajoJdbc extends QueryTestCaseBase {
       try {
         int numRows = 0;
         String[] resultData = {"136.0N", "238.0N"};
-        while(res.next()) {
+        while (res.next()) {
           assertEquals(resultData[numRows],
-              ("" + res.getObject(1).toString() + res.getObject(2).toString() + res.getObject(3).toString()));
+            ("" + res.getObject(1).toString() + res.getObject(2).toString() + res.getObject(3).toString()));
           numRows++;
         }
         assertEquals(2, numRows);
@@ -179,9 +178,9 @@ public class TestTajoJdbc extends QueryTestCaseBase {
       try {
         int numRows = 0;
         String[] resultData = {"345.0R", "349.0R"};
-        while(res.next()) {
+        while (res.next()) {
           assertEquals(resultData[numRows],
-              ("" + res.getObject(1).toString() + res.getObject(2).toString() + res.getObject(3).toString()));
+            ("" + res.getObject(1).toString() + res.getObject(2).toString() + res.getObject(3).toString()));
           numRows++;
         }
         assertEquals(2, numRows);
@@ -189,10 +188,10 @@ public class TestTajoJdbc extends QueryTestCaseBase {
         res.close();
       }
     } finally {
-      if(res != null) {
+      if (res != null) {
         res.close();
       }
-      if(stmt != null) {
+      if (stmt != null) {
         stmt.close();
       }
     }
@@ -201,7 +200,7 @@ public class TestTajoJdbc extends QueryTestCaseBase {
   @Test
   public void testDatabaseMetaDataGetTable() throws Exception {
     String connUri = buildConnectionUri(tajoMasterAddress.getHostName(), tajoMasterAddress.getPort(),
-        TajoConstants.DEFAULT_DATABASE_NAME);
+      TajoConstants.DEFAULT_DATABASE_NAME);
     Connection conn = DriverManager.getConnection(connUri);
     assertTrue(conn.isValid(100));
 
@@ -219,12 +218,12 @@ public class TestTajoJdbc extends QueryTestCaseBase {
       Set<String> retrivedViaJavaAPI = new HashSet<String>(client.getTableList("default"));
 
       Set<String> retrievedViaJDBC = new HashSet<String>();
-      while(rs.next()) {
+      while (rs.next()) {
         retrievedViaJDBC.add(rs.getString("TABLE_NAME"));
       }
       assertEquals(retrievedViaJDBC, retrivedViaJavaAPI);
     } finally {
-      if(rs != null) {
+      if (rs != null) {
         rs.close();
       }
     }
@@ -236,7 +235,7 @@ public class TestTajoJdbc extends QueryTestCaseBase {
   @Test
   public void testDatabaseMetaDataGetColumns() throws Exception {
     String connUri = buildConnectionUri(tajoMasterAddress.getHostName(), tajoMasterAddress.getPort(),
-        TajoConstants.DEFAULT_DATABASE_NAME);
+      TajoConstants.DEFAULT_DATABASE_NAME);
     Connection conn = DriverManager.getConnection(connUri);
     assertTrue(conn.isValid(100));
 
@@ -258,7 +257,7 @@ public class TestTajoJdbc extends QueryTestCaseBase {
 
       List<Column> columns = tableDesc.getSchema().getColumns();
 
-      while(rs.next()) {
+      while (rs.next()) {
         assertEquals(tableName, rs.getString("TABLE_NAME"));
         assertEquals(columns.get(numColumns).getSimpleName(), rs.getString("COLUMN_NAME"));
         // TODO assert type
@@ -267,7 +266,7 @@ public class TestTajoJdbc extends QueryTestCaseBase {
 
       assertEquals(16, numColumns);
     } finally {
-      if(rs != null) {
+      if (rs != null) {
         rs.close();
       }
     }
@@ -280,24 +279,24 @@ public class TestTajoJdbc extends QueryTestCaseBase {
   @Test
   public void testMultipleConnections() throws Exception {
     String connUri = buildConnectionUri(tajoMasterAddress.getHostName(), tajoMasterAddress.getPort(),
-        TajoConstants.DEFAULT_DATABASE_NAME);
+      TajoConstants.DEFAULT_DATABASE_NAME);
 
     Connection[] conns = new Connection[2];
     conns[0] = DriverManager.getConnection(connUri);
     conns[1] = DriverManager.getConnection(connUri);
 
     try {
-      for(int i = 0; i < conns.length; i++) {
+      for (int i = 0; i < conns.length; i++) {
         Statement stmt = null;
         ResultSet res = null;
         try {
           stmt = conns[i].createStatement();
 
           res = stmt.executeQuery("select l_returnflag, l_linestatus, count(*) as count_order from lineitem " +
-              "group by l_returnflag, l_linestatus order by l_returnflag, l_linestatus");
+            "group by l_returnflag, l_linestatus order by l_returnflag, l_linestatus");
 
           try {
-            Map<String,Integer> result = Maps.newHashMap();
+            Map<String, Integer> result = Maps.newHashMap();
             result.put("NO", 3);
             result.put("RF", 2);
 
@@ -317,10 +316,10 @@ public class TestTajoJdbc extends QueryTestCaseBase {
             res.close();
           }
         } finally {
-          if(res != null) {
+          if (res != null) {
             res.close();
           }
-          if(stmt != null) {
+          if (stmt != null) {
             stmt.close();
           }
         }
@@ -338,24 +337,24 @@ public class TestTajoJdbc extends QueryTestCaseBase {
   @Test
   public void testMultipleConnectionsSequentialClose() throws Exception {
     String connUri = buildConnectionUri(tajoMasterAddress.getHostName(), tajoMasterAddress.getPort(),
-        DEFAULT_DATABASE_NAME);
+      DEFAULT_DATABASE_NAME);
 
     Connection[] conns = new Connection[2];
     conns[0] = DriverManager.getConnection(connUri);
     conns[1] = DriverManager.getConnection(connUri);
 
     try {
-      for(int i = 0; i < conns.length; i++) {
+      for (int i = 0; i < conns.length; i++) {
         Statement stmt = null;
         ResultSet res = null;
         try {
           stmt = conns[i].createStatement();
 
           res = stmt.executeQuery("select l_returnflag, l_linestatus, count(*) as count_order from lineitem " +
-              "group by l_returnflag, l_linestatus order by l_returnflag, l_linestatus");
+            "group by l_returnflag, l_linestatus order by l_returnflag, l_linestatus");
 
           try {
-            Map<String,Integer> result = Maps.newHashMap();
+            Map<String, Integer> result = Maps.newHashMap();
             result.put("NO", 3);
             result.put("RF", 2);
 
@@ -375,22 +374,22 @@ public class TestTajoJdbc extends QueryTestCaseBase {
             res.close();
           }
         } finally {
-          if(res != null) {
+          if (res != null) {
             res.close();
           }
-          if(stmt != null) {
+          if (stmt != null) {
             stmt.close();
           }
           conns[i].close();
         }
       }
     } finally {
-      if(!conns[0].isClosed()) {
+      if (!conns[0].isClosed()) {
         assertTrue(conns[0].isValid(100));
         conns[0].close();
         assertFalse(conns[0].isValid(100));
       }
-      if(!conns[1].isClosed()) {
+      if (!conns[1].isClosed()) {
         assertTrue(conns[1].isValid(100));
         conns[1].close();
         assertFalse(conns[1].isValid(100));
@@ -405,7 +404,7 @@ public class TestTajoJdbc extends QueryTestCaseBase {
     assertFalse(TajoStatement.isSetVariableQuery("--SET JOIN_TASK_INPUT_SIZE 123"));
 
     String connUri = buildConnectionUri(tajoMasterAddress.getHostName(), tajoMasterAddress.getPort(),
-        DEFAULT_DATABASE_NAME);
+      DEFAULT_DATABASE_NAME);
 
     Connection conn = DriverManager.getConnection(connUri);
 
@@ -419,7 +418,7 @@ public class TestTajoJdbc extends QueryTestCaseBase {
       assertNotNull(rsmd);
       assertEquals(0, rsmd.getColumnCount());
 
-      QueryClient connTajoClient = ((JdbcConnection)stmt.getConnection()).getQueryClient();
+      QueryClient connTajoClient = ((JdbcConnection) stmt.getConnection()).getQueryClient();
       Map<String, String> variables = connTajoClient.getAllSessionVariables();
       String value = variables.get("JOIN_TASK_INPUT_SIZE");
       assertNotNull(value);
@@ -447,7 +446,7 @@ public class TestTajoJdbc extends QueryTestCaseBase {
   @Test
   public void testSetPreparedStatement() throws Exception {
     String connUri = buildConnectionUri(tajoMasterAddress.getHostName(), tajoMasterAddress.getPort(),
-        DEFAULT_DATABASE_NAME);
+      DEFAULT_DATABASE_NAME);
 
     Connection conn = DriverManager.getConnection(connUri);
 
@@ -461,7 +460,7 @@ public class TestTajoJdbc extends QueryTestCaseBase {
       assertNotNull(rsmd);
       assertEquals(0, rsmd.getColumnCount());
 
-      QueryClient connTajoClient = ((JdbcConnection)stmt.getConnection()).getQueryClient();
+      QueryClient connTajoClient = ((JdbcConnection) stmt.getConnection()).getQueryClient();
       Map<String, String> variables = connTajoClient.getAllSessionVariables();
       String value = variables.get("JOIN_TASK_INPUT_SIZE");
       assertNotNull(value);
@@ -487,4 +486,102 @@ public class TestTajoJdbc extends QueryTestCaseBase {
       }
     }
   }
+
+  @Test
+  public void testCreateTableWithDateAndTimestamp() throws Exception {
+    String tableName = CatalogUtil.normalizeIdentifier("testCreateTableWithDateAndTimestamp");
+
+    int result;
+    Statement stmt = null;
+    ResultSet res = null;
+
+    try {
+      String connUri = buildConnectionUri(tajoMasterAddress.getHostName(), tajoMasterAddress.getPort(),
+        DEFAULT_DATABASE_NAME);
+      Connection conn = DriverManager.getConnection(connUri);
+      assertTrue(conn.isValid(100));
+
+      stmt = conn.createStatement();
+      result = stmt.executeUpdate("create table " + tableName + " (id int, name text, score double"
+        + ", register_date timestamp, update_date date, send_date time)");
+      assertEquals(result, 1);
+
+      res = stmt.executeQuery("select * from " + tableName);
+      assertFalse(res.next());
+
+      ResultSetMetaData rsmd = res.getMetaData();
+      assertNotNull(rsmd);
+      assertEquals(6, rsmd.getColumnCount());
+
+      assertEquals("id", rsmd.getColumnName(1));
+      assertEquals("name", rsmd.getColumnName(2));
+      assertEquals("score", rsmd.getColumnName(3));
+      assertEquals("register_date", rsmd.getColumnName(4));
+      assertEquals("update_date", rsmd.getColumnName(5));
+      assertEquals("send_date", rsmd.getColumnName(6));
+
+      assertEquals("integer", rsmd.getColumnTypeName(1));
+      assertEquals("varchar", rsmd.getColumnTypeName(2));
+      assertEquals("float8", rsmd.getColumnTypeName(3));
+      assertEquals("timestamp", rsmd.getColumnTypeName(4));
+      assertEquals("date", rsmd.getColumnTypeName(5));
+      assertEquals("time", rsmd.getColumnTypeName(6));
+
+    } finally {
+      cleanupQuery(res);
+      if (stmt != null) {
+        stmt.close();
+      }
+    }
+  }
+
+  @Test
+  public void testSortWithDateTime() throws Exception {
+    Statement stmt = null;
+    ResultSet res = null;
+    int result;
+
+    // skip this test if catalog uses HCatalogStore.
+    // It is because HCatalogStore does not support Time data type.
+    TimeZone oldTimeZone = TajoConf.setCurrentTimeZone(TimeZone.getTimeZone("UTC"));
+    TimeZone systemOldTimeZone = TimeZone.getDefault();
+    TimeZone.setDefault(TimeZone.getTimeZone("UTC"));
+
+    try {
+      if (!testingCluster.isHCatalogStoreRunning()) {
+        executeDDL("create_table_with_date_ddl.sql", "table1");
+
+        String connUri = buildConnectionUri(tajoMasterAddress.getHostName(),
+          tajoMasterAddress.getPort(), "TestTajoJdbc");
+
+        Connection conn = DriverManager.getConnection(connUri);
+        assertTrue(conn.isValid(100));
+
+        stmt = conn.createStatement();
+        res = stmt.executeQuery("select col1, col2, col3 from table1 order by col1, col2, col3");
+
+        ResultSetMetaData rsmd = res.getMetaData();
+        assertNotNull(rsmd);
+        assertEquals(3, rsmd.getColumnCount());
+
+        assertEquals("timestamp", rsmd.getColumnTypeName(1));
+        assertEquals("date", rsmd.getColumnTypeName(2));
+        assertEquals("time", rsmd.getColumnTypeName(3));
+
+        assertResultSet(res);
+
+        result = stmt.executeUpdate("drop table table1");
+        assertEquals(result, 1);
+
+      }
+    } finally {
+      TajoConf.setCurrentTimeZone(oldTimeZone);
+      TimeZone.setDefault(systemOldTimeZone);
+
+      cleanupQuery(res);
+      if (stmt != null) {
+        stmt.close();
+      }
+    }
+  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/7d41c67c/tajo-core/src/test/resources/dataset/TestTajoJdbc/table1/table1.tbl
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/resources/dataset/TestTajoJdbc/table1/table1.tbl b/tajo-core/src/test/resources/dataset/TestTajoJdbc/table1/table1.tbl
new file mode 100644
index 0000000..52fa2fe
--- /dev/null
+++ b/tajo-core/src/test/resources/dataset/TestTajoJdbc/table1/table1.tbl
@@ -0,0 +1,5 @@
+1997-11-09 20:34:56|1996-04-12|15:34:56
+1997-11-09 20:34:56|1996-03-13|19:34:56
+1993-11-09 20:34:56|1997-01-28|08:34:56
+1995-11-09 20:34:56|1994-02-02|17:34:56
+1995-11-09 20:34:56|1993-11-09|20:34:56

http://git-wip-us.apache.org/repos/asf/tajo/blob/7d41c67c/tajo-core/src/test/resources/queries/TestTajoJdbc/create_table_with_date_ddl.sql
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/resources/queries/TestTajoJdbc/create_table_with_date_ddl.sql b/tajo-core/src/test/resources/queries/TestTajoJdbc/create_table_with_date_ddl.sql
new file mode 100644
index 0000000..846cbb6
--- /dev/null
+++ b/tajo-core/src/test/resources/queries/TestTajoJdbc/create_table_with_date_ddl.sql
@@ -0,0 +1,10 @@
+-- Sort Table
+-- It is used in TestSortQuery::testSortWithDate
+
+create external table table1 (
+  col1 timestamp,
+	col2 date,
+	col3 time
+) using csv
+with ('csvfile.delimiter'='|', 'csvfile.null'='NULL')
+location ${table.path};
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/7d41c67c/tajo-core/src/test/resources/results/TestTajoJdbc/testSortWithDateTime.result
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/resources/results/TestTajoJdbc/testSortWithDateTime.result b/tajo-core/src/test/resources/results/TestTajoJdbc/testSortWithDateTime.result
new file mode 100644
index 0000000..118909c
--- /dev/null
+++ b/tajo-core/src/test/resources/results/TestTajoJdbc/testSortWithDateTime.result
@@ -0,0 +1,7 @@
+col1,col2,col3
+-------------------------------
+1993-11-09 20:34:56,1997-01-28,08:34:56
+1995-11-09 20:34:56,1993-11-09,20:34:56
+1995-11-09 20:34:56,1994-02-02,17:34:56
+1997-11-09 20:34:56,1996-03-13,19:34:56
+1997-11-09 20:34:56,1996-04-12,15:34:56
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/7d41c67c/tajo-jdbc/src/main/java/org/apache/tajo/jdbc/MetaDataTuple.java
----------------------------------------------------------------------
diff --git a/tajo-jdbc/src/main/java/org/apache/tajo/jdbc/MetaDataTuple.java b/tajo-jdbc/src/main/java/org/apache/tajo/jdbc/MetaDataTuple.java
index 6c8ef5d..30fc9ee 100644
--- a/tajo-jdbc/src/main/java/org/apache/tajo/jdbc/MetaDataTuple.java
+++ b/tajo-jdbc/src/main/java/org/apache/tajo/jdbc/MetaDataTuple.java
@@ -1,4 +1,4 @@
-package org.apache.tajo.jdbc; /**
+/**
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -16,6 +16,8 @@ package org.apache.tajo.jdbc; /**
  * limitations under the License.
  */
 
+package org.apache.tajo.jdbc;
+
 import org.apache.tajo.datum.Datum;
 import org.apache.tajo.datum.IntervalDatum;
 import org.apache.tajo.datum.NullDatum;


[04/29] tajo git commit: TAJO-1053: ADD PARTITIONS for HCatalogStore. (jaehwa)

Posted by hj...@apache.org.
TAJO-1053: ADD PARTITIONS for HCatalogStore. (jaehwa)


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

Branch: refs/heads/hbase_storage
Commit: 3ae44b1d2a1cf49123eb1d1c30f081a8a8d0e7fb
Parents: 3d485ec
Author: JaeHwa Jung <bl...@apache.org>
Authored: Thu Nov 27 19:27:10 2014 +0900
Committer: JaeHwa Jung <bl...@apache.org>
Committed: Thu Nov 27 19:27:10 2014 +0900

----------------------------------------------------------------------
 CHANGES                                            |  2 ++
 tajo-docs/src/main/sphinx/hcatalog_integration.rst | 13 +++++++++++++
 2 files changed, 15 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tajo/blob/3ae44b1d/CHANGES
----------------------------------------------------------------------
diff --git a/CHANGES b/CHANGES
index 1657c20..f2141a6 100644
--- a/CHANGES
+++ b/CHANGES
@@ -13,6 +13,8 @@ Release 0.9.1 - unreleased
 
   IMPROVEMENT
 
+    TAJO-1053: ADD PARTITIONS for HCatalogStore. (jaehwa)
+
     TAJO-1195: Remove unused CachedDNSResolver Class. (DaeMyung Kang via jaehwa)
 
     TAJO-1184: Upgrade netty-buffer to 4.0.24.Final. (jinho)

http://git-wip-us.apache.org/repos/asf/tajo/blob/3ae44b1d/tajo-docs/src/main/sphinx/hcatalog_integration.rst
----------------------------------------------------------------------
diff --git a/tajo-docs/src/main/sphinx/hcatalog_integration.rst b/tajo-docs/src/main/sphinx/hcatalog_integration.rst
index 7337346..d81975d 100644
--- a/tajo-docs/src/main/sphinx/hcatalog_integration.rst
+++ b/tajo-docs/src/main/sphinx/hcatalog_integration.rst
@@ -37,3 +37,16 @@ Finally, you should specify HCatalogStore as Tajo catalog driver class in ``conf
     <name>tajo.catalog.store.class</name>
     <value>org.apache.tajo.catalog.store.HCatalogStore</value>
   </property>
+
+.. note::
+
+  Hive stores a list of partitions for each table in its metastore. If new partitions are
+  directly added to HDFS, HiveMetastore will not able aware of these partitions unless the user
+  ``ALTER TABLE table_name ADD PARTITION`` commands on each of the newly added partitions or
+  ``MSCK REPAIR TABLE  table_name`` command.
+
+  But current tajo doesn't provide ``ADD PARTITION`` command and hive doesn't provide an api for
+  responding to ``MSK REPAIR TABLE`` command. Thus, if you insert data to hive partitioned
+  table and you want to scan the updated partitions through Tajo, you must run following command on hive ::
+
+  $ MSCK REPAIR TABLE [table_name];


[13/29] tajo git commit: TAJO-1183: Keep command execution even with errors. (Jaewoong Jung via hyunsik)

Posted by hj...@apache.org.
TAJO-1183: Keep command execution even with errors. (Jaewoong Jung via hyunsik)

Closes #266


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

Branch: refs/heads/hbase_storage
Commit: 5066ac3e80da390ff0f5733d345cbc63ec252843
Parents: 1cdbe46
Author: Hyunsik Choi <hy...@apache.org>
Authored: Wed Dec 3 15:30:16 2014 +0900
Committer: Hyunsik Choi <hy...@apache.org>
Committed: Wed Dec 3 15:32:16 2014 +0900

----------------------------------------------------------------------
 CHANGES                                                |  3 +++
 .../main/java/org/apache/tajo/cli/tsql/TajoCli.java    | 13 +++++++------
 2 files changed, 10 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tajo/blob/5066ac3e/CHANGES
----------------------------------------------------------------------
diff --git a/CHANGES b/CHANGES
index 188e024..c03b72b 100644
--- a/CHANGES
+++ b/CHANGES
@@ -80,6 +80,9 @@ Release 0.9.1 - unreleased
 
   BUG FIXES
 
+    TAJO-1183: Keep command execution even with errors. (Jaewoong Jung via 
+    hyunsik)
+
     TAJO-1190: INSERT INTO to partition tables may cause NPE. (hyunsik)
 
     TAJO-1211: Staging directory for CTAS and INSERT should be in 

http://git-wip-us.apache.org/repos/asf/tajo/blob/5066ac3e/tajo-client/src/main/java/org/apache/tajo/cli/tsql/TajoCli.java
----------------------------------------------------------------------
diff --git a/tajo-client/src/main/java/org/apache/tajo/cli/tsql/TajoCli.java b/tajo-client/src/main/java/org/apache/tajo/cli/tsql/TajoCli.java
index e96017b..d4886cf 100644
--- a/tajo-client/src/main/java/org/apache/tajo/cli/tsql/TajoCli.java
+++ b/tajo-client/src/main/java/org/apache/tajo/cli/tsql/TajoCli.java
@@ -390,7 +390,7 @@ public class TajoCli {
   public int runShell() throws Exception {
     String line;
     String currentPrompt = context.getCurrentDatabase();
-    int exitCode = 0;
+    int exitCode;
 
     sout.write("Try \\? for help.\n");
 
@@ -416,7 +416,8 @@ public class TajoCli {
           exitCode = executeParsedResults(parsedResults);
           currentPrompt = updatePrompt(parser.getState());
 
-          if (exitCode != 0 && context.getBool(SessionVars.ON_ERROR_STOP)) {
+          // if at least one failed
+          if (exitCode != 0) {
             return exitCode;
           }
         }
@@ -430,11 +431,11 @@ public class TajoCli {
       
       throw e;
     }
-    return exitCode;
+    return 0;
   }
 
   private int executeParsedResults(Collection<ParsedResult> parsedResults) throws Exception {
-    int exitCode = 0;
+    int exitCode;
     for (ParsedResult parsedResult : parsedResults) {
       if (parsedResult.getType() == META) {
         exitCode = executeMetaCommand(parsedResult.getStatement());
@@ -442,12 +443,12 @@ public class TajoCli {
         exitCode = executeQuery(parsedResult.getStatement());
       }
 
-      if (exitCode != 0) {
+      if (exitCode != 0 && context.getBool(SessionVars.ON_ERROR_STOP)) {
         return exitCode;
       }
     }
 
-    return exitCode;
+    return 0;
   }
 
   public int executeMetaCommand(String line) throws Exception {


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

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

Conflicts:
	tajo-plan/src/main/java/org/apache/tajo/plan/util/PlannerUtil.java


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

Branch: refs/heads/hbase_storage
Commit: 1526b7d1073c24c653ce17b93df3290c65ffe4e2
Parents: 0073ef2 ab2efce
Author: HyoungJun Kim <ba...@babokim-MacBook-Pro.local>
Authored: Fri Dec 5 17:20:19 2014 +0900
Committer: HyoungJun Kim <ba...@babokim-MacBook-Pro.local>
Committed: Fri Dec 5 17:20:19 2014 +0900

----------------------------------------------------------------------
 CHANGES                                         |   7 +-
 .../org/apache/tajo/client/TestTajoClient.java  |  37 +++-
 .../tajo/engine/eval/TestIntervalType.java      | 172 ++++++++---------
 .../engine/function/TestDateTimeFunctions.java  | 184 +++++++++----------
 .../org/apache/tajo/plan/LogicalPlanner.java    |  20 +-
 .../org/apache/tajo/plan/util/PlannerUtil.java  |  17 ++
 6 files changed, 248 insertions(+), 189 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tajo/blob/1526b7d1/CHANGES
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/tajo/blob/1526b7d1/tajo-plan/src/main/java/org/apache/tajo/plan/util/PlannerUtil.java
----------------------------------------------------------------------
diff --cc tajo-plan/src/main/java/org/apache/tajo/plan/util/PlannerUtil.java
index b5d2db3,c55c203..4e61de4
--- a/tajo-plan/src/main/java/org/apache/tajo/plan/util/PlannerUtil.java
+++ b/tajo-plan/src/main/java/org/apache/tajo/plan/util/PlannerUtil.java
@@@ -21,11 -21,15 +21,13 @@@ package org.apache.tajo.plan.util
  import com.google.common.base.Preconditions;
  import com.google.common.collect.Lists;
  import com.google.common.collect.Sets;
+ import org.apache.tajo.OverridableConf;
+ import org.apache.tajo.SessionVars;
  import org.apache.tajo.algebra.*;
  import org.apache.tajo.annotation.Nullable;
 -import org.apache.tajo.catalog.Column;
 -import org.apache.tajo.catalog.Schema;
 -import org.apache.tajo.catalog.SchemaUtil;
 -import org.apache.tajo.catalog.SortSpec;
 +import org.apache.tajo.catalog.*;
  import org.apache.tajo.catalog.proto.CatalogProtos;
 +import org.apache.tajo.catalog.proto.CatalogProtos.StoreType;
  import org.apache.tajo.common.TajoDataTypes.DataType;
  import org.apache.tajo.plan.*;
  import org.apache.tajo.plan.expr.*;
@@@ -33,11 -37,15 +35,16 @@@ import org.apache.tajo.plan.logical.*
  import org.apache.tajo.plan.visitor.BasicLogicalPlanVisitor;
  import org.apache.tajo.plan.visitor.ExplainLogicalPlanVisitor;
  import org.apache.tajo.plan.visitor.SimpleAlgebraVisitor;
+ import org.apache.tajo.storage.StorageConstants;
+ import org.apache.tajo.util.KeyValueSet;
  import org.apache.tajo.util.TUtil;
  
 +import java.io.IOException;
  import java.util.*;
  
+ import static org.apache.tajo.catalog.proto.CatalogProtos.StoreType.CSV;
+ import static org.apache.tajo.catalog.proto.CatalogProtos.StoreType.TEXTFILE;
+ 
  public class PlannerUtil {
  
    public static boolean checkIfDDLPlan(LogicalNode node) {
@@@ -776,108 -784,13 +783,118 @@@
      return explains.toString();
    }
  
+   public static void applySessionToTableProperties(OverridableConf sessionVars,
+                                                    CatalogProtos.StoreType storeType,
+                                                    KeyValueSet tableProperties) {
+     if (storeType == CSV || storeType == TEXTFILE) {
+       if (sessionVars.containsKey(SessionVars.NULL_CHAR)) {
+         tableProperties.set(StorageConstants.TEXT_NULL, sessionVars.get(SessionVars.NULL_CHAR));
+       }
+     }
+   }
++
 +  public static boolean isFileStorageType(String storageType) {
 +    if (storageType.equalsIgnoreCase("hbase")) {
 +      return false;
 +    } else {
 +      return true;
 +    }
 +  }
 +
 +  public static boolean isFileStorageType(StoreType storageType) {
 +    if (storageType== StoreType.HBASE) {
 +      return false;
 +    } else {
 +      return true;
 +    }
 +  }
 +
 +  public static StoreType getStoreType(LogicalPlan plan) {
 +    LogicalRootNode rootNode = plan.getRootBlock().getRoot();
 +    NodeType nodeType = rootNode.getChild().getType();
 +    if (nodeType == NodeType.CREATE_TABLE) {
 +      return ((CreateTableNode)rootNode.getChild()).getStorageType();
 +    } else if (nodeType == NodeType.INSERT) {
 +      return ((InsertNode)rootNode.getChild()).getStorageType();
 +    } else {
 +      return null;
 +    }
 +  }
 +
 +  public static String getStoreTableName(LogicalPlan plan) {
 +    LogicalRootNode rootNode = plan.getRootBlock().getRoot();
 +    NodeType nodeType = rootNode.getChild().getType();
 +    if (nodeType == NodeType.CREATE_TABLE) {
 +      return ((CreateTableNode)rootNode.getChild()).getTableName();
 +    } else if (nodeType == NodeType.INSERT) {
 +      return ((InsertNode)rootNode.getChild()).getTableName();
 +    } else {
 +      return null;
 +    }
 +  }
 +
 +  public static TableDesc getTableDesc(CatalogService catalog, LogicalNode node) throws IOException {
 +    if (node.getType() == NodeType.CREATE_TABLE) {
 +      return createTableDesc((CreateTableNode)node);
 +    }
 +    String tableName = null;
 +    InsertNode insertNode = null;
 +    if (node.getType() == NodeType.INSERT) {
 +      insertNode = (InsertNode)node;
 +      tableName = insertNode.getTableName();
 +    } else {
 +      return null;
 +    }
 +
 +    if (tableName != null) {
 +      String[] tableTokens = tableName.split("\\.");
 +      if (tableTokens.length >= 2) {
 +        if (catalog.existsTable(tableTokens[0], tableTokens[1])) {
 +          return catalog.getTableDesc(tableTokens[0], tableTokens[1]);
 +        }
 +      }
 +    } else {
 +      if (insertNode.getPath() != null) {
 +        //insert ... location
 +        return createTableDesc(insertNode);
 +      }
 +    }
 +    return null;
 +  }
 +
 +  private static TableDesc createTableDesc(CreateTableNode createTableNode) {
 +    TableMeta meta = new TableMeta(createTableNode.getStorageType(), createTableNode.getOptions());
 +
 +    TableDesc tableDescTobeCreated =
 +        new TableDesc(
 +            createTableNode.getTableName(),
 +            createTableNode.getTableSchema(),
 +            meta,
 +            createTableNode.getPath() != null ? createTableNode.getPath().toUri() : null);
 +
 +    tableDescTobeCreated.setExternal(createTableNode.isExternal());
 +
 +    if (createTableNode.hasPartition()) {
 +      tableDescTobeCreated.setPartitionMethod(createTableNode.getPartitionMethod());
 +    }
 +
 +    return tableDescTobeCreated;
 +  }
 +
 +  private static TableDesc createTableDesc(InsertNode insertNode) {
 +    TableMeta meta = new TableMeta(insertNode.getStorageType(), insertNode.getOptions());
 +
 +    TableDesc tableDescTobeCreated =
 +        new TableDesc(
 +            insertNode.getTableName(),
 +            insertNode.getTableSchema(),
 +            meta,
 +            insertNode.getPath() != null ? insertNode.getPath().toUri() : null);
 +
 +    if (insertNode.hasPartition()) {
 +      tableDescTobeCreated.setPartitionMethod(insertNode.getPartitionMethod());
 +    }
 +
 +    return tableDescTobeCreated;
 +  }
  }


[10/29] tajo git commit: TAJO-1210: ByteBufLineReader does not handle the end of file, if newline is not appeared. (jinho)

Posted by hj...@apache.org.
TAJO-1210: ByteBufLineReader does not handle the end of file, if newline is not appeared. (jinho)

Closes #272


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

Branch: refs/heads/hbase_storage
Commit: bf68b770e6abbb4c63d696e264a348bb1ddb5982
Parents: cd38dff
Author: jhkim <jh...@apache.org>
Authored: Mon Dec 1 11:31:23 2014 +0900
Committer: jhkim <jh...@apache.org>
Committed: Mon Dec 1 11:31:23 2014 +0900

----------------------------------------------------------------------
 CHANGES                                         |  3 ++
 tajo-storage/pom.xml                            |  3 +-
 .../tajo/storage/text/ByteBufLineReader.java    | 22 ++++++++++++--
 .../org/apache/tajo/storage/TestLineReader.java | 32 +++++++++++++++++++-
 .../org/apache/tajo/storage/TestStorages.java   |  2 +-
 .../apache/tajo/storage/avro/TestAvroUtil.java  |  2 +-
 .../src/test/resources/dataset/testLineText.txt |  2 ++
 .../resources/dataset/testVariousTypes.avsc     | 20 ++++++++++++
 .../src/test/resources/testVariousTypes.avsc    | 20 ------------
 9 files changed, 78 insertions(+), 28 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tajo/blob/bf68b770/CHANGES
----------------------------------------------------------------------
diff --git a/CHANGES b/CHANGES
index 025ae88..33983bc 100644
--- a/CHANGES
+++ b/CHANGES
@@ -79,6 +79,9 @@ Release 0.9.1 - unreleased
 
   BUG FIXES
 
+    TAJO-1210: ByteBufLineReader does not handle the end of file, 
+    if newline is not appeared. (jinho)
+
     TAJO-1119: JDBC driver should support TIMESTAMP type. (jaehwa)
 
     TAJO-1166: S3 related storage causes compilation error in Hadoop 2.6.0-SNAPSHOT. (jaehwa)

http://git-wip-us.apache.org/repos/asf/tajo/blob/bf68b770/tajo-storage/pom.xml
----------------------------------------------------------------------
diff --git a/tajo-storage/pom.xml b/tajo-storage/pom.xml
index ef26a32..7ede2e1 100644
--- a/tajo-storage/pom.xml
+++ b/tajo-storage/pom.xml
@@ -71,8 +71,7 @@
         </executions>
         <configuration>
           <excludes>
-            <exclude>src/test/resources/testVariousTypes.avsc</exclude>
-            <exclude>src/test/resources/dataset/TestJsonSerDe/*.json</exclude>
+            <exclude>src/test/resources/dataset/**</exclude>
           </excludes>
         </configuration>
       </plugin>

http://git-wip-us.apache.org/repos/asf/tajo/blob/bf68b770/tajo-storage/src/main/java/org/apache/tajo/storage/text/ByteBufLineReader.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/text/ByteBufLineReader.java b/tajo-storage/src/main/java/org/apache/tajo/storage/text/ByteBufLineReader.java
index 1448885..86319e1 100644
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/text/ByteBufLineReader.java
+++ b/tajo-storage/src/main/java/org/apache/tajo/storage/text/ByteBufLineReader.java
@@ -32,6 +32,7 @@ public class ByteBufLineReader implements Closeable {
 
   private int bufferSize;
   private long readBytes;
+  private boolean eof = false;
   private ByteBuf buffer;
   private final ByteBufInputChannel channel;
   private final AtomicInteger tempReadBytes = new AtomicInteger();
@@ -92,6 +93,10 @@ public class ByteBufLineReader implements Closeable {
       for (; ; ) {
         int localReadBytes = buffer.writeBytes(channel, bufferSize - readBytes);
         if (localReadBytes < 0) {
+          if (tailBytes == readBytes) {
+            // no more bytes are in the channel
+            eof = true;
+          }
           break;
         }
         readBytes += localReadBytes;
@@ -101,7 +106,9 @@ public class ByteBufLineReader implements Closeable {
       }
       this.readBytes += (readBytes - tailBytes);
       release = false;
-      this.buffer.readerIndex(this.buffer.readerIndex() + tailBytes); //skip past buffer (tail)
+      if (!eof) {
+        this.buffer.readerIndex(this.buffer.readerIndex() + tailBytes); //skip past buffer (tail)
+      }
     } finally {
       if (release) {
         buffer.release();
@@ -113,6 +120,8 @@ public class ByteBufLineReader implements Closeable {
    * Read a line terminated by one of CR, LF, or CRLF.
    */
   public ByteBuf readLineBuf(AtomicInteger reads) throws IOException {
+    if(eof) return null;
+
     int startIndex = buffer.readerIndex();
     int readBytes;
     int readable;
@@ -127,14 +136,21 @@ public class ByteBufLineReader implements Closeable {
         if (!buffer.isReadable()) {
           return null;
         } else {
-          startIndex = 0; // reset the line start position
+          if (!eof) startIndex = 0; // reset the line start position
+          else startIndex = buffer.readerIndex();
         }
         readable = buffer.readableBytes();
       }
 
       int endIndex = buffer.forEachByte(buffer.readerIndex(), readable, processor);
       if (endIndex < 0) {
-        buffer.readerIndex(buffer.writerIndex());
+        //does not appeared terminating newline
+        buffer.readerIndex(buffer.writerIndex()); // set to end buffer
+        if(eof){
+          readBytes = buffer.readerIndex() - startIndex;
+          newlineLength = 0;
+          break loop;
+        }
       } else {
         buffer.readerIndex(endIndex + 1);
         readBytes = buffer.readerIndex() - startIndex;

http://git-wip-us.apache.org/repos/asf/tajo/blob/bf68b770/tajo-storage/src/test/java/org/apache/tajo/storage/TestLineReader.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/test/java/org/apache/tajo/storage/TestLineReader.java b/tajo-storage/src/test/java/org/apache/tajo/storage/TestLineReader.java
index ef6efdf..4512d00 100644
--- a/tajo-storage/src/test/java/org/apache/tajo/storage/TestLineReader.java
+++ b/tajo-storage/src/test/java/org/apache/tajo/storage/TestLineReader.java
@@ -34,11 +34,14 @@ import org.apache.tajo.datum.DatumFactory;
 import org.apache.tajo.datum.NullDatum;
 import org.apache.tajo.storage.fragment.FileFragment;
 import org.apache.tajo.storage.text.ByteBufLineReader;
-import org.apache.tajo.storage.text.DelimitedTextFile;
 import org.apache.tajo.storage.text.DelimitedLineReader;
+import org.apache.tajo.storage.text.DelimitedTextFile;
 import org.apache.tajo.util.CommonTestingUtil;
+import org.apache.tajo.util.FileUtil;
 import org.junit.Test;
 
+import java.io.File;
+import java.io.FileInputStream;
 import java.io.IOException;
 import java.util.concurrent.atomic.AtomicInteger;
 
@@ -160,4 +163,31 @@ public class TestLineReader {
     assertEquals(tupleNum, i);
 
   }
+
+  @Test
+  public void testByteBufLineReaderWithoutTerminating() throws IOException {
+    String path = FileUtil.getResourcePath("dataset/testLineText.txt").getFile();
+    File file = new File(path);
+    String data = FileUtil.readTextFile(file);
+
+    ByteBufInputChannel channel = new ByteBufInputChannel(new FileInputStream(file));
+
+    assertEquals(file.length(), channel.available());
+    ByteBufLineReader reader = new ByteBufLineReader(channel);
+    assertEquals(file.length(), reader.available());
+
+    long totalRead = 0;
+    int i = 0;
+    AtomicInteger bytes = new AtomicInteger();
+    for(;;){
+      ByteBuf buf = reader.readLineBuf(bytes);
+      if(buf == null) break;
+      totalRead += bytes.get();
+      i++;
+    }
+    IOUtils.cleanup(null, reader);
+    assertEquals(file.length(), totalRead);
+    assertEquals(file.length(), reader.readBytes());
+    assertEquals(data.split("\n").length, i);
+  }
 }

http://git-wip-us.apache.org/repos/asf/tajo/blob/bf68b770/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
index c581926..bd1a1f9 100644
--- a/tajo-storage/src/test/java/org/apache/tajo/storage/TestStorages.java
+++ b/tajo-storage/src/test/java/org/apache/tajo/storage/TestStorages.java
@@ -320,7 +320,7 @@ public class TestStorages {
     TableMeta meta = CatalogUtil.newTableMeta(storeType, options);
     meta.setOptions(CatalogUtil.newPhysicalProperties(storeType));
     if (storeType == StoreType.AVRO) {
-      String path = FileUtil.getResourcePath("testVariousTypes.avsc").toString();
+      String path = FileUtil.getResourcePath("dataset/testVariousTypes.avsc").toString();
       meta.putOption(StorageConstants.AVRO_SCHEMA_URL, path);
     }
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/bf68b770/tajo-storage/src/test/java/org/apache/tajo/storage/avro/TestAvroUtil.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/test/java/org/apache/tajo/storage/avro/TestAvroUtil.java b/tajo-storage/src/test/java/org/apache/tajo/storage/avro/TestAvroUtil.java
index 6186e9e..a79e8ab 100644
--- a/tajo-storage/src/test/java/org/apache/tajo/storage/avro/TestAvroUtil.java
+++ b/tajo-storage/src/test/java/org/apache/tajo/storage/avro/TestAvroUtil.java
@@ -49,7 +49,7 @@ public class TestAvroUtil {
 
   @Before
   public void setUp() throws Exception {
-    schemaUrl = FileUtil.getResourcePath("testVariousTypes.avsc");
+    schemaUrl = FileUtil.getResourcePath("dataset/testVariousTypes.avsc");
     assertNotNull(schemaUrl);
 
     File file = new File(schemaUrl.getPath());

http://git-wip-us.apache.org/repos/asf/tajo/blob/bf68b770/tajo-storage/src/test/resources/dataset/testLineText.txt
----------------------------------------------------------------------
diff --git a/tajo-storage/src/test/resources/dataset/testLineText.txt b/tajo-storage/src/test/resources/dataset/testLineText.txt
new file mode 100644
index 0000000..7403c26
--- /dev/null
+++ b/tajo-storage/src/test/resources/dataset/testLineText.txt
@@ -0,0 +1,2 @@
+1|25|emiya muljomdao
+2|25|emiya muljomdao
\ No newline at end of file

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

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