You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@tajo.apache.org by hy...@apache.org on 2013/05/09 07:38:33 UTC

[01/11] TAJO-57: Recognize Parser and Catalog Standard SQL data types. (hyunsik)

Updated Branches:
  refs/heads/master fc4743686 -> c1c6f83ea


http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-storage/src/test/java/tajo/storage/index/TestBSTIndex.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/test/java/tajo/storage/index/TestBSTIndex.java b/tajo-core/tajo-core-storage/src/test/java/tajo/storage/index/TestBSTIndex.java
index 85c7beb..8d06d9b 100644
--- a/tajo-core/tajo-core-storage/src/test/java/tajo/storage/index/TestBSTIndex.java
+++ b/tajo-core/tajo-core-storage/src/test/java/tajo/storage/index/TestBSTIndex.java
@@ -24,8 +24,8 @@ import org.apache.hadoop.fs.Path;
 import org.junit.Before;
 import org.junit.Test;
 import tajo.catalog.*;
-import tajo.catalog.proto.CatalogProtos.DataType;
 import tajo.catalog.proto.CatalogProtos.StoreType;
+import tajo.common.TajoDataTypes.Type;
 import tajo.conf.TajoConf;
 import tajo.datum.DatumFactory;
 import tajo.storage.*;
@@ -54,11 +54,11 @@ public class TestBSTIndex {
     conf = new TajoConf();
     conf.setVar(TajoConf.ConfVars.ROOT_DIR, TEST_PATH);
     schema = new Schema();
-    schema.addColumn(new Column("int", DataType.INT));
-    schema.addColumn(new Column("long", DataType.LONG));
-    schema.addColumn(new Column("double", DataType.DOUBLE));
-    schema.addColumn(new Column("float", DataType.FLOAT));
-    schema.addColumn(new Column("string", DataType.STRING));
+    schema.addColumn(new Column("int", Type.INT4));
+    schema.addColumn(new Column("long", Type.INT8));
+    schema.addColumn(new Column("double", Type.FLOAT8));
+    schema.addColumn(new Column("float", Type.FLOAT4));
+    schema.addColumn(new Column("string", Type.TEXT));
   }
 
    
@@ -70,7 +70,7 @@ public class TestBSTIndex {
   
   @Test
   public void testFindValueInCSV() throws IOException {
-    meta = TCatUtil.newTableMeta(schema, StoreType.CSV);
+    meta = CatalogUtil.newTableMeta(schema, StoreType.CSV);
     
     Path tablePath = new Path(testDir, "FindValueInCSV.csv");
     Appender appender  = StorageManager.getAppender(conf, meta, tablePath);
@@ -78,11 +78,11 @@ public class TestBSTIndex {
     Tuple tuple;
     for(int i = 0 ; i < TUPLE_NUM; i ++ ) {
         tuple = new VTuple(5);
-        tuple.put(0, DatumFactory.createInt(i));
-        tuple.put(1, DatumFactory.createLong(i));
-        tuple.put(2, DatumFactory.createDouble(i));
-        tuple.put(3, DatumFactory.createFloat(i));
-        tuple.put(4, DatumFactory.createString("field_"+i));
+        tuple.put(0, DatumFactory.createInt4(i));
+        tuple.put(1, DatumFactory.createInt8(i));
+        tuple.put(2, DatumFactory.createFloat8(i));
+        tuple.put(3, DatumFactory.createFloat4(i));
+        tuple.put(4, DatumFactory.createText("field_" + i));
         appender.addTuple(tuple);
       }
     appender.close();
@@ -96,8 +96,8 @@ public class TestBSTIndex {
     sortKeys[1] = new SortSpec(schema.getColumn("double"), true, false);
 
     Schema keySchema = new Schema();
-    keySchema.addColumn(new Column("long", DataType.LONG));
-    keySchema.addColumn(new Column("double", DataType.DOUBLE));
+    keySchema.addColumn(new Column("long", Type.INT8));
+    keySchema.addColumn(new Column("double", Type.FLOAT8));
 
     TupleComparator comp = new TupleComparator(keySchema, sortKeys);
     
@@ -131,13 +131,13 @@ public class TestBSTIndex {
     reader.open();
     scanner  = (SeekableScanner)(StorageManager.getScanner(conf, meta, tablet));
     for(int i = 0 ; i < TUPLE_NUM -1 ; i ++) {
-      tuple.put(0, DatumFactory.createLong(i));
-      tuple.put(1, DatumFactory.createDouble(i));
+      tuple.put(0, DatumFactory.createInt8(i));
+      tuple.put(1, DatumFactory.createFloat8(i));
       long offsets = reader.find(tuple);
       scanner.seek(offsets);
       tuple = scanner.next();
-      assertTrue("seek check [" + (i) + " ," +(tuple.get(1).asLong())+ "]" , (i) == (tuple.get(1).asLong()));
-      assertTrue("seek check [" + (i) + " ,"  +(tuple.get(2).asDouble())+"]" , (i) == (tuple.get(2).asDouble()));
+      assertTrue("seek check [" + (i) + " ," +(tuple.get(1).asInt8())+ "]" , (i) == (tuple.get(1).asInt8()));
+      assertTrue("seek check [" + (i) + " ,"  +(tuple.get(2).asFloat8())+"]" , (i) == (tuple.get(2).asFloat8()));
       
       offsets = reader.next();
       if (offsets == -1) {
@@ -145,14 +145,14 @@ public class TestBSTIndex {
       }
       scanner.seek(offsets);
       tuple = scanner.next();
-      assertTrue("[seek check " + (i + 1) + " ]" , (i + 1) == (tuple.get(0).asInt()));
-      assertTrue("[seek check " + (i + 1) + " ]" , (i + 1) == (tuple.get(1).asLong()));
+      assertTrue("[seek check " + (i + 1) + " ]" , (i + 1) == (tuple.get(0).asInt4()));
+      assertTrue("[seek check " + (i + 1) + " ]" , (i + 1) == (tuple.get(1).asInt8()));
     }
   }
 
   @Test
   public void testBuildIndexWithAppender() throws IOException {
-    meta = TCatUtil.newTableMeta(schema, StoreType.CSV);
+    meta = CatalogUtil.newTableMeta(schema, StoreType.CSV);
 
     Path tablePath = new Path(testDir, "BuildIndexWithAppender.csv");
     FileAppender appender  = (FileAppender) StorageManager.getAppender(conf, meta, tablePath);
@@ -163,8 +163,8 @@ public class TestBSTIndex {
     sortKeys[1] = new SortSpec(schema.getColumn("double"), true, false);
 
     Schema keySchema = new Schema();
-    keySchema.addColumn(new Column("long", DataType.LONG));
-    keySchema.addColumn(new Column("double", DataType.DOUBLE));
+    keySchema.addColumn(new Column("long", Type.INT8));
+    keySchema.addColumn(new Column("double", Type.FLOAT8));
 
     TupleComparator comp = new TupleComparator(keySchema, sortKeys);
 
@@ -178,11 +178,11 @@ public class TestBSTIndex {
     long offset;
     for(int i = 0 ; i < TUPLE_NUM; i ++ ) {
       tuple = new VTuple(5);
-      tuple.put(0, DatumFactory.createInt(i));
-      tuple.put(1, DatumFactory.createLong(i));
-      tuple.put(2, DatumFactory.createDouble(i));
-      tuple.put(3, DatumFactory.createFloat(i));
-      tuple.put(4, DatumFactory.createString("field_"+i));
+      tuple.put(0, DatumFactory.createInt4(i));
+      tuple.put(1, DatumFactory.createInt8(i));
+      tuple.put(2, DatumFactory.createFloat8(i));
+      tuple.put(3, DatumFactory.createFloat4(i));
+      tuple.put(4, DatumFactory.createText("field_" + i));
 
       offset = appender.getOffset();
       appender.addTuple(tuple);
@@ -205,13 +205,13 @@ public class TestBSTIndex {
     reader.open();
     SeekableScanner scanner  = (SeekableScanner)(StorageManager.getScanner(conf, meta, tablet));
     for(int i = 0 ; i < TUPLE_NUM -1 ; i ++) {
-      tuple.put(0, DatumFactory.createLong(i));
-      tuple.put(1, DatumFactory.createDouble(i));
+      tuple.put(0, DatumFactory.createInt8(i));
+      tuple.put(1, DatumFactory.createFloat8(i));
       long offsets = reader.find(tuple);
       scanner.seek(offsets);
       tuple = scanner.next();
-      assertTrue("[seek check " + (i) + " ]", (i) == (tuple.get(1).asLong()));
-      assertTrue("[seek check " + (i) + " ]" , (i) == (tuple.get(2).asDouble()));
+      assertTrue("[seek check " + (i) + " ]", (i) == (tuple.get(1).asInt8()));
+      assertTrue("[seek check " + (i) + " ]" , (i) == (tuple.get(2).asFloat8()));
 
       offsets = reader.next();
       if (offsets == -1) {
@@ -219,14 +219,14 @@ public class TestBSTIndex {
       }
       scanner.seek(offsets);
       tuple = scanner.next();
-      assertTrue("[seek check " + (i + 1) + " ]" , (i + 1) == (tuple.get(0).asInt()));
-      assertTrue("[seek check " + (i + 1) + " ]" , (i + 1) == (tuple.get(1).asLong()));
+      assertTrue("[seek check " + (i + 1) + " ]" , (i + 1) == (tuple.get(0).asInt4()));
+      assertTrue("[seek check " + (i + 1) + " ]" , (i + 1) == (tuple.get(1).asInt8()));
     }
   }
   
   @Test
   public void testFindOmittedValueInCSV() throws IOException {
-    meta = TCatUtil.newTableMeta(schema, StoreType.CSV);
+    meta = CatalogUtil.newTableMeta(schema, StoreType.CSV);
     
     Path tablePath = StorageUtil.concatPath(testDir, "FindOmittedValueInCSV.csv");
     Appender appender = StorageManager.getAppender(conf, meta, tablePath);
@@ -234,11 +234,11 @@ public class TestBSTIndex {
     Tuple tuple;
     for(int i = 0 ; i < TUPLE_NUM; i += 2 ) {
         tuple = new VTuple(5);
-        tuple.put(0, DatumFactory.createInt(i));
-        tuple.put(1, DatumFactory.createLong(i));
-        tuple.put(2, DatumFactory.createDouble(i));
-        tuple.put(3, DatumFactory.createFloat(i));
-        tuple.put(4, DatumFactory.createString("field_"+i));
+        tuple.put(0, DatumFactory.createInt4(i));
+        tuple.put(1, DatumFactory.createInt8(i));
+        tuple.put(2, DatumFactory.createFloat8(i));
+        tuple.put(3, DatumFactory.createFloat4(i));
+        tuple.put(4, DatumFactory.createText("field_" + i));
         appender.addTuple(tuple);
       }
     appender.close();
@@ -251,8 +251,8 @@ public class TestBSTIndex {
     sortKeys[1] = new SortSpec(schema.getColumn("double"), true, false);
 
     Schema keySchema = new Schema();
-    keySchema.addColumn(new Column("long", DataType.LONG));
-    keySchema.addColumn(new Column("double", DataType.DOUBLE));
+    keySchema.addColumn(new Column("long", Type.INT8));
+    keySchema.addColumn(new Column("double", Type.FLOAT8));
 
     TupleComparator comp = new TupleComparator(keySchema, sortKeys);
     
@@ -283,8 +283,8 @@ public class TestBSTIndex {
     BSTIndexReader reader = bst.getIndexReader(new Path(testDir, "FindOmittedValueInCSV.idx"), keySchema, comp);
     reader.open();
     for(int i = 1 ; i < TUPLE_NUM -1 ; i+=2) {
-      keyTuple.put(0, DatumFactory.createLong(i));
-      keyTuple.put(1, DatumFactory.createDouble(i));
+      keyTuple.put(0, DatumFactory.createInt8(i));
+      keyTuple.put(1, DatumFactory.createFloat8(i));
       long offsets = reader.find(keyTuple);
       assertEquals(-1, offsets);
     }
@@ -292,7 +292,7 @@ public class TestBSTIndex {
   
   @Test
   public void testFindNextKeyValueInCSV() throws IOException {
-    meta = TCatUtil.newTableMeta(schema, StoreType.CSV);
+    meta = CatalogUtil.newTableMeta(schema, StoreType.CSV);
 
     Path tablePath = new Path(testDir, "FindNextKeyValueInCSV.csv");
     Appender appender = StorageManager.getAppender(conf, meta, tablePath);
@@ -300,11 +300,11 @@ public class TestBSTIndex {
     Tuple tuple;
     for(int i = 0 ; i < TUPLE_NUM; i ++ ) {
       tuple = new VTuple(5);
-      tuple.put(0, DatumFactory.createInt(i));
-      tuple.put(1, DatumFactory.createLong(i));
-      tuple.put(2, DatumFactory.createDouble(i));
-      tuple.put(3, DatumFactory.createFloat(i));
-      tuple.put(4, DatumFactory.createString("field_"+i));
+      tuple.put(0, DatumFactory.createInt4(i));
+      tuple.put(1, DatumFactory.createInt8(i));
+      tuple.put(2, DatumFactory.createFloat8(i));
+      tuple.put(3, DatumFactory.createFloat4(i));
+      tuple.put(4, DatumFactory.createText("field_" + i));
       appender.addTuple(tuple);
     }
     appender.close();
@@ -318,8 +318,8 @@ public class TestBSTIndex {
     sortKeys[1] = new SortSpec(schema.getColumn("long"), true, false);
 
     Schema keySchema = new Schema();
-    keySchema.addColumn(new Column("int", DataType.INT));
-    keySchema.addColumn(new Column("long", DataType.LONG));
+    keySchema.addColumn(new Column("int", Type.INT4));
+    keySchema.addColumn(new Column("long", Type.INT8));
 
     TupleComparator comp = new TupleComparator(keySchema, sortKeys);
     
@@ -354,14 +354,14 @@ public class TestBSTIndex {
     Tuple result;
     for(int i = 0 ; i < TUPLE_NUM -1 ; i ++) {
       keyTuple = new VTuple(2);
-      keyTuple.put(0, DatumFactory.createInt(i));
-      keyTuple.put(1, DatumFactory.createLong(i));
+      keyTuple.put(0, DatumFactory.createInt4(i));
+      keyTuple.put(1, DatumFactory.createInt8(i));
       long offsets = reader.find(keyTuple, true);
       scanner.seek(offsets);
       result = scanner.next();
       assertTrue("[seek check " + (i + 1) + " ]",
-          (i + 1) == (result.get(0).asInt()));
-      assertTrue("[seek check " + (i + 1) + " ]" , (i + 1) == (result.get(1).asLong()));
+          (i + 1) == (result.get(0).asInt4()));
+      assertTrue("[seek check " + (i + 1) + " ]" , (i + 1) == (result.get(1).asInt8()));
       
       offsets = reader.next();
       if (offsets == -1) {
@@ -369,14 +369,14 @@ public class TestBSTIndex {
       }
       scanner.seek(offsets);
       result = scanner.next();
-      assertTrue("[seek check " + (i + 2) + " ]" , (i + 2) == (result.get(0).asLong()));
-      assertTrue("[seek check " + (i + 2) + " ]" , (i + 2) == (result.get(1).asDouble()));
+      assertTrue("[seek check " + (i + 2) + " ]" , (i + 2) == (result.get(0).asInt8()));
+      assertTrue("[seek check " + (i + 2) + " ]" , (i + 2) == (result.get(1).asFloat8()));
     }
   }
   
   @Test
   public void testFindNextKeyOmittedValueInCSV() throws IOException {
-    meta = TCatUtil.newTableMeta(schema, StoreType.CSV);
+    meta = CatalogUtil.newTableMeta(schema, StoreType.CSV);
 
     Path tablePath = new Path(testDir, "FindNextKeyOmittedValueInCSV.csv");
     Appender appender = StorageManager.getAppender(conf, meta, tablePath);
@@ -384,11 +384,11 @@ public class TestBSTIndex {
     Tuple tuple;
     for(int i = 0 ; i < TUPLE_NUM; i+=2) {
       tuple = new VTuple(5);
-      tuple.put(0, DatumFactory.createInt(i));
-      tuple.put(1, DatumFactory.createLong(i));
-      tuple.put(2, DatumFactory.createDouble(i));
-      tuple.put(3, DatumFactory.createFloat(i));
-      tuple.put(4, DatumFactory.createString("field_"+i));
+      tuple.put(0, DatumFactory.createInt4(i));
+      tuple.put(1, DatumFactory.createInt8(i));
+      tuple.put(2, DatumFactory.createFloat8(i));
+      tuple.put(3, DatumFactory.createFloat4(i));
+      tuple.put(4, DatumFactory.createText("field_" + i));
       appender.addTuple(tuple);
     }
     appender.close();
@@ -402,8 +402,8 @@ public class TestBSTIndex {
     sortKeys[1] = new SortSpec(schema.getColumn("long"), true, false);
 
     Schema keySchema = new Schema();
-    keySchema.addColumn(new Column("int", DataType.INT));
-    keySchema.addColumn(new Column("long", DataType.LONG));
+    keySchema.addColumn(new Column("int", Type.INT4));
+    keySchema.addColumn(new Column("long", Type.INT8));
 
     TupleComparator comp = new TupleComparator(keySchema, sortKeys);
     
@@ -438,13 +438,13 @@ public class TestBSTIndex {
     Tuple result;
     for(int i = 1 ; i < TUPLE_NUM -1 ; i+=2) {
       keyTuple = new VTuple(2);
-      keyTuple.put(0, DatumFactory.createInt(i));
-      keyTuple.put(1, DatumFactory.createLong(i));
+      keyTuple.put(0, DatumFactory.createInt4(i));
+      keyTuple.put(1, DatumFactory.createInt8(i));
       long offsets = reader.find(keyTuple, true);
       scanner.seek(offsets);
       result = scanner.next();
-      assertTrue("[seek check " + (i + 1) + " ]" , (i + 1) == (result.get(0).asInt()));
-      assertTrue("[seek check " + (i + 1) + " ]" , (i + 1) == (result.get(1).asLong()));
+      assertTrue("[seek check " + (i + 1) + " ]" , (i + 1) == (result.get(0).asInt4()));
+      assertTrue("[seek check " + (i + 1) + " ]" , (i + 1) == (result.get(1).asInt8()));
     }
   }
 
@@ -458,11 +458,11 @@ public class TestBSTIndex {
     Tuple tuple;
     for(int i = 0 ; i < TUPLE_NUM; i ++ ) {
         tuple = new VTuple(5);
-        tuple.put(0, DatumFactory.createInt(i));
-        tuple.put(1, DatumFactory.createLong(i));
-        tuple.put(2, DatumFactory.createDouble(i));
-        tuple.put(3, DatumFactory.createFloat(i));
-        tuple.put(4, DatumFactory.createString("field_"+i));
+        tuple.put(0, DatumFactory.createInt4(i));
+        tuple.put(1, DatumFactory.createInt8(i));
+        tuple.put(2, DatumFactory.createFloat8(i));
+        tuple.put(3, DatumFactory.createFloat4(i));
+        tuple.put(4, DatumFactory.createText("field_"+i));
         appender.addTuple(tuple);
       }
     appender.close();
@@ -512,8 +512,8 @@ public class TestBSTIndex {
     reader.open();
     scanner  = (SeekableScanner)(sm.getScanner(meta, new Fragment[]{tablet}));
     for(int i = 0 ; i < TUPLE_NUM -1 ; i ++) {
-      tuple.put(0, DatumFactory.createLong(i));
-      tuple.put(1, DatumFactory.createDouble(i));
+      tuple.put(0, DatumFactory.createInt8(i));
+      tuple.put(1, DatumFactory.createFloat8(i));
       long offsets = reader.find(tuple, false);
       scanner.seek(offsets);
       tuple = scanner.next();
@@ -531,11 +531,11 @@ public class TestBSTIndex {
     Tuple tuple;
     for(int i = 0 ; i < TUPLE_NUM; i += 2 ) {
         tuple = new VTuple(5);
-        tuple.put(0, DatumFactory.createInt(i));
-        tuple.put(1, DatumFactory.createLong(i));
-        tuple.put(2, DatumFactory.createDouble(i));
-        tuple.put(3, DatumFactory.createFloat(i));
-        tuple.put(4, DatumFactory.createString("field_"+i));
+        tuple.put(0, DatumFactory.createInt4(i));
+        tuple.put(1, DatumFactory.createInt8(i));
+        tuple.put(2, DatumFactory.createFloat8(i));
+        tuple.put(3, DatumFactory.createFloat4(i));
+        tuple.put(4, DatumFactory.createText("field_"+i));
         appender.addTuple(tuple);
       }
     appender.close();
@@ -585,8 +585,8 @@ public class TestBSTIndex {
         keySchema, comp);
     reader.open();
     for(int i = 1 ; i < TUPLE_NUM -1 ; i+=2) {
-      tuple.put(0, DatumFactory.createLong(i));
-      tuple.put(1, DatumFactory.createDouble(i));
+      tuple.put(0, DatumFactory.createInt8(i));
+      tuple.put(1, DatumFactory.createFloat8(i));
       long offsets = reader.find(tuple, false);
       assertEquals(-1, offsets);
     }
@@ -601,11 +601,11 @@ public class TestBSTIndex {
     Tuple tuple;
     for(int i = 0 ; i < TUPLE_NUM; i ++ ) {
       tuple = new VTuple(5);
-      tuple.put(0, DatumFactory.createInt(i));
-      tuple.put(1, DatumFactory.createLong(i));
-      tuple.put(2, DatumFactory.createDouble(i));
-      tuple.put(3, DatumFactory.createFloat(i));
-      tuple.put(4, DatumFactory.createString("field_"+i));
+      tuple.put(0, DatumFactory.createInt4(i));
+      tuple.put(1, DatumFactory.createInt8(i));
+      tuple.put(2, DatumFactory.createFloat8(i));
+      tuple.put(3, DatumFactory.createFloat4(i));
+      tuple.put(4, DatumFactory.createText("field_"+i));
       appender.addTuple(tuple);
     }
     appender.close();
@@ -654,8 +654,8 @@ public class TestBSTIndex {
     Tuple result;
     for(int i = 0 ; i < TUPLE_NUM -1 ; i ++) {
       keyTuple = new VTuple(2);
-      keyTuple.put(0, DatumFactory.createInt(i));
-      keyTuple.put(1, DatumFactory.createLong(i));
+      keyTuple.put(0, DatumFactory.createInt4(i));
+      keyTuple.put(1, DatumFactory.createInt8(i));
       long offsets = reader.find(keyTuple, true);
       scanner.seek(offsets);
       result = scanner.next();
@@ -683,11 +683,11 @@ public class TestBSTIndex {
     Tuple tuple;
     for(int i = 0 ; i < TUPLE_NUM; i+=2) {
       tuple = new VTuple(5);
-      tuple.put(0, DatumFactory.createInt(i));
-      tuple.put(1, DatumFactory.createLong(i));
-      tuple.put(2, DatumFactory.createDouble(i));
-      tuple.put(3, DatumFactory.createFloat(i));
-      tuple.put(4, DatumFactory.createString("field_"+i));
+      tuple.put(0, DatumFactory.createInt4(i));
+      tuple.put(1, DatumFactory.createInt8(i));
+      tuple.put(2, DatumFactory.createFloat8(i));
+      tuple.put(3, DatumFactory.createFloat4(i));
+      tuple.put(4, DatumFactory.createText("field_"+i));
       appender.addTuple(tuple);
     }
     appender.close();
@@ -737,8 +737,8 @@ public class TestBSTIndex {
     Tuple result;
     for(int i = 1 ; i < TUPLE_NUM -1 ; i+=2) {
       keyTuple = new VTuple(2);
-      keyTuple.put(0, DatumFactory.createInt(i));
-      keyTuple.put(1, DatumFactory.createLong(i));
+      keyTuple.put(0, DatumFactory.createInt4(i));
+      keyTuple.put(1, DatumFactory.createInt8(i));
       long offsets = reader.find(keyTuple, true);
       scanner.seek(offsets);
       result = scanner.next();
@@ -756,11 +756,11 @@ public class TestBSTIndex {
     Tuple tuple;
     for(int i = 0 ; i < TUPLE_NUM; i ++ ) {
       tuple = new VTuple(5);
-      tuple.put(0, DatumFactory.createInt(i));
-      tuple.put(1, DatumFactory.createLong(i));
-      tuple.put(2, DatumFactory.createDouble(i));
-      tuple.put(3, DatumFactory.createFloat(i));
-      tuple.put(4, DatumFactory.createString("field_"+i));
+      tuple.put(0, DatumFactory.createInt4(i));
+      tuple.put(1, DatumFactory.createInt8(i));
+      tuple.put(2, DatumFactory.createFloat8(i));
+      tuple.put(3, DatumFactory.createFloat4(i));
+      tuple.put(4, DatumFactory.createText("field_"+i));
       appender.addTuple(tuple);
     }
     appender.close();
@@ -810,8 +810,8 @@ public class TestBSTIndex {
     Tuple result;
 
     keyTuple = new VTuple(2);
-    keyTuple.put(0, DatumFactory.createInt(0));
-    keyTuple.put(1, DatumFactory.createLong(0));
+    keyTuple.put(0, DatumFactory.createInt4(0));
+    keyTuple.put(1, DatumFactory.createInt8(0));
     long offsets = reader.find(keyTuple);
     scanner.seek(offsets);
     result = scanner.next();
@@ -837,11 +837,11 @@ public class TestBSTIndex {
     Tuple tuple;
     for(int i = 5 ; i < TUPLE_NUM + 5; i ++ ) {
       tuple = new VTuple(5);
-      tuple.put(0, DatumFactory.createInt(i));
-      tuple.put(1, DatumFactory.createLong(i));
-      tuple.put(2, DatumFactory.createDouble(i));
-      tuple.put(3, DatumFactory.createFloat(i));
-      tuple.put(4, DatumFactory.createString("field_"+i));
+      tuple.put(0, DatumFactory.createInt4(i));
+      tuple.put(1, DatumFactory.createInt8(i));
+      tuple.put(2, DatumFactory.createFloat8(i));
+      tuple.put(3, DatumFactory.createFloat4(i));
+      tuple.put(4, DatumFactory.createText("field_"+i));
       appender.addTuple(tuple);
     }
     appender.close();
@@ -890,8 +890,8 @@ public class TestBSTIndex {
     BSTIndexReader reader = bst.getIndexReader(new Path(testDir, "Test.idx"), keySchema, comp);
     reader.open();
     scanner  = (SeekableScanner)(sm.getScanner(meta, new Fragment[]{tablet}));
-    tuple.put(0, DatumFactory.createLong(0));
-    tuple.put(1, DatumFactory.createDouble(0));
+    tuple.put(0, DatumFactory.createInt8(0));
+    tuple.put(1, DatumFactory.createFloat8(0));
 
     offset = reader.find(tuple);
     assertEquals(-1, offset);
@@ -913,11 +913,11 @@ public class TestBSTIndex {
     Tuple tuple;
     for(int i = 5 ; i < TUPLE_NUM; i ++ ) {
       tuple = new VTuple(5);
-      tuple.put(0, DatumFactory.createInt(i));
-      tuple.put(1, DatumFactory.createLong(i));
-      tuple.put(2, DatumFactory.createDouble(i));
-      tuple.put(3, DatumFactory.createFloat(i));
-      tuple.put(4, DatumFactory.createString("field_"+i));
+      tuple.put(0, DatumFactory.createInt4(i));
+      tuple.put(1, DatumFactory.createInt8(i));
+      tuple.put(2, DatumFactory.createFloat8(i));
+      tuple.put(3, DatumFactory.createFloat4(i));
+      tuple.put(4, DatumFactory.createText("field_"+i));
       appender.addTuple(tuple);
     }
     appender.close();
@@ -992,8 +992,8 @@ public class TestBSTIndex {
       int keyVal;
       for (int i = 0; i < 10000; i++) {
         keyVal = rnd.nextInt(10000);
-        findKey.put(0, DatumFactory.createInt(keyVal));
-        findKey.put(1, DatumFactory.createLong(keyVal));
+        findKey.put(0, DatumFactory.createInt4(keyVal));
+        findKey.put(1, DatumFactory.createInt8(keyVal));
         try {
           assertTrue(reader.find(findKey) != -1);
         } catch (Exception e) {
@@ -1013,11 +1013,11 @@ public class TestBSTIndex {
     Tuple tuple;
     for(int i = 0 ; i < TUPLE_NUM; i ++ ) {
       tuple = new VTuple(5);
-      tuple.put(0, DatumFactory.createInt(i));
-      tuple.put(1, DatumFactory.createLong(i));
-      tuple.put(2, DatumFactory.createDouble(i));
-      tuple.put(3, DatumFactory.createFloat(i));
-      tuple.put(4, DatumFactory.createString("field_"+i));
+      tuple.put(0, DatumFactory.createInt4(i));
+      tuple.put(1, DatumFactory.createInt8(i));
+      tuple.put(2, DatumFactory.createFloat8(i));
+      tuple.put(3, DatumFactory.createFloat4(i));
+      tuple.put(4, DatumFactory.createText("field_"+i));
       appender.addTuple(tuple);
     }
     appender.close();
@@ -1087,11 +1087,11 @@ public class TestBSTIndex {
     Tuple tuple;
     for(int i = (TUPLE_NUM - 1); i >= 0; i -- ) {
       tuple = new VTuple(5);
-      tuple.put(0, DatumFactory.createInt(i));
-      tuple.put(1, DatumFactory.createLong(i));
-      tuple.put(2, DatumFactory.createDouble(i));
-      tuple.put(3, DatumFactory.createFloat(i));
-      tuple.put(4, DatumFactory.createString("field_"+i));
+      tuple.put(0, DatumFactory.createInt4(i));
+      tuple.put(1, DatumFactory.createInt8(i));
+      tuple.put(2, DatumFactory.createFloat8(i));
+      tuple.put(3, DatumFactory.createFloat4(i));
+      tuple.put(4, DatumFactory.createText("field_"+i));
       appender.addTuple(tuple);
     }
     appender.close();
@@ -1141,8 +1141,8 @@ public class TestBSTIndex {
     reader.open();
     scanner  = (SeekableScanner)(sm.getScanner(meta, new Fragment[]{tablet}));
     for(int i = (TUPLE_NUM - 1) ; i > 0  ; i --) {
-      tuple.put(0, DatumFactory.createLong(i));
-      tuple.put(1, DatumFactory.createDouble(i));
+      tuple.put(0, DatumFactory.createInt8(i));
+      tuple.put(1, DatumFactory.createFloat8(i));
       long offsets = reader.find(tuple);
       scanner.seek(offsets);
       tuple = scanner.next();
@@ -1169,11 +1169,11 @@ public class TestBSTIndex {
     Tuple tuple;
     for(int i = (TUPLE_NUM - 1); i >= 0; i --) {
       tuple = new VTuple(5);
-      tuple.put(0, DatumFactory.createInt(i));
-      tuple.put(1, DatumFactory.createLong(i));
-      tuple.put(2, DatumFactory.createDouble(i));
-      tuple.put(3, DatumFactory.createFloat(i));
-      tuple.put(4, DatumFactory.createString("field_"+i));
+      tuple.put(0, DatumFactory.createInt4(i));
+      tuple.put(1, DatumFactory.createInt8(i));
+      tuple.put(2, DatumFactory.createFloat8(i));
+      tuple.put(3, DatumFactory.createFloat4(i));
+      tuple.put(4, DatumFactory.createText("field_"+i));
       appender.addTuple(tuple);
     }
     appender.close();
@@ -1226,8 +1226,8 @@ public class TestBSTIndex {
     Tuple result;
     for(int i = (TUPLE_NUM - 1) ; i > 0 ; i --) {
       keyTuple = new VTuple(2);
-      keyTuple.put(0, DatumFactory.createInt(i));
-      keyTuple.put(1, DatumFactory.createLong(i));
+      keyTuple.put(0, DatumFactory.createInt4(i));
+      keyTuple.put(1, DatumFactory.createInt8(i));
       long offsets = reader.find(keyTuple, true);
       scanner.seek(offsets);
       result = scanner.next();

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-storage/src/test/java/tajo/storage/index/TestSingleCSVFileBSTIndex.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/test/java/tajo/storage/index/TestSingleCSVFileBSTIndex.java b/tajo-core/tajo-core-storage/src/test/java/tajo/storage/index/TestSingleCSVFileBSTIndex.java
index e0ae7b3..23c4ee1 100644
--- a/tajo-core/tajo-core-storage/src/test/java/tajo/storage/index/TestSingleCSVFileBSTIndex.java
+++ b/tajo-core/tajo-core-storage/src/test/java/tajo/storage/index/TestSingleCSVFileBSTIndex.java
@@ -24,8 +24,8 @@ import org.apache.hadoop.fs.Path;
 import org.junit.Before;
 import org.junit.Test;
 import tajo.catalog.*;
-import tajo.catalog.proto.CatalogProtos.DataType;
 import tajo.catalog.proto.CatalogProtos.StoreType;
+import tajo.common.TajoDataTypes.Type;
 import tajo.conf.TajoConf;
 import tajo.conf.TajoConf.ConfVars;
 import tajo.datum.DatumFactory;
@@ -57,11 +57,11 @@ public class TestSingleCSVFileBSTIndex {
     conf = new TajoConf();
     conf.setVar(ConfVars.ROOT_DIR, TEST_PATH);
     schema = new Schema();
-    schema.addColumn(new Column("int", DataType.INT));
-    schema.addColumn(new Column("long", DataType.LONG));
-    schema.addColumn(new Column("double", DataType.DOUBLE));
-    schema.addColumn(new Column("float", DataType.FLOAT));
-    schema.addColumn(new Column("string", DataType.STRING));
+    schema.addColumn(new Column("int", Type.INT4));
+    schema.addColumn(new Column("long", Type.INT8));
+    schema.addColumn(new Column("double", Type.FLOAT8));
+    schema.addColumn(new Column("float", Type.FLOAT4));
+    schema.addColumn(new Column("string", Type.TEXT));
   }
 
   @Before
@@ -72,7 +72,7 @@ public class TestSingleCSVFileBSTIndex {
 
   @Test
   public void testFindValueInSingleCSV() throws IOException {
-    meta = TCatUtil.newTableMeta(schema, StoreType.CSV);
+    meta = CatalogUtil.newTableMeta(schema, StoreType.CSV);
 
     Path tablePath = StorageUtil.concatPath(testDir, "testFindValueInSingleCSV", "table.csv");
     fs.mkdirs(tablePath.getParent());
@@ -82,11 +82,11 @@ public class TestSingleCSVFileBSTIndex {
     Tuple tuple;
     for (int i = 0; i < TUPLE_NUM; i++) {
       tuple = new VTuple(5);
-      tuple.put(0, DatumFactory.createInt(i));
-      tuple.put(1, DatumFactory.createLong(i));
-      tuple.put(2, DatumFactory.createDouble(i));
-      tuple.put(3, DatumFactory.createFloat(i));
-      tuple.put(4, DatumFactory.createString("field_" + i));
+      tuple.put(0, DatumFactory.createInt4(i));
+      tuple.put(1, DatumFactory.createInt8(i));
+      tuple.put(2, DatumFactory.createFloat8(i));
+      tuple.put(3, DatumFactory.createFloat4(i));
+      tuple.put(4, DatumFactory.createText("field_" + i));
       appender.addTuple(tuple);
     }
     appender.close();
@@ -101,8 +101,8 @@ public class TestSingleCSVFileBSTIndex {
     sortKeys[1] = new SortSpec(schema.getColumn("double"), true, false);
 
     Schema keySchema = new Schema();
-    keySchema.addColumn(new Column("long", DataType.LONG));
-    keySchema.addColumn(new Column("double", DataType.DOUBLE));
+    keySchema.addColumn(new Column("long", Type.INT8));
+    keySchema.addColumn(new Column("double", Type.FLOAT8));
 
     TupleComparator comp = new TupleComparator(keySchema, sortKeys);
 
@@ -137,13 +137,13 @@ public class TestSingleCSVFileBSTIndex {
     reader.open();
     fileScanner = new CSVScanner(conf, meta, tablet);
     for (int i = 0; i < TUPLE_NUM - 1; i++) {
-      tuple.put(0, DatumFactory.createLong(i));
-      tuple.put(1, DatumFactory.createDouble(i));
+      tuple.put(0, DatumFactory.createInt8(i));
+      tuple.put(1, DatumFactory.createFloat8(i));
       long offsets = reader.find(tuple);
       fileScanner.seek(offsets);
       tuple = fileScanner.next();
-      assertEquals(i,  (tuple.get(1).asLong()));
-      assertEquals(i, (tuple.get(2).asDouble()) , 0.01);
+      assertEquals(i,  (tuple.get(1).asInt8()));
+      assertEquals(i, (tuple.get(2).asFloat8()) , 0.01);
 
       offsets = reader.next();
       if (offsets == -1) {
@@ -152,15 +152,15 @@ public class TestSingleCSVFileBSTIndex {
       fileScanner.seek(offsets);
       tuple = fileScanner.next();
       assertTrue("[seek check " + (i + 1) + " ]",
-          (i + 1) == (tuple.get(0).asInt()));
+          (i + 1) == (tuple.get(0).asInt4()));
       assertTrue("[seek check " + (i + 1) + " ]",
-          (i + 1) == (tuple.get(1).asLong()));
+          (i + 1) == (tuple.get(1).asInt8()));
     }
   }
 
   @Test
   public void testFindNextKeyValueInSingleCSV() throws IOException {
-    meta = TCatUtil.newTableMeta(schema, StoreType.CSV);
+    meta = CatalogUtil.newTableMeta(schema, StoreType.CSV);
 
     Path tablePath = StorageUtil.concatPath(testDir, "testFindNextKeyValueInSingleCSV",
         "table1.csv");
@@ -170,11 +170,11 @@ public class TestSingleCSVFileBSTIndex {
     Tuple tuple;
     for(int i = 0 ; i < TUPLE_NUM; i ++ ) {
       tuple = new VTuple(5);
-      tuple.put(0, DatumFactory.createInt(i));
-      tuple.put(1, DatumFactory.createLong(i));
-      tuple.put(2, DatumFactory.createDouble(i));
-      tuple.put(3, DatumFactory.createFloat(i));
-      tuple.put(4, DatumFactory.createString("field_"+i));
+      tuple.put(0, DatumFactory.createInt4(i));
+      tuple.put(1, DatumFactory.createInt8(i));
+      tuple.put(2, DatumFactory.createFloat8(i));
+      tuple.put(3, DatumFactory.createFloat4(i));
+      tuple.put(4, DatumFactory.createText("field_" + i));
       appender.addTuple(tuple);
     }
     appender.close();
@@ -188,8 +188,8 @@ public class TestSingleCSVFileBSTIndex {
     sortKeys[1] = new SortSpec(schema.getColumn("long"), true, false);
 
     Schema keySchema = new Schema();
-    keySchema.addColumn(new Column("int", DataType.INT));
-    keySchema.addColumn(new Column("long", DataType.LONG));
+    keySchema.addColumn(new Column("int", Type.INT4));
+    keySchema.addColumn(new Column("long", Type.INT8));
 
     TupleComparator comp = new TupleComparator(keySchema, sortKeys);
     
@@ -223,13 +223,13 @@ public class TestSingleCSVFileBSTIndex {
     Tuple result;
     for(int i = 0 ; i < TUPLE_NUM -1 ; i ++) {
       keyTuple = new VTuple(2);
-      keyTuple.put(0, DatumFactory.createInt(i));
-      keyTuple.put(1, DatumFactory.createLong(i));
+      keyTuple.put(0, DatumFactory.createInt4(i));
+      keyTuple.put(1, DatumFactory.createInt8(i));
       long offsets = reader.find(keyTuple, true);
       fileScanner.seek(offsets);
       result = fileScanner.next();
-      assertTrue("[seek check " + (i + 1) + " ]" , (i + 1) == (result.get(0).asInt()));
-      assertTrue("[seek check " + (i + 1) + " ]" , (i + 1) == (result.get(1).asLong()));
+      assertTrue("[seek check " + (i + 1) + " ]" , (i + 1) == (result.get(0).asInt4()));
+      assertTrue("[seek check " + (i + 1) + " ]" , (i + 1) == (result.get(1).asInt8()));
       
       offsets = reader.next();
       if (offsets == -1) {
@@ -237,8 +237,8 @@ public class TestSingleCSVFileBSTIndex {
       }
       fileScanner.seek(offsets);
       result = fileScanner.next();
-      assertTrue("[seek check " + (i + 2) + " ]" , (i + 2) == (result.get(0).asLong()));
-      assertTrue("[seek check " + (i + 2) + " ]" , (i + 2) == (result.get(1).asDouble()));
+      assertTrue("[seek check " + (i + 2) + " ]" , (i + 2) == (result.get(0).asInt8()));
+      assertTrue("[seek check " + (i + 2) + " ]" , (i + 2) == (result.get(1).asFloat8()));
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-storage/src/test/java/tajo/storage/rcfile/TestRCFile.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/test/java/tajo/storage/rcfile/TestRCFile.java b/tajo-core/tajo-core-storage/src/test/java/tajo/storage/rcfile/TestRCFile.java
index 315b0ed..0df4026 100644
--- a/tajo-core/tajo-core-storage/src/test/java/tajo/storage/rcfile/TestRCFile.java
+++ b/tajo-core/tajo-core-storage/src/test/java/tajo/storage/rcfile/TestRCFile.java
@@ -127,11 +127,11 @@ public class TestRCFile {
 
     Datum [] r1 = new Datum[7];
     int idx = 0;
-    r1[idx++] = DatumFactory.createInt(123);
-    r1[idx++] = DatumFactory.createLong(456);
-    r1[idx++] = DatumFactory.createFloat(7.89f);
-    r1[idx++] = DatumFactory.createDouble(10.11d);
-    r1[idx] = DatumFactory.createString("tajo and hadoop");
+    r1[idx++] = DatumFactory.createInt4(123);
+    r1[idx++] = DatumFactory.createInt8(456);
+    r1[idx++] = DatumFactory.createFloat4(7.89f);
+    r1[idx++] = DatumFactory.createFloat8(10.11d);
+    r1[idx] = DatumFactory.createText("tajo and hadoop");
 
 //    byte[][] record_1 = {
 //        "123".getBytes("UTF-8"), "456".getBytes("UTF-8"),
@@ -151,11 +151,11 @@ public class TestRCFile {
 
     Datum [] r2 = new Datum[7];
     idx = 0;
-    r2[idx++] = DatumFactory.createInt(100);
-    r2[idx++] = DatumFactory.createLong(200);
-    r2[idx++] = DatumFactory.createFloat(5.3f);
-    r2[idx++] = DatumFactory.createDouble(11.12d);
-    r2[idx] = DatumFactory.createString("the second str");
+    r2[idx++] = DatumFactory.createInt4(100);
+    r2[idx++] = DatumFactory.createInt8(200);
+    r2[idx++] = DatumFactory.createFloat4(5.3f);
+    r2[idx++] = DatumFactory.createFloat8(11.12d);
+    r2[idx] = DatumFactory.createText("the second str");
 
     byte [][] record_2 = {
         r2[0].asByteArray(),
@@ -221,22 +221,22 @@ public class TestRCFile {
     BytesRefArrayWritable cols = new BytesRefArrayWritable();
     reader.getCurrentRow(cols);
     cols.resetValid(7);
-    assertEquals(r1[0], new IntDatum(cols.get(0).getBytesCopy()));
-    assertEquals(r1[1], new LongDatum(cols.get(1).getBytesCopy()));
-    assertEquals(r1[2], new FloatDatum(cols.get(2).getBytesCopy()));
-    assertEquals(r1[3], new DoubleDatum(cols.get(3).getBytesCopy()));
-    assertEquals(r1[4], new StringDatum(cols.get(4).getBytesCopy()));
+    assertEquals(r1[0], new Int4Datum(cols.get(0).getBytesCopy()));
+    assertEquals(r1[1], new Int8Datum(cols.get(1).getBytesCopy()));
+    assertEquals(r1[2], new Float4Datum(cols.get(2).getBytesCopy()));
+    assertEquals(r1[3], new Float8Datum(cols.get(3).getBytesCopy()));
+    assertEquals(r1[4], new TextDatum(cols.get(4).getBytesCopy()));
 
 
     reader.next(rowID);
     cols = new BytesRefArrayWritable();
     reader.getCurrentRow(cols);
     cols.resetValid(7);
-    assertEquals(r2[0], new IntDatum(cols.get(0).getBytesCopy()));
-    assertEquals(r2[1], new LongDatum(cols.get(1).getBytesCopy()));
-    assertEquals(r2[2], new FloatDatum(cols.get(2).getBytesCopy()));
-    assertEquals(r2[3], new DoubleDatum(cols.get(3).getBytesCopy()));
-    assertEquals(r2[4], new StringDatum(cols.get(4).getBytesCopy()));
+    assertEquals(r2[0], new Int4Datum(cols.get(0).getBytesCopy()));
+    assertEquals(r2[1], new Int8Datum(cols.get(1).getBytesCopy()));
+    assertEquals(r2[2], new Float4Datum(cols.get(2).getBytesCopy()));
+    assertEquals(r2[3], new Float8Datum(cols.get(3).getBytesCopy()));
+    assertEquals(r2[4], new TextDatum(cols.get(4).getBytesCopy()));
     /*
       cols.resetValid(8);
       Object row = serDe.deserialize(cols);


[10/11] TAJO-57: Recognize Parser and Catalog Standard SQL data types. (hyunsik)

Posted by hy...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-catalog/tajo-catalog-server/src/main/java/tajo/catalog/AlreadyRegisteredURIException.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-server/src/main/java/tajo/catalog/AlreadyRegisteredURIException.java b/tajo-catalog/tajo-catalog-server/src/main/java/tajo/catalog/AlreadyRegisteredURIException.java
deleted file mode 100644
index 5aa1cb1..0000000
--- a/tajo-catalog/tajo-catalog-server/src/main/java/tajo/catalog/AlreadyRegisteredURIException.java
+++ /dev/null
@@ -1,43 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-/**
- * 
- */
-package tajo.catalog;
-
-import java.net.URI;
-
-public class AlreadyRegisteredURIException extends RuntimeException {
-
-	private static final long serialVersionUID = 747390434221048348L;
-
-	public AlreadyRegisteredURIException() {
-	}
-
-	/**
-	 * @param uri
-	 */
-	public AlreadyRegisteredURIException(String uri) {
-		super("Already registered TRID: "+uri);
-	}
-	
-	public AlreadyRegisteredURIException(URI uri) {
-		this("Already registered TRID: "+uri);
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-catalog/tajo-catalog-server/src/main/java/tajo/catalog/CatalogServer.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-server/src/main/java/tajo/catalog/CatalogServer.java b/tajo-catalog/tajo-catalog-server/src/main/java/tajo/catalog/CatalogServer.java
index 85600a8..8587b1e 100644
--- a/tajo-catalog/tajo-catalog-server/src/main/java/tajo/catalog/CatalogServer.java
+++ b/tajo-catalog/tajo-catalog-server/src/main/java/tajo/catalog/CatalogServer.java
@@ -30,6 +30,7 @@ import tajo.catalog.exception.*;
 import tajo.catalog.proto.CatalogProtos.*;
 import tajo.catalog.store.CatalogStore;
 import tajo.catalog.store.DBStore;
+import tajo.common.TajoDataTypes.DataType;
 import tajo.conf.TajoConf;
 import tajo.conf.TajoConf.ConfVars;
 import tajo.rpc.ProtoBlockingRpcServer;
@@ -226,7 +227,7 @@ public class CatalogServer extends AbstractService {
 
         // rewrite schema
         SchemaProto revisedSchema =
-            TCatUtil.getQualfiedSchema(tableDesc.getId(), tableDesc.getMeta()
+            CatalogUtil.getQualfiedSchema(tableDesc.getId(), tableDesc.getMeta()
                 .getSchema());
 
         TableProto.Builder metaBuilder = TableProto.newBuilder(tableDesc.getMeta());
@@ -396,7 +397,7 @@ public class CatalogServer extends AbstractService {
                                       FunctionDescProto funcDesc)
         throws ServiceException {
       String canonicalName =
-          TCatUtil.getCanonicalName(funcDesc.getSignature(),
+          CatalogUtil.getCanonicalName(funcDesc.getSignature(),
               funcDesc.getParameterTypesList());
       if (functions.containsKey(canonicalName)) {
         throw new AlreadyExistsFunctionException(canonicalName);
@@ -420,7 +421,7 @@ public class CatalogServer extends AbstractService {
       for (int i = 0; i < size; i++) {
         paramTypes.add(request.getParameterTypes(i));
       }
-      String canonicalName = TCatUtil.getCanonicalName(signature, paramTypes);
+      String canonicalName = CatalogUtil.getCanonicalName(signature, paramTypes);
       if (!functions.containsKey(canonicalName)) {
         throw new NoSuchFunctionException(canonicalName);
       }
@@ -440,7 +441,7 @@ public class CatalogServer extends AbstractService {
       for (int i = 0; i < size; i++) {
         paramTypes.add(request.getParameterTypes(i));
       }
-      return functions.get(TCatUtil.getCanonicalName(
+      return functions.get(CatalogUtil.getCanonicalName(
           request.getSignature().toLowerCase(), paramTypes));
     }
 
@@ -454,7 +455,7 @@ public class CatalogServer extends AbstractService {
         paramTypes.add(request.getParameterTypes(i));
       }
       boolean returnValue =
-          functions.containsKey(TCatUtil.getCanonicalName(
+          functions.containsKey(CatalogUtil.getCanonicalName(
               request.getSignature().toLowerCase(), paramTypes));
       return BoolProto.newBuilder().setValue(returnValue).build();
     }

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-catalog/tajo-catalog-server/src/main/java/tajo/catalog/store/DBStore.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-server/src/main/java/tajo/catalog/store/DBStore.java b/tajo-catalog/tajo-catalog-server/src/main/java/tajo/catalog/store/DBStore.java
index 136872d..6db6e49 100644
--- a/tajo-catalog/tajo-catalog-server/src/main/java/tajo/catalog/store/DBStore.java
+++ b/tajo-catalog/tajo-catalog-server/src/main/java/tajo/catalog/store/DBStore.java
@@ -28,6 +28,7 @@ import org.apache.hadoop.fs.Path;
 import tajo.catalog.*;
 import tajo.catalog.proto.CatalogProtos.*;
 import tajo.catalog.statistics.TableStat;
+import tajo.common.TajoDataTypes.Type;
 import tajo.conf.TajoConf;
 import tajo.exception.InternalException;
 
@@ -444,7 +445,7 @@ public class DBStore implements CatalogStore {
         + "'" + desc.getId() + "',"
         + columnId + ", "
         + "'" + col.getColumnName() + "',"
-        + "'" + col.getDataType().toString() + "'"
+        + "'" + col.getDataType().getType().name() + "'"
         + ")";
     
     return sql;
@@ -627,7 +628,7 @@ public class DBStore implements CatalogStore {
         while (res.next()) {
           String columnName = tableName + "." 
               + res.getString("column_name").trim();
-          DataType dataType = getDataType(res.getString("data_type")
+          Type dataType = getDataType(res.getString("data_type")
               .trim());
           schema.addColumn(columnName, dataType);
         }
@@ -695,37 +696,11 @@ public class DBStore implements CatalogStore {
     }
   }
   
-  private DataType getDataType(final String typeStr) {
-    if (typeStr.equals(DataType.BOOLEAN.toString())) {
-      return DataType.BOOLEAN;
-    } else if (typeStr.equals(DataType.BYTE.toString())) {
-      return DataType.BYTE;
-    } else if (typeStr.equals(DataType.CHAR.toString())) {
-      return DataType.CHAR;
-    } else if (typeStr.equals(DataType.SHORT.toString())) {
-      return DataType.SHORT;
-    } else if (typeStr.equals(DataType.INT.toString())) {
-      return DataType.INT;
-    } else if (typeStr.equals(DataType.LONG.toString())) {
-      return DataType.LONG;
-    } else if (typeStr.equals(DataType.FLOAT.toString())) {
-      return DataType.FLOAT;
-    } else if (typeStr.equals(DataType.DOUBLE.toString())) {
-      return DataType.DOUBLE;
-    } else if (typeStr.equals(DataType.STRING.toString())) {
-      return DataType.STRING;
-    } else if (typeStr.equals(DataType.IPv4.toString())) {
-      return DataType.IPv4;
-    } else if (typeStr.equals(DataType.IPv6.toString())) {
-      return DataType.IPv6;
-    } else if (typeStr.equals(DataType.BYTES.toString())) {
-      return DataType.BYTES;
-    } else if (typeStr.equals(DataType.DATE.toString())) {
-      return DataType.DATE;
-    } else {
-      LOG.error("Cannot find a matched type aginst from '"
-          + typeStr + "'");
-      // TODO - needs exception handling
+  private Type getDataType(final String typeStr) {
+    try {
+    return Enum.valueOf(Type.class, typeStr);
+    } catch (IllegalArgumentException iae) {
+      LOG.error("Cannot find a matched type aginst from '" + typeStr + "'");
       return null;
     }
   }
@@ -776,7 +751,7 @@ public class DBStore implements CatalogStore {
       stmt.setString(1, proto.getName());
       stmt.setString(2, proto.getTableId());
       stmt.setString(3, proto.getColumn().getColumnName());
-      stmt.setString(4, proto.getColumn().getDataType().toString());
+      stmt.setString(4, proto.getColumn().getDataType().getType().name());
       stmt.setString(5, proto.getIndexMethod().toString());
       stmt.setBoolean(6, proto.hasIsUnique() && proto.getIsUnique());
       stmt.setBoolean(7, proto.hasIsClustered() && proto.getIsClustered());
@@ -1001,7 +976,8 @@ public class DBStore implements CatalogStore {
   private ColumnProto resultToColumnProto(final ResultSet res) throws SQLException {
     ColumnProto.Builder builder = ColumnProto.newBuilder();
     builder.setColumnName(res.getString("column_name"));
-    builder.setDataType(getDataType(res.getString("data_type").trim()));
+    builder.setDataType(CatalogUtil.newDataTypeWithoutLen(
+        getDataType(res.getString("data_type").trim())));
     return builder.build();
   }
   

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-catalog/tajo-catalog-server/src/test/java/tajo/catalog/TestCatalog.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-server/src/test/java/tajo/catalog/TestCatalog.java b/tajo-catalog/tajo-catalog-server/src/test/java/tajo/catalog/TestCatalog.java
index 269d8b4..9e61d83 100644
--- a/tajo-catalog/tajo-catalog-server/src/test/java/tajo/catalog/TestCatalog.java
+++ b/tajo-catalog/tajo-catalog-server/src/test/java/tajo/catalog/TestCatalog.java
@@ -23,10 +23,11 @@ import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
 import tajo.catalog.function.GeneralFunction;
-import tajo.catalog.proto.CatalogProtos.DataType;
 import tajo.catalog.proto.CatalogProtos.FunctionType;
 import tajo.catalog.proto.CatalogProtos.IndexMethod;
 import tajo.catalog.proto.CatalogProtos.StoreType;
+import tajo.common.TajoDataTypes;
+import tajo.common.TajoDataTypes.Type;
 import tajo.conf.TajoConf;
 import tajo.datum.Datum;
 import tajo.storage.Tuple;
@@ -61,11 +62,11 @@ public class TestCatalog {
 	@Test
 	public void testGetTable() throws Exception {
 		schema1 = new Schema();
-		schema1.addColumn(FieldName1, DataType.BYTE);
-		schema1.addColumn(FieldName2, DataType.INT);
-		schema1.addColumn(FieldName3, DataType.LONG);
+		schema1.addColumn(FieldName1, Type.BLOB);
+		schema1.addColumn(FieldName2, Type.INT4);
+		schema1.addColumn(FieldName3, Type.INT8);
 		
-		TableDesc meta = TCatUtil.newTableDesc(
+		TableDesc meta = CatalogUtil.newTableDesc(
         "getTable",
         schema1,
         StoreType.CSV,
@@ -86,11 +87,11 @@ public class TestCatalog {
 	@Test(expected = Throwable.class)
 	public void testAddTableNoName() throws Exception {
 	  schema1 = new Schema();
-    schema1.addColumn(FieldName1, DataType.BYTE);
-    schema1.addColumn(FieldName2, DataType.INT);
-    schema1.addColumn(FieldName3, DataType.LONG);
+    schema1.addColumn(FieldName1, Type.BLOB);
+    schema1.addColumn(FieldName2, Type.INT4);
+    schema1.addColumn(FieldName3, Type.INT8);
     
-	  TableMeta info = TCatUtil.newTableMeta(schema1, StoreType.CSV);
+	  TableMeta info = CatalogUtil.newTableMeta(schema1, StoreType.CSV);
 	  TableDesc desc = new TableDescImpl();
 	  desc.setMeta(info);
 	  
@@ -103,15 +104,15 @@ public class TestCatalog {
 
   static {
     desc1 = new IndexDesc(
-        "idx_test", "indexed", new Column("id", DataType.INT),
+        "idx_test", "indexed", new Column("id", Type.INT4),
         IndexMethod.TWO_LEVEL_BIN_TREE, true, true, true);
 
     desc2 = new IndexDesc(
-        "idx_test2", "indexed", new Column("score", DataType.DOUBLE),
+        "idx_test2", "indexed", new Column("score", Type.FLOAT8),
         IndexMethod.TWO_LEVEL_BIN_TREE, false, false, false);
 
     desc3 = new IndexDesc(
-        "idx_test", "indexed", new Column("id", DataType.INT),
+        "idx_test", "indexed", new Column("id", Type.INT4),
         IndexMethod.TWO_LEVEL_BIN_TREE, true, true, true);
   }
 	
@@ -144,7 +145,7 @@ public class TestCatalog {
 		public TestFunc1() {
 			super(					
 					new Column [] {
-							new Column("name", DataType.INT)
+							new Column("name", TajoDataTypes.Type.INT4)
 					}
 			);
 		}
@@ -160,8 +161,8 @@ public class TestCatalog {
     public TestFunc2() {
       super(
           new Column [] {
-              new Column("name", DataType.INT),
-              new Column("bytes", DataType.BYTES)
+              new Column("name", TajoDataTypes.Type.INT4),
+              new Column("bytes", TajoDataTypes.Type.BLOB)
           }
       );
     }
@@ -176,12 +177,12 @@ public class TestCatalog {
 	public final void testRegisterFunc() throws Exception { 
 		assertFalse(catalog.containFunction("test2"));
 		FunctionDesc meta = new FunctionDesc("test2", TestFunc1.class, FunctionType.GENERAL,
-        new DataType [] {DataType.INT},
-		    new DataType [] {DataType.INT});
+        CatalogUtil.newDataTypesWithoutLen(Type.INT4),
+        CatalogUtil.newDataTypesWithoutLen(Type.INT4));
 
     catalog.registerFunction(meta);
-		assertTrue(catalog.containFunction("test2", DataType.INT));
-		FunctionDesc retrived = catalog.getFunction("test2", DataType.INT);
+		assertTrue(catalog.containFunction("test2", CatalogUtil.newDataTypesWithoutLen(Type.INT4)));
+		FunctionDesc retrived = catalog.getFunction("test2", CatalogUtil.newDataTypesWithoutLen(Type.INT4));
 
 		assertEquals(retrived.getSignature(),"test2");
 		assertEquals(retrived.getFuncClass(),TestFunc1.class);
@@ -191,20 +192,23 @@ public class TestCatalog {
   @Test
   public final void testUnregisterFunc() throws Exception {    
     assertFalse(catalog
-        .containFunction("test3", DataType.INT));
+        .containFunction("test3", CatalogUtil.newDataTypesWithoutLen(Type.INT4)));
     FunctionDesc meta = new FunctionDesc("test3", TestFunc1.class, FunctionType.GENERAL,
-        new DataType [] {DataType.INT}, new DataType[] { DataType.INT });
+        CatalogUtil.newDataTypesWithoutLen(Type.INT4),
+        CatalogUtil.newDataTypesWithoutLen(Type.INT4 ));
     catalog.registerFunction(meta);
-    assertTrue(catalog.containFunction("test3", DataType.INT));
-    catalog.unregisterFunction("test3", DataType.INT);
+    assertTrue(catalog.containFunction("test3", CatalogUtil.newDataTypesWithoutLen(Type.INT4)));
+    catalog.unregisterFunction("test3", CatalogUtil.newDataTypesWithoutLen(Type.INT4));
     assertFalse(catalog
-        .containFunction("test3", DataType.INT));
+        .containFunction("test3", CatalogUtil.newDataTypesWithoutLen(Type.INT4)));
 
-    assertFalse(catalog.containFunction("test3", DataType.INT, DataType.BYTES));
+    assertFalse(catalog.containFunction("test3",
+        CatalogUtil.newDataTypesWithoutLen(Type.INT4, Type.BLOB)));
     FunctionDesc overload = new FunctionDesc("test3", TestFunc2.class, FunctionType.GENERAL,
-        new DataType [] {DataType.INT},
-        new DataType [] {DataType.INT,DataType.BYTES});
+        CatalogUtil.newDataTypesWithoutLen(Type.INT4),
+        CatalogUtil.newDataTypesWithoutLen(Type.INT4, Type.BLOB));
     catalog.registerFunction(overload);
-    assertTrue(catalog.containFunction("test3", DataType.INT, DataType.BYTES));
+    assertTrue(catalog.containFunction("test3",
+        CatalogUtil.newDataTypesWithoutLen(Type.INT4, Type.BLOB)));
   }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-catalog/tajo-catalog-server/src/test/java/tajo/catalog/TestDBStore.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-server/src/test/java/tajo/catalog/TestDBStore.java b/tajo-catalog/tajo-catalog-server/src/test/java/tajo/catalog/TestDBStore.java
index 033efa3..8ed8fb3 100644
--- a/tajo-catalog/tajo-catalog-server/src/test/java/tajo/catalog/TestDBStore.java
+++ b/tajo-catalog/tajo-catalog-server/src/test/java/tajo/catalog/TestDBStore.java
@@ -25,10 +25,10 @@ import org.apache.hadoop.fs.Path;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
-import tajo.catalog.proto.CatalogProtos.DataType;
 import tajo.catalog.proto.CatalogProtos.StoreType;
 import tajo.catalog.statistics.TableStat;
 import tajo.catalog.store.DBStore;
+import tajo.common.TajoDataTypes.Type;
 import tajo.conf.TajoConf;
 import tajo.util.CommonTestingUtil;
 
@@ -59,15 +59,15 @@ public class TestDBStore {
   @Test
   public final void testAddAndDeleteTable() throws Exception {
     Schema schema = new Schema();
-    schema.addColumn("id", DataType.INT)
-    .addColumn("name", DataType.STRING)
-    .addColumn("age", DataType.INT)
-    .addColumn("score", DataType.DOUBLE);
+    schema.addColumn("id", Type.INT4)
+    .addColumn("name", Type.TEXT)
+    .addColumn("age", Type.INT4)
+    .addColumn("score", Type.FLOAT8);
     
     String tableName = "addedtable";
     Options opts = new Options();
     opts.put("file.delimiter", ",");
-    TableMeta meta = TCatUtil.newTableMeta(schema, StoreType.CSV, opts);
+    TableMeta meta = CatalogUtil.newTableMeta(schema, StoreType.CSV, opts);
     TableDesc desc = new TableDescImpl(tableName, meta, new Path("/addedtable"));
     assertFalse(store.existTable(tableName));
     store.addTable(desc);
@@ -83,15 +83,15 @@ public class TestDBStore {
   @Test
   public final void testGetTable() throws Exception {
     Schema schema = new Schema();
-    schema.addColumn("gettable.id", DataType.INT)
-    .addColumn("gettable.name", DataType.STRING)
-    .addColumn("gettable.age", DataType.INT)
-    .addColumn("gettable.score", DataType.DOUBLE);
+    schema.addColumn("gettable.id", Type.INT4)
+    .addColumn("gettable.name", Type.TEXT)
+    .addColumn("gettable.age", Type.INT4)
+    .addColumn("gettable.score", Type.FLOAT8);
     
     String tableName = "gettable";
     Options opts = new Options();
     opts.put("file.delimiter", ",");
-    TableMeta meta = TCatUtil.newTableMeta(schema, StoreType.CSV, opts);
+    TableMeta meta = CatalogUtil.newTableMeta(schema, StoreType.CSV, opts);
 
     TableStat stat = new TableStat();
     stat.setNumRows(957685);
@@ -114,15 +114,15 @@ public class TestDBStore {
   @Test
   public final void testGetAllTableNames() throws Exception {
     Schema schema = new Schema();
-    schema.addColumn("id", DataType.INT)
-    .addColumn("name", DataType.STRING)
-    .addColumn("age", DataType.INT)
-    .addColumn("score", DataType.DOUBLE);
+    schema.addColumn("id", Type.INT4)
+    .addColumn("name", Type.TEXT)
+    .addColumn("age", Type.INT4)
+    .addColumn("score", Type.FLOAT8);
     
     int numTables = 5;
     for (int i = 0; i < numTables; i++) {
       String tableName = "tableA_" + i;
-      TableMeta meta = TCatUtil.newTableMeta(schema, StoreType.CSV);
+      TableMeta meta = CatalogUtil.newTableMeta(schema, StoreType.CSV);
       TableDesc desc = new TableDescImpl(tableName, meta, 
           new Path("/tableA_" + i));
       store.addTable(desc);
@@ -196,14 +196,14 @@ public class TestDBStore {
   
   public static TableDesc prepareTable() {
     Schema schema = new Schema();
-    schema.addColumn("indexed.id", DataType.INT)
-    .addColumn("indexed.name", DataType.STRING)
-    .addColumn("indexed.age", DataType.INT)
-    .addColumn("indexed.score", DataType.DOUBLE);
+    schema.addColumn("indexed.id", Type.INT4)
+    .addColumn("indexed.name", Type.TEXT)
+    .addColumn("indexed.age", Type.INT4)
+    .addColumn("indexed.score", Type.FLOAT8);
     
     String tableName = "indexed";
     
-    TableMeta meta = TCatUtil.newTableMeta(schema, StoreType.CSV);
+    TableMeta meta = CatalogUtil.newTableMeta(schema, StoreType.CSV);
     return new TableDescImpl(tableName, meta, new Path("/indexed"));
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-catalog/tajo-catalog-server/src/test/java/tajo/catalog/statistics/TestColumnStat.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-server/src/test/java/tajo/catalog/statistics/TestColumnStat.java b/tajo-catalog/tajo-catalog-server/src/test/java/tajo/catalog/statistics/TestColumnStat.java
index 9af4453..cffd42f 100644
--- a/tajo-catalog/tajo-catalog-server/src/test/java/tajo/catalog/statistics/TestColumnStat.java
+++ b/tajo-catalog/tajo-catalog-server/src/test/java/tajo/catalog/statistics/TestColumnStat.java
@@ -20,7 +20,7 @@ package tajo.catalog.statistics;
 
 import org.junit.Test;
 import tajo.catalog.Column;
-import tajo.catalog.proto.CatalogProtos;
+import tajo.common.TajoDataTypes.Type;
 import tajo.datum.DatumFactory;
 
 import static org.junit.Assert.assertEquals;
@@ -30,7 +30,7 @@ public class TestColumnStat {
 
   @Test
   public final void testColumnStat() {
-    ColumnStat stat = new ColumnStat(new Column("test", CatalogProtos.DataType.LONG));
+    ColumnStat stat = new ColumnStat(new Column("test", Type.INT8));
     stat.setNumDistVals(1000);
     stat.setNumNulls(999);
     
@@ -44,11 +44,11 @@ public class TestColumnStat {
 
   @Test
   public final void testEqualsObject() {
-    ColumnStat stat = new ColumnStat(new Column("test", CatalogProtos.DataType.LONG));
+    ColumnStat stat = new ColumnStat(new Column("test", Type.INT8));
     stat.setNumDistVals(1000);
     stat.setNumNulls(999);
-    stat.setMinValue(DatumFactory.createLong(5));
-    stat.setMaxValue(DatumFactory.createLong(10));
+    stat.setMinValue(DatumFactory.createInt8(5));
+    stat.setMaxValue(DatumFactory.createInt8(10));
     
     ColumnStat stat2 = new ColumnStat(stat.getProto());
     assertEquals(stat, stat2);
@@ -56,11 +56,11 @@ public class TestColumnStat {
 
   @Test
   public final void testClone() throws CloneNotSupportedException {
-    ColumnStat stat = new ColumnStat(new Column("test", CatalogProtos.DataType.LONG));
+    ColumnStat stat = new ColumnStat(new Column("test", Type.INT8));
     stat.setNumDistVals(1000);
     stat.setNumNulls(999);
-    stat.setMinValue(DatumFactory.createLong(5));
-    stat.setMaxValue(DatumFactory.createLong(10));
+    stat.setMinValue(DatumFactory.createInt8(5));
+    stat.setMaxValue(DatumFactory.createInt8(10));
     
     ColumnStat stat2 = (ColumnStat) stat.clone();
     assertEquals(stat, stat2);

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-catalog/tajo-catalog-server/src/test/java/tajo/catalog/statistics/TestTableStat.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-server/src/test/java/tajo/catalog/statistics/TestTableStat.java b/tajo-catalog/tajo-catalog-server/src/test/java/tajo/catalog/statistics/TestTableStat.java
index 3237d25..d679c69 100644
--- a/tajo-catalog/tajo-catalog-server/src/test/java/tajo/catalog/statistics/TestTableStat.java
+++ b/tajo-catalog/tajo-catalog-server/src/test/java/tajo/catalog/statistics/TestTableStat.java
@@ -20,7 +20,7 @@ package tajo.catalog.statistics;
 
 import org.junit.Test;
 import tajo.catalog.Column;
-import tajo.catalog.proto.CatalogProtos;
+import tajo.common.TajoDataTypes.Type;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
@@ -38,7 +38,7 @@ public class TestTableStat {
     int numCols = 3;
     ColumnStat[] cols = new ColumnStat[numCols];
     for (int i = 0; i < numCols; i++) {
-      cols[i] = new ColumnStat(new Column("col_" + i, CatalogProtos.DataType.LONG));
+      cols[i] = new ColumnStat(new Column("col_" + i, Type.INT8));
       cols[i].setNumDistVals(1024 * i);
       cols[i].setNumNulls(100 * i);
       stat.addColumnStat(cols[i]);

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-common/pom.xml
----------------------------------------------------------------------
diff --git a/tajo-common/pom.xml b/tajo-common/pom.xml
index 3bd06ef..4674ec1 100644
--- a/tajo-common/pom.xml
+++ b/tajo-common/pom.xml
@@ -90,6 +90,7 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xs
                 <argument>--java_out=target/generated-sources/proto</argument>
                 <argument>src/main/proto/TestProtos.proto</argument>
                 <argument>src/main/proto/TajoIdProtos.proto</argument>
+                <argument>src/main/proto/DataTypes.proto</argument>
               </arguments>
             </configuration>
             <goals>

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-common/src/main/java/tajo/datum/ArrayDatum.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/tajo/datum/ArrayDatum.java b/tajo-common/src/main/java/tajo/datum/ArrayDatum.java
index 12151ed..e1bc84c 100644
--- a/tajo-common/src/main/java/tajo/datum/ArrayDatum.java
+++ b/tajo-common/src/main/java/tajo/datum/ArrayDatum.java
@@ -21,15 +21,17 @@ package tajo.datum;
 import com.google.gson.annotations.Expose;
 import tajo.datum.json.GsonCreator;
 
+import static tajo.common.TajoDataTypes.Type;
+
 public class ArrayDatum extends Datum {
   @Expose private Datum [] data;
   public ArrayDatum(Datum [] data) {
-    super(DatumType.ARRAY);
+    super(Type.ARRAY);
     this.data = data;
   }
 
   public ArrayDatum(int size) {
-    super(DatumType.ARRAY);
+    super(Type.ARRAY);
     this.data = new Datum[size];
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-common/src/main/java/tajo/datum/BoolDatum.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/tajo/datum/BoolDatum.java b/tajo-common/src/main/java/tajo/datum/BoolDatum.java
deleted file mode 100644
index 0b1982a..0000000
--- a/tajo-common/src/main/java/tajo/datum/BoolDatum.java
+++ /dev/null
@@ -1,172 +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 tajo.datum;
-
-import com.google.gson.annotations.Expose;
-import tajo.datum.exception.InvalidOperationException;
-import tajo.datum.json.GsonCreator;
-
-public class BoolDatum extends Datum {
-	@Expose private boolean val;
-
-  public BoolDatum() {
-    super(DatumType.BOOLEAN);
-  }
-
-	public BoolDatum(boolean val) {
-		this();
-		this.val = val;
-	}
-
-  public BoolDatum(byte byteVal) {
-    this();
-    this.val = byteVal == 1;
-  }
-
-  public BoolDatum(int byteVal) {
-    this();
-    this.val = byteVal == 1;
-  }
-
-
-  public BoolDatum(byte [] bytes) {
-    this(bytes[0]);
-  }
-	
-	public boolean asBool() {
-		return val;
-	}
-
-  public void setValue(boolean val) {
-    this.val = val;
-  }
-	
-	@Override
-	public short asShort() {	
-		return (short) (val ? 1 : 0);
-	}
-
-	/* (non-Javadoc)
-	 * @see nta.common.datum.Datum#asInt()
-	 */
-	@Override
-	public int asInt() {
-		return val ? 1 : 0;
-	}
-
-	/* (non-Javadoc)
-	 * @see nta.common.datum.Datum#asLong()
-	 */
-	@Override
-	public long asLong() {
-		return val ? 1 : 0;
-	}
-
-	/* (non-Javadoc)
-	 * @see nta.common.datum.Datum#asByte()
-	 */
-	@Override
-	public byte asByte() {
-		return (byte) (val ? 0x01 : 0x00);
-	}
-
-	/* (non-Javadoc)
-	 * @see nta.common.datum.Datum#asByteArray()
-	 */
-	@Override
-	public byte[] asByteArray() {
-	  byte [] bytes = new byte[1];
-    bytes[0] = asByte();
-	  return bytes;
-	}
-
-	/* (non-Javadoc)
-	 * @see nta.common.datum.Datum#asFloat()
-	 */
-	@Override
-	public float asFloat() {
-		return val ? 1 : 0;
-	}
-
-	/* (non-Javadoc)
-	 * @see nta.common.datum.Datum#asDouble()
-	 */
-	@Override
-	public double asDouble() {
-		return val ? 1 : 0;
-	}
-
-	/* (non-Javadoc)
-	 * @see nta.common.datum.Datum#asChars()
-	 */
-	@Override
-	public String asChars() {
-		return val ? "true" : "false";
-	}
-	
-	public String toJSON() {
-		return GsonCreator.getInstance().toJson(this, Datum.class);
-	}
-
-  @Override
-  public int size() {
-    return 1;
-  }
-  
-  @Override
-  public int hashCode() {
-    return val ? 7907 : 0; // 7907 is one of the prime numbers
-  }
-  
-  @Override
-  public boolean equals(Object obj) {
-    if (obj instanceof BoolDatum) {
-      BoolDatum other = (BoolDatum) obj;
-      return val == other.val;
-    }
-    
-    return false;
-  }
-  
-  // Datum Comparator
-  public BoolDatum equalsTo(Datum datum) {
-    switch(datum.type()) {
-      case BOOLEAN: return DatumFactory.createBool(this.val == 
-          ((BoolDatum)datum).val);
-      default:
-        throw new InvalidOperationException(datum.type());
-    }
-  }
-
-  @Override
-  public int compareTo(Datum datum) {
-    switch (datum.type()) {
-    case BOOLEAN:
-      if (val && !datum.asBool()) {
-        return -1;
-      } else if (val && datum.asBool()) {
-        return 1;
-      } else {
-        return 0;
-      }
-    default:
-      throw new InvalidOperationException(datum.type());
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-common/src/main/java/tajo/datum/ByteDatum.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/tajo/datum/ByteDatum.java b/tajo-common/src/main/java/tajo/datum/ByteDatum.java
deleted file mode 100644
index 3bce3ff..0000000
--- a/tajo-common/src/main/java/tajo/datum/ByteDatum.java
+++ /dev/null
@@ -1,134 +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 tajo.datum;
-
-import com.google.gson.annotations.Expose;
-import tajo.datum.exception.InvalidOperationException;
-import tajo.datum.json.GsonCreator;
-
-public class ByteDatum extends Datum {
-  private static final int size = 1;
-  
-  @Expose	byte val;
-	
-	public ByteDatum() {
-		super(DatumType.BYTE);
-	}
-	
-	public ByteDatum(byte val) {
-		this();
-		this.val = val;
-	}
-
-  public ByteDatum(byte [] bytes) {
-    this(bytes[0]);
-  }
-
-  @Override
-  public char asChar() {
-    return (char)val;
-  }
-
-	@Override
-	public int asInt() {		
-		return val;
-	}
-
-  @Override
-	public long asLong() {
-		return val;
-	}
-
-  @Override
-	public byte asByte() {
-		return val;
-	}
-
-  @Override
-	public byte[] asByteArray() {
-    byte [] bytes = new byte[1];
-    bytes[0] = this.val;
-		return bytes;
-	}
-
-  @Override
-	public float asFloat() {		
-		return val;
-	}
-
-  @Override
-	public double asDouble() {
-		return val;
-	}
-
-  @Override
-	public String asChars() {
-		return "0x"+val;
-	}
-	
-	public String toJSON() {
-		return GsonCreator.getInstance().toJson(this, Datum.class);
-	}
-
-  @Override
-  public int size() {
-    return size;
-  }
-  
-  @Override
-  public int hashCode() {
-    return val;
-  }
-
-  @Override
-  public boolean equals(Object obj) {
-    if (obj instanceof ByteDatum) {
-      ByteDatum other = (ByteDatum) obj;
-      return val == other.val;
-    }
-    
-    return false;
-  }
-
-  @Override
-  public BoolDatum equalsTo(Datum datum) {
-    switch (datum.type()) {
-    case BYTE:
-      return DatumFactory.createBool(this.val == (((ByteDatum) datum).val));
-    default:
-      throw new InvalidOperationException(datum.type());
-    }
-  }
-  
-  @Override
-  public int compareTo(Datum datum) {
-    switch (datum.type()) {
-    case BYTE:
-      if (val < datum.asByte() ) {
-        return -1;
-      } else if (val > datum.asByte()) {
-        return 1;
-      } else {
-        return 0;
-      }
-    default:
-      throw new InvalidOperationException(datum.type());
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-common/src/main/java/tajo/datum/BytesDatum.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/tajo/datum/BytesDatum.java b/tajo-common/src/main/java/tajo/datum/BytesDatum.java
deleted file mode 100644
index 1111ead..0000000
--- a/tajo-common/src/main/java/tajo/datum/BytesDatum.java
+++ /dev/null
@@ -1,161 +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 tajo.datum;
-
-import com.google.gson.annotations.Expose;
-import tajo.datum.exception.InvalidOperationException;
-import tajo.datum.json.GsonCreator;
-
-import java.nio.ByteBuffer;
-import java.nio.charset.Charset;
-import java.util.Arrays;
-
-public class BytesDatum extends Datum {
-	@Expose private byte[] val;
-	private ByteBuffer bb = null;
-
-	public BytesDatum() {
-		super(DatumType.BYTES);
-	}
-	
-	public BytesDatum(byte [] val) {
-		this();
-		this.val = val;
-		this.bb = ByteBuffer.wrap(val);	
-		bb.flip();
-	}
-	
-	public BytesDatum(ByteBuffer val) {
-		this();
-		this.val = val.array();
-		this.bb = val.duplicate();
-		bb.flip();
-	}
-	
-	public void initFromBytes() {
-		if (bb == null) {
-			bb = ByteBuffer.wrap(val);
-		}
-	}
-
-  @Override
-	public int asInt() {
-		initFromBytes();
-		bb.rewind();
-		return bb.getInt();
-	}
-
-  @Override
-	public long asLong() {
-		initFromBytes();
-		bb.rewind();
-		return bb.getLong();
-	}
-
-  @Override
-	public byte asByte() {
-		initFromBytes();
-		bb.rewind();
-		return bb.get();
-	}
-
-  @Override
-	public byte[] asByteArray() {
-		initFromBytes();
-		bb.rewind();
-		return bb.array();
-	}
-
-  @Override
-	public float asFloat() {
-		initFromBytes();
-		bb.rewind();
-		return bb.getFloat();
-	}
-
-  @Override
-	public double asDouble() {
-		initFromBytes();
-		bb.rewind();
-		return bb.getDouble();
-	}
-
-  @Override
-	public String asChars() {
-		initFromBytes();
-		bb.rewind();
-		return new String(bb.array(), Charset.defaultCharset());
-	}
-
-  @Override
-	public String toJSON() {
-		return GsonCreator.getInstance().toJson(this, Datum.class);
-	}
-
-  @Override
-  public int size() {
-	  return this.val.length;
-  }
-  
-  @Override
-  public int hashCode() {
-	  initFromBytes();
-	  bb.rewind();
-    return bb.hashCode();
-  }
-
-  @Override
-  public boolean equals(Object obj) {
-    if (obj instanceof BytesDatum) {
-      BytesDatum other = (BytesDatum) obj;
-      initFromBytes();
-      other.initFromBytes();
-      return bb.equals(other.bb);
-    }
-    
-    return false;
-  }
-
-  @Override
-  public BoolDatum equalsTo(Datum datum) {
-    switch (datum.type()) {
-    case BYTES:
-    	initFromBytes();
-    	((BytesDatum)datum).initFromBytes();
-      return DatumFactory.createBool(Arrays.equals(this.val, ((BytesDatum)datum).val));
-    default:
-      throw new InvalidOperationException(datum.type());
-    }
-  }
-
-  @Override
-  public int compareTo(Datum datum) {
-    switch (datum.type()) {
-    case BYTES:
-    	initFromBytes();
-    	((BytesDatum)datum).initFromBytes();
-      return bb.compareTo(((BytesDatum) datum).bb);
-    default:
-      throw new InvalidOperationException(datum.type());
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-common/src/main/java/tajo/datum/CharDatum.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/tajo/datum/CharDatum.java b/tajo-common/src/main/java/tajo/datum/CharDatum.java
index ff82493..0dc11f8 100644
--- a/tajo-common/src/main/java/tajo/datum/CharDatum.java
+++ b/tajo-common/src/main/java/tajo/datum/CharDatum.java
@@ -22,12 +22,14 @@ import com.google.gson.annotations.Expose;
 import tajo.datum.exception.InvalidOperationException;
 import tajo.datum.json.GsonCreator;
 
+import static tajo.common.TajoDataTypes.Type;
+
 public class CharDatum extends Datum {
   private static final int size = 1;
   @Expose char val;
 
 	public CharDatum() {
-		super(DatumType.CHAR);
+		super(Type.CHAR);
 	}
 
 	public CharDatum(byte val) {
@@ -50,12 +52,12 @@ public class CharDatum extends Datum {
   }
 
   @Override
-	public int asInt() {		
+	public int asInt4() {
 		return val;
 	}
 
   @Override
-	public long asLong() {
+	public long asInt8() {
 		return val;
 	}
 
@@ -72,12 +74,12 @@ public class CharDatum extends Datum {
 	}
 
   @Override
-	public float asFloat() {		
+	public float asFloat4() {
 		return val;
 	}
 
   @Override
-	public double asDouble() {
+	public double asFloat8() {
 		return val;
 	}
 
@@ -112,9 +114,9 @@ public class CharDatum extends Datum {
   }
 
   @Override
-  public BoolDatum equalsTo(Datum datum) {
+  public BooleanDatum equalsTo(Datum datum) {
     switch (datum.type()) {
-    case BYTE:
+    case CHAR:
       return DatumFactory.createBool(this.val == (((CharDatum) datum).val));
     default:
       throw new InvalidOperationException(datum.type());
@@ -124,7 +126,6 @@ public class CharDatum extends Datum {
   @Override
   public int compareTo(Datum datum) {
     switch (datum.type()) {
-      case BYTE:
       case CHAR:
       if (val < datum.asChar()) {
         return -1;

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-common/src/main/java/tajo/datum/Datum.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/tajo/datum/Datum.java b/tajo-common/src/main/java/tajo/datum/Datum.java
index c9e0fb9..f4772ef 100644
--- a/tajo-common/src/main/java/tajo/datum/Datum.java
+++ b/tajo-common/src/main/java/tajo/datum/Datum.java
@@ -22,24 +22,26 @@ import com.google.gson.annotations.Expose;
 import tajo.datum.exception.InvalidCastException;
 import tajo.datum.exception.InvalidOperationException;
 
+import static tajo.common.TajoDataTypes.Type;
+
 public abstract class Datum implements Comparable<Datum> {
 	@Expose
-	private DatumType type;
+	private Type type;
 	
 	@SuppressWarnings("unused")
   private Datum() {
 	}
 	
-	public Datum(DatumType type) {
+	public Datum(Type type) {
 		this.type = type;
 	}
 	
-	public DatumType type() {
+	public Type type() {
 		return this.type;
 	}
 
   public boolean isNull() {
-    return type == DatumType.NULL;
+    return false;
   }
 	
 	public boolean asBool() {
@@ -54,14 +56,14 @@ public abstract class Datum implements Comparable<Datum> {
     throw new InvalidCastException(type + " cannot be casted to CHAR type");
   }
 
-	public short asShort() {
+	public short asInt2() {
     throw new InvalidCastException(type + " cannot be casted to SHORT type");
   }
-	public int asInt() {
+	public int asInt4() {
     throw new InvalidCastException(type + " cannot be casted to INT type");
   }
 
-  public long asLong() {
+  public long asInt8() {
     throw new InvalidCastException(type + " cannot be casted to LONG type");
   }
 
@@ -69,11 +71,11 @@ public abstract class Datum implements Comparable<Datum> {
     throw new InvalidCastException(type + " cannot be casted to BYTES type");
   }
 
-	public float asFloat() {
+	public float asFloat4() {
     throw new InvalidCastException(type + " cannot be casted to FLOAT type");
   }
 
-	public double asDouble() {
+	public double asFloat8() {
     throw new InvalidCastException(type + " cannot be casted to DOUBLE type");
   }
 
@@ -87,15 +89,15 @@ public abstract class Datum implements Comparable<Datum> {
 	
 	public boolean isNumber() {
 	  return 
-	      this.type == DatumType.SHORT ||
-	      this.type == DatumType.INT ||
-	      this.type == DatumType.LONG;
+	      this.type == Type.INT2 ||
+	      this.type == Type.INT4 ||
+	      this.type == Type.INT8;
 	}
 	
 	public boolean isReal() {
     return 
-        this.type == DatumType.FLOAT ||
-        this.type == DatumType.DOUBLE;
+        this.type == Type.FLOAT4||
+        this.type == Type.FLOAT8;
   }
 	
 	public abstract int size();
@@ -120,8 +122,8 @@ public abstract class Datum implements Comparable<Datum> {
     throw new InvalidOperationException(datum.type);
   }
 	
-	public BoolDatum equalsTo(Datum datum) {
-    if (this.isNull() || datum.isNull()) {
+	public BooleanDatum equalsTo(Datum datum) {
+    if (this instanceof NullDatum || datum instanceof NullDatum) {
     // TODO - comparing any value against null will be always unknown
       return DatumFactory.createBool(false);
     } else {
@@ -129,19 +131,19 @@ public abstract class Datum implements Comparable<Datum> {
     }
 	}
 
-	public BoolDatum lessThan(Datum datum) {
+	public BooleanDatum lessThan(Datum datum) {
     return DatumFactory.createBool(compareTo(datum) < 0);
 	}
 	
-	public BoolDatum lessThanEqual(Datum datum) {
+	public BooleanDatum lessThanEqual(Datum datum) {
     return DatumFactory.createBool(compareTo(datum) <= 0);
 	}	
 	
-	public BoolDatum greaterThan(Datum datum) {
+	public BooleanDatum greaterThan(Datum datum) {
     return DatumFactory.createBool(compareTo(datum) > 0);
 	}
 	
-	public BoolDatum greaterThanEqual(Datum datum) {
+	public BooleanDatum greaterThanEqual(Datum datum) {
     return DatumFactory.createBool(compareTo(datum) >= 0);
 	}
 	

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-common/src/main/java/tajo/datum/DatumFactory.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/tajo/datum/DatumFactory.java b/tajo-common/src/main/java/tajo/datum/DatumFactory.java
index 7f19262..2e1cd79 100644
--- a/tajo-common/src/main/java/tajo/datum/DatumFactory.java
+++ b/tajo-common/src/main/java/tajo/datum/DatumFactory.java
@@ -23,18 +23,18 @@ public class DatumFactory {
     return NullDatum.get();
   }
   
-  public static BoolDatum createBool(String val) {
+  public static BooleanDatum createBool(String val) {
     boolean boolVal = val.equalsIgnoreCase("true");
-    return new BoolDatum(boolVal);
+    return new BooleanDatum(boolVal);
   }
   
-  public static BoolDatum createBool(byte val) {
+  public static BooleanDatum createBool(byte val) {
     boolean boolVal = val == 0x01;
-    return new BoolDatum(boolVal);
+    return new BooleanDatum(boolVal);
   }
   
-  public static BoolDatum createBool(boolean val) {
-    return new BoolDatum(val);
+  public static BooleanDatum createBool(boolean val) {
+    return new BooleanDatum(val);
   }
   /*
   public static BoolDatum createBool(int val) {
@@ -42,11 +42,11 @@ public class DatumFactory {
   }
   */
   
-	public static ByteDatum createByte(byte val) {
-		return new ByteDatum(val);
+	public static BitDatum createBit(byte val) {
+		return new BitDatum(val);
 	}
   /*
-  public static ByteDatum createByte(int val) {
+  public static ByteDatum createBit(int val) {
     return new ByteDatum(val);
   }*/
 
@@ -64,75 +64,67 @@ public class DatumFactory {
   }
   */
 	
-	public static ShortDatum createShort(short val) {
-		return new ShortDatum(val);
+	public static Int2Datum createInt2(short val) {
+		return new Int2Datum(val);
 	}
 	
-	public static ShortDatum createShort(String val) {
-	  return new ShortDatum(Short.valueOf(val));
+	public static Int2Datum createInt2(String val) {
+	  return new Int2Datum(Short.valueOf(val));
 	}
 	
-	public static IntDatum createInt(int val) {
-		return new IntDatum(val);
+	public static Int4Datum createInt4(int val) {
+		return new Int4Datum(val);
 	}
 	
-	public static IntDatum createInt(String val) {
-	  return new IntDatum(Integer.valueOf(val));
+	public static Int4Datum createInt4(String val) {
+	  return new Int4Datum(Integer.valueOf(val));
 	}
 	
-	public static LongDatum createLong(long val) {
-		return new LongDatum(val);
+	public static Int8Datum createInt8(long val) {
+		return new Int8Datum(val);
 	}
 	
-	public static LongDatum createLong(String val) {
-	  return new LongDatum(Long.valueOf(val));
+	public static Int8Datum createInt8(String val) {
+	  return new Int8Datum(Long.valueOf(val));
 	}
 	
-	public static FloatDatum createFloat(float val) {
-		return new FloatDatum(val);
+	public static Float4Datum createFloat4(float val) {
+		return new Float4Datum(val);
 	}
 	
-	public static FloatDatum createFloat(String val) {
-	  return new FloatDatum(Float.valueOf(val));
+	public static Float4Datum createFloat4(String val) {
+	  return new Float4Datum(Float.valueOf(val));
 	}
 	
-	public static DoubleDatum createDouble(double val) {
-		return new DoubleDatum(val);
+	public static Float8Datum createFloat8(double val) {
+		return new Float8Datum(val);
 	}
 	
-	public static DoubleDatum createDouble(String val) {
-	  return new DoubleDatum(Double.valueOf(val));
+	public static Float8Datum createFloat8(String val) {
+	  return new Float8Datum(Double.valueOf(val));
 	}
 	
-  public static StringDatum createString(String val) {
-    return new StringDatum(val);
-  }
-
-  public static StringDatum2 createString2(String val) {
-    return new StringDatum2(val);
-  }
-
-  public static StringDatum2 createString2(byte[] val) {
-    return new StringDatum2(val);
+  public static TextDatum createText(String val) {
+    return new TextDatum(val);
   }
 
-  public static StringDatum createString(byte[] val) {
-    return new StringDatum(val);
+  public static TextDatum createText(byte[] val) {
+    return new TextDatum(val);
   }
 	
-	public static BytesDatum createBytes(byte [] val) {
-    return new BytesDatum(val);
+	public static BlobDatum createBlob(byte[] val) {
+    return new BlobDatum(val);
   }
 	
-	public static BytesDatum createBytes(String val) {
-	  return new BytesDatum(val.getBytes());
+	public static BlobDatum createBlob(String val) {
+	  return new BlobDatum(val.getBytes());
 	}
 	
-	public static IPv4Datum createIPv4(byte [] val) {
-	  return new IPv4Datum(val);
+	public static Inet4Datum createInet4(byte[] val) {
+	  return new Inet4Datum(val);
 	}
 	
-	public static IPv4Datum createIPv4(String val) {
-	  return new IPv4Datum(val);
+	public static Inet4Datum createInet4(String val) {
+	  return new Inet4Datum(val);
 	}
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-common/src/main/java/tajo/datum/DatumType.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/tajo/datum/DatumType.java b/tajo-common/src/main/java/tajo/datum/DatumType.java
deleted file mode 100644
index 5a26cb8..0000000
--- a/tajo-common/src/main/java/tajo/datum/DatumType.java
+++ /dev/null
@@ -1,42 +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 tajo.datum;
-
-public enum DatumType {
-  NULL,
-	BOOLEAN,
-	BYTE,
-  CHAR,
-	BYTES,
-	SHORT,
-	INT,
-	LONG,
-	FLOAT,
-	DOUBLE,
-	STRING,
-  STRING2,
-	ENUM,
-	IPv4,
-	IPv6,
-	ALL,
-  ARRAY
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-common/src/main/java/tajo/datum/DoubleDatum.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/tajo/datum/DoubleDatum.java b/tajo-common/src/main/java/tajo/datum/DoubleDatum.java
deleted file mode 100644
index 0b48fa4..0000000
--- a/tajo-common/src/main/java/tajo/datum/DoubleDatum.java
+++ /dev/null
@@ -1,270 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-/**
- * 
- */
-package tajo.datum;
-
-import com.google.gson.annotations.Expose;
-import tajo.datum.exception.InvalidOperationException;
-import tajo.datum.json.GsonCreator;
-
-import java.nio.ByteBuffer;
-
-public class DoubleDatum extends NumericDatum {
-  private static final int size = 8;
-  @Expose private double val;
-
-	public DoubleDatum() {
-		super(DatumType.DOUBLE);
-	}
-	
-	public DoubleDatum(double val) {
-		this();
-		this.val = val;
-	}
-
-  public DoubleDatum(byte [] bytes) {
-    this();
-    ByteBuffer bb = ByteBuffer.wrap(bytes);
-    this.val = bb.getDouble();
-  }
-	
-	@Override
-	public short asShort() {	
-		return (short) val;
-	}
-
-	@Override
-	public int asInt() {		
-		return (int) val;
-	}
-
-  @Override
-	public long asLong() {
-		return (long) val;
-	}
-
-  @Override
-	public byte[] asByteArray() {
-		ByteBuffer bb = ByteBuffer.allocate(8);
-		bb.putDouble(val);
-		return bb.array();
-	}
-
-  @Override
-	public float asFloat() {		
-		return (float) val;
-	}
-
-  @Override
-	public double asDouble() {
-		return val;
-	}
-
-  @Override
-	public String asChars() {
-		return ""+val;
-	}
-
-  @Override
-	public String toJSON() {
-		return GsonCreator.getInstance().toJson(this, Datum.class);
-	}
-
-  @Override
-  public int size() {
-    return size;
-  }
-  
-  @Override
-  public int hashCode() {
-    return (int) val;
-  }
-  
-  public boolean equals(Object obj) {
-    if (obj instanceof DoubleDatum) {
-      DoubleDatum other = (DoubleDatum) obj;
-      return val == other.val;
-    }
-    
-    return false;
-  }
-
-  @Override
-  public BoolDatum equalsTo(Datum datum) {
-    switch (datum.type()) {
-    case SHORT:
-      return DatumFactory.createBool(val == datum.asShort());
-    case INT:
-      return DatumFactory.createBool(val == datum.asInt());
-    case LONG:
-      return DatumFactory.createBool(val == datum.asLong());
-    case FLOAT:
-      return DatumFactory.createBool(val == datum.asFloat());
-    case DOUBLE:
-      return DatumFactory.createBool(val == datum.asDouble());
-    default:
-      throw new InvalidOperationException(datum.type());
-    }
-  }
-
-  @Override
-  public int compareTo(Datum datum) {
-    switch (datum.type()) {
-      case SHORT:
-        if (val < datum.asShort()) {
-          return -1;
-        } else if (datum.asShort() < val) {
-          return 1;
-        } else {
-          return 0;
-        }
-      case INT:
-        if (val < datum.asInt()) {
-          return -1;
-        } else if (datum.asInt() < val) {
-          return 1;
-        } else {
-          return 0;
-        }
-      case LONG:
-        if (val < datum.asLong()) {
-          return -1;
-        } else if (datum.asLong() < val) {
-          return 1;
-        } else {
-          return 0;
-        }
-      case FLOAT:
-        if (val < datum.asFloat()) {
-          return -1;
-        } else if (datum.asFloat() < val) {
-          return 1;
-        } else {
-          return 0;
-        }
-      case DOUBLE:
-        if (val < datum.asDouble()) {
-          return -1;
-        } else if (datum.asDouble() < val) {
-          return 1;
-        } else {
-          return 0;
-        }
-      default:
-        throw new InvalidOperationException(datum.type());
-    }
-  }
-
-  @Override
-  public Datum plus(Datum datum) {
-    switch (datum.type()) {
-    case SHORT:
-      return DatumFactory.createDouble(val + datum.asShort());
-    case INT:
-      return DatumFactory.createDouble(val + datum.asInt());
-    case LONG:
-      return DatumFactory.createDouble(val + datum.asLong());
-    case FLOAT:
-      return DatumFactory.createDouble(val + datum.asFloat());
-    case DOUBLE:
-      return DatumFactory.createDouble(val + datum.asDouble());
-    default:
-      throw new InvalidOperationException(datum.type());
-    }
-  }
-
-  @Override
-  public Datum minus(Datum datum) {
-    switch (datum.type()) {
-    case SHORT:
-      return DatumFactory.createDouble(val - datum.asShort());
-    case INT:
-      return DatumFactory.createDouble(val - datum.asInt());
-    case LONG:
-      return DatumFactory.createDouble(val - datum.asLong());
-    case FLOAT:
-      return DatumFactory.createDouble(val - datum.asFloat());
-    case DOUBLE:
-      return DatumFactory.createDouble(val - datum.asDouble());
-    default:
-      throw new InvalidOperationException(datum.type());
-    }
-  }
-
-  @Override
-  public Datum multiply(Datum datum) {
-    switch (datum.type()) {
-    case SHORT:
-      return DatumFactory.createDouble(val * datum.asShort());
-    case INT:
-      return DatumFactory.createDouble(val * datum.asInt());
-    case LONG:
-      return DatumFactory.createDouble(val * datum.asLong());
-    case FLOAT:
-      return DatumFactory.createDouble(val * datum.asFloat());
-    case DOUBLE:
-      return DatumFactory.createDouble(val * datum.asDouble());
-    default:
-      throw new InvalidOperationException();
-    }
-  }
-
-  @Override
-  public Datum divide(Datum datum) {
-    switch (datum.type()) {
-    case SHORT:
-      return DatumFactory.createDouble(val / datum.asShort());
-    case INT:
-      return DatumFactory.createDouble(val / datum.asInt());
-    case LONG:
-      return DatumFactory.createDouble(val / datum.asLong());
-    case FLOAT:
-      return DatumFactory.createDouble(val / datum.asFloat());
-    case DOUBLE:
-      return DatumFactory.createDouble(val / datum.asDouble());
-    default:
-      throw new InvalidOperationException(datum.type());
-    }
-  }
-
-  @Override
-  public Datum modular(Datum datum) {
-    switch (datum.type()) {
-      case SHORT:
-        return DatumFactory.createDouble(val % datum.asShort());
-      case INT:
-        return DatumFactory.createDouble(val % datum.asInt());
-      case LONG:
-        return DatumFactory.createDouble(val % datum.asLong());
-      case FLOAT:
-        return DatumFactory.createDouble(val % datum.asFloat());
-      case DOUBLE:
-        return DatumFactory.createDouble(val % datum.asDouble());
-      default:
-        throw new InvalidOperationException(datum.type());
-    }
-  }
-  
-  @Override
-  public void inverseSign() {   
-    this.val = -val;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-common/src/main/java/tajo/datum/FloatDatum.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/tajo/datum/FloatDatum.java b/tajo-common/src/main/java/tajo/datum/FloatDatum.java
deleted file mode 100644
index 1a6da49..0000000
--- a/tajo-common/src/main/java/tajo/datum/FloatDatum.java
+++ /dev/null
@@ -1,278 +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 tajo.datum;
-
-import com.google.gson.annotations.Expose;
-import tajo.datum.exception.InvalidCastException;
-import tajo.datum.exception.InvalidOperationException;
-import tajo.datum.json.GsonCreator;
-
-import java.nio.ByteBuffer;
-
-public class FloatDatum extends NumericDatum {
-  private static final int size = 4;
-  @Expose float val;
-
-	public FloatDatum() {
-		super(DatumType.FLOAT);
-	}
-	
-	public FloatDatum(float val) {
-		this();
-		this.val = val;
-	}
-
-  public FloatDatum(byte [] bytes) {
-    this();
-    ByteBuffer bb = ByteBuffer.wrap(bytes);
-    this.val = bb.getFloat();
-  }
-	
-	public boolean asBool() {
-		throw new InvalidCastException();
-	}
-	
-	@Override
-	public short asShort() {	
-		return (short) val;
-	}
-
-  @Override
-	public int asInt() {		
-		return (int) val;
-	}
-
-  @Override
-	public long asLong() {
-		return (long) val;
-	}
-
-  @Override
-	public byte asByte() {
-		throw new InvalidCastException();
-	}
-
-  @Override
-	public byte[] asByteArray() {
-		ByteBuffer bb = ByteBuffer.allocate(4);
-		bb.putFloat(val);
-		return bb.array();
-	}
-
-  @Override
-	public float asFloat() {		
-		return val;
-	}
-
-  @Override
-	public double asDouble() {
-		return val;
-	}
-
-  @Override
-	public String asChars() {
-		return ""+val;
-	}
-
-  @Override
-	public String toJSON() {
-		return GsonCreator.getInstance().toJson(this, Datum.class);
-	}
-
-  @Override
-  public int size() {
-    return size;
-  }
-  
-  @Override
-  public int hashCode() {
-    return (int) val;
-  }
-
-  @Override
-  public boolean equals(Object obj) {
-    if (obj instanceof FloatDatum) {
-      FloatDatum other = (FloatDatum) obj;
-      return val == other.val;
-    }
-    
-    return false;
-  }
-
-  @Override
-  public BoolDatum equalsTo(Datum datum) {
-    switch (datum.type()) {
-    case SHORT:
-      return DatumFactory.createBool(val == datum.asShort());
-    case INT:
-      return DatumFactory.createBool(val == datum.asInt());
-    case LONG:
-      return DatumFactory.createBool(val == datum.asLong());
-    case FLOAT:
-      return DatumFactory.createBool(val == datum.asFloat());
-    case DOUBLE:
-      return DatumFactory.createBool(val == datum.asDouble());
-    default:
-      throw new InvalidOperationException(datum.type());
-    }
-  }
-
-  @Override
-  public int compareTo(Datum datum) {
-    switch (datum.type()) {
-      case SHORT:
-        if (val < datum.asShort()) {
-          return -1;
-        } else if (datum.asShort() < val) {
-          return 1;
-        } else {
-          return 0;
-        }
-      case INT:
-        if (val < datum.asInt()) {
-          return -1;
-        } else if (datum.asInt() < val) {
-          return 1;
-        } else {
-          return 0;
-        }
-      case LONG:
-        if (val < datum.asLong()) {
-          return -1;
-        } else if (datum.asLong() < val) {
-          return 1;
-        } else {
-          return 0;
-        }
-      case FLOAT:
-        if (val < datum.asFloat()) {
-          return -1;
-        } else if (datum.asFloat() < val) {
-          return 1;
-        } else {
-          return 0;
-        }
-      case DOUBLE:
-        if (val < datum.asDouble()) {
-          return -1;
-        } else if (datum.asDouble() < val) {
-          return 1;
-        } else {
-          return 0;
-        }
-      default:
-        throw new InvalidOperationException(datum.type());
-    }
-  }
-
-  @Override
-  public Datum plus(Datum datum) {
-    switch (datum.type()) {
-    case SHORT:
-      return DatumFactory.createFloat(val + datum.asShort());
-    case INT:
-      return DatumFactory.createFloat(val + datum.asInt());
-    case LONG:
-      return DatumFactory.createDouble(val + datum.asLong());
-    case FLOAT:
-      return DatumFactory.createDouble(val + datum.asFloat());
-    case DOUBLE:
-      return DatumFactory.createDouble(val + datum.asDouble());
-    default:
-      throw new InvalidOperationException(datum.type());
-    }
-  }
-
-  @Override
-  public Datum minus(Datum datum) {
-    switch (datum.type()) {
-    case SHORT:
-      return DatumFactory.createFloat(val - datum.asShort());
-    case INT:
-      return DatumFactory.createFloat(val - datum.asInt());
-    case LONG:
-      return DatumFactory.createDouble(val - datum.asLong());
-    case FLOAT:
-      return DatumFactory.createDouble(val - datum.asFloat());
-    case DOUBLE:
-      return DatumFactory.createDouble(val - datum.asDouble());
-    default:
-      throw new InvalidOperationException(datum.type());
-    }
-  }
-
-  @Override
-  public Datum multiply(Datum datum) {
-    switch (datum.type()) {
-    case SHORT:
-      return DatumFactory.createFloat(val * datum.asShort());
-    case INT:
-      return DatumFactory.createFloat(val * datum.asInt());
-    case LONG:
-      return DatumFactory.createDouble(val * datum.asLong());
-    case FLOAT:
-      return DatumFactory.createDouble(val * datum.asFloat());
-    case DOUBLE:
-      return DatumFactory.createDouble(val * datum.asDouble());
-    default:
-      throw new InvalidOperationException();
-    }
-  }
-
-  @Override
-  public Datum divide(Datum datum) {
-    switch (datum.type()) {
-    case SHORT:
-      return DatumFactory.createFloat(val / datum.asShort());
-    case INT:
-      return DatumFactory.createFloat(val / datum.asInt());
-    case LONG:
-      return DatumFactory.createDouble(val / datum.asLong());
-    case FLOAT:
-      return DatumFactory.createDouble(val / datum.asFloat());
-    case DOUBLE:
-      return DatumFactory.createDouble(val / datum.asDouble());
-    default:
-      throw new InvalidOperationException(datum.type());
-    }
-  }
-
-  @Override
-  public Datum modular(Datum datum) {
-    switch (datum.type()) {
-      case SHORT:
-        return DatumFactory.createFloat(val / datum.asShort());
-      case INT:
-        return DatumFactory.createFloat(val / datum.asInt());
-      case LONG:
-        return DatumFactory.createFloat(val / datum.asLong());
-      case FLOAT:
-        return DatumFactory.createFloat(val / datum.asFloat());
-      case DOUBLE:
-        return DatumFactory.createDouble(val / datum.asDouble());
-      default:
-        throw new InvalidOperationException(datum.type());
-    }
-  }
-
-  @Override
-  public void inverseSign() {
-    this.val = - val;    
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-common/src/main/java/tajo/datum/IPv4Datum.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/tajo/datum/IPv4Datum.java b/tajo-common/src/main/java/tajo/datum/IPv4Datum.java
deleted file mode 100644
index dd3e95b..0000000
--- a/tajo-common/src/main/java/tajo/datum/IPv4Datum.java
+++ /dev/null
@@ -1,136 +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 tajo.datum;
-
-import com.google.common.base.Preconditions;
-import com.google.gson.annotations.Expose;
-import tajo.datum.exception.InvalidOperationException;
-import tajo.datum.json.GsonCreator;
-
-public class IPv4Datum extends Datum {
-  private static final int size = 4;
-  @Expose private int address;
-
-	public IPv4Datum() {
-		super(DatumType.IPv4);
-	}
-	
-	public IPv4Datum(String addr) {
-		this();
-		String [] elems = addr.split("\\.");
-		address  = Integer.valueOf(elems[3]) & 0xFF;
-    address |= ((Integer.valueOf(elems[2]) << 8) & 0xFF00);
-    address |= ((Integer.valueOf(elems[1]) << 16) & 0xFF0000);
-    address |= ((Integer.valueOf(elems[0]) << 24) & 0xFF000000);
-	}
-	
-	public IPv4Datum(byte [] addr) {
-		this();
-		Preconditions.checkArgument(addr.length == size);
-		address  = addr[3] & 0xFF;
-    address |= ((addr[2] << 8) & 0xFF00);
-    address |= ((addr[1] << 16) & 0xFF0000);
-    address |= ((addr[0] << 24) & 0xFF000000);
-	}
-
-	@Override
-	public int asInt() {
-		return this.address;
-	}
-
-	@Override
-	public long asLong() {
-	  return this.address;
-	}
-
-	@Override
-	public byte[] asByteArray() {
-	  byte[] addr = new byte[size];
-	  addr[0] = (byte) ((address >>> 24) & 0xFF);
-	  addr[1] = (byte) ((address >>> 16) & 0xFF);
-	  addr[2] = (byte) ((address >>> 8) & 0xFF);
-	  addr[3] = (byte) (address & 0xFF);
-	  return addr;
-	}
-
-	@Override
-	public String asChars() {
-		return numericToTextFormat(asByteArray());
-	}
-
-  @Override
-	public String toJSON() {
-		return GsonCreator.getInstance().toJson(this, Datum.class);
-	}
-
-  @Override
-  public int size() {
-    return size;
-  }
-  
-  @Override
-  public int hashCode() {
-    return address;
-  }
-
-  @Override
-  public boolean equals(Object obj) {
-    if (obj instanceof IPv4Datum) {
-      IPv4Datum other = (IPv4Datum) obj;
-      return this.address == other.address;
-    }
-    
-    return false;
-  }
-
-  @Override
-  public BoolDatum equalsTo(Datum datum) {
-    switch (datum.type()) {
-    case IPv4:    	
-    	return DatumFactory.createBool(this.address == ((IPv4Datum)datum).address);
-    default:
-      throw new InvalidOperationException(datum.type());
-    }
-  }
-  
-  @Override
-  public int compareTo(Datum datum) {
-    switch (datum.type()) {
-    case IPv4:
-      byte [] bytes = asByteArray();
-      byte [] other = datum.asByteArray();
-      
-      for (int i = 0; i < 4; i++) {
-        if (bytes[i] > other[i]) {
-          return 1;
-        } else if (bytes[i] < other[i]) {
-          return -1;
-        }
-      }
-      
-    default:
-      throw new InvalidOperationException(datum.type());
-    }
-  }
-  
-  static String numericToTextFormat(byte[] src) {
-    return (src[0] & 0xff) + "." + (src[1] & 0xff) + "." + (src[2] & 0xff)
-        + "." + (src[3] & 0xff);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-common/src/main/java/tajo/datum/IntDatum.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/tajo/datum/IntDatum.java b/tajo-common/src/main/java/tajo/datum/IntDatum.java
deleted file mode 100644
index 281aba3..0000000
--- a/tajo-common/src/main/java/tajo/datum/IntDatum.java
+++ /dev/null
@@ -1,274 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package tajo.datum;
-
-import com.google.gson.annotations.Expose;
-import tajo.datum.exception.InvalidCastException;
-import tajo.datum.exception.InvalidOperationException;
-import tajo.datum.json.GsonCreator;
-
-import java.nio.ByteBuffer;
-
-public class IntDatum extends NumericDatum {
-  private static final int size = 4;
-  
-  @Expose private int val;
-	
-	public IntDatum() {
-		super(DatumType.INT);
-	}
-	
-	public IntDatum(int val) {
-		this();
-		this.val = val;
-	}
-
-  public IntDatum(byte [] bytes) {
-    this();
-    ByteBuffer bb = ByteBuffer.wrap(bytes);
-    this.val = bb.getInt();
-  }
-
-	@Override
-	public short asShort() {	
-		return (short) val;
-	}
-
-  @Override
-	public int asInt() {		
-		return val;
-	}
-
-  @Override
-	public long asLong() {
-		return val;
-	}
-
-  @Override
-	public byte asByte() {
-		throw new InvalidCastException();
-	}
-
-  @Override
-	public byte[] asByteArray() {
-		ByteBuffer bb = ByteBuffer.allocate(4);
-		bb.putInt(val);
-		return bb.array();
-	}
-
-  @Override
-	public float asFloat() {
-		return val;
-	}
-
-  @Override
-	public double asDouble() {
-		return val;
-	}
-
-  @Override
-	public String asChars() {
-		return ""+val;
-	}
-
-  @Override
-	public String toJSON() {
-		return GsonCreator.getInstance().toJson(this, Datum.class);
-	}
-
-  @Override
-  public int size() {
-    return size;
-  }
-  
-  @Override
-  public int hashCode() {
-    return val;
-  }
-  
-  public boolean equals(Object obj) {
-    if (obj instanceof IntDatum) {
-      IntDatum other = (IntDatum) obj;
-      return val == other.val;
-    }
-    
-    return false;
-  }
-
-  @Override
-  public BoolDatum equalsTo(Datum datum) {
-    switch (datum.type()) {
-    case SHORT:
-      return DatumFactory.createBool(val == datum.asShort());
-    case INT:
-      return DatumFactory.createBool(val == datum.asInt());
-    case LONG:
-      return DatumFactory.createBool(val == datum.asLong());
-    case FLOAT:
-      return DatumFactory.createBool(val == datum.asFloat());
-    case DOUBLE:
-      return DatumFactory.createBool(val == datum.asDouble());
-    default:
-      throw new InvalidOperationException();
-    }
-  }
-
-  @Override
-  public int compareTo(Datum datum) {
-    switch (datum.type()) {
-      case SHORT:
-        if (val < datum.asShort()) {
-          return -1;
-        } else if (datum.asShort() < val) {
-          return 1;
-        } else {
-          return 0;
-        }
-      case INT:
-        if (val < datum.asInt()) {
-          return -1;
-        } else if (datum.asInt() < val) {
-          return 1;
-        } else {
-          return 0;
-        }
-      case LONG:
-        if (val < datum.asLong()) {
-          return -1;
-        } else if (datum.asLong() < val) {
-          return 1;
-        } else {
-          return 0;
-        }
-      case FLOAT:
-        if (val < datum.asFloat()) {
-          return -1;
-        } else if (datum.asFloat() < val) {
-          return 1;
-        } else {
-          return 0;
-        }
-      case DOUBLE:
-        if (val < datum.asDouble()) {
-          return -1;
-        } else if (datum.asDouble() < val) {
-          return 1;
-        } else {
-          return 0;
-        }
-      default:
-        throw new InvalidOperationException(datum.type());
-    }
-  }
-
-  @Override
-  public Datum plus(Datum datum) {
-    switch (datum.type()) {
-    case SHORT:
-      return DatumFactory.createInt(val + datum.asShort());
-    case INT:
-      return DatumFactory.createInt(val + datum.asInt());
-    case LONG:
-      return DatumFactory.createLong(val + datum.asLong());
-    case FLOAT:
-      return DatumFactory.createFloat(val + datum.asFloat());
-    case DOUBLE:
-      return DatumFactory.createDouble(val + datum.asDouble());
-    default:
-      throw new InvalidOperationException(datum.type());
-    }
-  }
-
-  @Override
-  public Datum minus(Datum datum) {
-    switch (datum.type()) {
-    case SHORT:
-      return DatumFactory.createInt(val - datum.asShort());
-    case INT:
-      return DatumFactory.createInt(val - datum.asInt());
-    case LONG:
-      return DatumFactory.createLong(val - datum.asLong());
-    case FLOAT:
-      return DatumFactory.createFloat(val - datum.asFloat());
-    case DOUBLE:
-      return DatumFactory.createDouble(val - datum.asDouble());
-    default:
-      throw new InvalidOperationException(datum.type());
-    }
-  }
-
-  @Override
-  public Datum multiply(Datum datum) {
-    switch (datum.type()) {
-    case SHORT:
-      return DatumFactory.createInt(val * datum.asShort());
-    case INT:
-      return DatumFactory.createInt(val * datum.asInt());
-    case LONG:
-      return DatumFactory.createLong(val * datum.asLong());
-    case FLOAT:
-      return DatumFactory.createFloat(val * datum.asFloat());
-    case DOUBLE:
-      return DatumFactory.createDouble(val * datum.asDouble());
-    default:
-      throw new InvalidOperationException();
-    }
-  }
-
-  @Override
-  public Datum divide(Datum datum) {
-    switch (datum.type()) {
-    case SHORT:
-      return DatumFactory.createInt(val / datum.asShort());
-    case INT:
-      return DatumFactory.createInt(val / datum.asInt());
-    case LONG:
-      return DatumFactory.createLong(val / datum.asLong());
-    case FLOAT:
-      return DatumFactory.createFloat(val / datum.asFloat());
-    case DOUBLE:
-      return DatumFactory.createDouble(val / datum.asDouble());
-    default:
-      throw new InvalidOperationException(datum.type());
-    }
-  }
-
-  @Override
-  public Datum modular(Datum datum) {
-    switch (datum.type()) {
-      case SHORT:
-        return DatumFactory.createInt(val % datum.asShort());
-      case INT:
-        return DatumFactory.createInt(val % datum.asInt());
-      case LONG:
-        return DatumFactory.createLong(val % datum.asLong());
-      case FLOAT:
-        return DatumFactory.createFloat(val % datum.asFloat());
-      case DOUBLE:
-        return DatumFactory.createDouble(val % datum.asDouble());
-      default:
-        throw new InvalidOperationException(datum.type());
-    }
-  }
-
-  @Override
-  public void inverseSign() {
-    this.val = - val;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-common/src/main/java/tajo/datum/LongDatum.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/tajo/datum/LongDatum.java b/tajo-common/src/main/java/tajo/datum/LongDatum.java
deleted file mode 100644
index ae53224..0000000
--- a/tajo-common/src/main/java/tajo/datum/LongDatum.java
+++ /dev/null
@@ -1,279 +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 tajo.datum;
-
-import com.google.gson.annotations.Expose;
-import tajo.datum.exception.InvalidCastException;
-import tajo.datum.exception.InvalidOperationException;
-import tajo.datum.json.GsonCreator;
-
-import java.nio.ByteBuffer;
-
-public class LongDatum extends NumericDatum {
-  private static final int size = 8;
-  @Expose private long val;
-
-	public LongDatum() {
-		super(DatumType.LONG);
-	}
-	
-	public LongDatum(long val) {
-		this();
-		this.val = val;
-	}
-
-  public LongDatum(byte [] bytes) {
-    this();
-    ByteBuffer bb = ByteBuffer.wrap(bytes);
-    val = bb.getLong();
-  }
-
-  @Override
-	public boolean asBool() {
-		throw new InvalidCastException();
-	}
-	
-	@Override
-	public short asShort() {	
-		return (short) val;
-	}
-
-  @Override
-	public int asInt() {		
-		return (int) val;
-	}
-
-  @Override
-	public long asLong() {
-		return val;
-	}
-
-  @Override
-	public byte asByte() {
-		throw new InvalidCastException();
-	}
-
-  @Override
-	public byte[] asByteArray() {
-		ByteBuffer bb = ByteBuffer.allocate(8);
-		bb.putLong(val);
-		return bb.array();
-	}
-
-  @Override
-	public float asFloat() {		
-		return val;
-	}
-
-  @Override
-	public double asDouble() {
-		return val;
-	}
-
-  @Override
-	public String asChars() {
-		return ""+val;
-	}
-
-  @Override
-	public String toJSON() {
-		return GsonCreator.getInstance().toJson(this, Datum.class);
-	}
-	
-  @Override
-  public int size() {
-    return size;
-  }
-  
-  @Override
-  public int hashCode() {
-    return (int) val;
-  }
-
-  @Override
-  public boolean equals(Object obj) {
-    if (obj instanceof LongDatum) {
-      LongDatum other = (LongDatum) obj;
-      return val == other.val;
-    }
-    
-    return false;
-  }
-
-  @Override
-  public BoolDatum equalsTo(Datum datum) {
-    switch (datum.type()) {
-      case SHORT:
-        return DatumFactory.createBool(val == datum.asShort());
-      case INT:
-        return DatumFactory.createBool(val == datum.asInt());
-      case LONG:
-        return DatumFactory.createBool(val == datum.asLong());
-      case FLOAT:
-        return DatumFactory.createBool(val == datum.asFloat());
-      case DOUBLE:
-        return DatumFactory.createBool(val == datum.asDouble());
-      default:
-        throw new InvalidOperationException(datum.type());
-    }
-  }
-
-  @Override
-  public int compareTo(Datum datum) {
-    switch (datum.type()) {
-      case SHORT:
-        if (val < datum.asShort()) {
-          return -1;
-        } else if (datum.asShort() < val) {
-          return 1;
-        } else {
-          return 0;
-        }
-      case INT:
-        if (val < datum.asInt()) {
-          return -1;
-        } else if (datum.asInt() < val) {
-          return 1;
-        } else {
-          return 0;
-        }
-      case LONG:
-        if (val < datum.asLong()) {
-          return -1;
-        } else if (datum.asLong() < val) {
-          return 1;
-        } else {
-          return 0;
-        }
-      case FLOAT:
-        if (val < datum.asFloat()) {
-          return -1;
-        } else if (datum.asFloat() < val) {
-          return 1;
-        } else {
-          return 0;
-        }
-      case DOUBLE:
-        if (val < datum.asDouble()) {
-          return -1;
-        } else if (datum.asDouble() < val) {
-          return 1;
-        } else {
-          return 0;
-        }
-      default:
-        throw new InvalidOperationException(datum.type());
-    }
-  }
-
-  @Override
-  public Datum plus(Datum datum) {
-    switch (datum.type()) {
-    case SHORT:
-      return DatumFactory.createLong(val + datum.asShort());
-    case INT:
-      return DatumFactory.createLong(val + datum.asInt());
-    case LONG:
-      return DatumFactory.createLong(val + datum.asLong());
-    case FLOAT:
-      return DatumFactory.createDouble(val + datum.asFloat());
-    case DOUBLE:
-      return DatumFactory.createDouble(val + datum.asDouble());
-    default:
-      throw new InvalidOperationException(datum.type());
-    }
-  }
-
-  @Override
-  public Datum minus(Datum datum) {
-    switch (datum.type()) {
-    case SHORT:
-      return DatumFactory.createLong(val - datum.asShort());
-    case INT:
-      return DatumFactory.createLong(val - datum.asInt());
-    case LONG:
-      return DatumFactory.createLong(val - datum.asLong());
-    case FLOAT:
-      return DatumFactory.createDouble(val - datum.asFloat());
-    case DOUBLE:
-      return DatumFactory.createDouble(val - datum.asDouble());
-    default:
-      throw new InvalidOperationException(datum.type());
-    }
-  }
-
-  @Override
-  public Datum multiply(Datum datum) {
-    switch (datum.type()) {
-    case SHORT:
-      return DatumFactory.createLong(val * datum.asShort());
-    case INT:
-      return DatumFactory.createLong(val * datum.asInt());
-    case LONG:
-      return DatumFactory.createLong(val * datum.asLong());
-    case FLOAT:
-      return DatumFactory.createDouble(val * datum.asFloat());
-    case DOUBLE:
-      return DatumFactory.createDouble(val * datum.asDouble());
-    default:
-      throw new InvalidOperationException(datum.type());
-    }
-  }
-
-  @Override
-  public Datum divide(Datum datum) {
-    switch (datum.type()) {
-    case SHORT:
-      return DatumFactory.createLong(val / datum.asShort());
-    case INT:
-      return DatumFactory.createLong(val / datum.asInt());
-    case LONG:
-      return DatumFactory.createLong(val / datum.asLong());
-    case FLOAT:
-      return DatumFactory.createDouble(val / datum.asFloat());
-    case DOUBLE:
-      return DatumFactory.createDouble(val / datum.asDouble());
-    default:
-      throw new InvalidOperationException(datum.type());
-    }
-  }
-
-  @Override
-  public Datum modular(Datum datum) {
-    switch (datum.type()) {
-      case SHORT:
-        return DatumFactory.createLong(val % datum.asShort());
-      case INT:
-        return DatumFactory.createLong(val % datum.asInt());
-      case LONG:
-        return DatumFactory.createLong(val % datum.asLong());
-      case FLOAT:
-        return DatumFactory.createDouble(val % datum.asFloat());
-      case DOUBLE:
-        return DatumFactory.createDouble(val % datum.asDouble());
-      default:
-        throw new InvalidOperationException(datum.type());
-    }
-  }
-
-  @Override
-  public void inverseSign() {
-    this.val = -val;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-common/src/main/java/tajo/datum/NullDatum.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/tajo/datum/NullDatum.java b/tajo-common/src/main/java/tajo/datum/NullDatum.java
index 2460a89..b34e6dd 100644
--- a/tajo-common/src/main/java/tajo/datum/NullDatum.java
+++ b/tajo-common/src/main/java/tajo/datum/NullDatum.java
@@ -20,6 +20,8 @@ package tajo.datum;
 
 import tajo.util.Bytes;
 
+import static tajo.common.TajoDataTypes.Type;
+
 public class NullDatum extends Datum {
   private static final NullDatum instance;
   
@@ -28,7 +30,7 @@ public class NullDatum extends Datum {
   }
   
   private NullDatum() {
-    super(DatumType.NULL);
+    super(Type.ANY);
   }
   
   public static NullDatum get() {
@@ -46,17 +48,17 @@ public class NullDatum extends Datum {
   }
 
   @Override
-  public short asShort() {
+  public short asInt2() {
     return Short.MIN_VALUE;
   }
 
   @Override
-  public int asInt() {
+  public int asInt4() {
     return Integer.MIN_VALUE;
   }
 
   @Override
-  public long asLong() {
+  public long asInt8() {
     return Long.MIN_VALUE;
   }
 
@@ -66,12 +68,12 @@ public class NullDatum extends Datum {
   }
 
   @Override
-  public float asFloat() {
+  public float asFloat4() {
     return Float.NaN;
   }
 
   @Override
-  public double asDouble() {
+  public double asFloat8() {
     return Double.NaN;
   }
 
@@ -87,11 +89,7 @@ public class NullDatum extends Datum {
 
   @Override
   public boolean equals(Object obj) {
-    if (obj instanceof NullDatum) {
-      return true;
-    } else {
-      return false;
-    }
+    return obj instanceof NullDatum;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-common/src/main/java/tajo/datum/NumericDatum.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/tajo/datum/NumericDatum.java b/tajo-common/src/main/java/tajo/datum/NumericDatum.java
index 3a7cb29..4f58aa0 100644
--- a/tajo-common/src/main/java/tajo/datum/NumericDatum.java
+++ b/tajo-common/src/main/java/tajo/datum/NumericDatum.java
@@ -19,9 +19,11 @@
 package tajo.datum;
 
 
+import tajo.common.TajoDataTypes;
+
 public abstract class NumericDatum extends Datum {
 
-  public NumericDatum(DatumType type) {
+  public NumericDatum(TajoDataTypes.Type type) {
     super(type);
   }
   


[02/11] TAJO-57: Recognize Parser and Catalog Standard SQL data types. (hyunsik)

Posted by hy...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/reader/LongReader.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/reader/LongReader.java b/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/reader/LongReader.java
deleted file mode 100644
index 3c4b96e..0000000
--- a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/reader/LongReader.java
+++ /dev/null
@@ -1,37 +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 tajo.storage.hcfile.reader;
-
-import tajo.datum.Datum;
-import tajo.datum.DatumFactory;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-
-public class LongReader extends TypeReader {
-
-  @Override
-  public Datum read(ByteBuffer buffer) throws IOException {
-    if (buffer.hasRemaining()) {
-      return DatumFactory.createLong(buffer.getLong());
-    } else {
-      return null;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/reader/Reader.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/reader/Reader.java b/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/reader/Reader.java
deleted file mode 100644
index a9ab3aa..0000000
--- a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/reader/Reader.java
+++ /dev/null
@@ -1,32 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package tajo.storage.hcfile.reader;
-
-import tajo.storage.hcfile.BlockMeta;
-import tajo.storage.hcfile.UpdatableBlock;
-
-import java.io.Closeable;
-import java.io.IOException;
-import java.nio.ByteBuffer;
-
-public interface Reader extends Closeable {
-
-  UpdatableBlock read(BlockMeta meta, ByteBuffer buffer) throws IOException;
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/reader/ShortReader.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/reader/ShortReader.java b/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/reader/ShortReader.java
deleted file mode 100644
index 7c1bf98..0000000
--- a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/reader/ShortReader.java
+++ /dev/null
@@ -1,37 +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 tajo.storage.hcfile.reader;
-
-import tajo.datum.Datum;
-import tajo.datum.DatumFactory;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-
-public class ShortReader extends TypeReader {
-
-  @Override
-  public Datum read(ByteBuffer buffer) throws IOException {
-    if (buffer.hasRemaining()) {
-      return DatumFactory.createShort(buffer.getShort());
-    }else {
-      return null;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/reader/String2Reader.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/reader/String2Reader.java b/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/reader/String2Reader.java
deleted file mode 100644
index 7407d49..0000000
--- a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/reader/String2Reader.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package tajo.storage.hcfile.reader;
-
-import tajo.datum.Datum;
-import tajo.datum.DatumFactory;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-
-public class String2Reader extends TypeReader {
-  @Override
-  public Datum read(ByteBuffer buffer) throws IOException {
-    if (buffer.hasRemaining()) {
-      int size = buffer.getInt();
-      byte[] rawBytes = new byte[size];
-      buffer.get(rawBytes);
-      return DatumFactory.createString2(rawBytes);
-    } else {
-      return null;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/reader/StringReader.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/reader/StringReader.java b/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/reader/StringReader.java
deleted file mode 100644
index 5043018..0000000
--- a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/reader/StringReader.java
+++ /dev/null
@@ -1,40 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package tajo.storage.hcfile.reader;
-
-import tajo.datum.Datum;
-import tajo.datum.DatumFactory;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-
-public class StringReader extends TypeReader {
-
-  @Override
-  public Datum read(ByteBuffer buffer) throws IOException {
-    if (buffer.hasRemaining()) {
-      int size = buffer.getInt();
-      byte[] rawBytes = new byte[size];
-      buffer.get(rawBytes);
-      return DatumFactory.createString(rawBytes);
-    } else {
-      return null;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/reader/TypeReader.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/reader/TypeReader.java b/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/reader/TypeReader.java
deleted file mode 100644
index e9cf356..0000000
--- a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/reader/TypeReader.java
+++ /dev/null
@@ -1,69 +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 tajo.storage.hcfile.reader;
-
-import tajo.catalog.proto.CatalogProtos.DataType;
-import tajo.datum.Datum;
-import tajo.storage.exception.UnknownDataTypeException;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-
-public abstract class TypeReader {
-
-  public static TypeReader get(DataType type)
-      throws IOException, UnknownDataTypeException {
-
-    switch (type) {
-      case BOOLEAN:
-      case BYTE:
-        return new ByteReader();
-      case CHAR:
-        return new CharReader();
-      case BYTES:
-        return new BytesReader();
-      case SHORT:
-        return new ShortReader();
-      case INT:
-        return new IntReader();
-      case LONG:
-        return new LongReader();
-      case FLOAT:
-        return new FloatReader();
-      case DOUBLE:
-        return new DoubleReader();
-      case STRING2:
-        return new String2Reader();
-      case STRING:
-        return new StringReader();
-      case IPv4:
-        return new IPv4Reader();
-      case ARRAY:
-        return new ArrayReader();
-      default:
-        throw new UnknownDataTypeException(type.name());
-    }
-  }
-
-  public abstract Datum read(ByteBuffer buffer) throws IOException;
-
-  public void close() throws IOException {
-
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/writer/ArrayWriter.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/writer/ArrayWriter.java b/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/writer/ArrayWriter.java
deleted file mode 100644
index 526f746..0000000
--- a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/writer/ArrayWriter.java
+++ /dev/null
@@ -1,41 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package tajo.storage.hcfile.writer;
-
-import org.apache.hadoop.fs.FSDataOutputStream;
-import tajo.datum.ArrayDatum;
-import tajo.datum.Datum;
-
-import java.io.IOException;
-
-public class ArrayWriter extends TypeWriter {
-
-  public ArrayWriter(FSDataOutputStream out) {
-    super(out);
-  }
-
-  @Override
-  public void write(Datum data) throws IOException {
-    ArrayDatum array = (ArrayDatum) data;
-    String json = array.toJSON();
-    byte[] jsonBytes = json.getBytes();
-    out.writeInt(jsonBytes.length);
-    out.write(jsonBytes);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/writer/ByteWriter.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/writer/ByteWriter.java b/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/writer/ByteWriter.java
deleted file mode 100644
index e7b84e4..0000000
--- a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/writer/ByteWriter.java
+++ /dev/null
@@ -1,36 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package tajo.storage.hcfile.writer;
-
-import org.apache.hadoop.fs.FSDataOutputStream;
-import tajo.datum.Datum;
-
-import java.io.IOException;
-
-public class ByteWriter extends TypeWriter {
-
-  public ByteWriter(FSDataOutputStream out) {
-    super(out);
-  }
-
-  @Override
-  public void write(Datum data) throws IOException {
-    out.write(data.asByte());
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/writer/BytesWriter.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/writer/BytesWriter.java b/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/writer/BytesWriter.java
deleted file mode 100644
index 3835a3f..0000000
--- a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/writer/BytesWriter.java
+++ /dev/null
@@ -1,38 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package tajo.storage.hcfile.writer;
-
-import org.apache.hadoop.fs.FSDataOutputStream;
-import tajo.datum.Datum;
-
-import java.io.IOException;
-
-public class BytesWriter extends TypeWriter {
-
-  public BytesWriter(FSDataOutputStream out) {
-    super(out);
-  }
-
-  @Override
-  public void write(Datum data) throws IOException {
-    byte[] rawBytes = data.asByteArray();
-    out.writeInt(rawBytes.length);
-    out.write(rawBytes);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/writer/CharWriter.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/writer/CharWriter.java b/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/writer/CharWriter.java
deleted file mode 100644
index b1b365c..0000000
--- a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/writer/CharWriter.java
+++ /dev/null
@@ -1,36 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package tajo.storage.hcfile.writer;
-
-import org.apache.hadoop.fs.FSDataOutputStream;
-import tajo.datum.Datum;
-
-import java.io.IOException;
-
-public class CharWriter extends TypeWriter {
-
-  public CharWriter(FSDataOutputStream out) {
-    super(out);
-  }
-
-  @Override
-  public void write(Datum data) throws IOException {
-    out.write(data.asByte());
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/writer/DoubleWriter.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/writer/DoubleWriter.java b/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/writer/DoubleWriter.java
deleted file mode 100644
index 13c4dff..0000000
--- a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/writer/DoubleWriter.java
+++ /dev/null
@@ -1,36 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package tajo.storage.hcfile.writer;
-
-import org.apache.hadoop.fs.FSDataOutputStream;
-import tajo.datum.Datum;
-
-import java.io.IOException;
-
-public class DoubleWriter extends TypeWriter {
-
-  public DoubleWriter(FSDataOutputStream out) {
-    super(out);
-  }
-
-  @Override
-  public void write(Datum data) throws IOException {
-    out.writeDouble(data.asDouble());
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/writer/FloatWriter.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/writer/FloatWriter.java b/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/writer/FloatWriter.java
deleted file mode 100644
index 72fc77f..0000000
--- a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/writer/FloatWriter.java
+++ /dev/null
@@ -1,36 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package tajo.storage.hcfile.writer;
-
-import org.apache.hadoop.fs.FSDataOutputStream;
-import tajo.datum.Datum;
-
-import java.io.IOException;
-
-public class FloatWriter extends TypeWriter {
-
-  public FloatWriter(FSDataOutputStream out) {
-    super(out);
-  }
-
-  @Override
-  public void write(Datum data) throws IOException {
-    out.writeFloat(data.asFloat());
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/writer/IntWriter.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/writer/IntWriter.java b/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/writer/IntWriter.java
deleted file mode 100644
index 676e815..0000000
--- a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/writer/IntWriter.java
+++ /dev/null
@@ -1,36 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package tajo.storage.hcfile.writer;
-
-import org.apache.hadoop.fs.FSDataOutputStream;
-import tajo.datum.Datum;
-
-import java.io.IOException;
-
-public class IntWriter extends TypeWriter {
-
-  public IntWriter(FSDataOutputStream out) {
-    super(out);
-  }
-
-  @Override
-  public void write(Datum data) throws IOException {
-    out.writeInt(data.asInt());
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/writer/LongWriter.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/writer/LongWriter.java b/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/writer/LongWriter.java
deleted file mode 100644
index 2e1c3f2..0000000
--- a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/writer/LongWriter.java
+++ /dev/null
@@ -1,36 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package tajo.storage.hcfile.writer;
-
-import org.apache.hadoop.fs.FSDataOutputStream;
-import tajo.datum.Datum;
-
-import java.io.IOException;
-
-public class LongWriter extends TypeWriter {
-
-  public LongWriter(FSDataOutputStream out) {
-    super(out);
-  }
-
-  @Override
-  public void write(Datum data) throws IOException {
-    out.writeLong(data.asLong());
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/writer/ShortWriter.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/writer/ShortWriter.java b/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/writer/ShortWriter.java
deleted file mode 100644
index 5868456..0000000
--- a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/writer/ShortWriter.java
+++ /dev/null
@@ -1,36 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package tajo.storage.hcfile.writer;
-
-import org.apache.hadoop.fs.FSDataOutputStream;
-import tajo.datum.Datum;
-
-import java.io.IOException;
-
-public class ShortWriter extends TypeWriter {
-
-  public ShortWriter(FSDataOutputStream out) {
-    super(out);
-  }
-
-  @Override
-  public void write(Datum data) throws IOException {
-    out.writeShort(data.asShort());
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/writer/TypeWriter.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/writer/TypeWriter.java b/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/writer/TypeWriter.java
deleted file mode 100644
index 4f1f20a..0000000
--- a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/writer/TypeWriter.java
+++ /dev/null
@@ -1,90 +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 tajo.storage.hcfile.writer;
-
-import org.apache.hadoop.fs.FSDataOutputStream;
-import tajo.catalog.proto.CatalogProtos.DataType;
-import tajo.datum.Datum;
-import tajo.storage.exception.UnknownDataTypeException;
-
-import java.io.IOException;
-
-public abstract class TypeWriter {
-
-  protected final FSDataOutputStream out;
-
-  public static TypeWriter get(FSDataOutputStream out,
-                               DataType type)
-      throws IOException, UnknownDataTypeException {
-
-    switch (type) {
-      case BOOLEAN :
-      case BYTE :
-        return new ByteWriter(out);
-
-      case CHAR :
-        return new CharWriter(out);
-
-      case SHORT :
-        return new ShortWriter(out);
-
-      case INT :
-        return new IntWriter(out);
-
-      case LONG :
-        return new LongWriter(out);
-
-      case FLOAT :
-        return new FloatWriter(out);
-
-      case DOUBLE:
-        return new DoubleWriter(out);
-
-      case STRING2:
-      case STRING:
-      case BYTES:
-      case IPv4:
-        return new BytesWriter(out);
-
-      case ARRAY:
-        return new ArrayWriter(out);
-
-      default:
-        throw new UnknownDataTypeException(type.toString());
-    }
-  }
-
-  protected TypeWriter(FSDataOutputStream out) {
-    this.out = out;
-  }
-
-  public FSDataOutputStream getOutputStream() {
-    return this.out;
-  }
-
-  public abstract void write(Datum data) throws IOException;
-
-  public long getPos() throws IOException {
-    return out.getPos();
-  }
-
-  public void close() throws IOException {
-
-  }
-}

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

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-storage/src/main/java/tajo/storage/rcfile/RCFileWrapper.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/rcfile/RCFileWrapper.java b/tajo-core/tajo-core-storage/src/main/java/tajo/storage/rcfile/RCFileWrapper.java
index e04cd52..2018992 100644
--- a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/rcfile/RCFileWrapper.java
+++ b/tajo-core/tajo-core-storage/src/main/java/tajo/storage/rcfile/RCFileWrapper.java
@@ -94,24 +94,23 @@ public class RCFileWrapper {
           byteRef.set(i, cu);
         } else {
           col = schema.getColumn(i);
-          switch (col.getDataType()) {
+          switch (col.getDataType().getType()) {
             case BOOLEAN:
-            case BYTE:
-            case CHAR:
+            case BIT:
               cu = new BytesRefWritable(t.get(i).asByteArray(), 0, 1);
               byteRef.set(i, cu);
               break;
 
-            case SHORT:
-            case INT:
-            case LONG:
-            case FLOAT:
-            case DOUBLE:
-            case STRING:
-            case STRING2:
-            case BYTES:
-            case IPv4:
-            case IPv6:
+            case CHAR:
+            case INT2:
+            case INT4:
+            case INT8:
+            case FLOAT4:
+            case FLOAT8:
+            case TEXT:
+            case BLOB:
+            case INET4:
+            case INET6:
               bytes = t.get(i).asByteArray();
               cu = new BytesRefWritable(bytes, 0, bytes.length);
               byteRef.set(i, cu);
@@ -241,69 +240,69 @@ public class RCFileWrapper {
         if (column.get(tid).getLength() == 0) {
           tuple.put(tid, DatumFactory.createNullDatum());
         } else {
-          switch (targets[i].getDataType()) {
+          switch (targets[i].getDataType().getType()) {
             case BOOLEAN:
               tuple.put(tid,
                   DatumFactory.createBool(column.get(tid).getBytesCopy()[0]));
               break;
-            case BYTE:
+            case BIT:
               tuple.put(tid,
-                  DatumFactory.createByte(column.get(tid).getBytesCopy()[0]));
+                  DatumFactory.createBit(column.get(tid).getBytesCopy()[0]));
               break;
             case CHAR:
               tuple.put(tid,
                   DatumFactory.createChar(column.get(tid).getBytesCopy()[0]));
               break;
 
-            case SHORT:
+            case INT2:
               tuple.put(tid,
-                  DatumFactory.createShort(Bytes.toShort(
+                  DatumFactory.createInt2(Bytes.toShort(
                       column.get(tid).getBytesCopy())));
               break;
-            case INT:
+            case INT4:
               tuple.put(tid,
-                  DatumFactory.createInt(Bytes.toInt(
+                  DatumFactory.createInt4(Bytes.toInt(
                       column.get(tid).getBytesCopy())));
               break;
 
-            case LONG:
+            case INT8:
               tuple.put(tid,
-                  DatumFactory.createLong(Bytes.toLong(
+                  DatumFactory.createInt8(Bytes.toLong(
                       column.get(tid).getBytesCopy())));
               break;
 
-            case FLOAT:
+            case FLOAT4:
               tuple.put(tid,
-                  DatumFactory.createFloat(Bytes.toFloat(
+                  DatumFactory.createFloat4(Bytes.toFloat(
                       column.get(tid).getBytesCopy())));
               break;
 
-            case DOUBLE:
+            case FLOAT8:
               tuple.put(tid,
-                  DatumFactory.createDouble(Bytes.toDouble(
+                  DatumFactory.createFloat8(Bytes.toDouble(
                       column.get(tid).getBytesCopy())));
               break;
 
-            case IPv4:
+            case INET4:
               tuple.put(tid,
-                  DatumFactory.createIPv4(column.get(tid).getBytesCopy()));
+                  DatumFactory.createInet4(column.get(tid).getBytesCopy()));
               break;
 
-            case STRING:
-              tuple.put(tid,
-                  DatumFactory.createString(
-                      Bytes.toString(column.get(tid).getBytesCopy())));
-              break;
+//            case STRING:
+//              tuple.put(tid,
+//                  DatumFactory.createText(
+//                      Bytes.toString(column.get(tid).getBytesCopy())));
+//              break;
 
-            case STRING2:
+            case TEXT:
               tuple.put(tid,
-                  DatumFactory.createString2(
+                  DatumFactory.createText(
                       column.get(tid).getBytesCopy()));
               break;
 
-            case BYTES:
+            case BLOB:
               tuple.put(tid,
-                  DatumFactory.createBytes(column.get(tid).getBytesCopy()));
+                  DatumFactory.createBlob(column.get(tid).getBytesCopy()));
               break;
 
             default:

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-storage/src/main/java/tajo/storage/trevni/TrevniAppender.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/trevni/TrevniAppender.java b/tajo-core/tajo-core-storage/src/main/java/tajo/storage/trevni/TrevniAppender.java
index e7b30df..60bc11e 100644
--- a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/trevni/TrevniAppender.java
+++ b/tajo-core/tajo-core-storage/src/main/java/tajo/storage/trevni/TrevniAppender.java
@@ -30,6 +30,7 @@ import org.apache.trevni.ValueType;
 import tajo.catalog.Column;
 import tajo.catalog.TableMeta;
 import tajo.catalog.statistics.TableStat;
+import tajo.common.TajoDataTypes.Type;
 import tajo.storage.FileAppender;
 import tajo.storage.TableStatistics;
 import tajo.storage.Tuple;
@@ -37,8 +38,6 @@ import tajo.storage.Tuple;
 import java.io.FileNotFoundException;
 import java.io.IOException;
 
-import static tajo.catalog.proto.CatalogProtos.DataType;
-
 public class TrevniAppender extends FileAppender {
   private FileSystem fs;
   private ColumnFileWriter writer;
@@ -65,7 +64,7 @@ public class TrevniAppender extends FileAppender {
     int i = 0;
     for (Column column : meta.getSchema().getColumns()) {
       trevniMetas[i++] = new ColumnMetaData(column.getColumnName(),
-          getType(column.getDataType()));
+          getType(column.getDataType().getType()));
     }
 
     writer = new ColumnFileWriter(createFileMeta(), trevniMetas);
@@ -83,35 +82,34 @@ public class TrevniAppender extends FileAppender {
         .setChecksum("null");
   }
 
-  private static ValueType getType(DataType type) {
+  private static ValueType getType(Type type) {
     switch (type) {
       case BOOLEAN:
         return ValueType.INT;
-      case BYTE:
+      case BIT:
         return ValueType.INT;
       case CHAR:
         return ValueType.INT;
-      case SHORT:
+      case INT2:
         return ValueType.INT;
-      case INT:
+      case INT4:
         return ValueType.INT;
-      case LONG:
+      case INT8:
         return ValueType.LONG;
-      case FLOAT:
+      case FLOAT4:
         return ValueType.FLOAT;
-      case DOUBLE:
+      case FLOAT8:
         return ValueType.DOUBLE;
-      case STRING:
-      case STRING2:
+      case TEXT:
         return ValueType.STRING;
-      case BYTES:
+      case BLOB:
         return ValueType.BYTES;
-      case IPv4:
+      case INET4:
         return ValueType.BYTES;
-      case IPv6:
+      case INET6:
         return ValueType.BYTES;
       case ARRAY:
-        return ValueType.STRING;
+        return ValueType.BYTES;
       default:
         return null;
     }
@@ -133,35 +131,34 @@ public class TrevniAppender extends FileAppender {
 
       if (!t.isNull(i)) {
         col = schema.getColumn(i);
-        switch (col.getDataType()) {
+        switch (col.getDataType().getType()) {
           case BOOLEAN:
-          case BYTE:
+          case BIT:
           case CHAR:
-          case SHORT:
-          case INT:
-            writer.writeValue(t.get(i).asInt(), i);
+          case INT2:
+          case INT4:
+            writer.writeValue(t.get(i).asInt4(), i);
             break;
-          case LONG:
-            writer.writeValue(t.get(i).asLong(), i);
+          case INT8:
+            writer.writeValue(t.get(i).asInt8(), i);
             break;
-          case FLOAT:
-            writer.writeValue(t.get(i).asFloat(), i);
+          case FLOAT4:
+            writer.writeValue(t.get(i).asFloat4(), i);
             break;
-          case DOUBLE:
-            writer.writeValue(t.get(i).asDouble(), i);
+          case FLOAT8:
+            writer.writeValue(t.get(i).asFloat8(), i);
             break;
-          case STRING:
-          case STRING2:
+          case TEXT:
             writer.writeValue(t.get(i).asChars(), i);
             break;
-          case IPv4:
-          case IPv6:
+          case INET4:
+          case INET6:
             writer.writeValue(t.get(i).asByteArray(), i);
             break;
           case ARRAY:
             writer.writeValue(t.get(i).asChars(), i);
             break;
-          case BYTES:
+          case BLOB:
             writer.writeValue(t.get(i).asByteArray(), i);
           default:
             break;

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-storage/src/main/java/tajo/storage/trevni/TrevniScanner.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/trevni/TrevniScanner.java b/tajo-core/tajo-core-storage/src/main/java/tajo/storage/trevni/TrevniScanner.java
index 6fb05bd..9c0a5ed 100644
--- a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/trevni/TrevniScanner.java
+++ b/tajo-core/tajo-core-storage/src/main/java/tajo/storage/trevni/TrevniScanner.java
@@ -24,7 +24,7 @@ import org.apache.trevni.ColumnValues;
 import org.apache.trevni.avro.HadoopInput;
 import tajo.catalog.Column;
 import tajo.catalog.TableMeta;
-import tajo.datum.BytesDatum;
+import tajo.datum.BlobDatum;
 import tajo.datum.DatumFactory;
 import tajo.storage.FileScanner;
 import tajo.storage.Fragment;
@@ -83,62 +83,62 @@ public class TrevniScanner extends FileScanner {
     for (int i = 0; i < projectionMap.length; i++) {
       tid = projectionMap[i];
       columns[i].startRow();
-      switch (schema.getColumn(tid).getDataType()) {
+      switch (schema.getColumn(tid).getDataType().getType()) {
         case BOOLEAN:
           tuple.put(tid,
               DatumFactory.createBool(((Integer)columns[i].nextValue()).byteValue()));
           break;
-        case BYTE:
+        case BIT:
           tuple.put(tid,
-              DatumFactory.createByte(((Integer)columns[i].nextValue()).byteValue()));
+              DatumFactory.createBit(((Integer) columns[i].nextValue()).byteValue()));
           break;
         case CHAR:
           tuple.put(tid,
               DatumFactory.createChar(((Integer)columns[i].nextValue()).byteValue()));
           break;
 
-        case SHORT:
+        case INT2:
           tuple.put(tid,
-              DatumFactory.createShort(((Integer)columns[i].nextValue()).shortValue()));
+              DatumFactory.createInt2(((Integer) columns[i].nextValue()).shortValue()));
           break;
-        case INT:
+        case INT4:
           tuple.put(tid,
-              DatumFactory.createInt((Integer)columns[i].nextValue()));
+              DatumFactory.createInt4((Integer) columns[i].nextValue()));
           break;
 
-        case LONG:
+        case INT8:
           tuple.put(tid,
-              DatumFactory.createLong((Long)columns[i].nextValue()));
+              DatumFactory.createInt8((Long) columns[i].nextValue()));
           break;
 
-        case FLOAT:
+        case FLOAT4:
           tuple.put(tid,
-              DatumFactory.createFloat((Float)columns[i].nextValue()));
+              DatumFactory.createFloat4((Float) columns[i].nextValue()));
           break;
 
-        case DOUBLE:
+        case FLOAT8:
           tuple.put(tid,
-              DatumFactory.createDouble((Double)columns[i].nextValue()));
+              DatumFactory.createFloat8((Double) columns[i].nextValue()));
           break;
 
-        case IPv4:
+        case INET4:
           tuple.put(tid,
-              DatumFactory.createIPv4(((ByteBuffer) columns[i].nextValue()).array()));
+              DatumFactory.createInet4(((ByteBuffer) columns[i].nextValue()).array()));
           break;
 
-        case STRING:
-          tuple.put(tid,
-              DatumFactory.createString((String) columns[i].nextValue()));
-          break;
+//        case TEXT:
+//          tuple.put(tid,
+//              DatumFactory.createText((String) columns[i].nextValue()));
+//          break;
 
-        case STRING2:
+        case TEXT:
           tuple.put(tid,
-              DatumFactory.createString2((String) columns[i].nextValue()));
+              DatumFactory.createText((String) columns[i].nextValue()));
           break;
 
-        case BYTES:
+        case BLOB:
           tuple.put(tid,
-              new BytesDatum(((ByteBuffer) columns[i].nextValue())));
+              new BlobDatum(((ByteBuffer) columns[i].nextValue())));
           break;
 
         default:

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-storage/src/main/proto/CatalogProtos.proto
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/proto/CatalogProtos.proto b/tajo-core/tajo-core-storage/src/main/proto/CatalogProtos.proto
index 3910520..efb975e 100644
--- a/tajo-core/tajo-core-storage/src/main/proto/CatalogProtos.proto
+++ b/tajo-core/tajo-core-storage/src/main/proto/CatalogProtos.proto
@@ -22,33 +22,12 @@ option optimize_for = SPEED;
 option java_generic_services = false;
 option java_generate_equals_and_hash = true;
 
-enum DataType {
-    BOOLEAN = 0;
-    BYTE = 1;
-    CHAR = 2;
-    BYTES = 3;
-    SHORT = 4;
-    INT = 5;
-    BIGINT = 6;
-    LONG = 7;
-    FLOAT = 8;
-    DOUBLE = 9;
-    BIGDECIMAL = 10;
-    STRING = 11;
-    DATE = 12;
-    IPv4 = 13;
-    IPv6 = 14;
-    NULL = 15;
-    ANY = 16;
-    ALL = 17;
-    ARRAY = 18;
-    STRING2 = 19;
-}
+import "DataTypes.proto";
 
 enum StoreType {
-	MEM = 0;
-	CSV = 1;
-	RAW = 2;
+  MEM = 0;
+  CSV = 1;
+  RAW = 2;
   RCFILE = 3;
   ROWFILE = 4;
   HCFILE = 5;
@@ -56,147 +35,147 @@ enum StoreType {
 }
 
 enum OrderType {
-    ORDER_NONE = 0;
-    ASC = 1;
-    DSC = 2;
+  ORDER_NONE = 0;
+  ASC = 1;
+  DSC = 2;
 }
 
 enum CompressType {
-    COMP_NONE = 0;
-    NULL_SUPPRESS = 1;
-    RUN_LENGTH = 2;
-    BIT_VECTOR = 3;
-    DICTIONARY = 4;
-    SNAPPY = 5;
-    LZ = 6;
+  COMP_NONE = 0;
+  NULL_SUPPRESS = 1;
+  RUN_LENGTH = 2;
+  BIT_VECTOR = 3;
+  DICTIONARY = 4;
+  SNAPPY = 5;
+  LZ = 6;
 }
 
 message ColumnMetaProto {
-    required DataType dataType = 1;
-    required bool compressed = 2;
-    required bool sorted = 3;
-    required bool contiguous = 4;
-    required StoreType storeType = 5;
-    required CompressType compType = 6;
-    required int64 startRid = 7;
-    required int32 recordNum = 8;
-    required int32 offsetToIndex = 9;
+  required DataType dataType = 1;
+  required bool compressed = 2;
+  required bool sorted = 3;
+  required bool contiguous = 4;
+  required StoreType storeType = 5;
+  required CompressType compType = 6;
+  required int64 startRid = 7;
+  required int32 recordNum = 8;
+  required int32 offsetToIndex = 9;
 }
 
 message ColumnProto {
-	required string columnName = 1;
-	required DataType dataType = 2;
+  required string columnName = 1;
+  required DataType dataType = 2;
 }
 
 message SchemaProto {
-	repeated ColumnProto fields = 1;
+  repeated ColumnProto fields = 1;
 }
 
 message KeyValueProto {
-	required string key = 1;
-	required string value = 2;
+  required string key = 1;
+  required string value = 2;
 }
 
 message KeyValueSetProto {
-	repeated KeyValueProto keyval = 1;
+  repeated KeyValueProto keyval = 1;
 }
 
 message TabletProto {
-	required string id = 1;
-	required string path = 2;
-	required int64 startOffset = 3;
-	required int64 length = 4;
-	required TableProto meta = 5;
-	optional TableStatProto stat = 6;
+  required string id = 1;
+  required string path = 2;
+  required int64 startOffset = 3;
+  required int64 length = 4;
+  required TableProto meta = 5;
+  optional TableStatProto stat = 6;
   optional bool distCached = 7 [default = false];
 }
 
 message TableProto {
-    required SchemaProto schema = 1;
-    required StoreType storeType = 2;
-    required KeyValueSetProto params = 3;
-    optional TableStatProto stat = 4;
+  required SchemaProto schema = 1;
+  required StoreType storeType = 2;
+  required KeyValueSetProto params = 3;
+  optional TableStatProto stat = 4;
 }
 
 message TableDescProto {
-	required string id = 1;
-	required string path = 2;
-	required TableProto meta = 3;
+  required string id = 1;
+  required string path = 2;
+  required TableProto meta = 3;
 }
 
 enum FunctionType {
-	GENERAL = 0;
-	AGGREGATION = 1;
+  GENERAL = 0;
+  AGGREGATION = 1;
 }
 
 message FunctionDescProto {
-	required string signature = 1;
-	required string className = 2;
-	required FunctionType type = 3;
-	repeated DataType parameterTypes = 4;
-	required DataType returnType = 5;
+  required string signature = 1;
+  required string className = 2;
+  required FunctionType type = 3;
+  repeated Type parameterTypes = 4;
+  required Type returnType = 5;
 }
 
 message IndexDescProto {
-    required string name = 1;
-    required string tableId = 2;
-    required ColumnProto column = 3;
-    required IndexMethod indexMethod = 4;
-    optional bool isUnique = 5 [default = false];
-    optional bool isClustered = 6 [default = false];
-    optional bool isAscending = 7 [default = false];
+  required string name = 1;
+  required string tableId = 2;
+  required ColumnProto column = 3;
+  required IndexMethod indexMethod = 4;
+  optional bool isUnique = 5 [default = false];
+  optional bool isClustered = 6 [default = false];
+  optional bool isAscending = 7 [default = false];
 }
 
 enum IndexMethod {
-    TWO_LEVEL_BIN_TREE = 0;
-    BTREE = 1;
-    HASH = 2;
-    BITMAP = 3;
+  TWO_LEVEL_BIN_TREE = 0;
+  BTREE = 1;
+  HASH = 2;
+  BITMAP = 3;
 }
 
 message GetAllTableNamesResponse {
-    repeated string tableName = 1;
+  repeated string tableName = 1;
 }
 
 message GetIndexRequest {
-    required string tableName = 1;
-    required string columnName = 2;
+  required string tableName = 1;
+  required string columnName = 2;
 }
 
 message GetFunctionsResponse {
-	repeated FunctionDescProto functionDesc = 1;
+  repeated FunctionDescProto functionDesc = 1;
 }
 
 message UnregisterFunctionRequest {
-	required string signature = 1;
-	repeated DataType parameterTypes = 2;
+  required string signature = 1;
+  repeated Type parameterTypes = 2;
 }
 
 message GetFunctionMetaRequest {
-	required string signature = 1;
-	repeated DataType parameterTypes = 2;
+  required string signature = 1;
+  repeated Type parameterTypes = 2;
 }
 
 message ContainFunctionRequest {
-	required string signature = 1;
-	repeated DataType parameterTypes = 2;
+  required string signature = 1;
+  repeated Type parameterTypes = 2;
 }
 
 message TableStatProto {
-	required int64 numRows = 1;
-	required int64 numBytes = 2;
-	optional int32 numBlocks = 3;
-	optional int32 numPartitions = 4;
-	optional int64 avgRows = 5;
-	repeated ColumnStatProto colStat = 6;
+  required int64 numRows = 1;
+  required int64 numBytes = 2;
+  optional int32 numBlocks = 3;
+  optional int32 numPartitions = 4;
+  optional int64 avgRows = 5;
+  repeated ColumnStatProto colStat = 6;
 }
 
 message ColumnStatProto {
-    required ColumnProto column = 1;
-    optional int64 numDistVal = 2;
-    optional int64 numNulls = 3;
-    optional bytes minValue = 4;
-    optional bytes maxValue = 5;
+  required ColumnProto column = 1;
+  optional int64 numDistVal = 2;
+  optional int64 numNulls = 3;
+  optional bytes minValue = 4;
+  optional bytes maxValue = 5;
 }
 
 enum StatType {
@@ -229,4 +208,4 @@ message SortSpecProto {
   required int32 sortColumnId = 1;
   optional bool ascending = 2 [default = true];
   optional bool nullFirst = 3 [default = false];
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-storage/src/test/java/tajo/storage/TestFrameTuple.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/test/java/tajo/storage/TestFrameTuple.java b/tajo-core/tajo-core-storage/src/test/java/tajo/storage/TestFrameTuple.java
index 7287c6f..318b1d7 100644
--- a/tajo-core/tajo-core-storage/src/test/java/tajo/storage/TestFrameTuple.java
+++ b/tajo-core/tajo-core-storage/src/test/java/tajo/storage/TestFrameTuple.java
@@ -36,31 +36,31 @@ public class TestFrameTuple {
     tuple1 = new VTuple(11);
     tuple1.put(new Datum[] {
         DatumFactory.createBool(true),
-        DatumFactory.createByte((byte) 0x99),
+        DatumFactory.createBit((byte) 0x99),
         DatumFactory.createChar('9'),
-        DatumFactory.createShort((short) 17),
-        DatumFactory.createInt(59),
-        DatumFactory.createLong(23l),
-        DatumFactory.createFloat(77.9f),
-        DatumFactory.createDouble(271.9f),        
-        DatumFactory.createString("hyunsik"),
-        DatumFactory.createBytes("hyunsik".getBytes()),
-        DatumFactory.createIPv4("192.168.0.1")
+        DatumFactory.createInt2((short) 17),
+        DatumFactory.createInt4(59),
+        DatumFactory.createInt8(23l),
+        DatumFactory.createFloat4(77.9f),
+        DatumFactory.createFloat8(271.9f),
+        DatumFactory.createText("hyunsik"),
+        DatumFactory.createBlob("hyunsik".getBytes()),
+        DatumFactory.createInet4("192.168.0.1")
     });
     
     tuple2 = new VTuple(11);
     tuple2.put(new Datum[] {
         DatumFactory.createBool(true),
-        DatumFactory.createByte((byte) 0x99),
+        DatumFactory.createBit((byte) 0x99),
         DatumFactory.createChar('9'),
-        DatumFactory.createShort((short) 17),
-        DatumFactory.createInt(59),
-        DatumFactory.createLong(23l),
-        DatumFactory.createFloat(77.9f),
-        DatumFactory.createDouble(271.9f),        
-        DatumFactory.createString("hyunsik"),
-        DatumFactory.createBytes("hyunsik".getBytes()),
-        DatumFactory.createIPv4("192.168.0.1")
+        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")
     });
   }
 
@@ -76,9 +76,9 @@ public class TestFrameTuple {
       assertTrue(frame.contains(i));
     }
     
-    assertEquals(DatumFactory.createLong(23l), frame.get(5));
-    assertEquals(DatumFactory.createLong(23l), frame.get(16));
-    assertEquals(DatumFactory.createIPv4("192.168.0.1"), frame.get(10));
-    assertEquals(DatumFactory.createIPv4("192.168.0.1"), frame.get(21));
+    assertEquals(DatumFactory.createInt8(23l), frame.get(5));
+    assertEquals(DatumFactory.createInt8(23l), frame.get(16));
+    assertEquals(DatumFactory.createInet4("192.168.0.1"), frame.get(10));
+    assertEquals(DatumFactory.createInet4("192.168.0.1"), frame.get(21));
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-storage/src/test/java/tajo/storage/TestMergeScanner.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/test/java/tajo/storage/TestMergeScanner.java b/tajo-core/tajo-core-storage/src/test/java/tajo/storage/TestMergeScanner.java
index 7688a1d..40d368e 100644
--- a/tajo-core/tajo-core-storage/src/test/java/tajo/storage/TestMergeScanner.java
+++ b/tajo-core/tajo-core-storage/src/test/java/tajo/storage/TestMergeScanner.java
@@ -26,13 +26,13 @@ import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
 import org.junit.runners.Parameterized.Parameters;
+import tajo.catalog.CatalogUtil;
 import tajo.catalog.Options;
 import tajo.catalog.Schema;
-import tajo.catalog.TCatUtil;
 import tajo.catalog.TableMeta;
-import tajo.catalog.proto.CatalogProtos.DataType;
 import tajo.catalog.proto.CatalogProtos.StoreType;
 import tajo.catalog.statistics.TableStat;
+import tajo.common.TajoDataTypes.Type;
 import tajo.conf.TajoConf;
 import tajo.conf.TajoConf.ConfVars;
 import tajo.datum.DatumFactory;
@@ -83,13 +83,13 @@ public class TestMergeScanner {
   @Test
   public void testMultipleFiles() throws IOException {
     Schema schema = new Schema();
-    schema.addColumn("id", DataType.INT);
-    schema.addColumn("file", DataType.STRING);
-    schema.addColumn("name", DataType.STRING);
-    schema.addColumn("age", DataType.LONG);
+    schema.addColumn("id", Type.INT4);
+    schema.addColumn("file", Type.TEXT);
+    schema.addColumn("name", Type.TEXT);
+    schema.addColumn("age", Type.INT8);
     
     Options options = new Options();
-    TableMeta meta = TCatUtil.newTableMeta(schema, storeType, options);
+    TableMeta meta = CatalogUtil.newTableMeta(schema, storeType, options);
 
     Path table1Path = new Path(testDir, storeType + "_1.data");
     Appender appender1 = StorageManager.getAppender(conf, meta, table1Path);
@@ -100,10 +100,10 @@ public class TestMergeScanner {
 
     for(int i = 0; i < tupleNum; i++) {
       vTuple = new VTuple(4);
-      vTuple.put(0, DatumFactory.createInt(i+1));
-      vTuple.put(1, DatumFactory.createString("hyunsik"));
-      vTuple.put(2, DatumFactory.createString("jihoon"));
-      vTuple.put(3, DatumFactory.createLong(25l));
+      vTuple.put(0, DatumFactory.createInt4(i + 1));
+      vTuple.put(1, DatumFactory.createText("hyunsik"));
+      vTuple.put(2, DatumFactory.createText("jihoon"));
+      vTuple.put(3, DatumFactory.createInt8(25l));
       appender1.addTuple(vTuple);
     }
     appender1.close();
@@ -120,10 +120,10 @@ public class TestMergeScanner {
 
     for(int i = 0; i < tupleNum; i++) {
       vTuple = new VTuple(4);
-      vTuple.put(0, DatumFactory.createInt(i+1));
-      vTuple.put(1, DatumFactory.createString("hyunsik"));
-      vTuple.put(2, DatumFactory.createString("jihoon"));
-      vTuple.put(3, DatumFactory.createLong(25l));
+      vTuple.put(0, DatumFactory.createInt4(i + 1));
+      vTuple.put(1, DatumFactory.createText("hyunsik"));
+      vTuple.put(2, DatumFactory.createText("jihoon"));
+      vTuple.put(3, DatumFactory.createInt8(25l));
       appender2.addTuple(vTuple);
     }
     appender2.close();

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-storage/src/test/java/tajo/storage/TestStorageManager.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/test/java/tajo/storage/TestStorageManager.java b/tajo-core/tajo-core-storage/src/test/java/tajo/storage/TestStorageManager.java
index 00215a0..dab6705 100644
--- a/tajo-core/tajo-core-storage/src/test/java/tajo/storage/TestStorageManager.java
+++ b/tajo-core/tajo-core-storage/src/test/java/tajo/storage/TestStorageManager.java
@@ -23,11 +23,11 @@ import org.apache.hadoop.fs.Path;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
+import tajo.catalog.CatalogUtil;
 import tajo.catalog.Schema;
-import tajo.catalog.TCatUtil;
 import tajo.catalog.TableMeta;
-import tajo.catalog.proto.CatalogProtos.DataType;
 import tajo.catalog.proto.CatalogProtos.StoreType;
+import tajo.common.TajoDataTypes.Type;
 import tajo.conf.TajoConf;
 import tajo.datum.Datum;
 import tajo.datum.DatumFactory;
@@ -58,19 +58,19 @@ public class TestStorageManager {
   @Test
 	public final void testGetScannerAndAppender() throws IOException {
 		Schema schema = new Schema();
-		schema.addColumn("id",DataType.INT);
-		schema.addColumn("age",DataType.INT);
-		schema.addColumn("name",DataType.STRING);
+		schema.addColumn("id", Type.INT4);
+		schema.addColumn("age",Type.INT4);
+		schema.addColumn("name",Type.TEXT);
 
-		TableMeta meta = TCatUtil.newTableMeta(schema, StoreType.CSV);
+		TableMeta meta = CatalogUtil.newTableMeta(schema, 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.createInt(i),
-		      DatumFactory.createInt(i+32),
-		      DatumFactory.createString("name"+i)});
+          DatumFactory.createInt4(i),
+		      DatumFactory.createInt4(i + 32),
+		      DatumFactory.createText("name" + i)});
 		}
 
     Path path = StorageUtil.concatPath(testDir, "testGetScannerAndAppender", "table.csv");

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-storage/src/test/java/tajo/storage/TestStorages.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/test/java/tajo/storage/TestStorages.java b/tajo-core/tajo-core-storage/src/test/java/tajo/storage/TestStorages.java
index d351edf..d5c2b38 100644
--- a/tajo-core/tajo-core-storage/src/test/java/tajo/storage/TestStorages.java
+++ b/tajo-core/tajo-core-storage/src/test/java/tajo/storage/TestStorages.java
@@ -24,13 +24,13 @@ import org.apache.hadoop.fs.Path;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
+import tajo.catalog.CatalogUtil;
 import tajo.catalog.Options;
 import tajo.catalog.Schema;
-import tajo.catalog.TCatUtil;
 import tajo.catalog.TableMeta;
-import tajo.catalog.proto.CatalogProtos.DataType;
 import tajo.catalog.proto.CatalogProtos.StoreType;
 import tajo.catalog.statistics.TableStat;
+import tajo.common.TajoDataTypes.Type;
 import tajo.conf.TajoConf;
 import tajo.datum.Datum;
 import tajo.datum.DatumFactory;
@@ -85,10 +85,10 @@ public class TestStorages {
   public void testSplitable() throws IOException {
     if (splitable) {
       Schema schema = new Schema();
-      schema.addColumn("id", DataType.INT);
-      schema.addColumn("age", DataType.LONG);
+      schema.addColumn("id", Type.INT4);
+      schema.addColumn("age", Type.INT8);
 
-      TableMeta meta = TCatUtil.newTableMeta(schema, storeType);
+      TableMeta meta = CatalogUtil.newTableMeta(schema, storeType);
       Path tablePath = new Path(testDir, "Splitable.data");
       Appender appender = StorageManager.getAppender(conf, meta, tablePath);
       appender.enableStats();
@@ -98,8 +98,8 @@ public class TestStorages {
 
       for(int i = 0; i < tupleNum; i++) {
         vTuple = new VTuple(2);
-        vTuple.put(0, DatumFactory.createInt(i+1));
-        vTuple.put(1, DatumFactory.createLong(25l));
+        vTuple.put(0, DatumFactory.createInt4(i + 1));
+        vTuple.put(1, DatumFactory.createInt8(25l));
         appender.addTuple(vTuple);
       }
       appender.close();
@@ -138,11 +138,11 @@ public class TestStorages {
   @Test
   public void testProjection() throws IOException {
     Schema schema = new Schema();
-    schema.addColumn("id", DataType.INT);
-    schema.addColumn("age", DataType.LONG);
-    schema.addColumn("score", DataType.FLOAT);
+    schema.addColumn("id", Type.INT4);
+    schema.addColumn("age", Type.INT8);
+    schema.addColumn("score", Type.FLOAT4);
 
-    TableMeta meta = TCatUtil.newTableMeta(schema, storeType);
+    TableMeta meta = CatalogUtil.newTableMeta(schema, storeType);
 
     Path tablePath = new Path(testDir, "testProjection.data");
     Appender appender = StorageManager.getAppender(conf, meta, tablePath);
@@ -152,9 +152,9 @@ public class TestStorages {
 
     for(int i = 0; i < tupleNum; i++) {
       vTuple = new VTuple(3);
-      vTuple.put(0, DatumFactory.createInt(i+1));
-      vTuple.put(1, DatumFactory.createLong(i+2));
-      vTuple.put(2, DatumFactory.createFloat(i + 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();
@@ -163,8 +163,8 @@ public class TestStorages {
     Fragment fragment = new Fragment("testReadAndWrite", tablePath, meta, 0, status.getLen(), null);
 
     Schema target = new Schema();
-    target.addColumn("age", DataType.LONG);
-    target.addColumn("score", DataType.FLOAT);
+    target.addColumn("age", Type.INT8);
+    target.addColumn("score", Type.FLOAT4);
     Scanner scanner = StorageManager.getScanner(conf, meta, fragment, target);
     scanner.init();
     int tupleCnt = 0;
@@ -173,8 +173,8 @@ public class TestStorages {
       if (storeType == StoreType.RCFILE || storeType == StoreType.TREVNI) {
         assertNull(tuple.get(0));
       }
-      assertEquals(DatumFactory.createLong(tupleCnt + 2), tuple.getLong(1));
-      assertEquals(DatumFactory.createFloat(tupleCnt + 3), tuple.getFloat(2));
+      assertEquals(DatumFactory.createInt8(tupleCnt + 2), tuple.getLong(1));
+      assertEquals(DatumFactory.createFloat4(tupleCnt + 3), tuple.getFloat(2));
       tupleCnt++;
     }
     scanner.close();
@@ -185,40 +185,39 @@ public class TestStorages {
   @Test
   public void testVariousTypes() throws IOException {
     Schema schema = new Schema();
-    schema.addColumn("col1", DataType.BOOLEAN);
-    schema.addColumn("col2", DataType.BYTE);
-    schema.addColumn("col3", DataType.CHAR);
-    schema.addColumn("col4", DataType.SHORT);
-    schema.addColumn("col5", DataType.INT);
-    schema.addColumn("col6", DataType.LONG);
-    schema.addColumn("col7", DataType.FLOAT);
-    schema.addColumn("col8", DataType.DOUBLE);
-    schema.addColumn("col9", DataType.STRING);
-    schema.addColumn("col10", DataType.BYTES);
-    schema.addColumn("col11", DataType.IPv4);
-    schema.addColumn("col12", DataType.STRING2);
+    schema.addColumn("col1", Type.BOOLEAN);
+    schema.addColumn("col2", Type.BIT);
+    schema.addColumn("col3", Type.CHAR);
+    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);
 
     Options options = new Options();
-    TableMeta meta = TCatUtil.newTableMeta(schema, storeType, options);
+    TableMeta meta = CatalogUtil.newTableMeta(schema, storeType, options);
 
     Path tablePath = new Path(testDir, "testVariousTypes.data");
     Appender appender = StorageManager.getAppender(conf, meta, tablePath);
     appender.init();
 
-    Tuple tuple = new VTuple(12);
+    Tuple tuple = new VTuple(11);
     tuple.put(new Datum[] {
         DatumFactory.createBool(true),
-        DatumFactory.createByte((byte) 0x99),
+        DatumFactory.createBit((byte) 0x99),
         DatumFactory.createChar('7'),
-        DatumFactory.createShort((short) 17),
-        DatumFactory.createInt(59),
-        DatumFactory.createLong(23l),
-        DatumFactory.createFloat(77.9f),
-        DatumFactory.createDouble(271.9f),
-        DatumFactory.createString("hyunsik"),
-        DatumFactory.createBytes("hyunsik".getBytes()),
-        DatumFactory.createIPv4("192.168.0.1"),
-        DatumFactory.createString2("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"),
+        DatumFactory.createText("hyunsik")
     });
     appender.addTuple(tuple);
     appender.flush();

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-storage/src/test/java/tajo/storage/TestTupleComparator.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/test/java/tajo/storage/TestTupleComparator.java b/tajo-core/tajo-core-storage/src/test/java/tajo/storage/TestTupleComparator.java
index e37b3e5..af8d6d5 100644
--- a/tajo-core/tajo-core-storage/src/test/java/tajo/storage/TestTupleComparator.java
+++ b/tajo-core/tajo-core-storage/src/test/java/tajo/storage/TestTupleComparator.java
@@ -23,7 +23,7 @@ import org.junit.Before;
 import org.junit.Test;
 import tajo.catalog.Schema;
 import tajo.catalog.SortSpec;
-import tajo.catalog.proto.CatalogProtos.DataType;
+import tajo.common.TajoDataTypes.Type;
 import tajo.datum.Datum;
 import tajo.datum.DatumFactory;
 
@@ -42,29 +42,29 @@ public class TestTupleComparator {
   @Test
   public final void testCompare() {
     Schema schema = new Schema();
-    schema.addColumn("col1", DataType.INT);
-    schema.addColumn("col2", DataType.INT);
-    schema.addColumn("col3", DataType.INT);
-    schema.addColumn("col4", DataType.INT);
-    schema.addColumn("col5", DataType.STRING);
+    schema.addColumn("col1", Type.INT4);
+    schema.addColumn("col2", Type.INT4);
+    schema.addColumn("col3", Type.INT4);
+    schema.addColumn("col4", Type.INT4);
+    schema.addColumn("col5", Type.TEXT);
     
     Tuple tuple1 = new VTuple(5);
     Tuple tuple2 = new VTuple(5);
 
     tuple1.put(
         new Datum[] {
-        DatumFactory.createInt(9), 
-        DatumFactory.createInt(3),
-        DatumFactory.createInt(33), 
-        DatumFactory.createInt(4),
-        DatumFactory.createString("abc")});
+        DatumFactory.createInt4(9),
+        DatumFactory.createInt4(3),
+        DatumFactory.createInt4(33),
+        DatumFactory.createInt4(4),
+        DatumFactory.createText("abc")});
     tuple2.put(
         new Datum[] {
-        DatumFactory.createInt(1), 
-        DatumFactory.createInt(25),
-        DatumFactory.createInt(109),
-        DatumFactory.createInt(4),
-        DatumFactory.createString("abd")});
+        DatumFactory.createInt4(1),
+        DatumFactory.createInt4(25),
+        DatumFactory.createInt4(109),
+        DatumFactory.createInt4(4),
+        DatumFactory.createText("abd")});
 
     SortSpec sortKey1 = new SortSpec(schema.getColumn("col4"), true, false);
     SortSpec sortKey2 = new SortSpec(schema.getColumn("col5"), true, false);

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-storage/src/test/java/tajo/storage/TestVTuple.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/test/java/tajo/storage/TestVTuple.java b/tajo-core/tajo-core-storage/src/test/java/tajo/storage/TestVTuple.java
index cebbae7..18faf88 100644
--- a/tajo-core/tajo-core-storage/src/test/java/tajo/storage/TestVTuple.java
+++ b/tajo-core/tajo-core-storage/src/test/java/tajo/storage/TestVTuple.java
@@ -38,11 +38,11 @@ public class TestVTuple {
 	@Test
 	public void testContain() {
 		VTuple t1 = new VTuple(260);
-		t1.put(0, DatumFactory.createInt(1));
-		t1.put(1, DatumFactory.createInt(1));
-		t1.put(27, DatumFactory.createInt(1));
-		t1.put(96, DatumFactory.createInt(1));
-		t1.put(257, DatumFactory.createInt(1));
+		t1.put(0, DatumFactory.createInt4(1));
+		t1.put(1, DatumFactory.createInt4(1));
+		t1.put(27, DatumFactory.createInt4(1));
+		t1.put(96, DatumFactory.createInt4(1));
+		t1.put(257, DatumFactory.createInt4(1));
 		
 		assertTrue(t1.contains(0));
 		assertTrue(t1.contains(1));
@@ -60,36 +60,37 @@ public class TestVTuple {
 	@Test
 	public void testPut() {
 		VTuple t1 = new VTuple(260);
-		t1.put(0, DatumFactory.createString("str"));
-		t1.put(1, DatumFactory.createInt(2));
-		t1.put(257, DatumFactory.createFloat(0.76f));
+		t1.put(0, DatumFactory.createText("str"));
+		t1.put(1, DatumFactory.createInt4(2));
+		t1.put(257, DatumFactory.createFloat4(0.76f));
 		
 		assertTrue(t1.contains(0));
 		assertTrue(t1.contains(1));
 		
 		assertEquals(t1.getString(0).toString(),"str");
-		assertEquals(t1.getInt(1).asInt(),2);
-		assertTrue(t1.getFloat(257).asFloat() == 0.76f);		
+		assertEquals(t1.getInt(1).asInt4(),2);
+		assertTrue(t1.getFloat(257).asFloat4() == 0.76f);
 	}
-	
+
+  @Test
 	public void testEquals() {
 	  Tuple t1 = new VTuple(5);
 	  Tuple t2 = new VTuple(5);
 	  
-	  t1.put(0, DatumFactory.createInt(1));
-	  t1.put(1, DatumFactory.createInt(2));
-	  t1.put(3, DatumFactory.createInt(2));
+	  t1.put(0, DatumFactory.createInt4(1));
+	  t1.put(1, DatumFactory.createInt4(2));
+	  t1.put(3, DatumFactory.createInt4(2));
 	  
-	  t2.put(0, DatumFactory.createInt(1));
-    t2.put(1, DatumFactory.createInt(2));
-    t2.put(3, DatumFactory.createInt(2));
+	  t2.put(0, DatumFactory.createInt4(1));
+    t2.put(1, DatumFactory.createInt4(2));
+    t2.put(3, DatumFactory.createInt4(2));
     
     assertEquals(t1,t2);
     
     Tuple t3 = new VTuple(5);
-    t2.put(0, DatumFactory.createInt(1));
-    t2.put(1, DatumFactory.createInt(2));
-    t2.put(4, DatumFactory.createInt(2));
+    t2.put(0, DatumFactory.createInt4(1));
+    t2.put(1, DatumFactory.createInt4(2));
+    t2.put(4, DatumFactory.createInt4(2));
     
     assertNotSame(t1,t3);
 	}
@@ -99,22 +100,22 @@ public class TestVTuple {
 	  Tuple t1 = new VTuple(5);
     Tuple t2 = new VTuple(5);
     
-    t1.put(0, DatumFactory.createInt(1));
-    t1.put(1, DatumFactory.createInt(2));
-    t1.put(3, DatumFactory.createInt(2));
-    t1.put(4, DatumFactory.createString("hyunsik"));
+    t1.put(0, DatumFactory.createInt4(1));
+    t1.put(1, DatumFactory.createInt4(2));
+    t1.put(3, DatumFactory.createInt4(2));
+    t1.put(4, DatumFactory.createText("hyunsik"));
     
-    t2.put(0, DatumFactory.createInt(1));
-    t2.put(1, DatumFactory.createInt(2));
-    t2.put(3, DatumFactory.createInt(2));
-    t2.put(4, DatumFactory.createString("hyunsik"));
+    t2.put(0, DatumFactory.createInt4(1));
+    t2.put(1, DatumFactory.createInt4(2));
+    t2.put(3, DatumFactory.createInt4(2));
+    t2.put(4, DatumFactory.createText("hyunsik"));
     
     assertEquals(t1.hashCode(),t2.hashCode());
     
     Tuple t3 = new VTuple(5);
-    t3.put(0, DatumFactory.createInt(1));
-    t3.put(1, DatumFactory.createInt(2));
-    t3.put(4, DatumFactory.createInt(2));
+    t3.put(0, DatumFactory.createInt4(1));
+    t3.put(1, DatumFactory.createInt4(2));
+    t3.put(4, DatumFactory.createInt4(2));
     
     assertNotSame(t1.hashCode(),t3.hashCode());
 	}
@@ -123,18 +124,18 @@ public class TestVTuple {
   public void testPutTuple() {
     Tuple t1 = new VTuple(5);
 
-    t1.put(0, DatumFactory.createInt(1));
-    t1.put(1, DatumFactory.createInt(2));
-    t1.put(2, DatumFactory.createInt(3));
+    t1.put(0, DatumFactory.createInt4(1));
+    t1.put(1, DatumFactory.createInt4(2));
+    t1.put(2, DatumFactory.createInt4(3));
 
     Tuple t2 = new VTuple(2);
-    t2.put(0, DatumFactory.createInt(4));
-    t2.put(1, DatumFactory.createInt(5));
+    t2.put(0, DatumFactory.createInt4(4));
+    t2.put(1, DatumFactory.createInt4(5));
 
     t1.put(3, t2);
 
     for (int i = 0; i < 5; i++) {
-      assertEquals(i+1, t1.get(i).asInt());
+      assertEquals(i+1, t1.get(i).asInt4());
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-storage/src/test/java/tajo/storage/hcfile/IndexTest.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/test/java/tajo/storage/hcfile/IndexTest.java b/tajo-core/tajo-core-storage/src/test/java/tajo/storage/hcfile/IndexTest.java
deleted file mode 100644
index 0299a0e..0000000
--- a/tajo-core/tajo-core-storage/src/test/java/tajo/storage/hcfile/IndexTest.java
+++ /dev/null
@@ -1,66 +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 tajo.storage.hcfile;
-
-import org.junit.Test;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-
-public class IndexTest {
-
-  @Test
-  public void test() {
-    Index<Integer> index = new Index<Integer>();
-    index.add(new IndexItem(3, 400));
-    index.add(new IndexItem(0, 100));
-    index.add(new IndexItem(6, 700));
-    index.add(new IndexItem(1, 200));
-    index.sort();
-
-    IndexItem<Integer> result = index.get(0);
-    assertEquals(0, result.getRid());
-    result = index.get(1);
-    assertEquals(1, result.getRid());
-    result = index.get(2);
-    assertEquals(3, result.getRid());
-    result = index.get(3);
-    assertEquals(6, result.getRid());
-
-    result = index.searchExact(0);
-    assertNotNull(result);
-    assertEquals(0, result.getRid());
-    assertEquals(100, result.getValue().intValue());
-
-    result = index.searchLargestSmallerThan(5);
-    assertNotNull(result);
-    assertEquals(3, result.getRid());
-    assertEquals(400, result.getValue().intValue());
-
-    result = index.searchLargestSmallerThan(3);
-    assertNotNull(result);
-    assertEquals(3, result.getRid());
-    assertEquals(400, result.getValue().intValue());
-
-    result = index.searchSmallestLargerThan(5);
-    assertNotNull(result);
-    assertEquals(6, result.getRid());
-    assertEquals(700, result.getValue().intValue());
-  }
-}


[03/11] TAJO-57: Recognize Parser and Catalog Standard SQL data types. (hyunsik)

Posted by hy...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/HCFile.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/HCFile.java b/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/HCFile.java
deleted file mode 100644
index 83df49e..0000000
--- a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/HCFile.java
+++ /dev/null
@@ -1,373 +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 tajo.storage.hcfile;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.*;
-import tajo.catalog.proto.CatalogProtos.CompressType;
-import tajo.datum.Datum;
-import tajo.storage.exception.UnknownCodecException;
-import tajo.storage.exception.UnknownDataTypeException;
-import tajo.storage.hcfile.reader.Reader;
-import tajo.storage.hcfile.writer.Writer;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-
-public class HCFile {
-
-  public static class Scanner extends ColumnFileScanner {
-    private final Log LOG = LogFactory.getLog(Scanner.class);
-    public static final int HDFS_CHUNK_SIZE = 65536;
-    private HColumnMetaWritable meta;
-    private FileSystem fs;
-    private FSDataInputStream in;
-    private ByteBuffer buffer;
-    private UpdatableSeekableBlock block;
-    private Reader reader;
-    private Index<Long> index;
-    private int nextBlock;
-
-    public Scanner(Configuration conf, Path path)
-        throws IOException, UnknownDataTypeException, UnknownCodecException {
-      super(conf, path);
-      fs = path.getFileSystem(conf);
-      in = fs.open(path);
-      buffer = ByteBuffer.allocateDirect(HDFS_CHUNK_SIZE);
-      buffer.flip();
-      index = new Index<Long>();
-      FileStatus file = fs.getFileStatus(path);
-
-      readColumnHeader(file.getLen());
-      if (this.meta.isCompressed()) {
-        reader = new CompressedBlockReader(this.meta.getDataType(), this.meta.getCompressType());
-      } else {
-        reader = new BlockReader(this.meta.getDataType());
-      }
-    }
-
-    public Path getPath() {
-      return this.path;
-    }
-
-    protected boolean fillBuffer(int desired) throws IOException {
-      buffer.compact();
-      int read = in.read(buffer);
-      if (read == -1) {
-        return false;
-      } else {
-        if (read < desired) {
-          read += in.read(buffer);
-          if (read < desired) {
-            return false;
-          }
-        }
-        buffer.flip();
-        return true;
-      }
-    }
-
-    private void readColumnHeader(long fileLen) throws IOException {
-      in.seek(fileLen - HColumnMetaWritable.size());
-      long headerPos = in.getPos();
-      meta = new HColumnMetaWritable();
-      meta.readFields(in);
-
-      // read index
-      in.seek(meta.getOffsetToIndex());
-      while (in.getPos() < headerPos) {
-        index.add(new IndexItem<Long>(in.readLong(), in.readLong()));
-      }
-
-      in.seek(0);
-      nextBlock = 0;
-      block = null;
-    }
-
-//    private void readColumnHeader(long fileLen) throws IOException {
-//      in.seek(fileLen - Long.SIZE / 8);
-//      long headerPos = in.readLong();
-//      in.seek(headerPos);
-//
-//      // read header
-//      long headerLen = fileLen - Long.SIZE/8 - headerPos;
-//      byte[] buf = new byte[(int)headerLen];
-//      in.read(buf);
-//      ColumnMetaProto proto = ColumnMetaProto.parseFrom(buf);
-//      meta = new HCFileMetaImpl(proto);
-//
-//      // read index
-//      in.seek(((HCFileMetaImpl)meta).getOffsetToIndex());
-//      while (in.getPos() < headerPos) {
-//        index.add(new IndexItem(in.readLong(), in.readLong()));
-//      }
-//
-//      in.seek(0);
-//      currentBlock = 0;
-//      block = null;
-//    }
-
-    @Override
-    public ColumnMeta getMeta() throws IOException {
-      return this.meta;
-    }
-
-    @Override
-    public void first() throws IOException {
-      if (nextBlock != 1) {
-        pos(0);
-      }
-    }
-
-    @Override
-    public void last() throws IOException {
-      if (nextBlock != index.size()) {
-        IndexItem<Long> item = index.get(index.size()-1);
-        long lastOffset = item.getValue();
-        in.seek(lastOffset);
-        nextBlock = index.size()-1;
-        block = null;
-        buffer.clear();
-        buffer.flip();
-        getBlock();
-        block.last();
-      }
-    }
-
-    @Override
-    public void pos(long rid) throws IOException {
-      // Consider the case which the given rid is already involved in the block
-      if (block == null
-          || block.getMeta().getStartRid() > rid
-          || block.getMeta().getStartRid()+block.getMeta().getRecordNum() <= rid) {
-        IndexItem<Long> item = index.searchLargestSmallerThan(rid);
-        long offset = item.getValue();
-        in.seek(offset);
-        nextBlock = index.findPos(item);
-        block = null;
-        // block을 읽을 경우 buffer를 완전히 비워야 함
-        buffer.clear();
-        buffer.flip();
-        getBlock();
-      }
-
-      block.pos(rid-block.getMeta().getStartRid());
-    }
-
-    @Override
-    public Datum get() throws IOException {
-      Datum ret;
-      if (block == null || (ret=block.next()) == null) {
-        getBlock();
-        if (block == null) {
-          return null;
-        }
-        ret = block.next();
-      }
-      return ret;
-    }
-
-    @Override
-    public Block getBlock() throws IOException {
-      if (nextBlock < index.size()) {
-        if (buffer.remaining() < Integer.SIZE/8) {
-          if (!fillBuffer(Integer.SIZE/8-buffer.remaining())) {
-            return null;
-          }
-        }
-        int blockSize = buffer.getInt();
-        if (buffer.remaining() < blockSize) {
-          if (!fillBuffer(blockSize-buffer.remaining())) {
-            return null;
-          }
-        }
-
-        int originLimit = buffer.limit();
-        buffer.limit(buffer.position() + blockSize);
-        BlockMeta blockMeta = newBlockMeta();
-
-        block = (UpdatableSeekableBlock) reader.read(blockMeta, buffer);
-        buffer.limit(originLimit);
-        nextBlock++;
-        return block;
-      } else {
-        return null;
-      }
-    }
-
-    private BlockMeta newBlockMeta() throws IOException {
-      return new HBlockMetaImpl(
-          meta.getDataType(),
-          0, // not set
-          index.get(nextBlock).getRid(),
-          meta.isSorted(),
-          meta.isContiguous(),
-          meta.isCompressed());
-    }
-
-    @Override
-    public Datum[] getBlockAsArray() throws IOException {
-      return this.getBlock().asArray();
-    }
-
-    @Override
-    public long getPos() throws IOException {
-      // TODO
-      return block.getMeta().getStartRid() + block.getPos();
-    }
-
-    @Override
-    public void close() throws IOException {
-      reader.close();
-      in.close();
-    }
-  }
-
-  public static class Appender extends ColumnFileAppender {
-    private Log LOG = LogFactory.getLog(Appender.class);
-    private final int HDFS_BLOCK_SIZE;
-    private FSDataOutputStream out;
-    private FileSystem fs;
-    private UpdatableBlock block;
-    private BlockMeta blockMeta;
-    private long blockStartId;
-    private long startId;
-    private int blockRecordNum;
-    private int totalRecordNum;
-    private Writer writer;
-    private Index<Long> index;
-
-    public Appender(Configuration conf, ColumnMeta meta, Path path)
-        throws IOException, UnknownDataTypeException, UnknownCodecException {
-      super(conf, meta, path);
-      HDFS_BLOCK_SIZE = conf.getInt("dfs.blocksize", -1);
-      if (HDFS_BLOCK_SIZE == -1) {
-        throw new IOException("HDFS block size can not be initialized!");
-      } else {
-        LOG.info("HDFS block size: " + HDFS_BLOCK_SIZE);
-      }
-      fs = path.getFileSystem(conf);
-      out = fs.create(path);
-
-      if (this.meta.getCompressType() != CompressType.COMP_NONE) {
-        writer = new CompressedBlockWriter(out,
-            this.meta.getDataType(), this.meta.getCompressType());
-      } else {
-        writer = new BlockWriter(out, this.meta.getDataType());
-        blockMeta = new HBlockMetaImpl(this.meta.getDataType(), 0, 0,
-            this.meta.isSorted(), this.meta.isContiguous(), this.meta.isCompressed());
-        block = new BasicBlock();
-      }
-      index = new Index<Long>();
-      blockStartId = ((HColumnMetaWritable)this.meta).getStartRid();
-      startId = blockStartId;
-      blockRecordNum = 0;
-      totalRecordNum = 0;
-    }
-
-    public Path getPath() {
-      return this.path;
-    }
-
-    public boolean isAppendable(Datum datum) throws IOException {
-      // TODO: validation of (index.size()+1)
-      int indexSize = (index.size()+1) * 2 * Long.SIZE/8;
-      if (!block.isAppendable(datum)) {
-        indexSize += 2 * Long.SIZE/8;
-      }
-
-      return out.getPos() /* written size */
-          + block.getSize()
-          + Integer.SIZE/8 /* block size */
-          + ColumnStoreUtil.getWrittenSize(datum)
-          + HColumnMetaWritable.size()
-          + Long.SIZE/8 /* column header pos */
-          + indexSize
-          < HDFS_BLOCK_SIZE;
-    }
-
-    public long getStartId() {
-      return this.startId;
-    }
-
-    public int getRecordNum() {
-      return this.totalRecordNum;
-    }
-
-    @Override
-    public void append(Datum datum) throws IOException {
-      if (!block.isAppendable(datum)) {
-        flush();
-        blockRecordNum = 0;
-      }
-      block.appendValue(datum);
-      blockRecordNum++;
-      totalRecordNum++;
-    }
-
-    @Override
-    public void flush() throws IOException {
-      index.add(new IndexItem(blockStartId, writer.getPos()));
-      blockMeta.setStartRid(blockStartId)
-          .setRecordNum(blockRecordNum);
-      block.setMeta(blockMeta);
-      writer.write(block);
-      block.clear();
-      blockStartId += blockRecordNum;
-    }
-
-    private void writeColumnHeader() throws IOException {
-      long offsetToIndex = out.getPos();
-      // write index
-      for (IndexItem<Long> e : index.get()) {
-        out.writeLong(e.getRid());
-        out.writeLong(e.getValue());
-      }
-
-      // write header
-      HColumnMetaWritable columnMeta = (HColumnMetaWritable) meta;
-      columnMeta.setRecordNum(totalRecordNum);
-      columnMeta.setOffsetToIndex((int) offsetToIndex);
-      columnMeta.write(out);
-
-//      long headerPos = out.getPos();
-//
-//      HCFileMetaImpl hmeta = (HCFileMetaImpl) meta;
-//      hmeta.setRecordNum(totalRecordNum);
-//      hmeta.setOffsetToIndex((int)offsetToIndex);
-//      ColumnMetaProto proto = meta.getProto();
-//      FileUtil.writeProto(out, proto);
-//
-//      out.writeLong(headerPos);
-    }
-
-    @Override
-    public void close() throws IOException {
-      if (blockRecordNum > 0) {
-        flush();
-      }
-      writeColumnHeader();
-
-      writer.close();
-      out.close();
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/HCFileMetaImpl.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/HCFileMetaImpl.java b/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/HCFileMetaImpl.java
deleted file mode 100644
index e06e944..0000000
--- a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/HCFileMetaImpl.java
+++ /dev/null
@@ -1,301 +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 tajo.storage.hcfile;
-
-import tajo.catalog.proto.CatalogProtos.*;
-import tajo.catalog.proto.CatalogProtos.ColumnMetaProto.Builder;
-import tajo.common.ProtoObject;
-
-public class HCFileMetaImpl implements ColumnMeta, ProtoObject<ColumnMetaProto> {
-  private ColumnMetaProto proto = ColumnMetaProto.getDefaultInstance();
-  private Builder builder = null;
-  private boolean viaProto = false;
-
-  private Long startRid;
-  private Integer recordNum;
-  private Integer offsetToIndex;
-  private DataType dataType;
-  private StoreType storeType;
-  private CompressType compType;
-  private Boolean compressed;
-  private Boolean sorted;
-  private Boolean contiguous;
-
-  public HCFileMetaImpl() {
-    setModified();
-    storeType = StoreType.HCFILE;
-  }
-
-  public HCFileMetaImpl(long startRid, DataType dataType,
-                        CompressType compType, boolean compressed,
-                        boolean sorted, boolean contiguous) {
-    this();
-    this.startRid = startRid;
-    this.dataType = dataType;
-    this.compType = compType;
-    this.compressed = compressed;
-    this.sorted = sorted;
-    this.contiguous = contiguous;
-  }
-
-  public HCFileMetaImpl(ColumnMetaProto proto) {
-    this.proto = proto;
-    this.viaProto = true;
-  }
-
-  public void setStartRid(Long startRid) {
-    setModified();
-    this.startRid = startRid;
-  }
-
-  public void setRecordNum(Integer recordNum) {
-    setModified();
-    this.recordNum = recordNum;
-  }
-
-  public void setOffsetToIndex(Integer offsetToIndex) {
-    setModified();
-    this.offsetToIndex = offsetToIndex;
-  }
-
-  public void setDataType(DataType dataType) {
-    setModified();
-    this.dataType = dataType;
-  }
-
-  public void setCompressed(Boolean compressed) {
-    setModified();
-    this.compressed = compressed;
-  }
-
-  public void setSorted(Boolean sorted) {
-    setModified();
-    this.sorted = sorted;
-  }
-
-  public void setContiguous(Boolean contiguous) {
-    setModified();
-    this.contiguous = contiguous;
-  }
-
-  public void setCompType(CompressType compType) {
-    setModified();
-    this.compType = compType;
-  }
-
-  public Long getStartRid() {
-    ColumnMetaProtoOrBuilder p = viaProto ? proto : builder;
-    if (startRid != null) {
-      return this.startRid;
-    }
-    if (!p.hasStartRid()) {
-      return null;
-    }
-    this.startRid = p.getStartRid();
-
-    return startRid;
-  }
-
-  public Integer getRecordNum() {
-    ColumnMetaProtoOrBuilder p = viaProto ? proto : builder;
-    if (recordNum != null) {
-      return this.recordNum;
-    }
-    if (!p.hasRecordNum()) {
-      return null;
-    }
-    this.recordNum = p.getRecordNum();
-    return recordNum;
-  }
-
-  public Integer getOffsetToIndex() {
-    ColumnMetaProtoOrBuilder p = viaProto ? proto : builder;
-    if (offsetToIndex != null) {
-      return this.offsetToIndex;
-    }
-    if (!p.hasOffsetToIndex()) {
-      return null;
-    }
-    this.offsetToIndex = p.getOffsetToIndex();
-    return offsetToIndex;
-  }
-
-  @Override
-  public StoreType getStoreType() {
-    ColumnMetaProtoOrBuilder p = viaProto ? proto : builder;
-    if (storeType != null) {
-      return this.storeType;
-    }
-    if (!p.hasStoreType()) {
-      return null;
-    }
-    storeType = p.getStoreType();
-    return this.storeType;
-  }
-
-  @Override
-  public DataType getDataType() {
-    ColumnMetaProtoOrBuilder p = viaProto ? proto : builder;
-    if (dataType != null) {
-      return dataType;
-    }
-    if (!p.hasDataType()) {
-      return null;
-    }
-    dataType = p.getDataType();
-    return dataType;
-  }
-
-  @Override
-  public CompressType getCompressType() {
-    ColumnMetaProtoOrBuilder p = viaProto ? proto : builder;
-    if (compType != null) {
-      return compType;
-    }
-    if (!p.hasCompType()) {
-      return null;
-    }
-    compType = p.getCompType();
-    return compType;
-  }
-
-  @Override
-  public boolean isCompressed() {
-    ColumnMetaProtoOrBuilder p = viaProto ? proto : builder;
-    if (compressed != null) {
-      return compressed;
-    }
-    if (!p.hasCompressed()) {
-      return false;
-    }
-
-    compressed = p.getCompressed();
-    return compressed;
-  }
-
-  @Override
-  public boolean isSorted() {
-    ColumnMetaProtoOrBuilder p = viaProto ? proto : builder;
-    if (sorted != null) {
-      return sorted;
-    }
-    if (!p.hasSorted()) {
-      return false;
-    }
-
-    sorted = p.getSorted();
-    return sorted;
-  }
-
-  @Override
-  public boolean isContiguous() {
-    ColumnMetaProtoOrBuilder p = viaProto ? proto : builder;
-    if (contiguous != null) {
-      return contiguous;
-    }
-    if (!p.hasContiguous()) {
-      return false;
-    }
-    contiguous = p.getContiguous();
-    return contiguous;
-  }
-
-  @Override
-  public void initFromProto() {
-    ColumnMetaProtoOrBuilder p = viaProto ? proto : builder;
-    if (this.startRid == null && p.hasStartRid()) {
-      this.startRid = p.getStartRid();
-    }
-    if (this.recordNum == null && p.hasRecordNum()) {
-      this.recordNum = p.getRecordNum();
-    }
-    if (this.offsetToIndex == null && p.hasOffsetToIndex()) {
-      this.offsetToIndex = p.getOffsetToIndex();
-    }
-    if (this.dataType == null && p.hasDataType()) {
-      this.dataType = p.getDataType();
-    }
-    if (this.compressed == null && p.hasCompressed()) {
-      this.compressed = p.getCompressed();
-    }
-    if (this.sorted == null && p.hasSorted()) {
-      this.sorted = p.getSorted();
-    }
-    if (this.contiguous == null && p.hasContiguous()) {
-      this.contiguous = p.getContiguous();
-    }
-    if (this.storeType == null && p.hasStoreType()) {
-      this.storeType = p.getStoreType();
-    }
-    if (this.compType == null && p.hasCompType()) {
-      this.compType = p.getCompType();
-    }
-  }
-
-  @Override
-  public ColumnMetaProto getProto() {
-    if(!viaProto) {
-      mergeLocalToBuilder();
-      proto = builder.build();
-      viaProto = true;
-    }
-
-    return proto;
-  }
-
-  private void setModified() {
-    if (viaProto && builder == null) {
-      builder = ColumnMetaProto.newBuilder(proto);
-    }
-    viaProto = false;
-  }
-
-  private void mergeLocalToBuilder() {
-    if (builder == null) {
-      builder = ColumnMetaProto.newBuilder(proto);
-    }
-    if (this.startRid != null) {
-      builder.setStartRid(startRid);
-    }
-    if (this.recordNum != null) {
-      builder.setRecordNum(recordNum);
-    }
-    if (this.offsetToIndex != null) {
-      builder.setOffsetToIndex(offsetToIndex);
-    }
-    if (this.dataType != null) {
-      builder.setDataType(dataType);
-    }
-    if (this.compressed != null) {
-      builder.setCompressed(compressed);
-    }
-    if (this.sorted != null) {
-      builder.setSorted(sorted);
-    }
-    if (this.contiguous != null) {
-      builder.setContiguous(contiguous);
-    }
-    if (this.storeType != null) {
-      builder.setStoreType(storeType);
-    }
-    if (this.compType != null) {
-      builder.setCompType(compType);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/HCTupleAppender.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/HCTupleAppender.java b/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/HCTupleAppender.java
deleted file mode 100644
index 5de66a5..0000000
--- a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/HCTupleAppender.java
+++ /dev/null
@@ -1,194 +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 tajo.storage.hcfile;
-
-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.FSDataOutputStream;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import tajo.catalog.Column;
-import tajo.catalog.Schema;
-import tajo.catalog.TableMeta;
-import tajo.catalog.proto.CatalogProtos.CompressType;
-import tajo.catalog.proto.CatalogProtos.DataType;
-import tajo.storage.Tuple;
-import tajo.storage.exception.UnknownCodecException;
-import tajo.storage.exception.UnknownDataTypeException;
-import tajo.storage.hcfile.HCFile.Appender;
-
-import java.io.IOException;
-import java.util.Map;
-
-public class HCTupleAppender implements TupleAppender {
-  private final Log LOG = LogFactory.getLog(HCTupleAppender.class);
-  private Configuration conf;
-  private Schema schema;
-  private final Path dataDir;
-  private int shardId;
-  private Map<Column, Appender> columnAppenderMap = Maps.newHashMap();
-  private Map<Column, Integer> columnFileIdMap = Maps.newHashMap();
-  private Column baseColumn;
-  private int baseColumnIdx;
-
-  public HCTupleAppender(Configuration conf, TableMeta meta, int baseColumnIdx, Path tablePath)
-      throws IOException, UnknownCodecException, UnknownDataTypeException {
-    this.conf = conf;
-    this.schema = meta.getSchema();
-    this.baseColumnIdx = baseColumnIdx;
-    this.baseColumn = schema.getColumn(baseColumnIdx);
-    this.dataDir = new Path(tablePath, "data");
-    this.shardId = -1;
-    newShard();
-  }
-
-  @Override
-  public void addTuple(Tuple t) throws IOException {
-    Appender appender = columnAppenderMap.get(baseColumn);
-
-    // try base column
-    if (!appender.isAppendable(t.get(baseColumnIdx))) {
-      try {
-        newShard();
-        appender = columnAppenderMap.get(baseColumn);
-      } catch (UnknownDataTypeException e) {
-        throw new IOException(e);
-      } catch (UnknownCodecException e) {
-        throw new IOException(e);
-      }
-    }
-    appender.append(t.get(baseColumnIdx));
-
-    for (int i = 0; i < schema.getColumnNum(); i++) {
-      if (i == baseColumnIdx) continue;
-
-      appender = columnAppenderMap.get(schema.getColumn(i));
-      if (!appender.isAppendable(t.get(i))) {
-        try {
-          appender = newAppender(schema.getColumn(i));
-        } catch (UnknownDataTypeException e) {
-          LOG.info(e);
-        } catch (UnknownCodecException e) {
-          LOG.info(e);
-        }
-      }
-      appender.append(t.get(i));
-    }
-  }
-
-  private Appender newAppender(Column column)
-      throws UnknownCodecException, IOException, UnknownDataTypeException {
-    long nextStartId;
-
-    if (columnAppenderMap.containsKey(column)) {
-      Appender oldAppender = columnAppenderMap.get(column);
-      nextStartId = oldAppender.getStartId() + oldAppender.getRecordNum();
-      oldAppender.close();
-    } else {
-      nextStartId = 0;
-    }
-
-    ColumnMeta columnMeta = newColumnMeta(nextStartId, column.getDataType());
-    Path columnPath = new Path(dataDir, getColumnFileName(column));
-    LOG.info("new appender is initialized for " + column.getColumnName());
-    LOG.info("column path:  " + columnPath.toString());
-    Appender newAppender = new Appender(conf, columnMeta, columnPath);
-    columnAppenderMap.put(column, newAppender);
-    return newAppender;
-  }
-
-  private ColumnMeta newColumnMeta(long startId, DataType dataType) {
-    return new HColumnMetaWritable(startId, dataType, CompressType.COMP_NONE,
-        false, false, true);
-  }
-
-  private String getColumnFileName(Column column) {
-    int fileId;
-    if (columnFileIdMap.containsKey(column)) {
-      fileId = columnFileIdMap.get(column) + 1;
-    } else {
-      fileId = 0;
-    }
-    columnFileIdMap.put(column, fileId);
-    return shardId + "/" + column.getColumnName() + "_" + fileId;
-  }
-
-  private void newShard()
-      throws UnknownDataTypeException, IOException, UnknownCodecException {
-    ++shardId;
-    columnFileIdMap.clear();
-    LOG.info("new shard id: " + shardId);
-    long oldStartRid = -1, newStartRid = -1;
-
-    if (!columnAppenderMap.isEmpty()) {
-      oldStartRid = columnAppenderMap.get(schema.getColumn(0)).getStartId();
-    }
-
-    for (Column column : schema.getColumns()) {
-      newAppender(column);
-    }
-
-    if (oldStartRid != -1) {
-      newStartRid = columnAppenderMap.get(schema.getColumn(0)).getStartId();
-      writeMeta(shardId-1, oldStartRid, newStartRid-oldStartRid);
-    }
-
-//    index.add(new IndexItem(columnAppenderMap.get(schema.getColumn(0)).getStartId(), shardId));
-  }
-
-  @Override
-  public void flush() throws IOException {
-    for (Appender appender : columnAppenderMap.values()) {
-      appender.flush();
-    }
-  }
-
-  @Override
-  public void close() throws IOException {
-    Appender app = columnAppenderMap.get(schema.getColumn(0));
-    writeMeta(shardId, app.getStartId(), app.getRecordNum());
-    for (Appender appender : columnAppenderMap.values()) {
-      appender.close();
-    }
-//    writeIndex();
-  }
-
-  private void writeMeta(int shardId, long startRid, long length) throws IOException {
-    FileSystem fs = dataDir.getFileSystem(conf);
-    Path shardPath = new Path(dataDir, shardId+"");
-    Path metaPath = new Path(shardPath, ".smeta");
-    FSDataOutputStream out = fs.create(metaPath);
-    out.writeLong(startRid);
-    out.writeLong(length);
-    out.close();
-  }
-
-/*  private void writeIndex() throws IOException {
-    FileSystem fs = dataDir.getFileSystem(conf);
-    Path indexPath = new Path(dataDir, ".index");
-    FSDataOutputStream out = fs.create(indexPath);
-    for (IndexItem<Integer> item : index.get()) {
-      out.writeLong(item.getRid());
-      out.writeInt(item.getValue());
-    }
-    out.close();
-  }*/
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/HColumnMetaWritable.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/HColumnMetaWritable.java b/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/HColumnMetaWritable.java
deleted file mode 100644
index 4a49f9e..0000000
--- a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/HColumnMetaWritable.java
+++ /dev/null
@@ -1,164 +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 tajo.storage.hcfile;
-
-import org.apache.hadoop.io.Writable;
-import tajo.catalog.proto.CatalogProtos.CompressType;
-import tajo.catalog.proto.CatalogProtos.DataType;
-import tajo.catalog.proto.CatalogProtos.StoreType;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-
-public class HColumnMetaWritable implements ColumnMeta, Writable {
-  public static final int SIZE = Long.SIZE/8
-      + 2 * Integer.SIZE/8
-      + 2 * Short.SIZE/8
-      + 3 * Byte.SIZE/8;
-
-
-  private long startRid;
-  private int recordNum;
-  private int offsetToIndex;
-  private DataType dataType;
-  private CompressType compType;
-  private boolean compressed;
-  private boolean sorted;
-  private boolean contiguous;
-
-  public HColumnMetaWritable() {
-
-  }
-
-  public HColumnMetaWritable(long startRid, DataType dataType,
-                             CompressType compType, boolean compressed,
-                             boolean sorted, boolean contiguous) {
-    this.startRid = startRid;
-    this.dataType = dataType;
-    this.compType = compType;
-    this.compressed = compressed;
-    this.sorted = sorted;
-    this.contiguous = contiguous;
-  }
-
-  public void setStartRid(long startRid) {
-    this.startRid = startRid;
-  }
-
-  public void setRecordNum(int recordNum) {
-    this.recordNum = recordNum;
-  }
-
-  public void setOffsetToIndex(int offsetToIndex) {
-    this.offsetToIndex = offsetToIndex;
-  }
-
-  public void setDataType(DataType dataType) {
-    this.dataType = dataType;
-  }
-
-  public void setCompType(CompressType compType) {
-    this.compType = compType;
-  }
-
-  public void setCompressed(boolean compressed) {
-    this.compressed = compressed;
-  }
-
-  public void setSorted(boolean sorted) {
-    this.sorted = sorted;
-  }
-
-  public void setContiguous(boolean contiguous) {
-    this.contiguous = contiguous;
-  }
-
-  @Override
-  public StoreType getStoreType() {
-    return StoreType.HCFILE;
-  }
-
-  @Override
-  public DataType getDataType() {
-    return dataType;
-  }
-
-  @Override
-  public CompressType getCompressType() {
-    return compType;
-  }
-
-  @Override
-  public boolean isCompressed() {
-    return compressed;
-  }
-
-  @Override
-  public boolean isSorted() {
-    return sorted;
-  }
-
-  @Override
-  public boolean isContiguous() {
-    return contiguous;
-  }
-
-  public long getStartRid() {
-    return startRid;
-  }
-
-  public int getRecordNum() {
-    return recordNum;
-  }
-
-  public long getOffsetToIndex() {
-    return this.offsetToIndex;
-
-  }
-
-  @Override
-  public void write(DataOutput out) throws IOException {
-    out.writeLong(startRid);
-    out.writeInt(recordNum);
-    out.writeInt(offsetToIndex);
-    out.writeShort(dataType.getNumber());
-    out.writeShort(compType.getNumber());
-    out.writeBoolean(compressed);
-    out.writeBoolean(sorted);
-    out.writeBoolean(contiguous);
-  }
-
-  @Override
-  public void readFields(DataInput in) throws IOException {
-    startRid = in.readLong();
-    recordNum = in.readInt();
-    offsetToIndex = in.readInt();
-    dataType = DataType.valueOf(in.readShort());
-    compType = CompressType.valueOf(in.readShort());
-    compressed = in.readBoolean();
-    sorted = in.readBoolean();
-    contiguous = in.readBoolean();
-  }
-
-  public static int size() {
-    return SIZE;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/HColumnReader.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/HColumnReader.java b/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/HColumnReader.java
deleted file mode 100644
index 9994c68..0000000
--- a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/HColumnReader.java
+++ /dev/null
@@ -1,259 +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 tajo.storage.hcfile;
-
-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.FSDataInputStream;
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import tajo.catalog.Column;
-import tajo.catalog.Schema;
-import tajo.catalog.TableMeta;
-import tajo.catalog.TableMetaImpl;
-import tajo.catalog.proto.CatalogProtos.TableProto;
-import tajo.datum.Datum;
-import tajo.storage.exception.UnknownCodecException;
-import tajo.storage.exception.UnknownDataTypeException;
-import tajo.storage.hcfile.HCFile.Scanner;
-import tajo.util.FileUtil;
-
-import java.io.IOException;
-import java.util.Arrays;
-import java.util.List;
-
-public class HColumnReader implements ColumnReader {
-  private final Log LOG = LogFactory.getLog(HColumnReader.class);
-  private final Configuration conf;
-  private final FileSystem fs;
-  private Scanner scanner;
-  private Path dataDir;
-  private List<Path> dataPaths = Lists.newArrayList();
-  private int next;
-  private Index<Integer> index;
-  private Column target;
-
-  public HColumnReader(Configuration conf, Path tablePath, int targetColumnIdx)
-      throws IOException {
-    this.conf = conf;
-    this.fs = tablePath.getFileSystem(this.conf);
-    Schema schema = getSchema(tablePath);
-    init(conf, tablePath, schema.getColumn(targetColumnIdx));
-  }
-
-  public HColumnReader(Configuration conf, Path tablePath, String columnName)
-      throws IOException {
-    this.conf = conf;
-    this.fs = tablePath.getFileSystem(this.conf);
-    Schema schema = getSchema(tablePath);
-    init(conf, tablePath, schema.getColumnByName(columnName));
-  }
-
-  public HColumnReader(Configuration conf, Path tablePath, Column target)
-      throws IOException {
-    this.conf = conf;
-    this.fs = tablePath.getFileSystem(this.conf);
-    init(conf, tablePath, target);
-  }
-
-  public List<Path> getDataPaths() {
-    return this.dataPaths;
-  }
-
-  private void init(Configuration conf, Path tablePath, Column target) throws IOException {
-    this.dataDir = new Path(tablePath, "data");
-    FileStatus[] files = fs.listStatus(dataDir);
-    Path[] shardPaths = new Path[files.length];
-    int i, j = 0;
-    for (i = 0; i < files.length; i++) {
-      shardPaths[j++] = files[i].getPath();
-    }
-    Arrays.sort(shardPaths);
-//    next = 0;
-    index = new Index<Integer>();
-    this.target = target;
-    initDataPaths(shardPaths);
-  }
-
-  private void setTarget(Path tablePath, int targetColumnIdx) throws IOException {
-    Schema schema = getSchema(tablePath);
-    this.target = schema.getColumn(targetColumnIdx);
-  }
-
-  private void setTarget(Path tablePath, String columnName) throws IOException {
-    Schema schema = getSchema(tablePath);
-    this.target = schema.getColumn(columnName);
-  }
-
-  private Schema getSchema(Path tablePath) throws IOException {
-    Path metaPath = new Path(tablePath, ".meta");
-    TableProto proto = (TableProto) FileUtil.loadProto(fs, metaPath,
-        TableProto.getDefaultInstance());
-    TableMeta meta = new TableMetaImpl(proto);
-    return meta.getSchema();
-  }
-
-  private void initDataPaths(Path[] shardPaths) throws IOException {
-    Path indexPath;
-    FSDataInputStream in;
-    long rid;
-    String targetName = target.getColumnName();
-
-    for (int i = 0; i < shardPaths.length; i++) {
-      indexPath = new Path(shardPaths[i], ".smeta");
-      in = fs.open(indexPath);
-
-      rid = in.readLong();
-      index.add(new IndexItem(rid, i));
-
-      in.close();
-
-      FileStatus[] columnFiles = fs.listStatus(shardPaths[i]);
-      for (FileStatus file : columnFiles) {
-        String colName = file.getPath().getName();
-        if (colName.length() < targetName.length()) {
-          continue;
-        }
-        if (colName.substring(0, targetName.length()).equals(targetName)) {
-          LOG.info("column file: " + file.getPath().toString());
-          dataPaths.add(file.getPath());
-        }
-      }
-    }
-    next = 0;
-  }
-
-  @Override
-  public void first() throws IOException {
-    pos(0);
-  }
-
-  @Override
-  public void last() throws IOException {
-    // TODO
-  }
-
-  @Override
-  public Datum get() throws IOException {
-    Datum datum;
-
-    if (scanner == null) {
-      if (next < dataPaths.size()) {
-        scanner = getScanner(conf, dataPaths.get(next++));
-        return scanner.get();
-      } else {
-        return null;
-      }
-    } else {
-      datum = scanner.get();
-      if (datum == null) {
-        scanner.close();
-        scanner = null;
-        if (next < dataPaths.size()) {
-          scanner = getScanner(conf, dataPaths.get(next++));
-          return scanner.get();
-        } else {
-          return null;
-        }
-      }
-      return datum;
-    }
-  }
-
-  @Override
-  public void pos(long rid) throws IOException {
-    if (scanner != null) {
-      HColumnMetaWritable meta = (HColumnMetaWritable) scanner.getMeta();
-      if (meta.getStartRid() <= rid
-          && meta.getStartRid() + meta.getRecordNum() > rid) {
-        scanner.pos(rid);
-        return;
-      } else {
-        scanner.close();
-        scanner = null;
-      }
-    }
-
-    // scanner must be null
-    IndexItem<Integer> item = index.searchLargestSmallerThan(rid);
-    if (item == null) {
-      throw new IOException("Cannot find the column file containing " + rid);
-    }
-    Path shardPath = new Path(dataDir, item.getValue().toString());
-
-    int i;
-    FileStatus[] files = fs.listStatus(shardPath);
-    for (FileStatus file : files) {
-      if (dataPaths.contains(file.getPath())) {
-        LOG.info("matched path: " + file.getPath());
-        scanner = getScanner(conf, file.getPath());
-        HColumnMetaWritable meta = (HColumnMetaWritable) scanner.getMeta();
-        LOG.info("start: " + meta.getStartRid() + " len: " + meta.getRecordNum());
-        if (meta.getStartRid() <= rid
-            && meta.getStartRid() + meta.getRecordNum() > rid) {
-          scanner.pos(rid);
-
-          for (i = 0; i < dataPaths.size(); i++) {
-            if (file.getPath().equals(dataPaths.get(i))) {
-              next = i+1;
-              break;
-            }
-          }
-          if (i == dataPaths.size()) {
-            throw new IOException("Invalid path: " + file.getPath().toString());
-          }
-          return;
-        }
-        scanner.close();
-        scanner = null;
-      }
-    }
-
-    if (scanner == null) {
-      throw new IOException("Null scanner");
-    }
-  }
-
-  private Scanner getScanner(Configuration conf, Path columnPath) throws IOException {
-    try {
-      scanner = new Scanner(conf, columnPath);
-      return scanner;
-    } catch (UnknownDataTypeException e) {
-      throw new IOException(e);
-    } catch (UnknownCodecException e) {
-      throw new IOException(e);
-    }
-  }
-
-  @Override
-  public long getPos() throws IOException {
-    // TODO
-    return scanner.getPos();
-  }
-
-  @Override
-  public void close() throws  IOException {
-    if (scanner != null) {
-      scanner.close();
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/Index.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/Index.java b/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/Index.java
deleted file mode 100644
index 7e6dbf8..0000000
--- a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/Index.java
+++ /dev/null
@@ -1,170 +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 tajo.storage.hcfile;
-
-import com.google.common.collect.Lists;
-
-import java.util.Arrays;
-import java.util.List;
-
-public class Index<T> {
-  private List<IndexItem<T>> index = Lists.newArrayList();
-
-  public Index() {
-
-  }
-
-  public void add(IndexItem<T> item) {
-    index.add(item);
-  }
-
-  public void sort() {
-    IndexItem<T>[] array = index.toArray(new IndexItem[index.size()]);
-    index.clear();
-    Arrays.sort(array);
-    for (IndexItem<T> item : array) {
-      index.add(item);
-    }
-  }
-
-  public IndexItem<T> get(int i) {
-    return index.get(i);
-  }
-
-  public void clear() {
-    index.clear();
-  }
-
-  public int findPos(IndexItem<T> searchItem) {
-    for (int i = 0; i < index.size(); i++) {
-      if (index.get(i).equals(searchItem)) {
-        return i;
-      }
-    }
-    return -1;
-  }
-
-  public int size() {
-    return index.size();
-  }
-
-  public List<IndexItem<T>> get() {
-    return index;
-  }
-
-  /**
-   * For a given rid, search the index item
-   * @param rid search rid
-   * @return If the rid is in the index, return the index item which has the rid.
-   * Else, return null
-   */
-  public IndexItem<T> searchExact(long rid) {
-    if (index.isEmpty()) {
-      return null;
-    }
-
-    return searchExact(rid, 0, index.size());
-  }
-
-  private IndexItem<T> searchExact(long rid, int start, int length) {
-    int leftLength = length/2;
-    IndexItem<T> mid = index.get(start + leftLength);
-    if (mid.getRid() == rid) {
-      return mid;
-    } else if (length == 1) {
-      return null;
-    }
-
-    if (mid.getRid() > rid) {
-      return searchExact(rid, start, leftLength);
-    } else if (mid.getRid() < rid) {
-      return searchExact(rid, start + leftLength, length - leftLength);
-    }
-    return null;
-  }
-
-  /**
-   * For a given rid,
-   * search the index item of which rid is the largest smaller than the given rid.
-   *
-   * @param rid  search rid
-   * @return If the given rid is in the index, return the index item which has the rid.
-   * Else, return the index item which has the largest rid smaller than the given rid.
-   */
-  public IndexItem<T> searchLargestSmallerThan(long rid) {
-    if (index.isEmpty()) {
-      return null;
-    }
-    return searchLargestSmallerThan(rid, 0, index.size());
-  }
-
-  private IndexItem<T> searchLargestSmallerThan(long rid, int start, int length) {
-    int leftLength = length/2;
-    IndexItem<T> mid = index.get(start + leftLength);
-    if (mid.getRid() == rid) {
-      return mid;
-    } else if (length == 1) {
-      return index.get(start+leftLength);
-    }
-
-    if (mid.getRid() > rid) {
-      return searchLargestSmallerThan(rid, start, leftLength);
-    } else if (mid.getRid() < rid) {
-      return searchLargestSmallerThan(rid, start + leftLength, length - leftLength);
-    }
-
-    return null;
-  }
-
-  /**
-   * For a given rid,
-   * search the index item of which rid is the smallest larger than the given rid.
-   *
-   * @param rid  search rid
-   * @return If the given rid is in the index, return the index item which has the rid.
-   * Else, return the index item which has the smallest rid larger than the given rid.
-   */
-  public IndexItem<T> searchSmallestLargerThan(long rid) {
-    if (index.isEmpty()) {
-      return null;
-    }
-    return searchSmallestLargerThan(rid, 0, index.size());
-  }
-
-  private IndexItem<T> searchSmallestLargerThan(long rid, int start, int length) {
-    int leftLength = length/2;
-    IndexItem<T> mid = index.get(start + leftLength);
-    if (mid.getRid() == rid) {
-      return mid;
-    } else if (length == 1) {
-      if (start+leftLength+1 >= index.size()) {
-        return null;
-      } else {
-        return index.get(start+leftLength+1);
-      }
-    }
-
-    if (mid.getRid() > rid) {
-      return searchSmallestLargerThan(rid, start, leftLength);
-    } else if (mid.getRid() < rid) {
-      return searchSmallestLargerThan(rid, start + leftLength, length - leftLength);
-    }
-    return null;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/IndexItem.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/IndexItem.java b/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/IndexItem.java
deleted file mode 100644
index 8ce62c3..0000000
--- a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/IndexItem.java
+++ /dev/null
@@ -1,62 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package tajo.storage.hcfile;
-
-public class IndexItem<T> implements Comparable<IndexItem> {
-  private long rid;
-  private T value;
-
-  public IndexItem() {
-
-  }
-
-  public IndexItem(long rid, T value) {
-    this.set(rid, value);
-  }
-
-  public void set(long rid, T offset) {
-    this.rid = rid;
-    this.value = offset;
-  }
-
-  public T getValue() {
-    return value;
-  }
-
-  public long getRid() {
-    return rid;
-  }
-
-  @Override
-  public int compareTo(IndexItem indexItem) {
-    return (int)(this.rid - indexItem.rid);
-  }
-
-  @Override
-  public boolean equals(Object o) {
-    if (o instanceof IndexItem) {
-      IndexItem item = (IndexItem) o;
-      if (this.rid == item.rid &&
-          this.value.equals(item.value)) {
-        return true;
-      }
-    }
-    return false;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/Seekable.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/Seekable.java b/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/Seekable.java
deleted file mode 100644
index 1a82270..0000000
--- a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/Seekable.java
+++ /dev/null
@@ -1,30 +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 tajo.storage.hcfile;
-
-import java.io.IOException;
-
-public interface Seekable {
-
-  public void first() throws IOException;
-
-  public void last() throws IOException;
-
-  public void pos(long rid) throws IOException;
-}

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

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/Updatable.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/Updatable.java b/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/Updatable.java
deleted file mode 100644
index 5952d3d..0000000
--- a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/Updatable.java
+++ /dev/null
@@ -1,36 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package tajo.storage.hcfile;
-
-import tajo.datum.Datum;
-
-import java.io.IOException;
-
-public interface Updatable {
-
-  void appendValue(Datum data) throws IOException;
-
-  void setValues(Datum[] data) throws IOException;
-
-  Datum removeValue() throws IOException;
-
-  void clear() throws IOException;
-
-  boolean isAppendable(Datum datum) throws IOException;
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/UpdatableBlock.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/UpdatableBlock.java b/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/UpdatableBlock.java
deleted file mode 100644
index 8321075..0000000
--- a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/UpdatableBlock.java
+++ /dev/null
@@ -1,22 +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 tajo.storage.hcfile;
-
-public abstract class UpdatableBlock implements Updatable, Block {
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/UpdatableSeekableBlock.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/UpdatableSeekableBlock.java b/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/UpdatableSeekableBlock.java
deleted file mode 100644
index b2d6ea6..0000000
--- a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/UpdatableSeekableBlock.java
+++ /dev/null
@@ -1,23 +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 tajo.storage.hcfile;
-
-public abstract class UpdatableSeekableBlock extends UpdatableBlock implements Seekable {
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/compress/BitVector.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/compress/BitVector.java b/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/compress/BitVector.java
deleted file mode 100644
index 15944dd..0000000
--- a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/compress/BitVector.java
+++ /dev/null
@@ -1,44 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package tajo.storage.hcfile.compress;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-
-public class BitVector extends Codec {
-  @Override
-  public byte[] decompress(byte[] compressed) throws IOException {
-    return new byte[0];  //To change body of implemented methods use File | Settings | File Templates.
-  }
-
-  @Override
-  public byte[] decompress(ByteBuffer compressed) throws IOException {
-    return new byte[0];  //To change body of implemented methods use File | Settings | File Templates.
-  }
-
-  @Override
-  public byte[] compress(byte[] decompressed) throws IOException {
-    return new byte[0];  //To change body of implemented methods use File | Settings | File Templates.
-  }
-
-  @Override
-  public byte[] compress(ByteBuffer decompressed) throws IOException {
-    return new byte[0];  //To change body of implemented methods use File | Settings | File Templates.
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/compress/Codec.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/compress/Codec.java b/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/compress/Codec.java
deleted file mode 100644
index 2fe019b..0000000
--- a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/compress/Codec.java
+++ /dev/null
@@ -1,55 +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 tajo.storage.hcfile.compress;
-
-import tajo.catalog.proto.CatalogProtos.CompressType;
-import tajo.storage.exception.UnknownCodecException;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-
-public abstract class Codec {
-
-  public static Codec get(CompressType type) throws UnknownCodecException {
-    switch (type) {
-      case COMP_NONE:
-        return null;
-      case NULL_SUPPRESS:
-        return new NullSuppress();
-      case RUN_LENGTH:
-        return new RunLength();
-      case BIT_VECTOR:
-        return new BitVector();
-      case DICTIONARY:
-        return new Dictionary();
-      case SNAPPY:
-        return new Snappy();
-      case LZ:
-        return new LZ();
-      default:
-        throw new UnknownCodecException(type.name());
-    }
-  }
-
-  public abstract byte[] decompress(byte[] compressed) throws IOException;
-  public abstract byte[] decompress(ByteBuffer compressed) throws IOException;
-
-  public abstract byte[] compress(byte[] decompressed) throws IOException;
-  public abstract byte[] compress(ByteBuffer decompressed) throws IOException;
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/compress/Dictionary.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/compress/Dictionary.java b/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/compress/Dictionary.java
deleted file mode 100644
index 681b855..0000000
--- a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/compress/Dictionary.java
+++ /dev/null
@@ -1,44 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package tajo.storage.hcfile.compress;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-
-public class Dictionary extends Codec {
-  @Override
-  public byte[] decompress(byte[] compressed) throws IOException {
-    return new byte[0];  //To change body of implemented methods use File | Settings | File Templates.
-  }
-
-  @Override
-  public byte[] decompress(ByteBuffer compressed) throws IOException {
-    return new byte[0];  //To change body of implemented methods use File | Settings | File Templates.
-  }
-
-  @Override
-  public byte[] compress(byte[] decompressed) throws IOException {
-    return new byte[0];  //To change body of implemented methods use File | Settings | File Templates.
-  }
-
-  @Override
-  public byte[] compress(ByteBuffer decompressed) throws IOException {
-    return new byte[0];  //To change body of implemented methods use File | Settings | File Templates.
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/compress/LZ.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/compress/LZ.java b/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/compress/LZ.java
deleted file mode 100644
index ad1f9c8..0000000
--- a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/compress/LZ.java
+++ /dev/null
@@ -1,44 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package tajo.storage.hcfile.compress;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-
-public class LZ extends Codec {
-  @Override
-  public byte[] decompress(byte[] compressed) throws IOException {
-    return new byte[0];  //To change body of implemented methods use File | Settings | File Templates.
-  }
-
-  @Override
-  public byte[] decompress(ByteBuffer compressed) throws IOException {
-    return new byte[0];  //To change body of implemented methods use File | Settings | File Templates.
-  }
-
-  @Override
-  public byte[] compress(byte[] decompressed) throws IOException {
-    return new byte[0];  //To change body of implemented methods use File | Settings | File Templates.
-  }
-
-  @Override
-  public byte[] compress(ByteBuffer decompressed) throws IOException {
-    return new byte[0];  //To change body of implemented methods use File | Settings | File Templates.
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/compress/NullSuppress.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/compress/NullSuppress.java b/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/compress/NullSuppress.java
deleted file mode 100644
index 18dc09f..0000000
--- a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/compress/NullSuppress.java
+++ /dev/null
@@ -1,44 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package tajo.storage.hcfile.compress;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-
-public class NullSuppress extends Codec {
-  @Override
-  public byte[] decompress(byte[] compressed) throws IOException {
-    return new byte[0];  //To change body of implemented methods use File | Settings | File Templates.
-  }
-
-  @Override
-  public byte[] decompress(ByteBuffer compressed) throws IOException {
-    return new byte[0];  //To change body of implemented methods use File | Settings | File Templates.
-  }
-
-  @Override
-  public byte[] compress(byte[] decompressed) throws IOException {
-    return new byte[0];  //To change body of implemented methods use File | Settings | File Templates.
-  }
-
-  @Override
-  public byte[] compress(ByteBuffer decompressed) throws IOException {
-    return new byte[0];  //To change body of implemented methods use File | Settings | File Templates.
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/compress/RunLength.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/compress/RunLength.java b/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/compress/RunLength.java
deleted file mode 100644
index 9402181..0000000
--- a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/compress/RunLength.java
+++ /dev/null
@@ -1,44 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package tajo.storage.hcfile.compress;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-
-public class RunLength extends Codec {
-  @Override
-  public byte[] decompress(byte[] compressed) throws IOException {
-    return new byte[0];  //To change body of implemented methods use File | Settings | File Templates.
-  }
-
-  @Override
-  public byte[] decompress(ByteBuffer compressed) throws IOException {
-    return new byte[0];  //To change body of implemented methods use File | Settings | File Templates.
-  }
-
-  @Override
-  public byte[] compress(byte[] decompressed) throws IOException {
-    return new byte[0];  //To change body of implemented methods use File | Settings | File Templates.
-  }
-
-  @Override
-  public byte[] compress(ByteBuffer decompressed) throws IOException {
-    return new byte[0];  //To change body of implemented methods use File | Settings | File Templates.
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/compress/Snappy.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/compress/Snappy.java b/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/compress/Snappy.java
deleted file mode 100644
index 33cd64b..0000000
--- a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/compress/Snappy.java
+++ /dev/null
@@ -1,44 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package tajo.storage.hcfile.compress;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-
-public class Snappy extends Codec {
-  @Override
-  public byte[] decompress(byte[] compressed) throws IOException {
-    return new byte[0];  //To change body of implemented methods use File | Settings | File Templates.
-  }
-
-  @Override
-  public byte[] decompress(ByteBuffer compressed) throws IOException {
-    return new byte[0];  //To change body of implemented methods use File | Settings | File Templates.
-  }
-
-  @Override
-  public byte[] compress(byte[] decompressed) throws IOException {
-    return new byte[0];  //To change body of implemented methods use File | Settings | File Templates.
-  }
-
-  @Override
-  public byte[] compress(ByteBuffer decompressed) throws IOException {
-    return new byte[0];  //To change body of implemented methods use File | Settings | File Templates.
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/reader/ArrayReader.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/reader/ArrayReader.java b/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/reader/ArrayReader.java
deleted file mode 100644
index a7215b0..0000000
--- a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/reader/ArrayReader.java
+++ /dev/null
@@ -1,44 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package tajo.storage.hcfile.reader;
-
-import tajo.datum.ArrayDatum;
-import tajo.datum.Datum;
-import tajo.datum.json.GsonCreator;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-
-public class ArrayReader extends TypeReader {
-
-  @Override
-  public Datum read(ByteBuffer buffer) throws IOException {
-    if (buffer.hasRemaining()) {
-      int arrayByteSize = buffer.getInt();
-      byte [] arrayBytes = new byte[arrayByteSize];
-      buffer.get(arrayBytes);
-      String json = new String(arrayBytes);
-      ArrayDatum array = (ArrayDatum) GsonCreator
-          .getInstance().fromJson(json, Datum.class);
-      return array;
-    } else {
-      return null;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/reader/ByteReader.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/reader/ByteReader.java b/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/reader/ByteReader.java
deleted file mode 100644
index 3a37fd7..0000000
--- a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/reader/ByteReader.java
+++ /dev/null
@@ -1,37 +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 tajo.storage.hcfile.reader;
-
-import tajo.datum.Datum;
-import tajo.datum.DatumFactory;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-
-public class ByteReader extends TypeReader {
-
-  @Override
-  public Datum read(ByteBuffer buffer) throws IOException {
-    if (buffer.hasRemaining()) {
-      return DatumFactory.createByte(buffer.get());
-    } else {
-      return null;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/reader/BytesReader.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/reader/BytesReader.java b/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/reader/BytesReader.java
deleted file mode 100644
index 6a22904..0000000
--- a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/reader/BytesReader.java
+++ /dev/null
@@ -1,40 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package tajo.storage.hcfile.reader;
-
-import tajo.datum.Datum;
-import tajo.datum.DatumFactory;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-
-public class BytesReader extends TypeReader {
-
-  @Override
-  public Datum read(ByteBuffer buffer) throws IOException {
-    if (buffer.hasRemaining()) {
-      int size = buffer.getInt();
-      byte[] rawBytes = new byte[size];
-      buffer.get(rawBytes);
-      return DatumFactory.createBytes(rawBytes);
-    } else {
-      return null;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/reader/CharReader.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/reader/CharReader.java b/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/reader/CharReader.java
deleted file mode 100644
index 19dd59c..0000000
--- a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/reader/CharReader.java
+++ /dev/null
@@ -1,37 +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 tajo.storage.hcfile.reader;
-
-import tajo.datum.Datum;
-import tajo.datum.DatumFactory;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-
-public class CharReader extends TypeReader {
-
-  @Override
-  public Datum read(ByteBuffer buffer) throws IOException {
-    if (buffer.hasRemaining()) {
-      return DatumFactory.createChar(buffer.get());
-    } else {
-      return null;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/reader/DoubleReader.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/reader/DoubleReader.java b/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/reader/DoubleReader.java
deleted file mode 100644
index 021a42d..0000000
--- a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/reader/DoubleReader.java
+++ /dev/null
@@ -1,37 +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 tajo.storage.hcfile.reader;
-
-import tajo.datum.Datum;
-import tajo.datum.DatumFactory;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-
-public class DoubleReader extends TypeReader {
-
-  @Override
-  public Datum read(ByteBuffer buffer) throws IOException {
-    if (buffer.hasRemaining()) {
-      return DatumFactory.createDouble(buffer.getDouble());
-    } else {
-      return null;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/reader/FloatReader.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/reader/FloatReader.java b/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/reader/FloatReader.java
deleted file mode 100644
index e5f0e03..0000000
--- a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/reader/FloatReader.java
+++ /dev/null
@@ -1,37 +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 tajo.storage.hcfile.reader;
-
-import tajo.datum.Datum;
-import tajo.datum.DatumFactory;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-
-public class FloatReader extends TypeReader {
-
-  @Override
-  public Datum read(ByteBuffer buffer) throws IOException {
-    if (buffer.hasRemaining()) {
-      return DatumFactory.createFloat(buffer.getFloat());
-    } else {
-      return null;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/reader/IPv4Reader.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/reader/IPv4Reader.java b/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/reader/IPv4Reader.java
deleted file mode 100644
index d9094c7..0000000
--- a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/reader/IPv4Reader.java
+++ /dev/null
@@ -1,40 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package tajo.storage.hcfile.reader;
-
-import tajo.datum.Datum;
-import tajo.datum.DatumFactory;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-
-public class IPv4Reader extends TypeReader {
-
-  @Override
-  public Datum read(ByteBuffer buffer) throws IOException {
-    if (buffer.hasRemaining()) {
-      int size = buffer.getInt();
-      byte[] rawBytes = new byte[size];
-      buffer.get(rawBytes);
-      return DatumFactory.createIPv4(rawBytes);
-    } else {
-      return null;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/reader/IntReader.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/reader/IntReader.java b/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/reader/IntReader.java
deleted file mode 100644
index fa6ad62..0000000
--- a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/reader/IntReader.java
+++ /dev/null
@@ -1,37 +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 tajo.storage.hcfile.reader;
-
-import tajo.datum.Datum;
-import tajo.datum.DatumFactory;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-
-public class IntReader extends TypeReader {
-
-  @Override
-  public Datum read(ByteBuffer buffer) throws IOException {
-    if (buffer.hasRemaining()) {
-      return DatumFactory.createInt(buffer.getInt());
-    } else {
-      return null;
-    }
-  }
-}


[05/11] TAJO-57: Recognize Parser and Catalog Standard SQL data types. (hyunsik)

Posted by hy...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-backend/src/test/java/tajo/engine/planner/physical/TestBNLJoinExec.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/tajo/engine/planner/physical/TestBNLJoinExec.java b/tajo-core/tajo-core-backend/src/test/java/tajo/engine/planner/physical/TestBNLJoinExec.java
index ea74a8c..8ade9cf 100644
--- a/tajo-core/tajo-core-backend/src/test/java/tajo/engine/planner/physical/TestBNLJoinExec.java
+++ b/tajo-core/tajo-core-backend/src/test/java/tajo/engine/planner/physical/TestBNLJoinExec.java
@@ -25,8 +25,8 @@ import org.junit.Test;
 import tajo.TajoTestingCluster;
 import tajo.TaskAttemptContext;
 import tajo.catalog.*;
-import tajo.catalog.proto.CatalogProtos.DataType;
 import tajo.catalog.proto.CatalogProtos.StoreType;
+import tajo.common.TajoDataTypes.Type;
 import tajo.conf.TajoConf;
 import tajo.datum.Datum;
 import tajo.datum.DatumFactory;
@@ -71,48 +71,48 @@ public class TestBNLJoinExec {
     sm = StorageManager.get(conf, testDir);
 
     Schema schema = new Schema();
-    schema.addColumn("managerId", DataType.INT);
-    schema.addColumn("empId", DataType.INT);
-    schema.addColumn("memId", DataType.INT);
-    schema.addColumn("deptName", DataType.STRING);
+    schema.addColumn("managerId", Type.INT4);
+    schema.addColumn("empId", Type.INT4);
+    schema.addColumn("memId", Type.INT4);
+    schema.addColumn("deptName", Type.TEXT);
 
-    TableMeta employeeMeta = TCatUtil.newTableMeta(schema, StoreType.CSV);
+    TableMeta employeeMeta = CatalogUtil.newTableMeta(schema, StoreType.CSV);
     Path employeePath = new Path(testDir, "employee.csv");
     Appender appender = StorageManager.getAppender(conf, employeeMeta, employeePath);
     appender.init();
     Tuple tuple = new VTuple(employeeMeta.getSchema().getColumnNum());
     for (int i = 0; i < OUTER_TUPLE_NUM; i++) {
-      tuple.put(new Datum[] { DatumFactory.createInt(i),
-          DatumFactory.createInt(i), DatumFactory.createInt(10 + i),
-          DatumFactory.createString("dept_" + i) });
+      tuple.put(new Datum[] { DatumFactory.createInt4(i),
+          DatumFactory.createInt4(i), DatumFactory.createInt4(10 + i),
+          DatumFactory.createText("dept_" + i) });
       appender.addTuple(tuple);
     }
     appender.flush();
     appender.close();
-    employee = TCatUtil.newTableDesc("employee", employeeMeta, employeePath);
+    employee = CatalogUtil.newTableDesc("employee", employeeMeta, employeePath);
     catalog.addTable(employee);
 
     Schema peopleSchema = new Schema();
-    peopleSchema.addColumn("empId", DataType.INT);
-    peopleSchema.addColumn("fk_memId", DataType.INT);
-    peopleSchema.addColumn("name", DataType.STRING);
-    peopleSchema.addColumn("age", DataType.INT);
-    TableMeta peopleMeta = TCatUtil.newTableMeta(peopleSchema, StoreType.CSV);
+    peopleSchema.addColumn("empId", Type.INT4);
+    peopleSchema.addColumn("fk_memId", Type.INT4);
+    peopleSchema.addColumn("name", Type.TEXT);
+    peopleSchema.addColumn("age", Type.INT4);
+    TableMeta peopleMeta = CatalogUtil.newTableMeta(peopleSchema, StoreType.CSV);
     Path peoplePath = new Path(testDir, "people.csv");
     appender = StorageManager.getAppender(conf, peopleMeta, peoplePath);
     appender.init();
     tuple = new VTuple(peopleMeta.getSchema().getColumnNum());
     for (int i = 1; i < INNER_TUPLE_NUM; i += 2) {
-      tuple.put(new Datum[] { DatumFactory.createInt(i),
-          DatumFactory.createInt(10 + i),
-          DatumFactory.createString("name_" + i),
-          DatumFactory.createInt(30 + i) });
+      tuple.put(new Datum[] { DatumFactory.createInt4(i),
+          DatumFactory.createInt4(10 + i),
+          DatumFactory.createText("name_" + i),
+          DatumFactory.createInt4(30 + i) });
       appender.addTuple(tuple);
     }
     appender.flush();
     appender.close();
 
-    people = TCatUtil.newTableDesc("people", peopleMeta, peoplePath);
+    people = CatalogUtil.newTableDesc("people", peopleMeta, peoplePath);
     catalog.addTable(people);
     analyzer = new QueryAnalyzer(catalog);
     planner = new LogicalPlanner(catalog);
@@ -130,9 +130,9 @@ public class TestBNLJoinExec {
 
   @Test
   public final void testCrossJoin() throws IOException {
-    Fragment[] empFrags = sm.splitNG(conf, "employee", employee.getMeta(), employee.getPath(),
+    Fragment[] empFrags = StorageManager.splitNG(conf, "employee", employee.getMeta(), employee.getPath(),
         Integer.MAX_VALUE);
-    Fragment[] peopleFrags = sm.splitNG(conf, "people", people.getMeta(), people.getPath(),
+    Fragment[] peopleFrags = StorageManager.splitNG(conf, "people", people.getMeta(), people.getPath(),
         Integer.MAX_VALUE);
 
     Fragment[] merged = TUtil.concat(empFrags, peopleFrags);
@@ -167,9 +167,9 @@ public class TestBNLJoinExec {
 
   @Test
   public final void testInnerJoin() throws IOException {
-    Fragment[] empFrags = sm.splitNG(conf, "employee", employee.getMeta(), employee.getPath(),
+    Fragment[] empFrags = StorageManager.splitNG(conf, "employee", employee.getMeta(), employee.getPath(),
         Integer.MAX_VALUE);
-    Fragment[] peopleFrags = sm.splitNG(conf, "people", people.getMeta(), people.getPath(),
+    Fragment[] peopleFrags = StorageManager.splitNG(conf, "people", people.getMeta(), people.getPath(),
         Integer.MAX_VALUE);
 
     Fragment[] merged = TUtil.concat(empFrags, peopleFrags);
@@ -213,10 +213,10 @@ public class TestBNLJoinExec {
     exec.init();
     while ((tuple = exec.next()) != null) {
       count++;
-      assertTrue(i == tuple.getInt(0).asInt());
-      assertTrue(i == tuple.getInt(1).asInt());
+      assertTrue(i == tuple.getInt(0).asInt4());
+      assertTrue(i == tuple.getInt(1).asInt4());
       assertTrue(("dept_" + i).equals(tuple.getString(2).asChars()));
-      assertTrue(10 + i == tuple.getInt(3).asInt());
+      assertTrue(10 + i == tuple.getInt(3).asInt4());
       i += 2;
     }
     exec.close();

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-backend/src/test/java/tajo/engine/planner/physical/TestBSTIndexExec.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/tajo/engine/planner/physical/TestBSTIndexExec.java b/tajo-core/tajo-core-backend/src/test/java/tajo/engine/planner/physical/TestBSTIndexExec.java
index cff7f7f..96009f3 100644
--- a/tajo-core/tajo-core-backend/src/test/java/tajo/engine/planner/physical/TestBSTIndexExec.java
+++ b/tajo-core/tajo-core-backend/src/test/java/tajo/engine/planner/physical/TestBSTIndexExec.java
@@ -27,8 +27,8 @@ import org.junit.Test;
 import tajo.TajoTestingCluster;
 import tajo.TaskAttemptContext;
 import tajo.catalog.*;
-import tajo.catalog.proto.CatalogProtos.DataType;
 import tajo.catalog.proto.CatalogProtos.StoreType;
+import tajo.common.TajoDataTypes.Type;
 import tajo.conf.TajoConf;
 import tajo.datum.Datum;
 import tajo.datum.DatumFactory;
@@ -85,12 +85,12 @@ public class TestBSTIndexExec {
     idxPath = new Path(workDir, "test.idx");
 
     Schema schema = new Schema();
-    schema.addColumn("managerId", DataType.INT);
-    schema.addColumn("empId", DataType.INT);
-    schema.addColumn("deptName", DataType.STRING);
+    schema.addColumn("managerId", Type.INT4);
+    schema.addColumn("empId", Type.INT4);
+    schema.addColumn("deptName", Type.TEXT);
 
     this.idxSchema = new Schema();
-    idxSchema.addColumn("managerId", DataType.INT);
+    idxSchema.addColumn("managerId", Type.INT4);
     SortSpec[] sortKeys = new SortSpec[1];
     sortKeys[0] = new SortSpec(idxSchema.getColumn("managerId"), true, false);
     this.comp = new TupleComparator(idxSchema, sortKeys);
@@ -101,7 +101,7 @@ public class TestBSTIndexExec {
     writer.open();
     long offset;
 
-    meta = TCatUtil.newTableMeta(schema, StoreType.CSV);
+    meta = CatalogUtil.newTableMeta(schema, StoreType.CSV);
     tablePath = StorageUtil.concatPath(workDir, "employee", "table.csv");
     fs = tablePath.getFileSystem(conf);
     fs.mkdirs(tablePath.getParent());
@@ -120,10 +120,10 @@ public class TestBSTIndexExec {
         this.randomValues.put(rndKey, 1);
       }
       
-      key.put(new Datum[] { DatumFactory.createInt(rndKey) });
-      tuple.put(new Datum[] { DatumFactory.createInt(rndKey),
-          DatumFactory.createInt(rnd.nextInt(10)),
-          DatumFactory.createString("dept_" + rnd.nextInt(10)) });
+      key.put(new Datum[] { DatumFactory.createInt4(rndKey) });
+      tuple.put(new Datum[] { DatumFactory.createInt4(rndKey),
+          DatumFactory.createInt4(rnd.nextInt(10)),
+          DatumFactory.createText("dept_" + rnd.nextInt(10)) });
       offset = appender.getOffset();
       appender.addTuple(tuple);
       writer.write(key, offset);
@@ -151,7 +151,7 @@ public class TestBSTIndexExec {
     this.rndKey = rnd.nextInt(250);
     final String QUERY = "select * from employee where managerId = " + rndKey;
     
-    Fragment[] frags = sm.splitNG(conf, "employee", meta, tablePath, Integer.MAX_VALUE);
+    Fragment[] frags = StorageManager.splitNG(conf, "employee", meta, tablePath, Integer.MAX_VALUE);
     Path workDir = CommonTestingUtil.getTestDir("target/test-data/testEqual");
     TaskAttemptContext ctx = new TaskAttemptContext(conf,
         TUtil.newQueryUnitAttemptId(), new Fragment[] { frags[0] }, workDir);
@@ -190,7 +190,7 @@ public class TestBSTIndexExec {
 
       Fragment[] fragments = ctx.getTables(scanNode.getTableId());
       
-      Datum[] datum = new Datum[]{DatumFactory.createInt(rndKey)};
+      Datum[] datum = new Datum[]{DatumFactory.createInt4(rndKey)};
 
       return new BSTIndexScanExec(ctx, sm, scanNode, fragments[0], idxPath,
           idxSchema, comp , datum);

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-backend/src/test/java/tajo/engine/planner/physical/TestExternalSortExec.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/tajo/engine/planner/physical/TestExternalSortExec.java b/tajo-core/tajo-core-backend/src/test/java/tajo/engine/planner/physical/TestExternalSortExec.java
index b43c80a..1e5eca1 100644
--- a/tajo-core/tajo-core-backend/src/test/java/tajo/engine/planner/physical/TestExternalSortExec.java
+++ b/tajo-core/tajo-core-backend/src/test/java/tajo/engine/planner/physical/TestExternalSortExec.java
@@ -25,8 +25,8 @@ import org.junit.Test;
 import tajo.TajoTestingCluster;
 import tajo.TaskAttemptContext;
 import tajo.catalog.*;
-import tajo.catalog.proto.CatalogProtos.DataType;
 import tajo.catalog.proto.CatalogProtos.StoreType;
+import tajo.common.TajoDataTypes.Type;
 import tajo.conf.TajoConf;
 import tajo.datum.Datum;
 import tajo.datum.DatumFactory;
@@ -72,20 +72,20 @@ public class TestExternalSortExec {
     sm = StorageManager.get(conf, testDir);
 
     Schema schema = new Schema();
-    schema.addColumn("managerId", DataType.INT);
-    schema.addColumn("empId", DataType.INT);
-    schema.addColumn("deptName", DataType.STRING);
+    schema.addColumn("managerId", Type.INT4);
+    schema.addColumn("empId", Type.INT4);
+    schema.addColumn("deptName", Type.TEXT);
 
-    TableMeta employeeMeta = TCatUtil.newTableMeta(schema, StoreType.CSV);
+    TableMeta employeeMeta = CatalogUtil.newTableMeta(schema, StoreType.CSV);
     Path employeePath = new Path(testDir, "employee.csv");
     Appender appender = StorageManager.getAppender(conf, employeeMeta, employeePath);
     appender.enableStats();
     appender.init();
     Tuple tuple = new VTuple(employeeMeta.getSchema().getColumnNum());
     for (int i = 0; i < numTuple; i++) {
-      tuple.put(new Datum[] { DatumFactory.createInt(rnd.nextInt(50)),
-          DatumFactory.createInt(rnd.nextInt(100)),
-          DatumFactory.createString("dept_" + 123) });
+      tuple.put(new Datum[] { DatumFactory.createInt4(rnd.nextInt(50)),
+          DatumFactory.createInt4(rnd.nextInt(100)),
+          DatumFactory.createText("dept_" + 123) });
       appender.addTuple(tuple);
     }
     appender.flush();

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-backend/src/test/java/tajo/engine/planner/physical/TestHashJoinExec.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/tajo/engine/planner/physical/TestHashJoinExec.java b/tajo-core/tajo-core-backend/src/test/java/tajo/engine/planner/physical/TestHashJoinExec.java
index 47f743c..0b8612e 100644
--- a/tajo-core/tajo-core-backend/src/test/java/tajo/engine/planner/physical/TestHashJoinExec.java
+++ b/tajo-core/tajo-core-backend/src/test/java/tajo/engine/planner/physical/TestHashJoinExec.java
@@ -25,8 +25,8 @@ import org.junit.Test;
 import tajo.TajoTestingCluster;
 import tajo.TaskAttemptContext;
 import tajo.catalog.*;
-import tajo.catalog.proto.CatalogProtos.DataType;
 import tajo.catalog.proto.CatalogProtos.StoreType;
+import tajo.common.TajoDataTypes.Type;
 import tajo.conf.TajoConf;
 import tajo.datum.Datum;
 import tajo.datum.DatumFactory;
@@ -68,51 +68,51 @@ public class TestHashJoinExec {
     sm = StorageManager.get(conf, testDir);
 
     Schema employeeSchema = new Schema();
-    employeeSchema.addColumn("managerId", DataType.INT);
-    employeeSchema.addColumn("empId", DataType.INT);
-    employeeSchema.addColumn("memId", DataType.INT);
-    employeeSchema.addColumn("deptName", DataType.STRING);
+    employeeSchema.addColumn("managerId", Type.INT4);
+    employeeSchema.addColumn("empId", Type.INT4);
+    employeeSchema.addColumn("memId", Type.INT4);
+    employeeSchema.addColumn("deptName", Type.TEXT);
 
-    TableMeta employeeMeta = TCatUtil.newTableMeta(employeeSchema,
+    TableMeta employeeMeta = CatalogUtil.newTableMeta(employeeSchema,
         StoreType.CSV);
     Path employeePath = new Path(testDir, "employee.csv");
     Appender appender = StorageManager.getAppender(conf, employeeMeta, employeePath);
     appender.init();
     Tuple tuple = new VTuple(employeeMeta.getSchema().getColumnNum());
     for (int i = 0; i < 10; i++) {
-      tuple.put(new Datum[] { DatumFactory.createInt(i),
-          DatumFactory.createInt(i), DatumFactory.createInt(10 + i),
-          DatumFactory.createString("dept_" + i) });
+      tuple.put(new Datum[] { DatumFactory.createInt4(i),
+          DatumFactory.createInt4(i), DatumFactory.createInt4(10 + i),
+          DatumFactory.createText("dept_" + i) });
       appender.addTuple(tuple);
     }
 
     appender.flush();
     appender.close();
-    employee = TCatUtil.newTableDesc("employee", employeeMeta, employeePath);
+    employee = CatalogUtil.newTableDesc("employee", employeeMeta, employeePath);
     catalog.addTable(employee);
 
     Schema peopleSchema = new Schema();
-    peopleSchema.addColumn("empId", DataType.INT);
-    peopleSchema.addColumn("fk_memId", DataType.INT);
-    peopleSchema.addColumn("name", DataType.STRING);
-    peopleSchema.addColumn("age", DataType.INT);
-    TableMeta peopleMeta = TCatUtil.newTableMeta(peopleSchema, StoreType.CSV);
+    peopleSchema.addColumn("empId", Type.INT4);
+    peopleSchema.addColumn("fk_memId", Type.INT4);
+    peopleSchema.addColumn("name", Type.TEXT);
+    peopleSchema.addColumn("age", Type.INT4);
+    TableMeta peopleMeta = CatalogUtil.newTableMeta(peopleSchema, StoreType.CSV);
     Path peoplePath = new Path(testDir, "people.csv");
     appender = StorageManager.getAppender(conf, peopleMeta, peoplePath);
     appender.init();
     tuple = new VTuple(peopleMeta.getSchema().getColumnNum());
     for (int i = 1; i < 10; i += 2) {
-      tuple.put(new Datum[] { DatumFactory.createInt(i),
-          DatumFactory.createInt(10 + i),
-          DatumFactory.createString("name_" + i),
-          DatumFactory.createInt(30 + i) });
+      tuple.put(new Datum[] { DatumFactory.createInt4(i),
+          DatumFactory.createInt4(10 + i),
+          DatumFactory.createText("name_" + i),
+          DatumFactory.createInt4(30 + i) });
       appender.addTuple(tuple);
     }
 
     appender.flush();
     appender.close();
 
-    people = TCatUtil.newTableDesc("people", peopleMeta, peoplePath);
+    people = CatalogUtil.newTableDesc("people", peopleMeta, peoplePath);
     catalog.addTable(people);
     analyzer = new QueryAnalyzer(catalog);
     planner = new LogicalPlanner(catalog);
@@ -130,9 +130,9 @@ public class TestHashJoinExec {
 
   @Test
   public final void testInnerJoin() throws IOException {
-    Fragment[] empFrags = sm.splitNG(conf, "employee", employee.getMeta(), employee.getPath(),
+    Fragment[] empFrags = StorageManager.splitNG(conf, "employee", employee.getMeta(), employee.getPath(),
         Integer.MAX_VALUE);
-    Fragment[] peopleFrags = sm.splitNG(conf, "people", people.getMeta(), people.getPath(),
+    Fragment[] peopleFrags = StorageManager.splitNG(conf, "people", people.getMeta(), people.getPath(),
         Integer.MAX_VALUE);
 
     Fragment[] merged = TUtil.concat(empFrags, peopleFrags);
@@ -166,10 +166,10 @@ public class TestHashJoinExec {
     exec.init();
     while ((tuple = exec.next()) != null) {
       count++;
-      assertTrue(i == tuple.getInt(0).asInt());
-      assertTrue(i == tuple.getInt(1).asInt());
+      assertTrue(i == tuple.getInt(0).asInt4());
+      assertTrue(i == tuple.getInt(1).asInt4());
       assertTrue(("dept_" + i).equals(tuple.getString(2).asChars()));
-      assertTrue(10 + i == tuple.getInt(3).asInt());
+      assertTrue(10 + i == tuple.getInt(3).asInt4());
 
       i += 2;
     }

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-backend/src/test/java/tajo/engine/planner/physical/TestHashPartitioner.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/tajo/engine/planner/physical/TestHashPartitioner.java b/tajo-core/tajo-core-backend/src/test/java/tajo/engine/planner/physical/TestHashPartitioner.java
index dd2f53b..183c499 100644
--- a/tajo-core/tajo-core-backend/src/test/java/tajo/engine/planner/physical/TestHashPartitioner.java
+++ b/tajo-core/tajo-core-backend/src/test/java/tajo/engine/planner/physical/TestHashPartitioner.java
@@ -42,33 +42,33 @@ public class TestHashPartitioner {
   public final void testGetPartition() {   
     Tuple tuple1 = new VTuple(3);    
     tuple1.put(new Datum[] {
-        DatumFactory.createInt(1),
-        DatumFactory.createInt(2),
-        DatumFactory.createInt(3)
+        DatumFactory.createInt4(1),
+        DatumFactory.createInt4(2),
+        DatumFactory.createInt4(3)
     });
     Tuple tuple2 = new VTuple(3);    
     tuple2.put(new Datum[] {
-        DatumFactory.createInt(1),
-        DatumFactory.createInt(2),
-        DatumFactory.createInt(4)
+        DatumFactory.createInt4(1),
+        DatumFactory.createInt4(2),
+        DatumFactory.createInt4(4)
     });
     Tuple tuple3 = new VTuple(3);    
     tuple3.put(new Datum[] {
-        DatumFactory.createInt(1),
-        DatumFactory.createInt(2),
-        DatumFactory.createInt(5)
+        DatumFactory.createInt4(1),
+        DatumFactory.createInt4(2),
+        DatumFactory.createInt4(5)
     });
     Tuple tuple4 = new VTuple(3);    
     tuple4.put(new Datum[] {
-        DatumFactory.createInt(2),
-        DatumFactory.createInt(2),
-        DatumFactory.createInt(3)
+        DatumFactory.createInt4(2),
+        DatumFactory.createInt4(2),
+        DatumFactory.createInt4(3)
     });
     Tuple tuple5 = new VTuple(3);    
     tuple5.put(new Datum[] {
-        DatumFactory.createInt(2),
-        DatumFactory.createInt(2),
-        DatumFactory.createInt(4)
+        DatumFactory.createInt4(2),
+        DatumFactory.createInt4(2),
+        DatumFactory.createInt4(4)
     });
     
     int [] partKeys = {0,1};

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-backend/src/test/java/tajo/engine/planner/physical/TestMergeJoinExec.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/tajo/engine/planner/physical/TestMergeJoinExec.java b/tajo-core/tajo-core-backend/src/test/java/tajo/engine/planner/physical/TestMergeJoinExec.java
index 46d264b..afddea5 100644
--- a/tajo-core/tajo-core-backend/src/test/java/tajo/engine/planner/physical/TestMergeJoinExec.java
+++ b/tajo-core/tajo-core-backend/src/test/java/tajo/engine/planner/physical/TestMergeJoinExec.java
@@ -26,8 +26,8 @@ import org.junit.Test;
 import tajo.TajoTestingCluster;
 import tajo.TaskAttemptContext;
 import tajo.catalog.*;
-import tajo.catalog.proto.CatalogProtos.DataType;
 import tajo.catalog.proto.CatalogProtos.StoreType;
+import tajo.common.TajoDataTypes.Type;
 import tajo.conf.TajoConf;
 import tajo.datum.Datum;
 import tajo.datum.DatumFactory;
@@ -70,65 +70,65 @@ public class TestMergeJoinExec {
     sm = StorageManager.get(conf, testDir);
 
     Schema employeeSchema = new Schema();
-    employeeSchema.addColumn("managerId", DataType.INT);
-    employeeSchema.addColumn("empId", DataType.INT);
-    employeeSchema.addColumn("memId", DataType.INT);
-    employeeSchema.addColumn("deptName", DataType.STRING);
+    employeeSchema.addColumn("managerId", Type.INT4);
+    employeeSchema.addColumn("empId", Type.INT4);
+    employeeSchema.addColumn("memId", Type.INT4);
+    employeeSchema.addColumn("deptName", Type.TEXT);
 
-    TableMeta employeeMeta = TCatUtil.newTableMeta(employeeSchema,
+    TableMeta employeeMeta = CatalogUtil.newTableMeta(employeeSchema,
         StoreType.CSV);
     Path employeePath = new Path(testDir, "employee.csv");
     Appender appender = StorageManager.getAppender(conf, employeeMeta, employeePath);
     appender.init();
     Tuple tuple = new VTuple(employeeMeta.getSchema().getColumnNum());
     for (int i = 0; i < 10; i++) {
-      tuple.put(new Datum[] { DatumFactory.createInt(i),
-          DatumFactory.createInt(i), DatumFactory.createInt(10 + i),
-          DatumFactory.createString("dept_" + i) });
+      tuple.put(new Datum[] { DatumFactory.createInt4(i),
+          DatumFactory.createInt4(i), DatumFactory.createInt4(10 + i),
+          DatumFactory.createText("dept_" + i) });
       appender.addTuple(tuple);
     }
     for (int i = 11; i < 20; i+=2) {
-      tuple.put(new Datum[] { DatumFactory.createInt(i),
-          DatumFactory.createInt(i), DatumFactory.createInt(10 + i),
-          DatumFactory.createString("dept_" + i) });
+      tuple.put(new Datum[] { DatumFactory.createInt4(i),
+          DatumFactory.createInt4(i), DatumFactory.createInt4(10 + i),
+          DatumFactory.createText("dept_" + i) });
       appender.addTuple(tuple);
     }
 
     appender.flush();
     appender.close();
-    employee = TCatUtil.newTableDesc("employee", employeeMeta,
+    employee = CatalogUtil.newTableDesc("employee", employeeMeta,
         employeePath);
     catalog.addTable(employee);
 
     Schema peopleSchema = new Schema();
-    peopleSchema.addColumn("empId", DataType.INT);
-    peopleSchema.addColumn("fk_memId", DataType.INT);
-    peopleSchema.addColumn("name", DataType.STRING);
-    peopleSchema.addColumn("age", DataType.INT);
-    TableMeta peopleMeta = TCatUtil.newTableMeta(peopleSchema, StoreType.CSV);
+    peopleSchema.addColumn("empId", Type.INT4);
+    peopleSchema.addColumn("fk_memId", Type.INT4);
+    peopleSchema.addColumn("name", Type.TEXT);
+    peopleSchema.addColumn("age", Type.INT4);
+    TableMeta peopleMeta = CatalogUtil.newTableMeta(peopleSchema, StoreType.CSV);
     Path peoplePath = new Path(testDir, "people.csv");
     appender = StorageManager.getAppender(conf, peopleMeta, peoplePath);
     appender.init();
     tuple = new VTuple(peopleMeta.getSchema().getColumnNum());
     for (int i = 1; i < 10; i += 2) {
-      tuple.put(new Datum[] { DatumFactory.createInt(i),
-          DatumFactory.createInt(10 + i),
-          DatumFactory.createString("name_" + i),
-          DatumFactory.createInt(30 + i) });
+      tuple.put(new Datum[] { DatumFactory.createInt4(i),
+          DatumFactory.createInt4(10 + i),
+          DatumFactory.createText("name_" + i),
+          DatumFactory.createInt4(30 + i) });
       appender.addTuple(tuple);
     }
     for (int i = 10; i < 20; i++) {
-      tuple.put(new Datum[] { DatumFactory.createInt(i),
-          DatumFactory.createInt(10 + i),
-          DatumFactory.createString("name_" + i),
-          DatumFactory.createInt(30 + i) });
+      tuple.put(new Datum[] { DatumFactory.createInt4(i),
+          DatumFactory.createInt4(10 + i),
+          DatumFactory.createText("name_" + i),
+          DatumFactory.createInt4(30 + i) });
       appender.addTuple(tuple);
     }
 
     appender.flush();
     appender.close();
 
-    people = TCatUtil.newTableDesc("people", peopleMeta, peoplePath);
+    people = CatalogUtil.newTableDesc("people", peopleMeta, peoplePath);
     catalog.addTable(people);
     analyzer = new QueryAnalyzer(catalog);
     planner = new LogicalPlanner(catalog);
@@ -215,10 +215,10 @@ public class TestMergeJoinExec {
     exec.init();
     while ((tuple = exec.next()) != null) {
       count++;
-      assertTrue(i == tuple.getInt(0).asInt());
-      assertTrue(i == tuple.getInt(1).asInt());
+      assertTrue(i == tuple.getInt(0).asInt4());
+      assertTrue(i == tuple.getInt(1).asInt4());
       assertTrue(("dept_" + i).equals(tuple.getString(2).asChars()));
-      assertTrue(10 + i == tuple.getInt(3).asInt());
+      assertTrue(10 + i == tuple.getInt(3).asInt4());
 
       i += 2;
     }

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-backend/src/test/java/tajo/engine/planner/physical/TestNLJoinExec.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/tajo/engine/planner/physical/TestNLJoinExec.java b/tajo-core/tajo-core-backend/src/test/java/tajo/engine/planner/physical/TestNLJoinExec.java
index f8b8877..d7a562f 100644
--- a/tajo-core/tajo-core-backend/src/test/java/tajo/engine/planner/physical/TestNLJoinExec.java
+++ b/tajo-core/tajo-core-backend/src/test/java/tajo/engine/planner/physical/TestNLJoinExec.java
@@ -25,8 +25,8 @@ import org.junit.Test;
 import tajo.TajoTestingCluster;
 import tajo.TaskAttemptContext;
 import tajo.catalog.*;
-import tajo.catalog.proto.CatalogProtos.DataType;
 import tajo.catalog.proto.CatalogProtos.StoreType;
+import tajo.common.TajoDataTypes.Type;
 import tajo.conf.TajoConf;
 import tajo.datum.Datum;
 import tajo.datum.DatumFactory;
@@ -67,52 +67,52 @@ public class TestNLJoinExec {
     sm = StorageManager.get(conf, testDir);
 
     Schema schema = new Schema();
-    schema.addColumn("managerId", DataType.INT);
-    schema.addColumn("empId", DataType.INT);
-    schema.addColumn("memId", DataType.INT);
-    schema.addColumn("deptName", DataType.STRING);
+    schema.addColumn("managerId", Type.INT4);
+    schema.addColumn("empId", Type.INT4);
+    schema.addColumn("memId", Type.INT4);
+    schema.addColumn("deptName", Type.TEXT);
 
-    TableMeta employeeMeta = TCatUtil.newTableMeta(schema, StoreType.CSV);
+    TableMeta employeeMeta = CatalogUtil.newTableMeta(schema, StoreType.CSV);
     Path employeePath = new Path(testDir, "employee.csv");
     Appender appender = StorageManager.getAppender(conf, employeeMeta, employeePath);
     appender.init();
     Tuple tuple = new VTuple(employeeMeta.getSchema().getColumnNum());
     for (int i = 0; i < 50; i++) {
       tuple.put(new Datum[] {
-          DatumFactory.createInt(i),
-          DatumFactory.createInt(i),
-          DatumFactory.createInt(10+i),
-          DatumFactory.createString("dept_" + i)});
+          DatumFactory.createInt4(i),
+          DatumFactory.createInt4(i),
+          DatumFactory.createInt4(10 + i),
+          DatumFactory.createText("dept_" + i)});
       appender.addTuple(tuple);
     }
     appender.flush();
     appender.close();
-    employee = TCatUtil.newTableDesc("employee", employeeMeta,
+    employee = CatalogUtil.newTableDesc("employee", employeeMeta,
         employeePath);
     catalog.addTable(employee);
     
     Schema peopleSchema = new Schema();
-    peopleSchema.addColumn("empId", DataType.INT);
-    peopleSchema.addColumn("fk_memId", DataType.INT);
-    peopleSchema.addColumn("name", DataType.STRING);
-    peopleSchema.addColumn("age", DataType.INT);
-    TableMeta peopleMeta = TCatUtil.newTableMeta(peopleSchema, StoreType.CSV);
+    peopleSchema.addColumn("empId", Type.INT4);
+    peopleSchema.addColumn("fk_memId", Type.INT4);
+    peopleSchema.addColumn("name", Type.TEXT);
+    peopleSchema.addColumn("age", Type.INT4);
+    TableMeta peopleMeta = CatalogUtil.newTableMeta(peopleSchema, StoreType.CSV);
     Path peoplePath = new Path(testDir, "people.csv");
     appender = StorageManager.getAppender(conf, peopleMeta, peoplePath);
     appender.init();
     tuple = new VTuple(peopleMeta.getSchema().getColumnNum());
     for (int i = 1; i < 50; i += 2) {
       tuple.put(new Datum[] {
-          DatumFactory.createInt(i),
-          DatumFactory.createInt(10+i),
-          DatumFactory.createString("name_" + i),
-          DatumFactory.createInt(30 + i)});
+          DatumFactory.createInt4(i),
+          DatumFactory.createInt4(10 + i),
+          DatumFactory.createText("name_" + i),
+          DatumFactory.createInt4(30 + i)});
       appender.addTuple(tuple);
     }
     appender.flush();
     appender.close();
     
-    people = TCatUtil.newTableDesc("people", peopleMeta,
+    people = CatalogUtil.newTableDesc("people", peopleMeta,
         peoplePath);
     catalog.addTable(people);
     analyzer = new QueryAnalyzer(catalog);
@@ -132,9 +132,9 @@ public class TestNLJoinExec {
   
   @Test
   public final void testCrossJoin() throws IOException {
-    Fragment[] empFrags = sm.splitNG(conf, "employee", employee.getMeta(), employee.getPath(),
+    Fragment[] empFrags = StorageManager.splitNG(conf, "employee", employee.getMeta(), employee.getPath(),
         Integer.MAX_VALUE);
-    Fragment[] peopleFrags = sm.splitNG(conf, "people", people.getMeta(), people.getPath(),
+    Fragment[] peopleFrags = StorageManager.splitNG(conf, "people", people.getMeta(), people.getPath(),
         Integer.MAX_VALUE);
     
     Fragment [] merged = TUtil.concat(empFrags, peopleFrags);
@@ -160,9 +160,9 @@ public class TestNLJoinExec {
 
   @Test
   public final void testInnerJoin() throws IOException {
-    Fragment[] empFrags = sm.splitNG(conf, "employee", employee.getMeta(), employee.getPath(),
+    Fragment[] empFrags = StorageManager.splitNG(conf, "employee", employee.getMeta(), employee.getPath(),
         Integer.MAX_VALUE);
-    Fragment[] peopleFrags = sm.splitNG(conf, "people", people.getMeta(), people.getPath(),
+    Fragment[] peopleFrags = StorageManager.splitNG(conf, "people", people.getMeta(), people.getPath(),
         Integer.MAX_VALUE);
     
     Fragment [] merged = TUtil.concat(empFrags, peopleFrags);
@@ -183,10 +183,10 @@ public class TestNLJoinExec {
     exec.init();
     while ((tuple = exec.next()) != null) {
       count++;
-      assertTrue(i == tuple.getInt(0).asInt());
-      assertTrue(i == tuple.getInt(1).asInt());
+      assertTrue(i == tuple.getInt(0).asInt4());
+      assertTrue(i == tuple.getInt(1).asInt4());
       assertTrue(("dept_" + i).equals(tuple.getString(2).asChars()));
-      assertTrue(10 + i == tuple.getInt(3).asInt());
+      assertTrue(10 + i == tuple.getInt(3).asInt4());
       i += 2;
     }
     exec.close();

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-backend/src/test/java/tajo/engine/planner/physical/TestPhysicalPlanner.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/tajo/engine/planner/physical/TestPhysicalPlanner.java b/tajo-core/tajo-core-backend/src/test/java/tajo/engine/planner/physical/TestPhysicalPlanner.java
index 8b5dee5..c54b39a 100644
--- a/tajo-core/tajo-core-backend/src/test/java/tajo/engine/planner/physical/TestPhysicalPlanner.java
+++ b/tajo-core/tajo-core-backend/src/test/java/tajo/engine/planner/physical/TestPhysicalPlanner.java
@@ -34,8 +34,8 @@ import tajo.QueryUnitAttemptId;
 import tajo.TajoTestingCluster;
 import tajo.TaskAttemptContext;
 import tajo.catalog.*;
-import tajo.catalog.proto.CatalogProtos.DataType;
 import tajo.catalog.proto.CatalogProtos.StoreType;
+import tajo.common.TajoDataTypes.Type;
 import tajo.conf.TajoConf;
 import tajo.datum.Datum;
 import tajo.datum.DatumFactory;
@@ -92,21 +92,21 @@ public class TestPhysicalPlanner {
     }
 
     Schema schema = new Schema();
-    schema.addColumn("name", DataType.STRING);
-    schema.addColumn("empId", DataType.INT);
-    schema.addColumn("deptName", DataType.STRING);
+    schema.addColumn("name", Type.TEXT);
+    schema.addColumn("empId", Type.INT4);
+    schema.addColumn("deptName", Type.TEXT);
 
     Schema schema2 = new Schema();
-    schema2.addColumn("deptName", DataType.STRING);
-    schema2.addColumn("manager", DataType.STRING);
+    schema2.addColumn("deptName", Type.TEXT);
+    schema2.addColumn("manager", Type.TEXT);
 
     Schema scoreSchema = new Schema();
-    scoreSchema.addColumn("deptName", DataType.STRING);
-    scoreSchema.addColumn("class", DataType.STRING);
-    scoreSchema.addColumn("score", DataType.INT);
-    scoreSchema.addColumn("nullable", DataType.STRING);
+    scoreSchema.addColumn("deptName", Type.TEXT);
+    scoreSchema.addColumn("class", Type.TEXT);
+    scoreSchema.addColumn("score", Type.INT4);
+    scoreSchema.addColumn("nullable", Type.TEXT);
 
-    TableMeta employeeMeta = TCatUtil.newTableMeta(schema, StoreType.CSV);
+    TableMeta employeeMeta = CatalogUtil.newTableMeta(schema, StoreType.CSV);
 
 
     Path employeePath = new Path(testDir, "employee.csv");
@@ -114,8 +114,8 @@ public class TestPhysicalPlanner {
     appender.init();
     Tuple tuple = new VTuple(employeeMeta.getSchema().getColumnNum());
     for (int i = 0; i < 100; i++) {
-      tuple.put(new Datum[] {DatumFactory.createString("name_" + i),
-          DatumFactory.createInt(i), DatumFactory.createString("dept_" + i)});
+      tuple.put(new Datum[] {DatumFactory.createText("name_" + i),
+          DatumFactory.createInt4(i), DatumFactory.createText("dept_" + i)});
       appender.addTuple(tuple);
     }
     appender.flush();
@@ -125,7 +125,7 @@ public class TestPhysicalPlanner {
     catalog.addTable(employee);
 
     Path scorePath = new Path(testDir, "score");
-    TableMeta scoreMeta = TCatUtil.newTableMeta(scoreSchema, StoreType.CSV, new Options());
+    TableMeta scoreMeta = CatalogUtil.newTableMeta(scoreSchema, StoreType.CSV, new Options());
     appender = StorageManager.getAppender(conf, scoreMeta, scorePath);
     appender.init();
     score = new TableDescImpl("score", scoreMeta, scorePath);
@@ -136,10 +136,10 @@ public class TestPhysicalPlanner {
         for (int j = 1; j <= 3; j++) {
           tuple.put(
               new Datum[] {
-                  DatumFactory.createString("name_" + i), // name_1 ~ 5 (cad: // 5)
-                  DatumFactory.createString(k + "rd"), // 3 or 4rd (cad: 2)
-                  DatumFactory.createInt(j), // 1 ~ 3
-              m % 3 == 1 ? DatumFactory.createString("one") : NullDatum.get()});
+                  DatumFactory.createText("name_" + i), // name_1 ~ 5 (cad: // 5)
+                  DatumFactory.createText(k + "rd"), // 3 or 4rd (cad: 2)
+                  DatumFactory.createInt4(j), // 1 ~ 3
+              m % 3 == 1 ? DatumFactory.createText("one") : NullDatum.get()});
           appender.addTuple(tuple);
           m++;
         }
@@ -170,7 +170,7 @@ public class TestPhysicalPlanner {
       "select count(deptName) from score", // 9
       "select managerId, empId, deptName from employee order by managerId, empId desc", // 10
       "select deptName, nullable from score group by deptName, nullable", // 11
-      "select 3 < 4 as ineq, 3.5 * 2 as real", // 12
+      "select 3 < 4 as ineq, 3.5 * 2 as score", // 12
 //      "select (3 > 2) = (1 > 0) and 3 > 1", // 12
       "select (1 > 0) and 3 > 1", // 13
       "select deptName, class, sum(score), max(score), min(score) from score", // 14
@@ -224,9 +224,9 @@ public class TestPhysicalPlanner {
     Tuple tuple;
     exec.init();
     while ((tuple = exec.next()) != null) {
-      assertEquals(6, tuple.get(2).asInt()); // sum
-      assertEquals(3, tuple.get(3).asInt()); // max
-      assertEquals(1, tuple.get(4).asInt()); // min
+      assertEquals(6, tuple.get(2).asInt4()); // sum
+      assertEquals(3, tuple.get(3).asInt4()); // max
+      assertEquals(1, tuple.get(4).asInt4()); // min
       i++;
     }
     exec.close();
@@ -254,9 +254,9 @@ public class TestPhysicalPlanner {
     exec.init();
     while ((tuple = exec.next()) != null) {
       assertEquals(DatumFactory.createNullDatum(), tuple.get(1));
-      assertEquals(12, tuple.get(2).asInt()); // sum
-      assertEquals(3, tuple.get(3).asInt()); // max
-      assertEquals(1, tuple.get(4).asInt()); // min
+      assertEquals(12, tuple.get(2).asInt4()); // sum
+      assertEquals(3, tuple.get(3).asInt4()); // max
+      assertEquals(1, tuple.get(4).asInt4()); // min
       i++;
     }
     exec.close();
@@ -296,9 +296,9 @@ public class TestPhysicalPlanner {
     Tuple tuple;
     exec.init();
     while ((tuple = exec.next()) != null) {
-      assertEquals(6, tuple.get(2).asInt()); // sum
-      assertEquals(3, tuple.get(3).asInt()); // max
-      assertEquals(1, tuple.get(4).asInt()); // min
+      assertEquals(6, tuple.get(2).asInt4()); // sum
+      assertEquals(3, tuple.get(3).asInt4()); // max
+      assertEquals(1, tuple.get(4).asInt4()); // min
       i++;
     }
     assertEquals(10, i);
@@ -306,9 +306,9 @@ public class TestPhysicalPlanner {
     exec.rescan();
     i = 0;
     while ((tuple = exec.next()) != null) {
-      assertEquals(6, tuple.getInt(2).asInt()); // sum
-      assertEquals(3, tuple.getInt(3).asInt()); // max
-      assertEquals(1, tuple.getInt(4).asInt()); // min
+      assertEquals(6, tuple.getInt(2).asInt4()); // sum
+      assertEquals(3, tuple.getInt(3).asInt4()); // max
+      assertEquals(1, tuple.getInt(4).asInt4()); // min
       i++;
     }
     exec.close();
@@ -334,7 +334,7 @@ public class TestPhysicalPlanner {
     LogicalNode plan = planner.createPlan(context);
     plan = LogicalOptimizer.optimize(context, plan);
 
-    TableMeta outputMeta = TCatUtil.newTableMeta(plan.getOutSchema(),
+    TableMeta outputMeta = CatalogUtil.newTableMeta(plan.getOutSchema(),
         StoreType.CSV);
 
     PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf,sm);
@@ -348,9 +348,9 @@ public class TestPhysicalPlanner {
     Tuple tuple;
     int i = 0;
     while ((tuple = scanner.next()) != null) {
-      assertEquals(6, tuple.get(2).asInt()); // sum
-      assertEquals(3, tuple.get(3).asInt()); // max
-      assertEquals(1, tuple.get(4).asInt()); // min
+      assertEquals(6, tuple.get(2).asInt4()); // sum
+      assertEquals(3, tuple.get(3).asInt4()); // max
+      assertEquals(1, tuple.get(4).asInt4()); // min
       i++;
     }
     assertEquals(10, i);
@@ -374,7 +374,7 @@ public class TestPhysicalPlanner {
     LogicalNode plan = planner.createPlan(context);
     plan = LogicalOptimizer.optimize(context, plan);
 
-    TableMeta outputMeta = TCatUtil.newTableMeta(plan.getOutSchema(),
+    TableMeta outputMeta = CatalogUtil.newTableMeta(plan.getOutSchema(),
         StoreType.RCFILE);
 
     PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf,sm);
@@ -388,9 +388,9 @@ public class TestPhysicalPlanner {
     Tuple tuple;
     int i = 0;
     while ((tuple = scanner.next()) != null) {
-      assertEquals(6, tuple.get(2).asInt()); // sum
-      assertEquals(3, tuple.get(3).asInt()); // max
-      assertEquals(1, tuple.get(4).asInt()); // min
+      assertEquals(6, tuple.get(2).asInt4()); // sum
+      assertEquals(3, tuple.get(3).asInt4()); // max
+      assertEquals(1, tuple.get(4).asInt4()); // min
       i++;
     }
     assertEquals(10, i);
@@ -421,14 +421,14 @@ public class TestPhysicalPlanner {
     LogicalNode plan = planner.createPlan(context);
 
     int numPartitions = 3;
-    Column key1 = new Column("score.deptName", DataType.STRING);
-    Column key2 = new Column("score.class", DataType.STRING);
+    Column key1 = new Column("score.deptName", Type.TEXT);
+    Column key2 = new Column("score.class", Type.TEXT);
     StoreTableNode storeNode = new StoreTableNode("partition");
     storeNode.setPartitions(PartitionType.HASH, new Column[]{key1, key2}, numPartitions);
     PlannerUtil.insertNode(plan, storeNode);
     plan = LogicalOptimizer.optimize(context, plan);
 
-    TableMeta outputMeta = TCatUtil.newTableMeta(plan.getOutSchema(),
+    TableMeta outputMeta = CatalogUtil.newTableMeta(plan.getOutSchema(),
         StoreType.CSV);
 
     FileSystem fs = sm.getFileSystem();
@@ -454,9 +454,9 @@ public class TestPhysicalPlanner {
     Tuple tuple;
     i = 0;
     while ((tuple = scanner.next()) != null) {
-      assertEquals(6, tuple.get(2).asInt()); // sum
-      assertEquals(3, tuple.get(3).asInt()); // max
-      assertEquals(1, tuple.get(4).asInt()); // min
+      assertEquals(6, tuple.get(2).asInt4()); // sum
+      assertEquals(3, tuple.get(3).asInt4()); // max
+      assertEquals(1, tuple.get(4).asInt4()); // min
       i++;
     }
     assertEquals(10, i);
@@ -486,7 +486,7 @@ public class TestPhysicalPlanner {
     PlannerUtil.insertNode(plan, storeNode);
     plan = LogicalOptimizer.optimize(context, plan);
 
-    TableMeta outputMeta = TCatUtil.newTableMeta(plan.getOutSchema(),
+    TableMeta outputMeta = CatalogUtil.newTableMeta(plan.getOutSchema(),
         StoreType.CSV);
 
     PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf,sm);
@@ -511,9 +511,9 @@ public class TestPhysicalPlanner {
     Tuple tuple;
     i = 0;
     while ((tuple = scanner.next()) != null) {
-      assertEquals(60, tuple.get(2).asInt()); // sum
-      assertEquals(3, tuple.get(3).asInt()); // max
-      assertEquals(1, tuple.get(4).asInt()); // min
+      assertEquals(60, tuple.get(2).asInt4()); // sum
+      assertEquals(3, tuple.get(3).asInt4()); // max
+      assertEquals(1, tuple.get(4).asInt4()); // min
       i++;
     }
     assertEquals(1, i);
@@ -539,9 +539,9 @@ public class TestPhysicalPlanner {
 
     exec.init();
     Tuple tuple = exec.next();
-    assertEquals(30, tuple.get(0).asLong());
-    assertEquals(3, tuple.get(1).asInt());
-    assertEquals(1, tuple.get(2).asInt());
+    assertEquals(30, tuple.get(0).asInt8());
+    assertEquals(3, tuple.get(1).asInt4());
+    assertEquals(1, tuple.get(2).asInt4());
     assertNull(exec.next());
     exec.close();
   }
@@ -561,7 +561,7 @@ public class TestPhysicalPlanner {
     PhysicalExec exec = phyPlanner.createPlan(ctx, plan);
 
     Tuple tuple = exec.next();
-    assertEquals(30, tuple.get(0).asLong());
+    assertEquals(30, tuple.get(0).asInt8());
     assertNull(exec.next());
   }
 
@@ -630,7 +630,7 @@ public class TestPhysicalPlanner {
     tuple = exec.next();
     exec.close();
     assertEquals(true, tuple.get(0).asBool());
-    assertTrue(7.0d == tuple.get(1).asDouble());
+    assertTrue(7.0d == tuple.get(1).asFloat8());
 
     context = analyzer.parse(QUERIES[13]);
     plan = planner.createPlan(context);
@@ -710,8 +710,8 @@ public class TestPhysicalPlanner {
   @Test
   public final void testBug() throws IOException {
     Schema s1 = new Schema();
-    s1.addColumn("o_orderdate", DataType.STRING);
-    s1.addColumn("o_shippriority", DataType.INT);
+    s1.addColumn("o_orderdate", Type.TEXT);
+    s1.addColumn("o_shippriority", Type.INT4);
     s1.addColumn("o_orderkey", DataType.LONG);
 
     Options opt = new Options();
@@ -777,7 +777,7 @@ public class TestPhysicalPlanner {
     exec.close();
 
     Schema keySchema = new Schema();
-    keySchema.addColumn("?empId", DataType.INT);
+    keySchema.addColumn("?empId", Type.INT4);
     SortSpec[] sortSpec = new SortSpec[1];
     sortSpec[0] = new SortSpec(keySchema.getColumn(0), true, false);
     TupleComparator comp = new TupleComparator(keySchema, sortSpec);
@@ -786,7 +786,7 @@ public class TestPhysicalPlanner {
         keySchema, comp);
     reader.open();
     Path outputPath = StorageUtil.concatPath(workDir, "output", "output");
-    TableMeta meta = TCatUtil.newTableMeta(plan.getOutSchema(), StoreType.CSV, new Options());
+    TableMeta meta = CatalogUtil.newTableMeta(plan.getOutSchema(), StoreType.CSV, new Options());
     SeekableScanner scanner = (SeekableScanner)
         StorageManager.getScanner(conf, meta, outputPath);
     scanner.init();
@@ -801,12 +801,12 @@ public class TestPhysicalPlanner {
 
     Tuple keytuple = new VTuple(1);
     for(int i = 1 ; i < 100 ; i ++) {
-      keytuple.put(0, DatumFactory.createInt(i));
+      keytuple.put(0, DatumFactory.createInt4(i));
       long offsets = reader.find(keytuple);
       scanner.seek(offsets);
       tuple = scanner.next();
       assertTrue("[seek check " + (i) + " ]" , ("name_" + i).equals(tuple.get(0).asChars()));
-      assertTrue("[seek check " + (i) + " ]" , i == tuple.get(1).asInt());
+      assertTrue("[seek check " + (i) + " ]" , i == tuple.get(1).asInt4());
     }
 
 
@@ -815,12 +815,12 @@ public class TestPhysicalPlanner {
         new File(new Path(workDir, "output").toUri()), keySchema, comp);
     Map<String,List<String>> kvs = Maps.newHashMap();
     Tuple startTuple = new VTuple(1);
-    startTuple.put(0, DatumFactory.createInt(50));
+    startTuple.put(0, DatumFactory.createInt4(50));
     kvs.put("start", Lists.newArrayList(
         new String(Base64.encodeBase64(
             RowStoreUtil.RowStoreEncoder.toBytes(keySchema, startTuple), false))));
     Tuple endTuple = new VTuple(1);
-    endTuple.put(0, DatumFactory.createInt(80));
+    endTuple.put(0, DatumFactory.createInt4(80));
     kvs.put("end", Lists.newArrayList(
         new String(Base64.encodeBase64(
             RowStoreUtil.RowStoreEncoder.toBytes(keySchema, endTuple), false))));
@@ -828,11 +828,11 @@ public class TestPhysicalPlanner {
 
     scanner.seek(chunk.startOffset());
     keytuple = scanner.next();
-    assertEquals(50, keytuple.get(1).asInt());
+    assertEquals(50, keytuple.get(1).asInt4());
 
     long endOffset = chunk.startOffset() + chunk.length();
     while((keytuple = scanner.next()) != null && scanner.getNextOffset() <= endOffset) {
-      assertTrue(keytuple.get(1).asInt() <= 80);
+      assertTrue(keytuple.get(1).asInt4() <= 80);
     }
 
     scanner.close();

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-backend/src/test/java/tajo/engine/planner/physical/TestSortExec.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/tajo/engine/planner/physical/TestSortExec.java b/tajo-core/tajo-core-backend/src/test/java/tajo/engine/planner/physical/TestSortExec.java
index 694875f..f91a37c 100644
--- a/tajo-core/tajo-core-backend/src/test/java/tajo/engine/planner/physical/TestSortExec.java
+++ b/tajo-core/tajo-core-backend/src/test/java/tajo/engine/planner/physical/TestSortExec.java
@@ -25,8 +25,8 @@ import org.junit.Test;
 import tajo.TajoTestingCluster;
 import tajo.TaskAttemptContext;
 import tajo.catalog.*;
-import tajo.catalog.proto.CatalogProtos.DataType;
 import tajo.catalog.proto.CatalogProtos.StoreType;
+import tajo.common.TajoDataTypes.Type;
 import tajo.conf.TajoConf;
 import tajo.datum.Datum;
 import tajo.datum.DatumFactory;
@@ -65,11 +65,11 @@ public class TestSortExec {
     sm = StorageManager.get(conf, workDir);
 
     Schema schema = new Schema();
-    schema.addColumn("managerId", DataType.INT);
-    schema.addColumn("empId", DataType.INT);
-    schema.addColumn("deptName", DataType.STRING);
+    schema.addColumn("managerId", Type.INT4);
+    schema.addColumn("empId", Type.INT4);
+    schema.addColumn("deptName", Type.TEXT);
 
-    employeeMeta = TCatUtil.newTableMeta(schema, StoreType.CSV);
+    employeeMeta = CatalogUtil.newTableMeta(schema, StoreType.CSV);
 
     tablePath = StorageUtil.concatPath(workDir, "employee", "table1");
     sm.getFileSystem().mkdirs(tablePath.getParent());
@@ -79,9 +79,9 @@ public class TestSortExec {
     Tuple tuple = new VTuple(employeeMeta.getSchema().getColumnNum());
     for (int i = 0; i < 100; i++) {
       tuple.put(new Datum[] {
-          DatumFactory.createInt(rnd.nextInt(5)),
-          DatumFactory.createInt(rnd.nextInt(10)),
-          DatumFactory.createString("dept_" + rnd.nextInt(10))});
+          DatumFactory.createInt4(rnd.nextInt(5)),
+          DatumFactory.createInt4(rnd.nextInt(10)),
+          DatumFactory.createText("dept_" + rnd.nextInt(10))});
       appender.addTuple(tuple);
     }
     appender.flush();
@@ -139,11 +139,11 @@ public class TestSortExec {
    */
   public void testTAJO_946() {
     Schema schema = new Schema();
-    schema.addColumn("l_orderkey", DataType.LONG);
+    schema.addColumn("l_orderkey", Type.INT8);
     Tuple s = new VTuple(1);
-    s.put(0, DatumFactory.createLong(0));
+    s.put(0, DatumFactory.createInt8(0));
     Tuple e = new VTuple(1);
-    e.put(0, DatumFactory.createLong(6000000000l));
+    e.put(0, DatumFactory.createInt8(6000000000l));
     TupleRange expected = new TupleRange(schema, s, e);
     RangePartitionAlgorithm partitioner
         = new UniformRangePartition(schema, expected, true);

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-backend/src/test/java/tajo/engine/query/TestNullValues.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/tajo/engine/query/TestNullValues.java b/tajo-core/tajo-core-backend/src/test/java/tajo/engine/query/TestNullValues.java
index 167b15e..c3b37aa 100644
--- a/tajo-core/tajo-core-backend/src/test/java/tajo/engine/query/TestNullValues.java
+++ b/tajo-core/tajo-core-backend/src/test/java/tajo/engine/query/TestNullValues.java
@@ -24,7 +24,7 @@ import tajo.IntegrationTest;
 import tajo.TajoTestingCluster;
 import tajo.catalog.Options;
 import tajo.catalog.Schema;
-import tajo.catalog.proto.CatalogProtos;
+import tajo.common.TajoDataTypes.Type;
 import tajo.storage.CSVFile;
 
 import java.sql.ResultSet;
@@ -42,8 +42,8 @@ public class TestNullValues {
   public final void testIsNull() throws Exception {
     String [] table = new String[] {"nulltable1"};
     Schema schema = new Schema();
-    schema.addColumn("col1", CatalogProtos.DataType.INT);
-    schema.addColumn("col2", CatalogProtos.DataType.STRING);
+    schema.addColumn("col1", Type.INT4);
+    schema.addColumn("col2", Type.TEXT);
     Schema [] schemas = new Schema[] {schema};
     String [] data = {
         "1|filled|",
@@ -64,8 +64,8 @@ public class TestNullValues {
   public final void testIsNotNull() throws Exception {
     String [] table = new String[] {"nulltable2"};
     Schema schema = new Schema();
-    schema.addColumn("col1", CatalogProtos.DataType.INT);
-    schema.addColumn("col2", CatalogProtos.DataType.STRING);
+    schema.addColumn("col1", Type.INT4);
+    schema.addColumn("col2", Type.TEXT);
     Schema [] schemas = new Schema[] {schema};
     String [] data = {
         "1|filled|",
@@ -88,16 +88,16 @@ public class TestNullValues {
   public final void testIsNotNull2() throws Exception {
     String [] table = new String[] {"nulltable3"};
     Schema schema = new Schema();
-    schema.addColumn("col1", CatalogProtos.DataType.LONG);
-    schema.addColumn("col2", CatalogProtos.DataType.LONG);
-    schema.addColumn("col3", CatalogProtos.DataType.LONG);
-    schema.addColumn("col4", CatalogProtos.DataType.LONG);
-    schema.addColumn("col5", CatalogProtos.DataType.LONG);
-    schema.addColumn("col6", CatalogProtos.DataType.LONG);
-    schema.addColumn("col7", CatalogProtos.DataType.LONG);
-    schema.addColumn("col8", CatalogProtos.DataType.LONG);
-    schema.addColumn("col9", CatalogProtos.DataType.LONG);
-    schema.addColumn("col10", CatalogProtos.DataType.LONG);
+    schema.addColumn("col1", Type.INT8);
+    schema.addColumn("col2", Type.INT8);
+    schema.addColumn("col3", Type.INT8);
+    schema.addColumn("col4", Type.INT8);
+    schema.addColumn("col5", Type.INT8);
+    schema.addColumn("col6", Type.INT8);
+    schema.addColumn("col7", Type.INT8);
+    schema.addColumn("col8", Type.INT8);
+    schema.addColumn("col9", Type.INT8);
+    schema.addColumn("col10", Type.INT8);
     Schema [] schemas = new Schema[] {schema};
     String [] data = {
         ",,,,672287821,1301460,1,313895860387,126288907,1024",

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-backend/src/test/java/tajo/engine/query/TestResultSetImpl.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/tajo/engine/query/TestResultSetImpl.java b/tajo-core/tajo-core-backend/src/test/java/tajo/engine/query/TestResultSetImpl.java
index f1f09ef..ebeecd3 100644
--- a/tajo-core/tajo-core-backend/src/test/java/tajo/engine/query/TestResultSetImpl.java
+++ b/tajo-core/tajo-core-backend/src/test/java/tajo/engine/query/TestResultSetImpl.java
@@ -28,12 +28,12 @@ import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import tajo.IntegrationTest;
 import tajo.TajoTestingCluster;
+import tajo.catalog.CatalogUtil;
 import tajo.catalog.Schema;
-import tajo.catalog.TCatUtil;
 import tajo.catalog.TableMeta;
-import tajo.catalog.proto.CatalogProtos.DataType;
 import tajo.catalog.proto.CatalogProtos.StoreType;
 import tajo.catalog.statistics.TableStat;
+import tajo.common.TajoDataTypes.Type;
 import tajo.conf.TajoConf;
 import tajo.datum.DatumFactory;
 import tajo.storage.Appender;
@@ -62,9 +62,9 @@ public class TestResultSetImpl {
     sm = new StorageManager(conf);
 
     Schema scoreSchema = new Schema();
-    scoreSchema.addColumn("deptname", DataType.STRING);
-    scoreSchema.addColumn("score", DataType.INT);
-    scoreMeta = TCatUtil.newTableMeta(scoreSchema, StoreType.CSV);
+    scoreSchema.addColumn("deptname", Type.TEXT);
+    scoreSchema.addColumn("score", Type.INT4);
+    scoreMeta = CatalogUtil.newTableMeta(scoreSchema, StoreType.CSV);
     TableStat stat = new TableStat();
 
     Path p = sm.getTablePath("score");
@@ -78,8 +78,8 @@ public class TestResultSetImpl {
     for (int i = 0; i < tupleNum; i++) {
       tuple = new VTuple(2);
       String key = "test" + (i % deptSize);
-      tuple.put(0, DatumFactory.createString(key));
-      tuple.put(1, DatumFactory.createInt(i + 1));
+      tuple.put(0, DatumFactory.createText(key));
+      tuple.put(1, DatumFactory.createInt4(i + 1));
       written += key.length() + Integer.SIZE;
       appender.addTuple(tuple);
     }

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-backend/src/test/java/tajo/engine/util/TestTupleUtil.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/tajo/engine/util/TestTupleUtil.java b/tajo-core/tajo-core-backend/src/test/java/tajo/engine/util/TestTupleUtil.java
index 9a23083..bee9b39 100644
--- a/tajo-core/tajo-core-backend/src/test/java/tajo/engine/util/TestTupleUtil.java
+++ b/tajo-core/tajo-core-backend/src/test/java/tajo/engine/util/TestTupleUtil.java
@@ -20,7 +20,7 @@ package tajo.engine.util;
 
 import org.junit.Test;
 import tajo.catalog.Schema;
-import tajo.catalog.proto.CatalogProtos.DataType;
+import tajo.common.TajoDataTypes.Type;
 import tajo.datum.Datum;
 import tajo.datum.DatumFactory;
 import tajo.engine.planner.PlannerUtil;
@@ -40,32 +40,32 @@ public class TestTupleUtil {
   @Test
   public final void testToBytesAndToTuple() {
     Schema schema = new Schema();
-    schema.addColumn("col1", DataType.BOOLEAN);
-    schema.addColumn("col2", DataType.BYTE);
-    schema.addColumn("col3", DataType.CHAR);
-    schema.addColumn("col4", DataType.SHORT);
-    schema.addColumn("col5", DataType.INT);
-    schema.addColumn("col6", DataType.LONG);
-    schema.addColumn("col7", DataType.FLOAT);
-    schema.addColumn("col8", DataType.DOUBLE);
-    schema.addColumn("col9", DataType.STRING);
-    schema.addColumn("col10", DataType.BYTES);
-    schema.addColumn("col11", DataType.IPv4);
+    schema.addColumn("col1", Type.BOOLEAN);
+    schema.addColumn("col2", Type.BIT);
+    schema.addColumn("col3", Type.CHAR);
+    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("col11", DataType.IPv6);
     
     Tuple tuple = new VTuple(11);
     tuple.put(new Datum[] {
         DatumFactory.createBool(true),
-        DatumFactory.createByte((byte) 0x99),
+        DatumFactory.createBit((byte) 0x99),
         DatumFactory.createChar('7'),
-        DatumFactory.createShort((short) 17),
-        DatumFactory.createInt(59),
-        DatumFactory.createLong(23l),
-        DatumFactory.createFloat(77.9f),
-        DatumFactory.createDouble(271.9f),        
-        DatumFactory.createString("hyunsik"),
-        DatumFactory.createBytes("hyunsik".getBytes()),
-        DatumFactory.createIPv4("192.168.0.1")
+        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")
     });
     
     byte [] bytes = RowStoreUtil.RowStoreEncoder.toBytes(schema, tuple);
@@ -80,29 +80,29 @@ public class TestTupleUtil {
     Tuple eTuple = new VTuple(7);
 
     Schema schema = new Schema();
-    schema.addColumn("numByte", DataType.BYTE);
-    schema.addColumn("numChar", DataType.CHAR);
-    schema.addColumn("numShort", DataType.SHORT);
-    schema.addColumn("numInt", DataType.INT);
-    schema.addColumn("numLong", DataType.LONG);
-    schema.addColumn("numFloat", DataType.FLOAT);
-    schema.addColumn("numDouble", DataType.FLOAT);
-
-    sTuple.put(0, DatumFactory.createByte((byte) 44));
+    schema.addColumn("numByte", Type.BIT);
+    schema.addColumn("numChar", Type.CHAR);
+    schema.addColumn("numShort", Type.INT2);
+    schema.addColumn("numInt", Type.INT4);
+    schema.addColumn("numLong", Type.INT8);
+    schema.addColumn("numFloat", Type.FLOAT4);
+    schema.addColumn("numDouble", Type.FLOAT4);
+
+    sTuple.put(0, DatumFactory.createBit((byte) 44));
     sTuple.put(1, DatumFactory.createChar('a'));
-    sTuple.put(2, DatumFactory.createShort((short) 10));
-    sTuple.put(3, DatumFactory.createInt(5));
-    sTuple.put(4, DatumFactory.createLong(100));
-    sTuple.put(5, DatumFactory.createFloat(100));
-    sTuple.put(6, DatumFactory.createDouble(100));
+    sTuple.put(2, DatumFactory.createInt2((short) 10));
+    sTuple.put(3, DatumFactory.createInt4(5));
+    sTuple.put(4, DatumFactory.createInt8(100));
+    sTuple.put(5, DatumFactory.createFloat4(100));
+    sTuple.put(6, DatumFactory.createFloat8(100));
 
-    eTuple.put(0, DatumFactory.createByte((byte) 99));
+    eTuple.put(0, DatumFactory.createBit((byte) 99));
     eTuple.put(1, DatumFactory.createChar('p'));
-    eTuple.put(2, DatumFactory.createShort((short) 70));
-    eTuple.put(3, DatumFactory.createInt(70));
-    eTuple.put(4, DatumFactory.createLong(10000));
-    eTuple.put(5, DatumFactory.createFloat(150));
-    eTuple.put(6, DatumFactory.createDouble(170));
+    eTuple.put(2, DatumFactory.createInt2((short) 70));
+    eTuple.put(3, DatumFactory.createInt4(70));
+    eTuple.put(4, DatumFactory.createInt8(10000));
+    eTuple.put(5, DatumFactory.createFloat4(150));
+    eTuple.put(6, DatumFactory.createFloat8(170));
 
     RangePartitionAlgorithm partitioner = new UniformRangePartition(schema, new TupleRange(schema, sTuple, eTuple));
     TupleRange [] ranges = partitioner.partition(5);
@@ -119,16 +119,16 @@ public class TestTupleUtil {
   @Test
   public void testQueryToRange() throws UnsupportedEncodingException {
     Schema schema = new Schema();
-    schema.addColumn("intval", DataType.INT);
-    schema.addColumn("floatval", DataType.FLOAT);
+    schema.addColumn("intval", Type.INT4);
+    schema.addColumn("floatval", Type.FLOAT4);
 
     Tuple s = new VTuple(2);
-    s.put(0, DatumFactory.createInt(5));
-    s.put(1, DatumFactory.createFloat(10));
+    s.put(0, DatumFactory.createInt4(5));
+    s.put(1, DatumFactory.createFloat4(10));
 
     Tuple e = new VTuple(2);
-    e.put(0, DatumFactory.createInt(10));
-    e.put(1, DatumFactory.createFloat(20));
+    e.put(0, DatumFactory.createInt4(10));
+    e.put(1, DatumFactory.createFloat4(20));
 
     TupleRange expected = new TupleRange(schema, s, e);
     int card = (int) TupleUtil.computeCardinality(schema, expected);
@@ -155,12 +155,12 @@ public class TestTupleUtil {
   @Test
   public void testQueryToRangeWithOneRange() throws UnsupportedEncodingException {
     Schema schema = new Schema();
-    schema.addColumn("partkey", DataType.FLOAT);
+    schema.addColumn("partkey", Type.FLOAT4);
 
     Tuple s = new VTuple(1);
-    s.put(0, DatumFactory.createFloat(28082));
+    s.put(0, DatumFactory.createFloat4(28082));
     Tuple e = new VTuple(1);
-    e.put(0, DatumFactory.createFloat(28082));
+    e.put(0, DatumFactory.createFloat4(28082));
 
     TupleRange expected = new TupleRange(schema, s, e);
     int card = (int) TupleUtil.computeCardinality(schema, expected);
@@ -186,11 +186,11 @@ public class TestTupleUtil {
    */
   public void testRangeToQueryHeavyTest() throws UnsupportedEncodingException {
     Schema schema = new Schema();
-    schema.addColumn("c_custkey", DataType.INT);
+    schema.addColumn("c_custkey", Type.INT4);
     Tuple s = new VTuple(1);
-    s.put(0, DatumFactory.createInt(4));
+    s.put(0, DatumFactory.createInt4(4));
     Tuple e = new VTuple(1);
-    e.put(0, DatumFactory.createInt(149995));
+    e.put(0, DatumFactory.createInt4(149995));
     TupleRange expected = new TupleRange(schema, s, e);
     TupleRange [] ranges = TupleUtil.getPartitions(schema, 31, expected);
 
@@ -208,14 +208,14 @@ public class TestTupleUtil {
    */
   public void testRangeToQueryTest() throws UnsupportedEncodingException {
     Schema schema = new Schema();
-    schema.addColumn("l_returnflag", DataType.STRING);
-    schema.addColumn("l_linestatus", DataType.STRING);
+    schema.addColumn("l_returnflag", Type.TEXT);
+    schema.addColumn("l_linestatus", Type.TEXT);
     Tuple s = new VTuple(2);
-    s.put(0, DatumFactory.createString("A"));
-    s.put(1, DatumFactory.createString("F"));
+    s.put(0, DatumFactory.createText("A"));
+    s.put(1, DatumFactory.createText("F"));
     Tuple e = new VTuple(2);
-    e.put(0, DatumFactory.createString("R"));
-    e.put(1, DatumFactory.createString("O"));
+    e.put(0, DatumFactory.createText("R"));
+    e.put(1, DatumFactory.createText("O"));
     TupleRange expected = new TupleRange(schema, s, e);
 
     RangePartitionAlgorithm partitioner = new UniformRangePartition(schema, expected, true);

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-backend/src/test/java/tajo/master/TestExecutionBlockCursor.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/tajo/master/TestExecutionBlockCursor.java b/tajo-core/tajo-core-backend/src/test/java/tajo/master/TestExecutionBlockCursor.java
index c6a5d43..36cdef3 100644
--- a/tajo-core/tajo-core-backend/src/test/java/tajo/master/TestExecutionBlockCursor.java
+++ b/tajo-core/tajo-core-backend/src/test/java/tajo/master/TestExecutionBlockCursor.java
@@ -21,10 +21,7 @@ import org.junit.Test;
 import tajo.QueryIdFactory;
 import tajo.TajoTestingCluster;
 import tajo.benchmark.TPCH;
-import tajo.catalog.CatalogService;
-import tajo.catalog.TCatUtil;
-import tajo.catalog.TableDesc;
-import tajo.catalog.TableMeta;
+import tajo.catalog.*;
 import tajo.catalog.proto.CatalogProtos;
 import tajo.conf.TajoConf;
 import tajo.engine.parser.QueryAnalyzer;
@@ -56,8 +53,8 @@ public class TestExecutionBlockCursor {
     tpch.loadSchemas();
     tpch.loadOutSchema();
     for (String table : tpch.getTableNames()) {
-      TableMeta m = TCatUtil.newTableMeta(tpch.getSchema(table), CatalogProtos.StoreType.CSV);
-      TableDesc d = TCatUtil.newTableDesc(table, m, new Path("file:///"));
+      TableMeta m = CatalogUtil.newTableMeta(tpch.getSchema(table), CatalogProtos.StoreType.CSV);
+      TableDesc d = CatalogUtil.newTableDesc(table, m, new Path("file:///"));
       catalog.addTable(d);
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-backend/src/test/java/tajo/storage/TestFragment.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/tajo/storage/TestFragment.java b/tajo-core/tajo-core-backend/src/test/java/tajo/storage/TestFragment.java
index 87a45ed..674eaa6 100644
--- a/tajo-core/tajo-core-backend/src/test/java/tajo/storage/TestFragment.java
+++ b/tajo-core/tajo-core-backend/src/test/java/tajo/storage/TestFragment.java
@@ -23,11 +23,11 @@ import com.google.gson.Gson;
 import org.apache.hadoop.fs.Path;
 import org.junit.Before;
 import org.junit.Test;
+import tajo.catalog.CatalogUtil;
 import tajo.catalog.Schema;
-import tajo.catalog.TCatUtil;
 import tajo.catalog.TableMeta;
-import tajo.catalog.proto.CatalogProtos.DataType;
 import tajo.catalog.proto.CatalogProtos.StoreType;
+import tajo.common.TajoDataTypes.Type;
 import tajo.engine.json.GsonCreator;
 
 import java.util.Arrays;
@@ -43,9 +43,9 @@ public class TestFragment {
   @Before
   public final void setUp() throws Exception {
     schema1 = new Schema();
-    schema1.addColumn("id", DataType.INT);
-    schema1.addColumn("name", DataType.STRING);
-    meta1 = TCatUtil.newTableMeta(schema1, StoreType.CSV);
+    schema1.addColumn("id", Type.INT4);
+    schema1.addColumn("name", Type.TEXT);
+    meta1 = CatalogUtil.newTableMeta(schema1, StoreType.CSV);
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-backend/src/test/java/tajo/storage/TestHCFile.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/tajo/storage/TestHCFile.java b/tajo-core/tajo-core-backend/src/test/java/tajo/storage/TestHCFile.java
deleted file mode 100644
index aa4e9f0..0000000
--- a/tajo-core/tajo-core-backend/src/test/java/tajo/storage/TestHCFile.java
+++ /dev/null
@@ -1,268 +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 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.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-import tajo.TajoTestingCluster;
-import tajo.catalog.Schema;
-import tajo.catalog.TCatUtil;
-import tajo.catalog.TableMeta;
-import tajo.catalog.proto.CatalogProtos.CompressType;
-import tajo.catalog.proto.CatalogProtos.DataType;
-import tajo.catalog.proto.CatalogProtos.StoreType;
-import tajo.conf.TajoConf;
-import tajo.datum.Datum;
-import tajo.datum.DatumFactory;
-import tajo.storage.exception.UnknownCodecException;
-import tajo.storage.exception.UnknownDataTypeException;
-import tajo.storage.hcfile.ColumnMeta;
-import tajo.storage.hcfile.HCFile.Appender;
-import tajo.storage.hcfile.HCFile.Scanner;
-import tajo.storage.hcfile.HCTupleAppender;
-import tajo.storage.hcfile.HColumnMetaWritable;
-
-import java.io.IOException;
-import java.util.Arrays;
-import java.util.List;
-import java.util.Random;
-
-import static org.junit.Assert.assertEquals;
-
-public class TestHCFile {
-
-  private static TajoTestingCluster util;
-  private static TajoConf conf;
-  private static Random random;
-
-  @Before
-  public void setUp() throws Exception {
-    util = new TajoTestingCluster();
-    util.startMiniDFSCluster(1);
-    conf = util.getConfiguration();
-    conf.setInt("dfs.blocksize", 65535);
-    random = new Random(System.currentTimeMillis());
-  }
-
-  @After
-  public void tearDown() throws Exception {
-    util.shutdownMiniDFSCluster();
-  }
-
-  @Test
-  public void testInt()
-      throws UnknownCodecException, IOException, UnknownDataTypeException {
-    int rowNum = 1000;
-    Path path = new Path("hdfs:///hcfile.int");
-    List<Integer> data = Lists.newArrayList();
-
-    ColumnMeta
-        meta = new HColumnMetaWritable(0, DataType.INT, CompressType.COMP_NONE,
-        false, false, true);
-    long before = System.currentTimeMillis();
-    Appender appender = new Appender(conf, meta, path);
-
-    for (int i = 0; i < rowNum; i++) {
-      data.add(i);
-      appender.append(DatumFactory.createInt(data.get(data.size()-1)));
-    }
-    appender.close();
-    long after = System.currentTimeMillis();
-    System.out.println("write time: " + (after-before));
-
-    before = System.currentTimeMillis();
-    Scanner scanner = new Scanner(conf, path);
-
-    for (Integer i : data) {
-      assertEquals(i.intValue(), scanner.get().asInt());
-    }
-    after = System.currentTimeMillis();
-    System.out.println("sequential read time: " + (after-before));
-    scanner.close();
-
-    before = System.currentTimeMillis();
-    scanner = new Scanner(conf, path);
-    scanner.first();
-    assertEquals(data.get(0).intValue(), scanner.get().asInt());
-    after = System.currentTimeMillis();
-    System.out.println("after first() read time: " + (after-before));
-    scanner.close();
-
-    before = System.currentTimeMillis();
-    scanner = new Scanner(conf, path);
-    scanner.last();
-    assertEquals(data.get(data.size()-1).intValue(), scanner.get().asInt());
-    after = System.currentTimeMillis();
-    System.out.println("after last() read time: " + (after-before));
-    scanner.close();
-
-    before = System.currentTimeMillis();
-    scanner = new Scanner(conf, path);
-    int randomIndex = random.nextInt(rowNum);
-    scanner.pos(randomIndex);
-    assertEquals(data.get(randomIndex).intValue(), scanner.get().asInt());
-    after = System.currentTimeMillis();
-    System.out.println("after pos() read time: " + (after-before));
-    scanner.close();
-  }
-
-  @Test
-  public void testString()
-      throws IOException, UnknownCodecException, UnknownDataTypeException {
-    int rowNum = 1000;
-    Path path = new Path("hdfs:///hcfile.string");
-    List<String> data = Lists.newArrayList();
-
-    ColumnMeta meta = new HColumnMetaWritable(0, DataType.STRING, CompressType.COMP_NONE,
-        false, false, true);
-    Appender appender = new Appender(conf, meta, path);
-
-    String randomStr;
-    for (int i = 0; i < rowNum; i++) {
-      randomStr = getRandomString(10);
-      data.add(randomStr);
-      appender.append(DatumFactory.createString(randomStr));
-    }
-    appender.close();
-
-    Scanner scanner = new Scanner(conf, path);
-    for (String s : data) {
-      assertEquals(s, scanner.get().asChars());
-    }
-    scanner.close();
-  }
-
-  @Test
-  public void testHCTupleAppender()
-      throws UnknownCodecException, IOException, UnknownDataTypeException {
-    int tupleNum = 1000;
-
-    Path tablePath = new Path("hdfs:///table");
-    Schema schema = new Schema();
-    schema.addColumn("id", DataType.INT);
-    schema.addColumn("age", DataType.LONG);
-    schema.addColumn("description", DataType.STRING);
-    schema.addColumn("char", DataType.CHAR);
-    TableMeta meta = TCatUtil.newTableMeta(schema, StoreType.HCFILE);
-
-    HCTupleAppender appender = new HCTupleAppender(conf, meta, 2, tablePath);
-    Datum stringDatum = DatumFactory.createString("abcdefghijklmnopqrstuvwxyz");
-
-    int i;
-    Tuple tuple = new VTuple(4);
-    for(i = 0; i < tupleNum; i++) {
-      tuple.put(0, DatumFactory.createInt(i));
-      tuple.put(1, DatumFactory.createLong(25l));
-      tuple.put(2, stringDatum);
-      tuple.put(3, DatumFactory.createChar('a'));
-      appender.addTuple(tuple);
-    }
-    appender.close();
-
-    FileSystem fs = tablePath.getFileSystem(conf);
-    FileStatus[] files = fs.listStatus(new Path(tablePath, "data"));
-    Path[] shardDirs = new Path[files.length];
-    for (i = 0; i < files.length; i++) {
-      shardDirs[i] = files[i].getPath();
-    }
-    Arrays.sort(shardDirs, new NumericPathComparator());
-
-    Scanner scanner;
-    Datum datum;
-    int cnt = 0;
-
-    for (i = 0; i < shardDirs.length; i++) {
-      scanner = new Scanner(conf, new Path(shardDirs[i], "id_0"));
-      while ((datum=scanner.get()) != null) {
-        assertEquals(cnt++, datum.asInt());
-      }
-      scanner.close();
-
-      scanner = new Scanner(conf, new Path(shardDirs[i], "age_0"));
-      while ((datum=scanner.get()) != null) {
-        assertEquals(25l, datum.asLong());
-      }
-      scanner.close();
-
-      scanner = new Scanner(conf, new Path(shardDirs[i], "description_0"));
-      while ((datum=scanner.get()) != null) {
-        assertEquals("abcdefghijklmnopqrstuvwxyz", datum.asChars());
-      }
-      scanner.close();
-
-      scanner = new Scanner(conf, new Path(shardDirs[i], "char_0"));
-      while ((datum=scanner.get()) != null) {
-        assertEquals('a', datum.asChar());
-      }
-      scanner.close();
-    }
-  }
-
-//  @Test
-//  public void testOrders()
-//      throws IOException, UnknownCodecException, UnknownDataTypeException {
-//    Path tablePath = new Path("file:///home/jihoon/work/develop/tpch/customer");
-//    Path metaPath = new Path(tablePath, ".meta");
-//    Path dataDir = new Path(tablePath, "data");
-//    Path outPath = new Path("file:///home/jihoon/work/develop/ColumnLoader/target/test-data/customer");
-//    FileSystem fs = metaPath.getFileSystem(conf);
-//
-//    FSDataInputStream in = fs.open(metaPath);
-//    TableProto proto = (TableProto) FileUtil.loadProto(in, TableProto.getDefaultInstance());
-//    TableMeta meta = new TableMetaImpl(proto);
-//    in.close();
-//
-//    Tuple tuple;
-//    Fragment fragment;
-//    CSVFile.CSVScanner scanner;
-//    HCTupleAppender appender = new HCTupleAppender(conf, meta, meta.getSchema().getColumn(0), outPath);
-//
-//    for (FileStatus file : fs.listStatus(dataDir)) {
-//      if (file.getPath().getName().equals(".index")) {
-//        continue;
-//      }
-//      fragment = new Fragment("0", file.getPath(), meta, 0, file.getLen());
-//      scanner = new CSVScanner(conf, meta.getSchema(), fragment);
-//      while ((tuple=scanner.next()) != null) {
-//        appender.addTuple(tuple);
-//      }
-//      scanner.close();
-//    }
-//    appender.close();
-//
-//  }
-
-  private static String getRandomString(int length) {
-    StringBuffer buffer = new StringBuffer();
-
-    String chars[] =
-        "a,b,c,d,e,f,g,h,i,j,k,l,m,n,o,p,q,r,s,t,u,v,w,x,y,z".split(",");
-
-    for (int i=0 ; i<length ; i++)
-    {
-      buffer.append(chars[random.nextInt(chars.length)]);
-    }
-    return buffer.toString();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-backend/src/test/java/tajo/storage/TestHColumnReader.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/tajo/storage/TestHColumnReader.java b/tajo-core/tajo-core-backend/src/test/java/tajo/storage/TestHColumnReader.java
deleted file mode 100644
index df72f39..0000000
--- a/tajo-core/tajo-core-backend/src/test/java/tajo/storage/TestHColumnReader.java
+++ /dev/null
@@ -1,145 +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 tajo.storage;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.Path;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-import tajo.TajoTestingCluster;
-import tajo.catalog.Column;
-import tajo.catalog.Schema;
-import tajo.catalog.TCatUtil;
-import tajo.catalog.TableMeta;
-import tajo.catalog.proto.CatalogProtos.DataType;
-import tajo.catalog.proto.CatalogProtos.StoreType;
-import tajo.datum.Datum;
-import tajo.datum.DatumFactory;
-import tajo.storage.hcfile.HCTupleAppender;
-import tajo.storage.hcfile.HColumnReader;
-import tajo.util.FileUtil;
-
-import java.io.IOException;
-
-import static org.junit.Assert.assertEquals;
-
-public class TestHColumnReader {
-  private TajoTestingCluster util;
-  private Configuration conf;
-  private int i, tupleNum = 150000;
-  private Path tablePath = new Path("hdfs:///customer");
-
-
-  @Before
-  public void setup() throws Exception {
-    util = new TajoTestingCluster();
-    util.startMiniDFSCluster(1);
-    conf = util.getConfiguration();
-
-    Schema schema = new Schema(
-        new Column[]{
-            new Column("id", DataType.INT),
-            new Column("name", DataType.STRING2)});
-    TableMeta tableMeta = TCatUtil.newTableMeta(schema, StoreType.HCFILE);
-    FileUtil.writeProto(util.getDefaultFileSystem(),
-        new Path(tablePath, ".meta"), tableMeta.getProto());
-
-    HCTupleAppender appender = new HCTupleAppender(conf, tableMeta, 1, tablePath);
-    Tuple tuple = new VTuple(2);
-
-    for (i = 0; i < tupleNum; i++) {
-      tuple.put(0, DatumFactory.createInt(i));
-      tuple.put(1, DatumFactory.createString2("abcdefghijklmnopqrstuvwxyz"));
-      appender.addTuple(tuple);
-    }
-
-    appender.close();
-  }
-
-  @After
-  public void teardown() throws Exception {
-    util.shutdownMiniDFSCluster();
-  }
-
-  @Test
-  public void testSeqscan() throws IOException {
-
-    Datum datum;
-    HColumnReader reader = new HColumnReader(conf, tablePath, "id");
-    for (i = 0; (datum=reader.get()) != null; i++) {
-      assertEquals(i, datum.asInt());
-    }
-
-    reader.close();
-
-    assertEquals(i, tupleNum);
-
-    reader = new HColumnReader(conf, tablePath, "name");
-    for (i = 0; (datum=reader.get()) != null; i++) {
-      assertEquals("abcdefghijklmnopqrstuvwxyz", datum.asChars());
-    }
-
-    reader.close();
-
-    assertEquals(i, tupleNum);
-  }
-
-  @Test
-  public void testRandscan() throws IOException {
-    Datum datum;
-    HColumnReader idReader = new HColumnReader(conf, tablePath, 0);
-    HColumnReader nameReader = new HColumnReader(conf, tablePath, "name");
-    idReader.pos(100000);
-    nameReader.pos(100000);
-    for (i = 100000; (datum=idReader.get()) != null; i++) {
-      assertEquals(i, datum.asInt());
-      assertEquals("abcdefghijklmnopqrstuvwxyz", nameReader.get().asChars());
-    }
-    assertEquals(i, tupleNum);
-
-    idReader.pos(3000);
-    nameReader.pos(3000);
-    for (i = 3000; i < 50000; i++) {
-      datum = idReader.get();
-      assertEquals(i, datum.asInt());
-      assertEquals("abcdefghijklmnopqrstuvwxyz", nameReader.get().asChars());
-    }
-    assertEquals(50000, i);
-
-    idReader.pos(30000);
-    nameReader.pos(30000);
-    for (i = 30000; (datum=idReader.get()) != null; i++) {
-      assertEquals(i, datum.asInt());
-      assertEquals("abcdefghijklmnopqrstuvwxyz", nameReader.get().asChars());
-    }
-    assertEquals(i, tupleNum);
-
-    idReader.pos(0);
-    nameReader.pos(0);
-    for (i = 0; (datum=idReader.get()) != null; i++) {
-      assertEquals(i, datum.asInt());
-      assertEquals("abcdefghijklmnopqrstuvwxyz", nameReader.get().asChars());
-    }
-    assertEquals(i, tupleNum);
-
-    idReader.close();
-  }
-}
-

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-backend/src/test/java/tajo/storage/TestRowFile.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/tajo/storage/TestRowFile.java b/tajo-core/tajo-core-backend/src/test/java/tajo/storage/TestRowFile.java
index cd3058f..b7b5b0a 100644
--- a/tajo-core/tajo-core-backend/src/test/java/tajo/storage/TestRowFile.java
+++ b/tajo-core/tajo-core-backend/src/test/java/tajo/storage/TestRowFile.java
@@ -27,14 +27,14 @@ import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
 import tajo.TajoTestingCluster;
+import tajo.catalog.CatalogUtil;
 import tajo.catalog.Schema;
-import tajo.catalog.TCatUtil;
 import tajo.catalog.TableMeta;
 import tajo.catalog.TableMetaImpl;
-import tajo.catalog.proto.CatalogProtos.DataType;
 import tajo.catalog.proto.CatalogProtos.StoreType;
 import tajo.catalog.proto.CatalogProtos.TableProto;
 import tajo.catalog.statistics.TableStat;
+import tajo.common.TajoDataTypes.Type;
 import tajo.conf.TajoConf.ConfVars;
 import tajo.datum.Datum;
 import tajo.datum.DatumFactory;
@@ -65,11 +65,11 @@ public class TestRowFile {
   @Test
   public void test() throws IOException {
     Schema schema = new Schema();
-    schema.addColumn("id", DataType.INT);
-    schema.addColumn("age", DataType.LONG);
-    schema.addColumn("description", DataType.STRING2);
+    schema.addColumn("id", Type.INT4);
+    schema.addColumn("age", Type.INT8);
+    schema.addColumn("description", Type.TEXT);
 
-    TableMeta meta = TCatUtil.newTableMeta(schema, StoreType.ROWFILE);
+    TableMeta meta = CatalogUtil.newTableMeta(schema, StoreType.ROWFILE);
 
     Path tablePath = new Path("hdfs:///test");
     Path metaPath = new Path(tablePath, ".meta");
@@ -87,14 +87,14 @@ public class TestRowFile {
 
     int tupleNum = 100000;
     Tuple tuple;
-    Datum stringDatum = DatumFactory.createString2("abcdefghijklmnopqrstuvwxyz");
+    Datum stringDatum = DatumFactory.createText("abcdefghijklmnopqrstuvwxyz");
     Set<Integer> idSet = Sets.newHashSet();
 
     tuple = new VTuple(3);
     long start = System.currentTimeMillis();
     for(int i = 0; i < tupleNum; i++) {
-      tuple.put(0, DatumFactory.createInt(i + 1));
-      tuple.put(1, DatumFactory.createLong(25l));
+      tuple.put(0, DatumFactory.createInt4(i + 1));
+      tuple.put(1, DatumFactory.createInt8(25l));
       tuple.put(2, stringDatum);
       appender.addTuple(tuple);
       idSet.add(i+1);
@@ -140,8 +140,8 @@ public class TestRowFile {
       scanner = new RowFile.RowFileScanner(conf, meta, fragment);
       scanner.init();
       while ((tuple=scanner.next()) != null) {
-        if (!idSet.remove(tuple.get(0).asInt())) {
-          System.out.println("duplicated! " + tuple.get(0).asInt());
+        if (!idSet.remove(tuple.get(0).asInt4())) {
+          System.out.println("duplicated! " + tuple.get(0).asInt4());
         }
         tupleCnt++;
       }

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-backend/src/test/java/tajo/worker/SlowFunc.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/tajo/worker/SlowFunc.java b/tajo-core/tajo-core-backend/src/test/java/tajo/worker/SlowFunc.java
deleted file mode 100644
index a6f2a3a..0000000
--- a/tajo-core/tajo-core-backend/src/test/java/tajo/worker/SlowFunc.java
+++ /dev/null
@@ -1,65 +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 tajo.worker;
-
-import org.mortbay.log.Log;
-import tajo.catalog.Column;
-import tajo.catalog.function.AggFunction;
-import tajo.catalog.function.FunctionContext;
-import tajo.catalog.proto.CatalogProtos.DataType;
-import tajo.datum.Datum;
-import tajo.storage.Tuple;
-
-public class SlowFunc extends AggFunction {
-  private Datum param;
-
-  public SlowFunc() {
-    super(new Column[] { new Column("name", DataType.STRING) });
-  }
-
-  @Override
-  public FunctionContext newContext() {
-    return null;
-  }
-
-  @Override
-  public void eval(FunctionContext ctx, Tuple params) {
-    param = params.get(0);
-  }
-
-  @Override
-  public Datum getPartialResult(FunctionContext ctx) {
-    return null;
-  }
-
-  @Override
-  public DataType[] getPartialResultType() {
-    return new DataType[] {DataType.STRING};
-  }
-
-  @Override
-  public Datum terminate(FunctionContext ctx) {
-    try {
-      Thread.sleep(1000);
-      Log.info("Sleepy... z...z...z");
-    } catch (InterruptedException ie) {
-    }
-    return param;
-  }
-}


[07/11] TAJO-57: Recognize Parser and Catalog Standard SQL data types. (hyunsik)

Posted by hy...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-backend/src/main/java/tajo/engine/planner/RangePartitionAlgorithm.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/planner/RangePartitionAlgorithm.java b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/planner/RangePartitionAlgorithm.java
index 570e416..7d4ce4a 100644
--- a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/planner/RangePartitionAlgorithm.java
+++ b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/planner/RangePartitionAlgorithm.java
@@ -20,7 +20,7 @@ package tajo.engine.planner;
 
 import tajo.catalog.Column;
 import tajo.catalog.Schema;
-import tajo.catalog.proto.CatalogProtos;
+import tajo.common.TajoDataTypes.DataType;
 import tajo.datum.Datum;
 import tajo.storage.Tuple;
 import tajo.storage.TupleRange;
@@ -55,33 +55,33 @@ public abstract class RangePartitionAlgorithm {
    * @param end
    * @return
    */
-  public static BigDecimal computeCardinality(CatalogProtos.DataType dataType, Datum start, Datum end,
+  public static BigDecimal computeCardinality(DataType dataType, Datum start, Datum end,
                                               boolean inclusive) {
     BigDecimal columnCard;
 
-    switch (dataType) {
+    switch (dataType.getType()) {
       case CHAR:
         columnCard = new BigDecimal(end.asChar() - start.asChar());
         break;
-      case BYTE:
+      case BIT:
         columnCard = new BigDecimal(end.asByte() - start.asByte());
         break;
-      case SHORT:
-        columnCard = new BigDecimal(end.asShort() - start.asShort());
+      case INT2:
+        columnCard = new BigDecimal(end.asInt2() - start.asInt2());
         break;
-      case INT:
-        columnCard = new BigDecimal(end.asInt() - start.asInt());
+      case INT4:
+        columnCard = new BigDecimal(end.asInt4() - start.asInt4());
         break;
-      case LONG:
-        columnCard = new BigDecimal(end.asLong() - start.asLong());
+      case INT8:
+        columnCard = new BigDecimal(end.asInt8() - start.asInt8());
         break;
-      case FLOAT:
-        columnCard = new BigDecimal(end.asInt() - start.asInt());
+      case FLOAT4:
+        columnCard = new BigDecimal(end.asInt4() - start.asInt4());
         break;
-      case DOUBLE:
-        columnCard = new BigDecimal(end.asLong() - start.asLong());
+      case FLOAT8:
+        columnCard = new BigDecimal(end.asInt8() - start.asInt8());
         break;
-      case STRING:
+      case TEXT:
         columnCard = new BigDecimal(end.asChars().charAt(0) - start.asChars().charAt(0));
         break;
       default:

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-backend/src/main/java/tajo/engine/planner/UniformRangePartition.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/planner/UniformRangePartition.java b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/planner/UniformRangePartition.java
index bee8534..87dac08 100644
--- a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/planner/UniformRangePartition.java
+++ b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/planner/UniformRangePartition.java
@@ -112,8 +112,8 @@ public class UniformRangePartition extends RangePartitionAlgorithm {
     Column column = schema.getColumn(colId);
     BigDecimal candidate;
     boolean overflow = false;
-    switch (column.getDataType()) {
-      case BYTE: {
+    switch (column.getDataType().getType()) {
+      case BIT: {
         candidate = inc.add(new BigDecimal(last.asByte()));
         return new BigDecimal(range.getEnd().get(colId).asByte()).compareTo(candidate) < 0;
       }
@@ -121,27 +121,27 @@ public class UniformRangePartition extends RangePartitionAlgorithm {
         candidate = inc.add(new BigDecimal((int)last.asChar()));
         return new BigDecimal((int)range.getEnd().get(colId).asChar()).compareTo(candidate) < 0;
       }
-      case SHORT: {
-        candidate = inc.add(new BigDecimal(last.asShort()));
-        return new BigDecimal(range.getEnd().get(colId).asShort()).compareTo(candidate) < 0;
+      case INT2: {
+        candidate = inc.add(new BigDecimal(last.asInt2()));
+        return new BigDecimal(range.getEnd().get(colId).asInt2()).compareTo(candidate) < 0;
       }
-      case INT: {
-        candidate = inc.add(new BigDecimal(last.asInt()));
-        return new BigDecimal(range.getEnd().get(colId).asInt()).compareTo(candidate) < 0;
+      case INT4: {
+        candidate = inc.add(new BigDecimal(last.asInt4()));
+        return new BigDecimal(range.getEnd().get(colId).asInt4()).compareTo(candidate) < 0;
       }
-      case LONG: {
-        candidate = inc.add(new BigDecimal(last.asLong()));
-        return new BigDecimal(range.getEnd().get(colId).asLong()).compareTo(candidate) < 0;
+      case INT8: {
+        candidate = inc.add(new BigDecimal(last.asInt8()));
+        return new BigDecimal(range.getEnd().get(colId).asInt8()).compareTo(candidate) < 0;
       }
-      case FLOAT: {
-        candidate = inc.add(new BigDecimal(last.asFloat()));
-        return new BigDecimal(range.getEnd().get(colId).asFloat()).compareTo(candidate) < 0;
+      case FLOAT4: {
+        candidate = inc.add(new BigDecimal(last.asFloat4()));
+        return new BigDecimal(range.getEnd().get(colId).asFloat4()).compareTo(candidate) < 0;
       }
-      case DOUBLE: {
-        candidate = inc.add(new BigDecimal(last.asDouble()));
-        return new BigDecimal(range.getEnd().get(colId).asDouble()).compareTo(candidate) < 0;
+      case FLOAT8: {
+        candidate = inc.add(new BigDecimal(last.asFloat8()));
+        return new BigDecimal(range.getEnd().get(colId).asFloat8()).compareTo(candidate) < 0;
       }
-      case STRING: {
+      case TEXT: {
         candidate = inc.add(new BigDecimal((int)(last.asChars().charAt(0))));
         return new BigDecimal(range.getEnd().get(colId).asChars().charAt(0)).compareTo(candidate) < 0;
       }
@@ -152,54 +152,47 @@ public class UniformRangePartition extends RangePartitionAlgorithm {
   public long incrementAndGetReminder(int colId, Datum last, long inc) {
     Column column = schema.getColumn(colId);
     long reminder = 0;
-    switch (column.getDataType()) {
-      case BYTE: {
+    switch (column.getDataType().getType()) {
+      case BIT: {
         long candidate = last.asByte() + inc;
         byte end = range.getEnd().get(colId).asByte();
-        long longReminder = candidate - end;
-        reminder = longReminder;
+        reminder = candidate - end;
         break;
       }
       case CHAR: {
         long candidate = last.asChar() + inc;
         char end = range.getEnd().get(colId).asChar();
-        long longReminder = candidate - end;
-        reminder = longReminder;
+        reminder = candidate - end;
         break;
       }
-      case INT: {
-        int candidate = (int) (last.asInt() + inc);
-        int end = range.getEnd().get(colId).asInt();
-        int longReminder = candidate - end;
-        reminder = longReminder;
+      case INT4: {
+        int candidate = (int) (last.asInt4() + inc);
+        int end = range.getEnd().get(colId).asInt4();
+        reminder = candidate - end;
         break;
       }
-      case LONG: {
-        long candidate = last.asLong() + inc;
-        long end = range.getEnd().get(colId).asLong();
-        long longReminder = candidate - end;
-        reminder = longReminder;
+      case INT8: {
+        long candidate = last.asInt8() + inc;
+        long end = range.getEnd().get(colId).asInt8();
+        reminder = candidate - end;
         break;
       }
-      case FLOAT: {
-        float candidate = last.asFloat() + inc;
-        float end = range.getEnd().get(colId).asFloat();
-        float longReminder = candidate - end;
-        reminder = (long) longReminder;
+      case FLOAT4: {
+        float candidate = last.asFloat4() + inc;
+        float end = range.getEnd().get(colId).asFloat4();
+        reminder = (long) (candidate - end);
         break;
       }
-      case DOUBLE: {
-        double candidate = last.asDouble() + inc;
-        double end = range.getEnd().get(colId).asDouble();
-        double longReminder = candidate - end;
-        reminder = (long) Math.ceil(longReminder);
+      case FLOAT8: {
+        double candidate = last.asFloat8() + inc;
+        double end = range.getEnd().get(colId).asFloat8();
+        reminder = (long) Math.ceil(candidate - end);
         break;
       }
-      case STRING: {
+      case TEXT: {
         char candidate = ((char)(last.asChars().charAt(0) + inc));
         char end = range.getEnd().get(colId).asChars().charAt(0);
-        char charReminder = (char) (candidate - end);
-        reminder = charReminder;
+        reminder = (char) (candidate - end);
         break;
       }
     }
@@ -263,7 +256,7 @@ public class UniformRangePartition extends RangePartitionAlgorithm {
     Column column;
     for (int i = 0; i < last.size(); i++) {
       column = schema.getColumn(i);
-      switch (column.getDataType()) {
+      switch (column.getDataType().getType()) {
         case CHAR:
           if (overflowFlag[i]) {
             end.put(i, DatumFactory.createChar((char) (range.getStart().get(i).asChar() + incs[i].longValue())));
@@ -271,54 +264,61 @@ public class UniformRangePartition extends RangePartitionAlgorithm {
             end.put(i, DatumFactory.createChar((char) (last.get(i).asChar() + incs[i].longValue())));
           }
           break;
-        case BYTE:
+        case BIT:
           if (overflowFlag[i]) {
-            end.put(i, DatumFactory.createByte((byte) (range.getStart().get(i).asByte() + incs[i].longValue())));
+            end.put(i, DatumFactory.createBit(
+                (byte) (range.getStart().get(i).asByte() + incs[i].longValue())));
           } else {
-            end.put(i, DatumFactory.createByte((byte) (last.get(i).asByte() + incs[i].longValue())));
+            end.put(i, DatumFactory.createBit((byte) (last.get(i).asByte() + incs[i].longValue())));
           }
           break;
-        case SHORT:
+        case INT2:
           if (overflowFlag[i]) {
-            end.put(i, DatumFactory.createShort((short) (range.getStart().get(i).asShort() + incs[i].longValue())));
+            end.put(i, DatumFactory.createInt2(
+                (short) (range.getStart().get(i).asInt2() + incs[i].longValue())));
           } else {
-            end.put(i, DatumFactory.createShort((short) (last.get(i).asShort() + incs[i].longValue())));
+            end.put(i, DatumFactory.createInt2((short) (last.get(i).asInt2() + incs[i].longValue())));
           }
           break;
-        case INT:
+        case INT4:
           if (overflowFlag[i]) {
-            end.put(i, DatumFactory.createInt((int) (range.getStart().get(i).asInt() + incs[i].longValue())));
+            end.put(i, DatumFactory.createInt4(
+                (int) (range.getStart().get(i).asInt4() + incs[i].longValue())));
           } else {
-            end.put(i, DatumFactory.createInt((int) (last.get(i).asInt() + incs[i].longValue())));
+            end.put(i, DatumFactory.createInt4((int) (last.get(i).asInt4() + incs[i].longValue())));
           }
           break;
-        case LONG:
+        case INT8:
           if (overflowFlag[i]) {
-            end.put(i, DatumFactory.createLong(range.getStart().get(i).asInt() + incs[i].longValue()));
+            end.put(i, DatumFactory.createInt8(
+                range.getStart().get(i).asInt4() + incs[i].longValue()));
           } else {
-            end.put(i, DatumFactory.createLong(last.get(i).asLong() + incs[i].longValue()));
+            end.put(i, DatumFactory.createInt8(last.get(i).asInt8() + incs[i].longValue()));
           }
           break;
-        case FLOAT:
+        case FLOAT4:
           if (overflowFlag[i]) {
-            end.put(i, DatumFactory.createFloat(range.getStart().get(i).asFloat() + incs[i].longValue()));
+            end.put(i, DatumFactory.createFloat4(
+                range.getStart().get(i).asFloat4() + incs[i].longValue()));
           } else {
-            end.put(i, DatumFactory.createFloat(last.get(i).asFloat() + incs[i].longValue()));
+            end.put(i, DatumFactory.createFloat4(last.get(i).asFloat4() + incs[i].longValue()));
           }
           break;
-        case DOUBLE:
+        case FLOAT8:
           if (overflowFlag[i]) {
-            end.put(i, DatumFactory.createDouble(range.getStart().get(i).asDouble() + incs[i].longValue()));
+            end.put(i, DatumFactory.createFloat8(
+                range.getStart().get(i).asFloat8() + incs[i].longValue()));
           } else {
-            end.put(i, DatumFactory.createDouble(last.get(i).asDouble() + incs[i].longValue()));
+            end.put(i, DatumFactory.createFloat8(last.get(i).asFloat8() + incs[i].longValue()));
           }
           break;
-        case STRING:
+        case TEXT:
           if (overflowFlag[i]) {
-            end.put(i, DatumFactory.createString(((char)(range.getStart().get(i).asChars().charAt(0)
+            end.put(i, DatumFactory.createText(((char) (range.getStart().get(i).asChars().charAt(0)
                 + incs[i].longValue())) + ""));
           } else {
-            end.put(i, DatumFactory.createString(((char)(last.get(i).asChars().charAt(0) + incs[i].longValue())) + ""));
+            end.put(i, DatumFactory.createText(
+                ((char) (last.get(i).asChars().charAt(0) + incs[i].longValue())) + ""));
           }
           break;
         default:

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-backend/src/main/java/tajo/engine/planner/physical/ExternalSortExec.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/planner/physical/ExternalSortExec.java b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/planner/physical/ExternalSortExec.java
index 20fa0b0..62877e4 100644
--- a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/planner/physical/ExternalSortExec.java
+++ b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/planner/physical/ExternalSortExec.java
@@ -21,7 +21,7 @@ package tajo.engine.planner.physical;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import tajo.TaskAttemptContext;
-import tajo.catalog.TCatUtil;
+import tajo.catalog.CatalogUtil;
 import tajo.catalog.TableMeta;
 import tajo.catalog.proto.CatalogProtos.StoreType;
 import tajo.conf.TajoConf.ConfVars;
@@ -56,7 +56,7 @@ public class ExternalSortExec extends SortExec {
 
     this.sortTmpDir = new Path(context.getWorkDir(), UUID.randomUUID().toString());
     this.localFS = FileSystem.getLocal(context.getConf());
-    meta = TCatUtil.newTableMeta(inSchema, StoreType.ROWFILE);
+    meta = CatalogUtil.newTableMeta(inSchema, StoreType.ROWFILE);
   }
 
   public void init() throws IOException {
@@ -70,7 +70,7 @@ public class ExternalSortExec extends SortExec {
 
   private void sortAndStoreChunk(int chunkId, List<Tuple> tupleSlots)
       throws IOException {
-    TableMeta meta = TCatUtil.newTableMeta(inSchema, StoreType.RAW);
+    TableMeta meta = CatalogUtil.newTableMeta(inSchema, StoreType.RAW);
     Collections.sort(tupleSlots, getComparator());
     // TODO - RawFile requires the local file path.
     // So, I add the scheme 'file:/' to path. But, it should be improved.

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-backend/src/main/java/tajo/engine/planner/physical/IndexedStoreExec.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/planner/physical/IndexedStoreExec.java b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/planner/physical/IndexedStoreExec.java
index 4a693b4..d0c6a7c 100644
--- a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/planner/physical/IndexedStoreExec.java
+++ b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/planner/physical/IndexedStoreExec.java
@@ -67,7 +67,7 @@ public class IndexedStoreExec extends UnaryPhysicalExec {
     this.comp = new TupleComparator(keySchema, sortSpecs);
     Path storeTablePath = new Path(context.getWorkDir(), "output");
     LOG.info("Output data directory: " + storeTablePath);
-    this.meta = TCatUtil
+    this.meta = CatalogUtil
         .newTableMeta(this.outSchema, CatalogProtos.StoreType.CSV);
     FileSystem fs = new RawLocalFileSystem();
     fs.mkdirs(storeTablePath);

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-backend/src/main/java/tajo/engine/planner/physical/JoinTupleComparator.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/planner/physical/JoinTupleComparator.java b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/planner/physical/JoinTupleComparator.java
index 88c8485..0ed11cd 100644
--- a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/planner/physical/JoinTupleComparator.java
+++ b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/planner/physical/JoinTupleComparator.java
@@ -22,7 +22,7 @@ import com.google.common.base.Preconditions;
 import tajo.catalog.Schema;
 import tajo.catalog.SortSpec;
 import tajo.datum.Datum;
-import tajo.datum.DatumType;
+import tajo.datum.NullDatum;
 import tajo.storage.Tuple;
 
 import java.util.Comparator;
@@ -64,11 +64,11 @@ public class JoinTupleComparator implements Comparator<Tuple> {
       outer = outerTuple.get(outerSortKeyIds[i]);
       inner = innerTuple.get(innerSortKeyIds[i]);
 
-      if (outer.type() == DatumType.NULL || inner.type() == DatumType.NULL) {
+      if (outer instanceof NullDatum || inner instanceof NullDatum) {
         if (!outer.equals(inner)) {
-          if (outer.type() == DatumType.NULL) {
+          if (outer instanceof NullDatum) {
             compVal = 1;
-          } else if (inner.type() == DatumType.NULL) {
+          } else if (inner instanceof NullDatum) {
             compVal = -1;
           }
         } else {

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-backend/src/main/java/tajo/engine/planner/physical/PartitionedStoreExec.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/planner/physical/PartitionedStoreExec.java b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/planner/physical/PartitionedStoreExec.java
index 056d963..a42f9bb 100644
--- a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/planner/physical/PartitionedStoreExec.java
+++ b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/planner/physical/PartitionedStoreExec.java
@@ -26,8 +26,8 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.RawLocalFileSystem;
 import tajo.TaskAttemptContext;
+import tajo.catalog.CatalogUtil;
 import tajo.catalog.Column;
-import tajo.catalog.TCatUtil;
 import tajo.catalog.TableMeta;
 import tajo.catalog.proto.CatalogProtos.StoreType;
 import tajo.catalog.statistics.StatisticsUtil;
@@ -69,7 +69,7 @@ public final class PartitionedStoreExec extends UnaryPhysicalExec {
     super(context, plan.getInSchema(), plan.getOutSchema(), child);
     Preconditions.checkArgument(plan.hasPartitionKey());
     this.plan = plan;
-    this.meta = TCatUtil.newTableMeta(this.outSchema, StoreType.CSV);
+    this.meta = CatalogUtil.newTableMeta(this.outSchema, StoreType.CSV);
     
     // about the partitions
     this.numPartitions = this.plan.getNumPartitions();

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-backend/src/main/java/tajo/engine/planner/physical/StoreTableExec.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/planner/physical/StoreTableExec.java b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/planner/physical/StoreTableExec.java
index 70a7a3c..f250438 100644
--- a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/planner/physical/StoreTableExec.java
+++ b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/planner/physical/StoreTableExec.java
@@ -25,7 +25,7 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.RawLocalFileSystem;
 import tajo.TaskAttemptContext;
-import tajo.catalog.TCatUtil;
+import tajo.catalog.CatalogUtil;
 import tajo.catalog.TableMeta;
 import tajo.engine.planner.logical.StoreTableNode;
 import tajo.storage.Appender;
@@ -59,9 +59,9 @@ public class StoreTableExec extends UnaryPhysicalExec {
 
     TableMeta meta;
     if (plan.hasOptions()) {
-      meta = TCatUtil.newTableMeta(outSchema, plan.getStorageType(), plan.getOptions());
+      meta = CatalogUtil.newTableMeta(outSchema, plan.getStorageType(), plan.getOptions());
     } else {
-      meta = TCatUtil.newTableMeta(outSchema, plan.getStorageType());
+      meta = CatalogUtil.newTableMeta(outSchema, plan.getStorageType());
     }
 
     if (context.isInterQuery()) {

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-backend/src/main/java/tajo/engine/query/ResultSetImpl.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/query/ResultSetImpl.java b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/query/ResultSetImpl.java
index abe78c0..89d52e0 100644
--- a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/query/ResultSetImpl.java
+++ b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/query/ResultSetImpl.java
@@ -28,7 +28,7 @@ import tajo.catalog.TableMeta;
 import tajo.catalog.TableMetaImpl;
 import tajo.catalog.proto.CatalogProtos.TableProto;
 import tajo.datum.Datum;
-import tajo.datum.DatumType;
+import tajo.datum.NullDatum;
 import tajo.exception.UnsupportedException;
 import tajo.storage.Fragment;
 import tajo.storage.MergeScanner;
@@ -585,7 +585,7 @@ public class ResultSetImpl implements ResultSet {
   public double getDouble(int fieldId) throws SQLException {
     Datum datum = cur.getDouble(fieldId - 1);
     handleNull(datum);
-    return datum.asDouble();
+    return datum.asFloat8();
   }
 
   /*
@@ -597,7 +597,7 @@ public class ResultSetImpl implements ResultSet {
   public double getDouble(String name) throws SQLException {
     Datum datum = cur.get(findColumn(name));
     handleNull(datum);
-    return datum.asDouble();
+    return datum.asFloat8();
   }
 
   /*
@@ -629,7 +629,7 @@ public class ResultSetImpl implements ResultSet {
   public float getFloat(int fieldId) throws SQLException {
     Datum datum = cur.get(fieldId - 1);
     handleNull(datum);
-    return datum.asFloat();
+    return datum.asFloat4();
   }
 
   /*
@@ -641,7 +641,7 @@ public class ResultSetImpl implements ResultSet {
   public float getFloat(String name) throws SQLException {
     Datum datum = cur.get(findColumn(name));
     handleNull(datum);
-    return datum.asFloat();
+    return datum.asFloat4();
   }
 
   /*
@@ -663,7 +663,7 @@ public class ResultSetImpl implements ResultSet {
   public int getInt(int fieldId) throws SQLException {
     Datum datum = cur.get(fieldId - 1);
     handleNull(datum);
-    return datum.asInt();
+    return datum.asInt4();
   }
 
   /*
@@ -675,7 +675,7 @@ public class ResultSetImpl implements ResultSet {
   public int getInt(String name) throws SQLException {
     Datum datum = cur.get(findColumn(name));
     handleNull(datum);
-    return datum.asInt();
+    return datum.asInt4();
   }
 
   /*
@@ -687,7 +687,7 @@ public class ResultSetImpl implements ResultSet {
   public long getLong(int fieldId) throws SQLException {
     Datum datum = cur.get(fieldId - 1);
     handleNull(datum);
-    return datum.asLong();
+    return datum.asInt8();
   }
 
   /*
@@ -699,7 +699,7 @@ public class ResultSetImpl implements ResultSet {
   public long getLong(String name) throws SQLException {
     Datum datum = cur.get(findColumn(name));
     handleNull(datum);
-    return datum.asLong();
+    return datum.asInt8();
   }
 
   /*
@@ -907,7 +907,7 @@ public class ResultSetImpl implements ResultSet {
   public short getShort(int fieldId) throws SQLException {
     Datum datum = cur.get(fieldId - 1);
     handleNull(datum);
-    return datum.asShort();
+    return datum.asInt2();
   }
 
   /*
@@ -919,7 +919,7 @@ public class ResultSetImpl implements ResultSet {
   public short getShort(String name) throws SQLException {
     Datum datum = cur.get(findColumn(name));
     handleNull(datum);
-    return datum.asShort();
+    return datum.asInt2();
   }
 
   /*
@@ -2195,6 +2195,6 @@ public class ResultSetImpl implements ResultSet {
   }
 
   private void handleNull(Datum d) {
-    wasNull = (d.type() == DatumType.NULL);
+    wasNull = (d instanceof NullDatum);
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-backend/src/main/java/tajo/engine/query/ResultSetMetaDataImpl.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/query/ResultSetMetaDataImpl.java b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/query/ResultSetMetaDataImpl.java
index ee9b7cf..baea966 100644
--- a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/query/ResultSetMetaDataImpl.java
+++ b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/query/ResultSetMetaDataImpl.java
@@ -22,7 +22,7 @@
 package tajo.engine.query;
 
 import tajo.catalog.TableMeta;
-import tajo.catalog.proto.CatalogProtos.DataType;
+import tajo.common.TajoDataTypes.DataType;
 import tajo.exception.UnsupportedException;
 
 import java.nio.channels.UnsupportedAddressTypeException;
@@ -109,35 +109,35 @@ public class ResultSetMetaDataImpl implements ResultSetMetaData {
   public int getColumnType(int column) throws SQLException {
     // TODO
     DataType type = meta.getSchema().getColumn(column - 1).getDataType();
-    switch (type) {
-    case BOOLEAN:
-      return Types.BOOLEAN;
-    case BIGDECIMAL:
-      return Types.DECIMAL;
-    case BIGINT:
-      return Types.BIGINT;
-    case BYTE:
-      return Types.TINYINT;
-    case BYTES:
-      return Types.VARBINARY;
-    case CHAR:
-      return Types.CHAR;
-    case DATE:
-      return Types.DATE;
-    case DOUBLE:
-      return Types.DOUBLE;
-    case FLOAT:
-      return Types.FLOAT;
-    case INT:
-      return Types.INTEGER;
-    case LONG:
-      return Types.BIGINT;
-    case SHORT:
-      return Types.SMALLINT;
-    case STRING:
-      return Types.VARCHAR;
-    default:
-      throw new UnsupportedException();
+    switch (type.getType()) {
+      case BOOLEAN:
+        return Types.BOOLEAN;
+      case INT1:
+        return Types.TINYINT;
+      case INT2:
+        return Types.SMALLINT;
+      case INT4:
+        return Types.INTEGER;
+      case INT8:
+        return Types.BIGINT;
+      case FLOAT4:
+        return Types.FLOAT;
+      case FLOAT8:
+        return Types.DOUBLE;
+      case DECIMAL:
+        return Types.DECIMAL;
+      case VARBINARY:
+        return Types.VARBINARY;
+      case CHAR:
+        return Types.CHAR;
+      case DATE:
+        return Types.DATE;
+      case VARCHAR:
+        return Types.VARCHAR;
+      case TEXT:
+        return Types.VARCHAR;
+      default:
+        throw new UnsupportedException();
     }
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-backend/src/main/java/tajo/engine/query/exception/InvalidCastException.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/query/exception/InvalidCastException.java b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/query/exception/InvalidCastException.java
deleted file mode 100644
index 9732391..0000000
--- a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/query/exception/InvalidCastException.java
+++ /dev/null
@@ -1,35 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-/**
- * 
- */
-package tajo.engine.query.exception;
-
-import tajo.catalog.proto.CatalogProtos.DataType;
-
-public class InvalidCastException extends InvalidQueryException {
-	private static final long serialVersionUID = -5090530469575858320L;
-
-	/**
-	 * @param message
-	 */
-	public InvalidCastException(DataType src, DataType target) {
-		super("Error: cannot cast " + src + " to " + target);
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-backend/src/main/java/tajo/engine/query/exception/InvalidEvalException.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/query/exception/InvalidEvalException.java b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/query/exception/InvalidEvalException.java
deleted file mode 100644
index 121f89b..0000000
--- a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/query/exception/InvalidEvalException.java
+++ /dev/null
@@ -1,30 +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 tajo.engine.query.exception;
-
-public class InvalidEvalException extends InvalidQueryException {
-	private static final long serialVersionUID = -2897003028483298256L;
-
-	/**
-	 * @param message
-	 */
-	public InvalidEvalException(String message) {
-		super(message);
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-backend/src/main/java/tajo/engine/utils/SchemaUtil.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/utils/SchemaUtil.java b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/utils/SchemaUtil.java
index bde92c1..894b0d4 100644
--- a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/utils/SchemaUtil.java
+++ b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/utils/SchemaUtil.java
@@ -20,7 +20,7 @@ package tajo.engine.utils;
 
 import tajo.catalog.Column;
 import tajo.catalog.Schema;
-import tajo.catalog.proto.CatalogProtos.DataType;
+import tajo.common.TajoDataTypes.DataType;
 import tajo.engine.parser.QueryBlock;
 
 import java.util.Collection;
@@ -56,7 +56,7 @@ public class SchemaUtil {
     for (Column outer : left.getColumns()) {
       for (Column inner : right.getColumns()) {
         if (outer.getColumnName().equals(inner.getColumnName()) &&
-            outer.getDataType() == inner.getDataType()) {
+            outer.getDataType().equals(inner.getDataType())) {
           common.addColumn(outer.getColumnName(), outer.getDataType());
         }
       }

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-backend/src/main/java/tajo/engine/utils/TupleUtil.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/utils/TupleUtil.java b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/utils/TupleUtil.java
index be4a595..8b95816 100644
--- a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/utils/TupleUtil.java
+++ b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/utils/TupleUtil.java
@@ -26,15 +26,14 @@ import org.apache.commons.logging.LogFactory;
 import tajo.catalog.Column;
 import tajo.catalog.Schema;
 import tajo.catalog.SortSpec;
-import tajo.catalog.proto.CatalogProtos;
 import tajo.catalog.statistics.ColumnStat;
-import tajo.datum.*;
+import tajo.datum.Datum;
+import tajo.datum.DatumFactory;
 import tajo.engine.planner.PlannerUtil;
 import tajo.storage.RowStoreUtil;
 import tajo.storage.Tuple;
 import tajo.storage.TupleRange;
 import tajo.storage.VTuple;
-import tajo.util.Bytes;
 import tajo.worker.dataserver.HttpUtil;
 
 import java.io.UnsupportedEncodingException;
@@ -63,29 +62,29 @@ public class TupleUtil {
     long columnCard;
     for (int i = 0; i < schema.getColumnNum(); i++) {
       col = schema.getColumn(i);
-      switch (col.getDataType()) {
+      switch (col.getDataType().getType()) {
         case CHAR:
           columnCard = end.get(i).asChar() - start.get(i).asChar();
           break;
-        case BYTE:
+        case BIT:
           columnCard = end.get(i).asByte() - start.get(i).asByte();
           break;
-        case SHORT:
-          columnCard = end.get(i).asShort() - start.get(i).asShort();
+        case INT2:
+          columnCard = end.get(i).asInt2() - start.get(i).asInt2();
           break;
-        case INT:
-          columnCard = end.get(i).asInt() - start.get(i).asInt();
+        case INT4:
+          columnCard = end.get(i).asInt4() - start.get(i).asInt4();
           break;
-        case LONG:
-          columnCard = end.get(i).asLong() - start.get(i).asLong();
+        case INT8:
+          columnCard = end.get(i).asInt8() - start.get(i).asInt8();
           break;
-        case FLOAT:
-          columnCard = end.get(i).asInt() - start.get(i).asInt();
+        case FLOAT4:
+          columnCard = end.get(i).asInt4() - start.get(i).asInt4();
           break;
-        case DOUBLE:
-          columnCard = end.get(i).asLong() - start.get(i).asLong();
+        case FLOAT8:
+          columnCard = end.get(i).asInt8() - start.get(i).asInt8();
           break;
-        case STRING:
+        case TEXT:
           columnCard = end.get(i).asChars().charAt(0) - start.get(i).asChars().charAt(0);
           break;
         default:
@@ -113,7 +112,7 @@ public class TupleUtil {
     for (int i = 0; i < schema.getColumnNum(); i++) {
       col = schema.getColumn(i);
       prevValues[i] = start.get(i);
-      switch (col.getDataType()) {
+      switch (col.getDataType().getType()) {
         case CHAR:
           int sChar = start.get(i).asChar();
           int eChar = end.get(i).asChar();
@@ -123,8 +122,8 @@ public class TupleUtil {
           } else {
             rangeChar = 1;
           }
-          term[i] = DatumFactory.createInt(rangeChar);
-        case BYTE:
+          term[i] = DatumFactory.createInt4(rangeChar);
+        case BIT:
           byte sByte = start.get(i).asByte();
           byte eByte = end.get(i).asByte();
           int rangeByte;
@@ -133,68 +132,68 @@ public class TupleUtil {
           } else {
             rangeByte = 1;
           }
-          term[i] = DatumFactory.createByte((byte)rangeByte);
+          term[i] = DatumFactory.createBit((byte) rangeByte);
           break;
 
-        case SHORT:
-          short sShort = start.get(i).asShort();
-          short eShort = end.get(i).asShort();
+        case INT2:
+          short sShort = start.get(i).asInt2();
+          short eShort = end.get(i).asInt2();
           int rangeShort;
           if ((eShort - sShort) > partNum) {
             rangeShort = (eShort - sShort) / partNum;
           } else {
             rangeShort = 1;
           }
-          term[i] = DatumFactory.createShort((short) rangeShort);
+          term[i] = DatumFactory.createInt2((short) rangeShort);
           break;
 
-        case INT:
-          int sInt = start.get(i).asInt();
-          int eInt = end.get(i).asInt();
+        case INT4:
+          int sInt = start.get(i).asInt4();
+          int eInt = end.get(i).asInt4();
           int rangeInt;
           if ((eInt - sInt) > partNum) {
             rangeInt = (eInt - sInt) / partNum;
           } else {
             rangeInt = 1;
           }
-          term[i] = DatumFactory.createInt(rangeInt);
+          term[i] = DatumFactory.createInt4(rangeInt);
           break;
 
-        case LONG:
-          long sLong = start.get(i).asLong();
-          long eLong = end.get(i).asLong();
+        case INT8:
+          long sLong = start.get(i).asInt8();
+          long eLong = end.get(i).asInt8();
           long rangeLong;
           if ((eLong - sLong) > partNum) {
             rangeLong = ((eLong - sLong) / partNum);
           } else {
             rangeLong = 1;
           }
-          term[i] = DatumFactory.createLong(rangeLong);
+          term[i] = DatumFactory.createInt8(rangeLong);
           break;
 
-        case FLOAT:
-          float sFloat = start.get(i).asFloat();
-          float eFloat = end.get(i).asFloat();
+        case FLOAT4:
+          float sFloat = start.get(i).asFloat4();
+          float eFloat = end.get(i).asFloat4();
           float rangeFloat;
           if ((eFloat - sFloat) > partNum) {
             rangeFloat = ((eFloat - sFloat) / partNum);
           } else {
             rangeFloat = 1;
           }
-          term[i] = DatumFactory.createFloat(rangeFloat);
+          term[i] = DatumFactory.createFloat4(rangeFloat);
           break;
-        case DOUBLE:
-          double sDouble = start.get(i).asDouble();
-          double eDouble = end.get(i).asDouble();
+        case FLOAT8:
+          double sDouble = start.get(i).asFloat8();
+          double eDouble = end.get(i).asFloat8();
           double rangeDouble;
           if ((eDouble - sDouble) > partNum) {
             rangeDouble = ((eDouble - sDouble) / partNum);
           } else {
             rangeDouble = 1;
           }
-          term[i] = DatumFactory.createDouble(rangeDouble);
+          term[i] = DatumFactory.createFloat8(rangeDouble);
           break;
-        case STRING:
+        case TEXT:
           char sChars = start.get(i).asChars().charAt(0);
           char eChars = end.get(i).asChars().charAt(0);
           int rangeString;
@@ -203,11 +202,11 @@ public class TupleUtil {
           } else {
             rangeString = 1;
           }
-          term[i] = DatumFactory.createString(((char)rangeString) + "");
+          term[i] = DatumFactory.createText(((char) rangeString) + "");
           break;
-        case IPv4:
+        case INET4:
           throw new UnsupportedOperationException();
-        case BYTES:
+        case BLOB:
           throw new UnsupportedOperationException();
         default:
           throw new UnsupportedOperationException();
@@ -220,7 +219,7 @@ public class TupleUtil {
       for (int i = 0; i < schema.getColumnNum(); i++) {
         col = schema.getColumn(i);
         sTuple.put(i, prevValues[i]);
-        switch (col.getDataType()) {
+        switch (col.getDataType().getType()) {
           case CHAR:
             char endChar = (char) (prevValues[i].asChar() + term[i].asChar());
             if (endChar > end.get(i).asByte()) {
@@ -230,80 +229,80 @@ public class TupleUtil {
             }
             prevValues[i] = DatumFactory.createChar(endChar);
             break;
-          case BYTE:
+          case BIT:
             byte endByte = (byte) (prevValues[i].asByte() + term[i].asByte());
             if (endByte > end.get(i).asByte()) {
               eTuple.put(i, end.get(i));
             } else {
-              eTuple.put(i, DatumFactory.createByte(endByte));
+              eTuple.put(i, DatumFactory.createBit(endByte));
             }
-            prevValues[i] = DatumFactory.createByte(endByte);
+            prevValues[i] = DatumFactory.createBit(endByte);
             break;
-          case SHORT:
-            int endShort = (short) (prevValues[i].asShort() + term[i].asShort());
-            if (endShort > end.get(i).asShort()) {
+          case INT2:
+            int endShort = (short) (prevValues[i].asInt2() + term[i].asInt2());
+            if (endShort > end.get(i).asInt2()) {
               eTuple.put(i, end.get(i));
             } else {
               // TODO - to consider overflow
-              eTuple.put(i, DatumFactory.createShort((short) endShort));
+              eTuple.put(i, DatumFactory.createInt2((short) endShort));
             }
-            prevValues[i] = DatumFactory.createShort((short) endShort);
+            prevValues[i] = DatumFactory.createInt2((short) endShort);
             break;
-          case INT:
-            int endInt = (prevValues[i].asInt() + term[i].asInt());
-            if (endInt > end.get(i).asInt()) {
+          case INT4:
+            int endInt = (prevValues[i].asInt4() + term[i].asInt4());
+            if (endInt > end.get(i).asInt4()) {
               eTuple.put(i, end.get(i));
             } else {
               // TODO - to consider overflow
-              eTuple.put(i, DatumFactory.createInt(endInt));
+              eTuple.put(i, DatumFactory.createInt4(endInt));
             }
-            prevValues[i] = DatumFactory.createInt(endInt);
+            prevValues[i] = DatumFactory.createInt4(endInt);
             break;
 
-          case LONG:
-            long endLong = (prevValues[i].asLong() + term[i].asLong());
-            if (endLong > end.get(i).asLong()) {
+          case INT8:
+            long endLong = (prevValues[i].asInt8() + term[i].asInt8());
+            if (endLong > end.get(i).asInt8()) {
               eTuple.put(i, end.get(i));
             } else {
               // TODO - to consider overflow
-              eTuple.put(i, DatumFactory.createLong(endLong));
+              eTuple.put(i, DatumFactory.createInt8(endLong));
             }
-            prevValues[i] = DatumFactory.createLong(endLong);
+            prevValues[i] = DatumFactory.createInt8(endLong);
             break;
 
-          case FLOAT:
-            float endFloat = (prevValues[i].asFloat() + term[i].asFloat());
-            if (endFloat > end.get(i).asFloat()) {
+          case FLOAT4:
+            float endFloat = (prevValues[i].asFloat4() + term[i].asFloat4());
+            if (endFloat > end.get(i).asFloat4()) {
               eTuple.put(i, end.get(i));
             } else {
               // TODO - to consider overflow
-              eTuple.put(i, DatumFactory.createFloat(endFloat));
+              eTuple.put(i, DatumFactory.createFloat4(endFloat));
             }
-            prevValues[i] = DatumFactory.createFloat(endFloat);
+            prevValues[i] = DatumFactory.createFloat4(endFloat);
             break;
-          case DOUBLE:
-            double endDouble = (prevValues[i].asDouble() + term[i].asDouble());
-            if (endDouble > end.get(i).asDouble()) {
+          case FLOAT8:
+            double endDouble = (prevValues[i].asFloat8() + term[i].asFloat8());
+            if (endDouble > end.get(i).asFloat8()) {
               eTuple.put(i, end.get(i));
             } else {
               // TODO - to consider overflow
-              eTuple.put(i, DatumFactory.createDouble(endDouble));
+              eTuple.put(i, DatumFactory.createFloat8(endDouble));
             }
-            prevValues[i] = DatumFactory.createDouble(endDouble);
+            prevValues[i] = DatumFactory.createFloat8(endDouble);
             break;
-          case STRING:
+          case TEXT:
             String endString = ((char)(prevValues[i].asChars().charAt(0) + term[i].asChars().charAt(0))) + "";
             if (endString.charAt(0) > end.get(i).asChars().charAt(0)) {
               eTuple.put(i, end.get(i));
             } else {
               // TODO - to consider overflow
-              eTuple.put(i, DatumFactory.createString(endString));
+              eTuple.put(i, DatumFactory.createText(endString));
             }
-            prevValues[i] = DatumFactory.createString(endString);
+            prevValues[i] = DatumFactory.createText(endString);
             break;
-          case IPv4:
+          case INET4:
             throw new UnsupportedOperationException();
-          case BYTES:
+          case BLOB:
             throw new UnsupportedOperationException();
           default:
             throw new UnsupportedOperationException();
@@ -386,64 +385,4 @@ public class TupleUtil {
     }
     return new TupleRange(target, startTuple, endTuple);
   }
-
-  public static Datum createFromBytes(CatalogProtos.DataType type, byte [] bytes) {
-    switch (type) {
-      case BOOLEAN:
-        return new BoolDatum(bytes);
-      case BYTE:
-        return new ByteDatum(bytes);
-      case CHAR:
-        return new CharDatum(bytes);
-      case SHORT:
-        return new ShortDatum(bytes);
-      case INT:
-        return new IntDatum(bytes);
-      case LONG:
-        return new LongDatum(bytes);
-      case FLOAT:
-        return new FloatDatum(bytes);
-      case DOUBLE:
-        return new DoubleDatum(bytes);
-      case STRING:
-        return new StringDatum(bytes);
-      case IPv4:
-        return new IPv4Datum(bytes);
-      default: throw new UnsupportedOperationException(type + " is not supported yet");
-    }
-  }
-
-  private final static byte [] TRUE_BYTES = new byte[] {(byte)1};
-  private final static byte [] FALSE_BYTES = new byte[] {(byte)0};
-
-  public static byte [] toBytes(CatalogProtos.DataType type, Datum datum) {
-    switch (type) {
-      case BOOLEAN:
-        if (datum.asBool()) {
-          return TRUE_BYTES;
-        } else {
-          return FALSE_BYTES;
-        }
-      case BYTE:
-      case CHAR:
-        return new byte[] {datum.asByte()};
-
-      case SHORT:
-        return Bytes.toBytes(datum.asShort());
-      case INT:
-        return Bytes.toBytes(datum.asInt());
-      case LONG:
-        return Bytes.toBytes(datum.asLong());
-      case FLOAT:
-        return Bytes.toBytes(datum.asFloat());
-      case DOUBLE:
-        return Bytes.toBytes(datum.asDouble());
-      case STRING:
-        return Bytes.toBytes(datum.asChars());
-      case IPv4:
-        return datum.asByteArray();
-
-      default: throw new UnsupportedOperationException(type + " is not supported yet");
-    }
-  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-backend/src/main/java/tajo/master/GlobalEngine.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/tajo/master/GlobalEngine.java b/tajo-core/tajo-core-backend/src/main/java/tajo/master/GlobalEngine.java
index 255e500..080dbdd 100644
--- a/tajo-core/tajo-core-backend/src/main/java/tajo/master/GlobalEngine.java
+++ b/tajo-core/tajo-core-backend/src/main/java/tajo/master/GlobalEngine.java
@@ -34,7 +34,7 @@ import org.apache.hadoop.yarn.service.AbstractService;
 import org.apache.hadoop.yarn.util.Records;
 import tajo.QueryConf;
 import tajo.QueryId;
-import tajo.catalog.TCatUtil;
+import tajo.catalog.CatalogUtil;
 import tajo.catalog.TableDesc;
 import tajo.catalog.TableMeta;
 import tajo.catalog.statistics.TableStat;
@@ -115,10 +115,10 @@ public class GlobalEngine extends AbstractService {
     CreateTableNode createTable = (CreateTableNode) root.getSubNode();
     TableMeta meta;
     if (createTable.hasOptions()) {
-      meta = TCatUtil.newTableMeta(createTable.getSchema(),
+      meta = CatalogUtil.newTableMeta(createTable.getSchema(),
           createTable.getStorageType(), createTable.getOptions());
     } else {
-      meta = TCatUtil.newTableMeta(createTable.getSchema(),
+      meta = CatalogUtil.newTableMeta(createTable.getSchema(),
           createTable.getStorageType());
     }
 
@@ -138,7 +138,7 @@ public class GlobalEngine extends AbstractService {
     meta.setStat(stat);
 
     StorageUtil.writeTableMeta(context.getConf(), createTable.getPath(), meta);
-    TableDesc desc = TCatUtil.newTableDesc(createTable.getTableName(), meta,
+    TableDesc desc = CatalogUtil.newTableDesc(createTable.getTableName(), meta,
         createTable.getPath());
     context.getCatalog().addTable(desc);
     return desc.getId();

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-backend/src/main/java/tajo/master/GlobalPlanner.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/tajo/master/GlobalPlanner.java b/tajo-core/tajo-core-backend/src/main/java/tajo/master/GlobalPlanner.java
index b9fb587..376f15d 100644
--- a/tajo-core/tajo-core-backend/src/main/java/tajo/master/GlobalPlanner.java
+++ b/tajo-core/tajo-core-backend/src/main/java/tajo/master/GlobalPlanner.java
@@ -482,7 +482,7 @@ public class GlobalPlanner {
     
     if (union.getOuterNode().getType() == ExprType.STORE) {
       outerStore = (StoreTableNode) union.getOuterNode();
-      TableMeta outerMeta = TCatUtil.newTableMeta(outerStore.getOutSchema(),
+      TableMeta outerMeta = CatalogUtil.newTableMeta(outerStore.getOutSchema(),
           StoreType.CSV);
       insertOuterScan(union, outerStore.getTableName(), outerMeta);
       prev = convertMap.get(outerStore);
@@ -498,7 +498,7 @@ public class GlobalPlanner {
     
     if (union.getInnerNode().getType() == ExprType.STORE) {
       innerStore = (StoreTableNode) union.getInnerNode();
-      TableMeta innerMeta = TCatUtil.newTableMeta(innerStore.getOutSchema(),
+      TableMeta innerMeta = CatalogUtil.newTableMeta(innerStore.getOutSchema(),
           StoreType.CSV);
       insertInnerScan(union, innerStore.getTableName(), innerMeta);
       prev = convertMap.get(innerStore);
@@ -597,7 +597,7 @@ public class GlobalPlanner {
     // outer
     if (join.getOuterNode().getType() == ExprType.STORE) {
       outerStore = (StoreTableNode) join.getOuterNode();
-      TableMeta outerMeta = TCatUtil.newTableMeta(outerStore.getOutSchema(),
+      TableMeta outerMeta = CatalogUtil.newTableMeta(outerStore.getOutSchema(),
           StoreType.CSV);
       insertOuterScan(join, outerStore.getTableName(), outerMeta);
       prev = convertMap.get(outerStore);
@@ -617,7 +617,7 @@ public class GlobalPlanner {
     // inner
     if (join.getInnerNode().getType() == ExprType.STORE) {
       innerStore = (StoreTableNode) join.getInnerNode();
-      TableMeta innerMeta = TCatUtil.newTableMeta(innerStore.getOutSchema(),
+      TableMeta innerMeta = CatalogUtil.newTableMeta(innerStore.getOutSchema(),
           StoreType.CSV);
       insertInnerScan(join, innerStore.getTableName(), innerMeta);
       prev = convertMap.get(innerStore);
@@ -654,7 +654,7 @@ public class GlobalPlanner {
     
     if (union.getOuterNode().getType() == ExprType.STORE) {
       store = (StoreTableNode) union.getOuterNode();
-      meta = TCatUtil.newTableMeta(store.getOutSchema(), StoreType.CSV);
+      meta = CatalogUtil.newTableMeta(store.getOutSchema(), StoreType.CSV);
       insertOuterScan(union, store.getTableName(), meta);
       prev = convertMap.get(store);
       if (prev != null) {
@@ -673,7 +673,7 @@ public class GlobalPlanner {
     
     if (union.getInnerNode().getType() == ExprType.STORE) {
       store = (StoreTableNode) union.getInnerNode();
-      meta = TCatUtil.newTableMeta(store.getOutSchema(), StoreType.CSV);
+      meta = CatalogUtil.newTableMeta(store.getOutSchema(), StoreType.CSV);
       insertInnerScan(union, store.getTableName(), meta);
       prev = convertMap.get(store);
       if (prev != null) {
@@ -762,7 +762,7 @@ public class GlobalPlanner {
   
   private LogicalNode insertOuterScan(BinaryNode parent, String tableId,
       TableMeta meta) throws IOException {
-    TableDesc desc = TCatUtil.newTableDesc(tableId, meta, sm.getTablePath(tableId));
+    TableDesc desc = CatalogUtil.newTableDesc(tableId, meta, sm.getTablePath(tableId));
     ScanNode scan = new ScanNode(new FromTable(desc));
     scan.setLocal(true);
     scan.setInSchema(meta.getSchema());
@@ -773,7 +773,7 @@ public class GlobalPlanner {
   
   private LogicalNode insertInnerScan(BinaryNode parent, String tableId, 
       TableMeta meta) throws IOException {
-    TableDesc desc = TCatUtil.newTableDesc(tableId, meta, sm.getTablePath(tableId));
+    TableDesc desc = CatalogUtil.newTableDesc(tableId, meta, sm.getTablePath(tableId));
     ScanNode scan = new ScanNode(new FromTable(desc));
     scan.setLocal(true);
     scan.setInSchema(meta.getSchema());

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-backend/src/main/java/tajo/master/GlobalPlannerUtils.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/tajo/master/GlobalPlannerUtils.java b/tajo-core/tajo-core-backend/src/main/java/tajo/master/GlobalPlannerUtils.java
index 1ecfa0e..089da1b 100644
--- a/tajo-core/tajo-core-backend/src/main/java/tajo/master/GlobalPlannerUtils.java
+++ b/tajo-core/tajo-core-backend/src/main/java/tajo/master/GlobalPlannerUtils.java
@@ -173,8 +173,8 @@ public class GlobalPlannerUtils {
   public static ScanNode newScanPlan(Schema inputSchema,
                                      String inputTableId,
                                      Path inputPath) {
-    TableMeta meta = TCatUtil.newTableMeta(inputSchema, StoreType.CSV);
-    TableDesc desc = TCatUtil.newTableDesc(inputTableId, meta, inputPath);
+    TableMeta meta = CatalogUtil.newTableMeta(inputSchema, StoreType.CSV);
+    TableDesc desc = CatalogUtil.newTableDesc(inputTableId, meta, inputPath);
     ScanNode newScan = new ScanNode(new QueryBlock.FromTable(desc));
     newScan.setInSchema(desc.getMeta().getSchema());
     newScan.setOutSchema(desc.getMeta().getSchema());

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-backend/src/main/java/tajo/master/Query.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/tajo/master/Query.java b/tajo-core/tajo-core-backend/src/main/java/tajo/master/Query.java
index d740cbd..98fb754 100644
--- a/tajo-core/tajo-core-backend/src/main/java/tajo/master/Query.java
+++ b/tajo-core/tajo-core-backend/src/main/java/tajo/master/Query.java
@@ -31,10 +31,7 @@ import tajo.QueryConf;
 import tajo.QueryId;
 import tajo.SubQueryId;
 import tajo.TajoProtos.QueryState;
-import tajo.catalog.TCatUtil;
-import tajo.catalog.TableDesc;
-import tajo.catalog.TableDescImpl;
-import tajo.catalog.TableMeta;
+import tajo.catalog.*;
 import tajo.catalog.proto.CatalogProtos.StoreType;
 import tajo.engine.json.GsonCreator;
 import tajo.engine.planner.global.MasterPlan;
@@ -418,7 +415,7 @@ public class Query implements EventHandler<QueryEvent> {
       if (sm.getFileSystem().exists(new Path(indexPath, ".meta"))) {
         meta = sm.getTableMeta(indexPath);
       } else {
-        meta = TCatUtil
+        meta = CatalogUtil
             .newTableMeta(execBlock.getOutputSchema(), StoreType.CSV);
       }
       String indexName = IndexUtil.getIndexName(index.getTableName(),

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-backend/src/main/java/tajo/master/Repartitioner.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/tajo/master/Repartitioner.java b/tajo-core/tajo-core-backend/src/main/java/tajo/master/Repartitioner.java
index 51aff94..622c4f1 100644
--- a/tajo-core/tajo-core-backend/src/main/java/tajo/master/Repartitioner.java
+++ b/tajo-core/tajo-core-backend/src/main/java/tajo/master/Repartitioner.java
@@ -76,7 +76,7 @@ public class Repartitioner {
 
       if (scans[i].isLocal()) { // it only requires a dummy fragment.
         fragments[i] = new Fragment(scans[i].getTableId(), tablePath,
-            TCatUtil.newTableMeta(scans[i].getInSchema(), StoreType.CSV),
+            CatalogUtil.newTableMeta(scans[i].getInSchema(), StoreType.CSV),
             0, 0, null);
       } else {
         fragments[i] = subQuery.getStorageManager().getSplits(scans[i].getTableId(),
@@ -291,7 +291,7 @@ public class Repartitioner {
     TupleRange [] ranges = partitioner.partition(determinedTaskNum);
 
     Fragment dummyFragment = new Fragment(scan.getTableId(), tablePath,
-        TCatUtil.newTableMeta(scan.getInSchema(), StoreType.CSV),
+        CatalogUtil.newTableMeta(scan.getInSchema(), StoreType.CSV),
         0, 0, null);
 
     List<String> basicFetchURIs = new ArrayList<String>();
@@ -389,7 +389,7 @@ public class Repartitioner {
     }
 
     Fragment frag = new Fragment(scan.getTableId(), tablePath,
-        TCatUtil.newTableMeta(scan.getInSchema(), StoreType.CSV),
+        CatalogUtil.newTableMeta(scan.getInSchema(), StoreType.CSV),
         0, 0, null);
 
     Map<Integer, List<IntermediateEntry>> hashed = hashByKey(partitions);

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-backend/src/main/java/tajo/master/SubQuery.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/tajo/master/SubQuery.java b/tajo-core/tajo-core-backend/src/main/java/tajo/master/SubQuery.java
index 986ecaf..d965982 100644
--- a/tajo-core/tajo-core-backend/src/main/java/tajo/master/SubQuery.java
+++ b/tajo-core/tajo-core-backend/src/main/java/tajo/master/SubQuery.java
@@ -35,7 +35,7 @@ import tajo.QueryIdFactory;
 import tajo.QueryUnitId;
 import tajo.SubQueryId;
 import tajo.catalog.CatalogService;
-import tajo.catalog.TCatUtil;
+import tajo.catalog.CatalogUtil;
 import tajo.catalog.TableDesc;
 import tajo.catalog.TableMeta;
 import tajo.catalog.statistics.ColumnStat;
@@ -358,10 +358,10 @@ public class SubQuery implements EventHandler<SubQueryEvent> {
 
   private static TableMeta toTableMeta(StoreTableNode store) {
     if (store.hasOptions()) {
-      return TCatUtil.newTableMeta(store.getOutSchema(),
+      return CatalogUtil.newTableMeta(store.getOutSchema(),
           store.getStorageType(), store.getOptions());
     } else {
-      return TCatUtil.newTableMeta(store.getOutSchema(),
+      return CatalogUtil.newTableMeta(store.getOutSchema(),
           store.getStorageType());
     }
   }
@@ -766,4 +766,4 @@ public class SubQuery implements EventHandler<SubQueryEvent> {
 
     }
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-backend/src/main/java/tajo/master/TajoMaster.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/tajo/master/TajoMaster.java b/tajo-core/tajo-core-backend/src/main/java/tajo/master/TajoMaster.java
index 66f4446..82b1445 100644
--- a/tajo-core/tajo-core-backend/src/main/java/tajo/master/TajoMaster.java
+++ b/tajo-core/tajo-core-backend/src/main/java/tajo/master/TajoMaster.java
@@ -39,12 +39,9 @@ import org.apache.hadoop.yarn.util.RackResolver;
 import tajo.QueryId;
 import tajo.QueryIdFactory;
 import tajo.TajoConstants;
-import tajo.catalog.CatalogServer;
-import tajo.catalog.CatalogService;
-import tajo.catalog.FunctionDesc;
-import tajo.catalog.LocalCatalog;
-import tajo.catalog.proto.CatalogProtos.DataType;
+import tajo.catalog.*;
 import tajo.catalog.proto.CatalogProtos.FunctionType;
+import tajo.common.TajoDataTypes.Type;
 import tajo.conf.TajoConf;
 import tajo.conf.TajoConf.ConfVars;
 import tajo.engine.MasterWorkerProtos.TaskStatusProto;
@@ -169,93 +166,93 @@ public class TajoMaster extends CompositeService {
 
     // Sum
     sqlFuncs.add(new FunctionDesc("sum", SumInt.class, FunctionType.AGGREGATION,
-        new DataType[] {DataType.INT},
-        new DataType[] {DataType.INT}));
+        CatalogUtil.newDataTypesWithoutLen(Type.INT4),
+        CatalogUtil.newDataTypesWithoutLen(Type.INT4)));
     sqlFuncs.add(new FunctionDesc("sum", SumLong.class, FunctionType.AGGREGATION,
-        new DataType[] {DataType.LONG},
-        new DataType[] {DataType.LONG}));
+        CatalogUtil.newDataTypesWithoutLen(Type.INT8),
+        CatalogUtil.newDataTypesWithoutLen(Type.INT8)));
     sqlFuncs.add(new FunctionDesc("sum", SumFloat.class, FunctionType.AGGREGATION,
-        new DataType[] {DataType.FLOAT},
-        new DataType[] {DataType.FLOAT}));
+        CatalogUtil.newDataTypesWithoutLen(Type.FLOAT4),
+        CatalogUtil.newDataTypesWithoutLen(Type.FLOAT4)));
     sqlFuncs.add(new FunctionDesc("sum", SumDouble.class, FunctionType.AGGREGATION,
-        new DataType[] {DataType.DOUBLE},
-        new DataType[] {DataType.DOUBLE}));
+        CatalogUtil.newDataTypesWithoutLen(Type.FLOAT8),
+        CatalogUtil.newDataTypesWithoutLen(Type.FLOAT8)));
 
     // Max
     sqlFuncs.add(new FunctionDesc("max", MaxInt.class, FunctionType.AGGREGATION,
-        new DataType[] {DataType.INT},
-        new DataType[] {DataType.INT}));
+        CatalogUtil.newDataTypesWithoutLen(Type.INT4),
+        CatalogUtil.newDataTypesWithoutLen(Type.INT4)));
     sqlFuncs.add(new FunctionDesc("max", MaxLong.class, FunctionType.AGGREGATION,
-        new DataType[] {DataType.LONG},
-        new DataType[] {DataType.LONG}));
+        CatalogUtil.newDataTypesWithoutLen(Type.INT8),
+        CatalogUtil.newDataTypesWithoutLen(Type.INT8)));
     sqlFuncs.add(new FunctionDesc("max", MaxFloat.class, FunctionType.AGGREGATION,
-        new DataType[] {DataType.FLOAT},
-        new DataType[] {DataType.FLOAT}));
+        CatalogUtil.newDataTypesWithoutLen(Type.FLOAT4),
+        CatalogUtil.newDataTypesWithoutLen(Type.FLOAT4)));
     sqlFuncs.add(new FunctionDesc("max", MaxDouble.class, FunctionType.AGGREGATION,
-        new DataType[] {DataType.DOUBLE},
-        new DataType[] {DataType.DOUBLE}));
+        CatalogUtil.newDataTypesWithoutLen(Type.FLOAT8),
+        CatalogUtil.newDataTypesWithoutLen(Type.FLOAT8)));
 
     // Min
     sqlFuncs.add(new FunctionDesc("min", MinInt.class, FunctionType.AGGREGATION,
-        new DataType[] {DataType.INT},
-        new DataType[] {DataType.INT}));
+        CatalogUtil.newDataTypesWithoutLen(Type.INT4),
+        CatalogUtil.newDataTypesWithoutLen(Type.INT4)));
     sqlFuncs.add(new FunctionDesc("min", MinLong.class, FunctionType.AGGREGATION,
-        new DataType[] {DataType.LONG},
-        new DataType[] {DataType.LONG}));
+        CatalogUtil.newDataTypesWithoutLen(Type.INT8),
+        CatalogUtil.newDataTypesWithoutLen(Type.INT8)));
     sqlFuncs.add(new FunctionDesc("min", MinFloat.class, FunctionType.AGGREGATION,
-        new DataType[] {DataType.FLOAT},
-        new DataType[] {DataType.FLOAT }));
+        CatalogUtil.newDataTypesWithoutLen(Type.FLOAT4),
+        CatalogUtil.newDataTypesWithoutLen(Type.FLOAT4 )));
     sqlFuncs.add(new FunctionDesc("min", MinDouble.class, FunctionType.AGGREGATION,
-        new DataType[] {DataType.DOUBLE},
-        new DataType[] {DataType.DOUBLE}));
+        CatalogUtil.newDataTypesWithoutLen(Type.FLOAT8),
+        CatalogUtil.newDataTypesWithoutLen(Type.FLOAT8)));
     sqlFuncs.add(new FunctionDesc("min", MinString.class, FunctionType.AGGREGATION,
-        new DataType[] {DataType.STRING},
-        new DataType[] {DataType.STRING}));
+        CatalogUtil.newDataTypesWithoutLen(Type.TEXT),
+        CatalogUtil.newDataTypesWithoutLen(Type.TEXT)));
 
     // AVG
     sqlFuncs.add(new FunctionDesc("avg", AvgInt.class, FunctionType.AGGREGATION,
-        new DataType[] {DataType.FLOAT},
-        new DataType[] {DataType.INT}));
+        CatalogUtil.newDataTypesWithoutLen(Type.FLOAT4),
+        CatalogUtil.newDataTypesWithoutLen(Type.INT4)));
     sqlFuncs.add(new FunctionDesc("avg", AvgLong.class, FunctionType.AGGREGATION,
-        new DataType[] {DataType.DOUBLE},
-        new DataType[] {DataType.LONG}));
+        CatalogUtil.newDataTypesWithoutLen(Type.FLOAT8),
+        CatalogUtil.newDataTypesWithoutLen(Type.INT8)));
     sqlFuncs.add(new FunctionDesc("avg", AvgFloat.class, FunctionType.AGGREGATION,
-        new DataType[] {DataType.FLOAT},
-        new DataType[] {DataType.FLOAT}));
+        CatalogUtil.newDataTypesWithoutLen(Type.FLOAT4),
+        CatalogUtil.newDataTypesWithoutLen(Type.FLOAT4)));
     sqlFuncs.add(new FunctionDesc("avg", AvgDouble.class, FunctionType.AGGREGATION,
-        new DataType[] {DataType.DOUBLE},
-        new DataType[] {DataType.DOUBLE}));
+        CatalogUtil.newDataTypesWithoutLen(Type.FLOAT8),
+        CatalogUtil.newDataTypesWithoutLen(Type.FLOAT8)));
 
     // Count
     sqlFuncs.add(new FunctionDesc("count", CountValue.class, FunctionType.AGGREGATION,
-        new DataType[] {DataType.LONG},
-        new DataType[] {DataType.ANY}));
+        CatalogUtil.newDataTypesWithoutLen(Type.INT8),
+        CatalogUtil.newDataTypesWithoutLen(Type.ANY)));
     sqlFuncs.add(new FunctionDesc("count", CountRows.class, FunctionType.AGGREGATION,
-        new DataType[] {DataType.LONG},
-        new DataType[] {}));
+        CatalogUtil.newDataTypesWithoutLen(Type.INT8),
+        CatalogUtil.newDataTypesWithoutLen()));
 
     // GeoIP
     sqlFuncs.add(new FunctionDesc("in_country", InCountry.class, FunctionType.GENERAL,
-        new DataType[] {DataType.BOOLEAN},
-        new DataType[] {DataType.STRING, DataType.STRING}));
+        CatalogUtil.newDataTypesWithoutLen(Type.BOOLEAN),
+        CatalogUtil.newDataTypesWithoutLen(Type.TEXT, Type.TEXT)));
     sqlFuncs.add(new FunctionDesc("country", Country.class, FunctionType.GENERAL,
-        new DataType[] {DataType.STRING},
-        new DataType[] {DataType.STRING}));
+        CatalogUtil.newDataTypesWithoutLen(Type.TEXT),
+        CatalogUtil.newDataTypesWithoutLen(Type.TEXT)));
 
     // Date
     sqlFuncs.add(new FunctionDesc("date", Date.class, FunctionType.GENERAL,
-        new DataType[] {DataType.LONG},
-        new DataType[] {DataType.STRING}));
+        CatalogUtil.newDataTypesWithoutLen(Type.INT8),
+        CatalogUtil.newDataTypesWithoutLen(Type.TEXT)));
 
     // Today
     sqlFuncs.add(new FunctionDesc("today", Date.class, FunctionType.GENERAL,
-        new DataType[] {DataType.LONG},
-        new DataType[] {}));
+        CatalogUtil.newDataTypesWithoutLen(Type.INT8),
+        CatalogUtil.newDataTypesWithoutLen()));
 
     sqlFuncs.add(
         new FunctionDesc("random", RandomInt.class, FunctionType.GENERAL,
-            new DataType[]{DataType.INT},
-            new DataType[]{DataType.INT}));
+            CatalogUtil.newDataTypesWithoutLen(Type.INT4),
+            CatalogUtil.newDataTypesWithoutLen(Type.INT4)));
 
     return sqlFuncs;
   }

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-backend/src/main/proto/CatalogProtos.proto
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/proto/CatalogProtos.proto b/tajo-core/tajo-core-backend/src/main/proto/CatalogProtos.proto
index dd1bf69..36b33c1 100644
--- a/tajo-core/tajo-core-backend/src/main/proto/CatalogProtos.proto
+++ b/tajo-core/tajo-core-backend/src/main/proto/CatalogProtos.proto
@@ -22,33 +22,12 @@ option optimize_for = SPEED;
 option java_generic_services = false;
 option java_generate_equals_and_hash = true;
 
-enum DataType {
-    BOOLEAN = 0;
-    BYTE = 1;
-    CHAR = 2;
-    BYTES = 3;
-    SHORT = 4;
-    INT = 5;
-    BIGINT = 6;
-    LONG = 7;
-    FLOAT = 8;
-    DOUBLE = 9;
-    BIGDECIMAL = 10;
-    STRING = 11;
-    DATE = 12;
-    IPv4 = 13;
-    IPv6 = 14;
-    NULL = 15;
-    ANY = 16;
-    ALL = 17;
-    ARRAY = 18;
-    STRING2 = 19;
-}
+import "DataTypes.proto";
 
 enum StoreType {
-	MEM = 0;
-	CSV = 1;
-	RAW = 2;
+  MEM = 0;
+  CSV = 1;
+  RAW = 2;
   RCFILE = 3;
   ROWFILE = 4;
   HCFILE = 5;
@@ -56,147 +35,147 @@ enum StoreType {
 }
 
 enum OrderType {
-    ORDER_NONE = 0;
-    ASC = 1;
-    DSC = 2;
+  ORDER_NONE = 0;
+  ASC = 1;
+  DSC = 2;
 }
 
 enum CompressType {
-    COMP_NONE = 0;
-    NULL_SUPPRESS = 1;
-    RUN_LENGTH = 2;
-    BIT_VECTOR = 3;
-    DICTIONARY = 4;
-    SNAPPY = 5;
-    LZ = 6;
+  COMP_NONE = 0;
+  NULL_SUPPRESS = 1;
+  RUN_LENGTH = 2;
+  BIT_VECTOR = 3;
+  DICTIONARY = 4;
+  SNAPPY = 5;
+  LZ = 6;
 }
 
 message ColumnMetaProto {
-    required DataType dataType = 1;
-    required bool compressed = 2;
-    required bool sorted = 3;
-    required bool contiguous = 4;
-    required StoreType storeType = 5;
-    required CompressType compType = 6;
-    required int64 startRid = 7;
-    required int32 recordNum = 8;
-    required int32 offsetToIndex = 9;
+  required DataType dataType = 1;
+  required bool compressed = 2;
+  required bool sorted = 3;
+  required bool contiguous = 4;
+  required StoreType storeType = 5;
+  required CompressType compType = 6;
+  required int64 startRid = 7;
+  required int32 recordNum = 8;
+  required int32 offsetToIndex = 9;
 }
 
 message ColumnProto {
-	required string columnName = 1;
-	required DataType dataType = 2;
+  required string columnName = 1;
+  required DataType dataType = 2;
 }
 
 message SchemaProto {
-	repeated ColumnProto fields = 1;
+  repeated ColumnProto fields = 1;
 }
 
 message KeyValueProto {
-	required string key = 1;
-	required string value = 2;
+  required string key = 1;
+  required string value = 2;
 }
 
 message KeyValueSetProto {
-	repeated KeyValueProto keyval = 1;
+  repeated KeyValueProto keyval = 1;
 }
 
 message TabletProto {
-	required string id = 1;
-	required string path = 2;
-	required int64 startOffset = 3;
-	required int64 length = 4;
-	required TableProto meta = 5;
-	optional TableStatProto stat = 6;
+  required string id = 1;
+  required string path = 2;
+  required int64 startOffset = 3;
+  required int64 length = 4;
+  required TableProto meta = 5;
+  optional TableStatProto stat = 6;
   optional bool distCached = 7 [default = false];
 }
 
 message TableProto {
-    required SchemaProto schema = 1;
-    required StoreType storeType = 2;
-    required KeyValueSetProto params = 3;
-    optional TableStatProto stat = 4;
+  required SchemaProto schema = 1;
+  required StoreType storeType = 2;
+  required KeyValueSetProto params = 3;
+  optional TableStatProto stat = 4;
 }
 
 message TableDescProto {
-	required string id = 1;
-	required string path = 2;
-	required TableProto meta = 3;
+  required string id = 1;
+  required string path = 2;
+  required TableProto meta = 3;
 }
 
 enum FunctionType {
-	GENERAL = 0;
-	AGGREGATION = 1;
+  GENERAL = 0;
+  AGGREGATION = 1;
 }
 
 message FunctionDescProto {
-	required string signature = 1;
-	required string className = 2;
-	required FunctionType type = 3;
-	repeated DataType parameterTypes = 4;
-	required DataType returnType = 5;
+  required string signature = 1;
+  required string className = 2;
+  required FunctionType type = 3;
+  repeated DataType parameterTypes = 4;
+  required DataType returnType = 5;
 }
 
 message IndexDescProto {
-    required string name = 1;
-    required string tableId = 2;
-    required ColumnProto column = 3;
-    required IndexMethod indexMethod = 4;
-    optional bool isUnique = 5 [default = false];
-    optional bool isClustered = 6 [default = false];
-    optional bool isAscending = 7 [default = false];
+  required string name = 1;
+  required string tableId = 2;
+  required ColumnProto column = 3;
+  required IndexMethod indexMethod = 4;
+  optional bool isUnique = 5 [default = false];
+  optional bool isClustered = 6 [default = false];
+  optional bool isAscending = 7 [default = false];
 }
 
 enum IndexMethod {
-    TWO_LEVEL_BIN_TREE = 0;
-    BTREE = 1;
-    HASH = 2;
-    BITMAP = 3;
+  TWO_LEVEL_BIN_TREE = 0;
+  BTREE = 1;
+  HASH = 2;
+  BITMAP = 3;
 }
 
 message GetAllTableNamesResponse {
-    repeated string tableName = 1;
+  repeated string tableName = 1;
 }
 
 message GetIndexRequest {
-    required string tableName = 1;
-    required string columnName = 2;
+  required string tableName = 1;
+  required string columnName = 2;
 }
 
 message GetFunctionsResponse {
-	repeated FunctionDescProto functionDesc = 1;
+  repeated FunctionDescProto functionDesc = 1;
 }
 
 message UnregisterFunctionRequest {
-	required string signature = 1;
-	repeated DataType parameterTypes = 2;
+  required string signature = 1;
+  repeated DataType parameterTypes = 2;
 }
 
 message GetFunctionMetaRequest {
-	required string signature = 1;
-	repeated DataType parameterTypes = 2;
+  required string signature = 1;
+  repeated DataType parameterTypes = 2;
 }
 
 message ContainFunctionRequest {
-	required string signature = 1;
-	repeated DataType parameterTypes = 2;
+  required string signature = 1;
+  repeated DataType parameterTypes = 2;
 }
 
 message TableStatProto {
-	required int64 numRows = 1;
-	required int64 numBytes = 2;
-	optional int32 numBlocks = 3;
-	optional int32 numPartitions = 4;
-	optional int64 avgRows = 5;
-	repeated ColumnStatProto colStat = 6;
+  required int64 numRows = 1;
+  required int64 numBytes = 2;
+  optional int32 numBlocks = 3;
+  optional int32 numPartitions = 4;
+  optional int64 avgRows = 5;
+  repeated ColumnStatProto colStat = 6;
 }
 
 message ColumnStatProto {
-    required ColumnProto column = 1;
-    optional int64 numDistVal = 2;
-    optional int64 numNulls = 3;
-    optional bytes minValue = 4;
-    optional bytes maxValue = 5;
+  required ColumnProto column = 1;
+  optional int64 numDistVal = 2;
+  optional int64 numNulls = 3;
+  optional bytes minValue = 4;
+  optional bytes maxValue = 5;
 }
 
 enum StatType {

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-backend/src/test/java/tajo/BackendTestingUtil.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/tajo/BackendTestingUtil.java b/tajo-core/tajo-core-backend/src/test/java/tajo/BackendTestingUtil.java
index 1db61c2..1b2e60b 100644
--- a/tajo-core/tajo-core-backend/src/test/java/tajo/BackendTestingUtil.java
+++ b/tajo-core/tajo-core-backend/src/test/java/tajo/BackendTestingUtil.java
@@ -26,8 +26,8 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import tajo.catalog.*;
-import tajo.catalog.proto.CatalogProtos.DataType;
 import tajo.catalog.proto.CatalogProtos.StoreType;
+import tajo.common.TajoDataTypes.Type;
 import tajo.conf.TajoConf;
 import tajo.datum.DatumFactory;
 import tajo.engine.parser.QueryAnalyzer;
@@ -51,9 +51,9 @@ public class BackendTestingUtil {
 
 	static {
     mockupSchema = new Schema();
-    mockupSchema.addColumn("deptname", DataType.STRING);
-    mockupSchema.addColumn("score", DataType.INT);
-    mockupMeta = TCatUtil.newTableMeta(mockupSchema, StoreType.CSV);
+    mockupSchema.addColumn("deptname", Type.TEXT);
+    mockupSchema.addColumn("score", Type.INT4);
+    mockupMeta = CatalogUtil.newTableMeta(mockupSchema, StoreType.CSV);
 	}
 
   public static void writeTmpTable(TajoConf conf, Path path,
@@ -81,8 +81,8 @@ public class BackendTestingUtil {
     for (int i = 0; i < tupleNum; i++) {
       tuple = new VTuple(2);
       String key = "test" + (i % deptSize);
-      tuple.put(0, DatumFactory.createString(key));
-      tuple.put(1, DatumFactory.createInt(i + 1));
+      tuple.put(0, DatumFactory.createText(key));
+      tuple.put(1, DatumFactory.createInt4(i + 1));
       appender.addTuple(tuple);
     }
     appender.close();

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-backend/src/test/java/tajo/LocalTajoTestingUtility.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/tajo/LocalTajoTestingUtility.java b/tajo-core/tajo-core-backend/src/test/java/tajo/LocalTajoTestingUtility.java
index 9ce3383..405eb11 100644
--- a/tajo-core/tajo-core-backend/src/test/java/tajo/LocalTajoTestingUtility.java
+++ b/tajo-core/tajo-core-backend/src/test/java/tajo/LocalTajoTestingUtility.java
@@ -21,10 +21,7 @@ package tajo;
 import com.google.protobuf.ServiceException;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import tajo.catalog.Options;
-import tajo.catalog.Schema;
-import tajo.catalog.TCatUtil;
-import tajo.catalog.TableMeta;
+import tajo.catalog.*;
 import tajo.catalog.proto.CatalogProtos;
 import tajo.client.TajoClient;
 import tajo.conf.TajoConf;
@@ -59,7 +56,7 @@ public class LocalTajoTestingUtility {
       fs.mkdirs(dataPath);
       Path dfsPath = new Path(dataPath, localPath.getName());
       fs.copyFromLocalFile(localPath, dfsPath);
-      TableMeta meta = TCatUtil.newTableMeta(schemas[i],
+      TableMeta meta = CatalogUtil.newTableMeta(schemas[i],
           CatalogProtos.StoreType.CSV, option);
       client.createTable(names[i], tablePath, meta);
     }

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-backend/src/test/java/tajo/TajoTestingCluster.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/tajo/TajoTestingCluster.java b/tajo-core/tajo-core-backend/src/test/java/tajo/TajoTestingCluster.java
index 75fde3c..8cac708 100644
--- a/tajo-core/tajo-core-backend/src/test/java/tajo/TajoTestingCluster.java
+++ b/tajo-core/tajo-core-backend/src/test/java/tajo/TajoTestingCluster.java
@@ -425,7 +425,7 @@ public class TajoTestingCluster {
       dataDir.mkdirs();
       File tableFile = new File(dataDir, tableNames[i]);
       writeLines(tableFile, tables[i]);
-      TableMeta meta = TCatUtil
+      TableMeta meta = CatalogUtil
           .newTableMeta(schemas[i], CatalogProtos.StoreType.CSV, option);
       client.createTable(tableNames[i], new Path(tableDir.getAbsolutePath()), meta);
     }
@@ -457,7 +457,7 @@ public class TajoTestingCluster {
       fs.mkdirs(dataPath);
       Path dfsPath = new Path(dataPath, localPath.getName());
       fs.copyFromLocalFile(localPath, dfsPath);
-      TableMeta meta = TCatUtil.newTableMeta(schemas[i],
+      TableMeta meta = CatalogUtil.newTableMeta(schemas[i],
           CatalogProtos.StoreType.CSV, option);
       client.createTable(names[i], tablePath, meta);
     }
@@ -492,7 +492,7 @@ public class TajoTestingCluster {
         out.write((tables[i][j]+"\n").getBytes());
       }
       out.close();
-      TableMeta meta = TCatUtil.newTableMeta(schemas[i],
+      TableMeta meta = CatalogUtil.newTableMeta(schemas[i],
           CatalogProtos.StoreType.CSV, option);
       client.createTable(names[i], tablePath, meta);
     }

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-backend/src/test/java/tajo/client/TestTajoClient.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/tajo/client/TestTajoClient.java b/tajo-core/tajo-core-backend/src/test/java/tajo/client/TestTajoClient.java
index ff7a292..afc75e7 100644
--- a/tajo-core/tajo-core-backend/src/test/java/tajo/client/TestTajoClient.java
+++ b/tajo-core/tajo-core-backend/src/test/java/tajo/client/TestTajoClient.java
@@ -89,7 +89,7 @@ public class TestTajoClient {
 
     assertFalse(tajo.existTable(tableName));
     String tql =
-        "create external table " + tableName + " (deptname string, score int) "
+        "create external table " + tableName + " (deptname text, score integer) "
             + "using csv location '" + tablePath + "'";
     tajo.updateQuery(tql);
     assertTrue(tajo.existTable(tableName));
@@ -118,7 +118,7 @@ public class TestTajoClient {
 
     assertFalse(tajo.existTable(tableName));
     String tql =
-        "create external table " + tableName + " (deptname string, score int) "
+        "create external table " + tableName + " (deptname text, score int4) "
             + "using csv location 'file:///tmp/" + tableName + "'";
     tajo.executeQueryAndGetResult(tql);
     assertTrue(tajo.existTable(tableName));


[11/11] git commit: TAJO-57: Recognize Parser and Catalog Standard SQL data types. (hyunsik)

Posted by hy...@apache.org.
TAJO-57: Recognize Parser and Catalog Standard SQL data types. (hyunsik)


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

Branch: refs/heads/master
Commit: c1c6f83eafdea80d931094285f8da1186f062cf5
Parents: fc47436
Author: Hyunsik Choi <hy...@apache.org>
Authored: Thu May 9 14:37:35 2013 +0900
Committer: Hyunsik Choi <hy...@apache.org>
Committed: Thu May 9 14:37:35 2013 +0900

----------------------------------------------------------------------
 .../java/tajo/catalog/AbstractCatalogClient.java   |   18 +-
 .../catalog/AlreadyRegisteredURIException.java     |   40 --
 .../src/main/java/tajo/catalog/AttributeType.java  |   25 -
 .../src/main/java/tajo/catalog/CatalogReader.java  |   31 --
 .../src/main/java/tajo/catalog/CatalogService.java |    2 +-
 .../java/tajo/catalog/CatalogServiceProtocol.java  |  105 ----
 .../src/main/proto/CatalogProtos.proto             |   23 +-
 .../src/main/java/tajo/catalog/CatalogUtil.java    |  138 ++++---
 .../src/main/java/tajo/catalog/Column.java         |   16 +-
 .../src/main/java/tajo/catalog/FunctionDesc.java   |    6 +-
 .../src/main/java/tajo/catalog/Schema.java         |    9 +-
 .../src/main/java/tajo/catalog/TCatUtil.java       |  130 -----
 .../java/tajo/catalog/function/AggFunction.java    |    4 +-
 .../main/java/tajo/catalog/json/GsonCreator.java   |    3 +
 .../java/tajo/catalog/statistics/TupleUtil.java    |   45 +-
 .../src/main/proto/CatalogProtos.proto             |  183 ++++----
 .../test/java/tajo/catalog/TestCatalogUtil.java    |    8 +-
 .../src/test/java/tajo/catalog/TestColumn.java     |   22 +-
 .../test/java/tajo/catalog/TestFunctionDesc.java   |   37 +-
 .../src/test/java/tajo/catalog/TestIndexDesc.java  |   12 +-
 .../src/test/java/tajo/catalog/TestSchema.java     |   26 +-
 .../src/test/java/tajo/catalog/TestTableDesc.java  |   14 +-
 .../src/test/java/tajo/catalog/TestTableInfo.java  |   48 +-
 .../src/test/java/tajo/catalog/TestTableMeta.java  |   48 +-
 .../tajo/catalog/statistics/TestColumnStat.java    |   16 +-
 .../tajo/catalog/statistics/TestTableStat.java     |    4 +-
 .../catalog/AlreadyRegisteredURIException.java     |   43 --
 .../src/main/java/tajo/catalog/CatalogServer.java  |   11 +-
 .../src/main/java/tajo/catalog/store/DBStore.java  |   46 +--
 .../src/test/java/tajo/catalog/TestCatalog.java    |   60 ++--
 .../src/test/java/tajo/catalog/TestDBStore.java    |   42 +-
 .../tajo/catalog/statistics/TestColumnStat.java    |   16 +-
 .../tajo/catalog/statistics/TestTableStat.java     |    4 +-
 tajo-common/pom.xml                                |    1 +
 .../src/main/java/tajo/datum/ArrayDatum.java       |    6 +-
 .../src/main/java/tajo/datum/BoolDatum.java        |  172 -------
 .../src/main/java/tajo/datum/ByteDatum.java        |  134 ------
 .../src/main/java/tajo/datum/BytesDatum.java       |  161 -------
 .../src/main/java/tajo/datum/CharDatum.java        |   17 +-
 tajo-common/src/main/java/tajo/datum/Datum.java    |   42 +-
 .../src/main/java/tajo/datum/DatumFactory.java     |   90 ++--
 .../src/main/java/tajo/datum/DatumType.java        |   42 --
 .../src/main/java/tajo/datum/DoubleDatum.java      |  270 -----------
 .../src/main/java/tajo/datum/FloatDatum.java       |  278 -----------
 .../src/main/java/tajo/datum/IPv4Datum.java        |  136 ------
 tajo-common/src/main/java/tajo/datum/IntDatum.java |  274 -----------
 .../src/main/java/tajo/datum/LongDatum.java        |  279 -----------
 .../src/main/java/tajo/datum/NullDatum.java        |   20 +-
 .../src/main/java/tajo/datum/NumericDatum.java     |    4 +-
 .../src/main/java/tajo/datum/ShortDatum.java       |  268 -----------
 .../src/main/java/tajo/datum/StringDatum.java      |  157 ------
 .../src/main/java/tajo/datum/StringDatum2.java     |  140 ------
 .../datum/exception/InvalidOperationException.java |    4 +-
 tajo-common/src/main/java/tajo/storage/Tuple.java  |   91 ++--
 .../src/test/java/tajo/datum/TestBoolDatum.java    |    9 +-
 .../src/test/java/tajo/datum/TestByteDatum.java    |   75 ---
 .../src/test/java/tajo/datum/TestBytesDatum.java   |   11 +-
 .../src/test/java/tajo/datum/TestCharDatum.java    |   13 +-
 .../src/test/java/tajo/datum/TestDatum.java        |  251 +++++-----
 .../src/test/java/tajo/datum/TestDatumFactory.java |   43 +-
 .../src/test/java/tajo/datum/TestDoubleDatum.java  |   69 ---
 .../src/test/java/tajo/datum/TestFloatDatum.java   |   25 +-
 .../src/test/java/tajo/datum/TestIPv4Datum.java    |   75 ---
 .../src/test/java/tajo/datum/TestIntDatum.java     |   69 ---
 .../src/test/java/tajo/datum/TestLongDatum.java    |   69 ---
 .../src/test/java/tajo/datum/TestShortDatum.java   |   69 ---
 .../src/test/java/tajo/datum/TestStringDatum.java  |   69 ---
 .../src/main/antlr3/tajo/engine/parser/NQL.g       |  342 ++++++++++----
 .../src/main/java/tajo/benchmark/TPCH.java         |  144 +++---
 .../src/main/java/tajo/client/TajoClient.java      |   10 +-
 .../java/tajo/engine/eval/AggFuncCallEval.java     |    4 +-
 .../src/main/java/tajo/engine/eval/BinaryEval.java |   66 ++--
 .../main/java/tajo/engine/eval/CaseWhenEval.java   |   10 +-
 .../src/main/java/tajo/engine/eval/ConstEval.java  |   29 +-
 .../src/main/java/tajo/engine/eval/EvalNode.java   |    2 +-
 .../main/java/tajo/engine/eval/EvalTreeUtil.java   |    4 +-
 .../src/main/java/tajo/engine/eval/FieldEval.java  |    4 +-
 .../src/main/java/tajo/engine/eval/FuncEval.java   |    2 +-
 .../src/main/java/tajo/engine/eval/IsNullEval.java |   19 +-
 .../src/main/java/tajo/engine/eval/LikeEval.java   |   16 +-
 .../src/main/java/tajo/engine/eval/NotEval.java    |    9 +-
 .../java/tajo/engine/eval/PartialBinaryExpr.java   |    4 +-
 .../main/java/tajo/engine/function/Country.java    |   11 +-
 .../main/java/tajo/engine/function/InCountry.java  |   12 +-
 .../tajo/engine/function/builtin/AvgDouble.java    |   23 +-
 .../tajo/engine/function/builtin/AvgFloat.java     |   26 +-
 .../java/tajo/engine/function/builtin/AvgInt.java  |   28 +-
 .../java/tajo/engine/function/builtin/AvgLong.java |   27 +-
 .../tajo/engine/function/builtin/CountRows.java    |   16 +-
 .../tajo/engine/function/builtin/CountValue.java   |    8 +-
 .../java/tajo/engine/function/builtin/Date.java    |   11 +-
 .../tajo/engine/function/builtin/MaxDouble.java    |   21 +-
 .../tajo/engine/function/builtin/MaxFloat.java     |   15 +-
 .../java/tajo/engine/function/builtin/MaxInt.java  |   17 +-
 .../java/tajo/engine/function/builtin/MaxLong.java |   23 +-
 .../tajo/engine/function/builtin/MinDouble.java    |   17 +-
 .../tajo/engine/function/builtin/MinFloat.java     |   22 +-
 .../java/tajo/engine/function/builtin/MinInt.java  |   15 +-
 .../java/tajo/engine/function/builtin/MinLong.java |   19 +-
 .../tajo/engine/function/builtin/MinString.java    |   19 +-
 .../tajo/engine/function/builtin/RandomInt.java    |    7 +-
 .../tajo/engine/function/builtin/SumDouble.java    |   21 +-
 .../tajo/engine/function/builtin/SumFloat.java     |   17 +-
 .../java/tajo/engine/function/builtin/SumInt.java  |   17 +-
 .../java/tajo/engine/function/builtin/SumLong.java |   21 +-
 .../java/tajo/engine/function/builtin/Today.java   |    6 +-
 .../main/java/tajo/engine/json/GsonCreator.java    |    2 +
 .../main/java/tajo/engine/parser/EvalTreeBin.java  |   31 --
 .../main/java/tajo/engine/parser/ParseUtil.java    |   44 --
 .../java/tajo/engine/parser/QueryAnalyzer.java     |  112 +++--
 .../main/java/tajo/engine/parser/QueryBlock.java   |    3 +-
 .../java/tajo/engine/planner/LogicalPlanner.java   |    2 +-
 .../main/java/tajo/engine/planner/PlannerUtil.java |   10 +-
 .../engine/planner/RangePartitionAlgorithm.java    |   30 +-
 .../tajo/engine/planner/UniformRangePartition.java |  138 +++---
 .../engine/planner/physical/ExternalSortExec.java  |    6 +-
 .../engine/planner/physical/IndexedStoreExec.java  |    2 +-
 .../planner/physical/JoinTupleComparator.java      |    8 +-
 .../planner/physical/PartitionedStoreExec.java     |    4 +-
 .../engine/planner/physical/StoreTableExec.java    |    6 +-
 .../main/java/tajo/engine/query/ResultSetImpl.java |   24 +-
 .../tajo/engine/query/ResultSetMetaDataImpl.java   |   60 ++--
 .../query/exception/InvalidCastException.java      |   35 --
 .../query/exception/InvalidEvalException.java      |   30 --
 .../main/java/tajo/engine/utils/SchemaUtil.java    |    4 +-
 .../src/main/java/tajo/engine/utils/TupleUtil.java |  215 +++------
 .../src/main/java/tajo/master/GlobalEngine.java    |    8 +-
 .../src/main/java/tajo/master/GlobalPlanner.java   |   16 +-
 .../main/java/tajo/master/GlobalPlannerUtils.java  |    4 +-
 .../src/main/java/tajo/master/Query.java           |    7 +-
 .../src/main/java/tajo/master/Repartitioner.java   |    6 +-
 .../src/main/java/tajo/master/SubQuery.java        |    8 +-
 .../src/main/java/tajo/master/TajoMaster.java      |  103 ++--
 .../src/main/proto/CatalogProtos.proto             |  183 ++++----
 .../src/test/java/tajo/BackendTestingUtil.java     |   12 +-
 .../test/java/tajo/LocalTajoTestingUtility.java    |    7 +-
 .../src/test/java/tajo/TajoTestingCluster.java     |    6 +-
 .../src/test/java/tajo/client/TestTajoClient.java  |    4 +-
 .../test/java/tajo/engine/eval/TestEvalTree.java   |  147 +++---
 .../java/tajo/engine/eval/TestEvalTreeUtil.java    |   64 ++--
 .../java/tajo/engine/function/TestAggFunction.java |   10 +-
 .../tajo/engine/function/TestGeneralFunction.java  |   10 +-
 .../java/tajo/engine/parser/TestNQLParser.java     |   92 ++++-
 .../java/tajo/engine/parser/TestQueryAnalyzer.java |  116 +++---
 .../engine/plan/global/TestGlobalQueryPlanner.java |   28 +-
 .../tajo/engine/planner/TestLogicalOptimizer.java  |   24 +-
 .../tajo/engine/planner/TestLogicalPlanner.java    |   74 ++--
 .../java/tajo/engine/planner/TestPlannerUtil.java  |   85 ++--
 .../engine/planner/TestUniformRangePartition.java  |  148 +++---
 .../planner/global/TestGlobalQueryOptimizer.java   |   23 +-
 .../engine/planner/physical/TestBNLJoinExec.java   |   54 +-
 .../engine/planner/physical/TestBSTIndexExec.java  |   24 +-
 .../planner/physical/TestExternalSortExec.java     |   16 +-
 .../engine/planner/physical/TestHashJoinExec.java  |   50 +-
 .../planner/physical/TestHashPartitioner.java      |   30 +-
 .../engine/planner/physical/TestMergeJoinExec.java |   60 ++--
 .../engine/planner/physical/TestNLJoinExec.java    |   56 ++--
 .../planner/physical/TestPhysicalPlanner.java      |  128 +++---
 .../tajo/engine/planner/physical/TestSortExec.java |   22 +-
 .../java/tajo/engine/query/TestNullValues.java     |   30 +-
 .../java/tajo/engine/query/TestResultSetImpl.java  |   14 +-
 .../test/java/tajo/engine/util/TestTupleUtil.java  |  118 +++---
 .../java/tajo/master/TestExecutionBlockCursor.java |    9 +-
 .../src/test/java/tajo/storage/TestFragment.java   |   10 +-
 .../src/test/java/tajo/storage/TestHCFile.java     |  268 -----------
 .../test/java/tajo/storage/TestHColumnReader.java  |  145 ------
 .../src/test/java/tajo/storage/TestRowFile.java    |   22 +-
 .../src/test/java/tajo/worker/SlowFunc.java        |   65 ---
 .../tajo/worker/TestRangeRetrieverHandler.java     |   30 +-
 .../src/main/java/tajo/storage/CSVFile.java        |   82 ++--
 .../src/main/java/tajo/storage/Fragment.java       |    2 +-
 .../src/main/java/tajo/storage/FrameTuple.java     |   46 +-
 .../src/main/java/tajo/storage/RawFile.java        |  108 ++--
 .../src/main/java/tajo/storage/RowFile.java        |  114 +++---
 .../src/main/java/tajo/storage/RowStoreUtil.java   |   94 ++--
 .../src/main/java/tajo/storage/StorageUtil.java    |   21 +-
 .../main/java/tajo/storage/TableStatistics.java    |   14 +-
 .../src/main/java/tajo/storage/Tuple.java          |   22 +-
 .../main/java/tajo/storage/TupleComparator.java    |    8 +-
 .../src/main/java/tajo/storage/VTuple.java         |   46 +-
 .../main/java/tajo/storage/hcfile/BasicBlock.java  |  140 ------
 .../src/main/java/tajo/storage/hcfile/Block.java   |   71 ---
 .../main/java/tajo/storage/hcfile/BlockMeta.java   |   50 --
 .../main/java/tajo/storage/hcfile/BlockReader.java |   57 ---
 .../main/java/tajo/storage/hcfile/BlockWriter.java |   57 ---
 .../java/tajo/storage/hcfile/ColumnAppender.java   |   35 --
 .../tajo/storage/hcfile/ColumnFileAppender.java    |   41 --
 .../tajo/storage/hcfile/ColumnFileScanner.java     |   33 --
 .../main/java/tajo/storage/hcfile/ColumnMeta.java  |   48 --
 .../java/tajo/storage/hcfile/ColumnReader.java     |   42 --
 .../java/tajo/storage/hcfile/ColumnScanner.java    |  107 ----
 .../java/tajo/storage/hcfile/ColumnStoreUtil.java  |   36 --
 .../java/tajo/storage/hcfile/CompressedBlock.java  |   26 -
 .../tajo/storage/hcfile/CompressedBlockReader.java |   53 --
 .../tajo/storage/hcfile/CompressedBlockWriter.java |   62 ---
 .../java/tajo/storage/hcfile/HBlockMetaImpl.java   |   94 ----
 .../src/main/java/tajo/storage/hcfile/HCFile.java  |  373 ---------------
 .../java/tajo/storage/hcfile/HCFileMetaImpl.java   |  301 ------------
 .../java/tajo/storage/hcfile/HCTupleAppender.java  |  194 --------
 .../tajo/storage/hcfile/HColumnMetaWritable.java   |  164 -------
 .../java/tajo/storage/hcfile/HColumnReader.java    |  259 ----------
 .../src/main/java/tajo/storage/hcfile/Index.java   |  170 -------
 .../main/java/tajo/storage/hcfile/IndexItem.java   |   62 ---
 .../main/java/tajo/storage/hcfile/Seekable.java    |   30 --
 .../java/tajo/storage/hcfile/TupleAppender.java    |   31 --
 .../main/java/tajo/storage/hcfile/Updatable.java   |   36 --
 .../java/tajo/storage/hcfile/UpdatableBlock.java   |   22 -
 .../storage/hcfile/UpdatableSeekableBlock.java     |   23 -
 .../tajo/storage/hcfile/compress/BitVector.java    |   44 --
 .../java/tajo/storage/hcfile/compress/Codec.java   |   55 ---
 .../tajo/storage/hcfile/compress/Dictionary.java   |   44 --
 .../main/java/tajo/storage/hcfile/compress/LZ.java |   44 --
 .../tajo/storage/hcfile/compress/NullSuppress.java |   44 --
 .../tajo/storage/hcfile/compress/RunLength.java    |   44 --
 .../java/tajo/storage/hcfile/compress/Snappy.java  |   44 --
 .../tajo/storage/hcfile/reader/ArrayReader.java    |   44 --
 .../tajo/storage/hcfile/reader/ByteReader.java     |   37 --
 .../tajo/storage/hcfile/reader/BytesReader.java    |   40 --
 .../tajo/storage/hcfile/reader/CharReader.java     |   37 --
 .../tajo/storage/hcfile/reader/DoubleReader.java   |   37 --
 .../tajo/storage/hcfile/reader/FloatReader.java    |   37 --
 .../tajo/storage/hcfile/reader/IPv4Reader.java     |   40 --
 .../java/tajo/storage/hcfile/reader/IntReader.java |   37 --
 .../tajo/storage/hcfile/reader/LongReader.java     |   37 --
 .../java/tajo/storage/hcfile/reader/Reader.java    |   32 --
 .../tajo/storage/hcfile/reader/ShortReader.java    |   37 --
 .../tajo/storage/hcfile/reader/String2Reader.java  |   39 --
 .../tajo/storage/hcfile/reader/StringReader.java   |   40 --
 .../tajo/storage/hcfile/reader/TypeReader.java     |   69 ---
 .../tajo/storage/hcfile/writer/ArrayWriter.java    |   41 --
 .../tajo/storage/hcfile/writer/ByteWriter.java     |   36 --
 .../tajo/storage/hcfile/writer/BytesWriter.java    |   38 --
 .../tajo/storage/hcfile/writer/CharWriter.java     |   36 --
 .../tajo/storage/hcfile/writer/DoubleWriter.java   |   36 --
 .../tajo/storage/hcfile/writer/FloatWriter.java    |   36 --
 .../java/tajo/storage/hcfile/writer/IntWriter.java |   36 --
 .../tajo/storage/hcfile/writer/LongWriter.java     |   36 --
 .../tajo/storage/hcfile/writer/ShortWriter.java    |   36 --
 .../tajo/storage/hcfile/writer/TypeWriter.java     |   90 ----
 .../java/tajo/storage/hcfile/writer/Writer.java    |   31 --
 .../java/tajo/storage/rcfile/RCFileWrapper.java    |   73 ++--
 .../java/tajo/storage/trevni/TrevniAppender.java   |   61 ++--
 .../java/tajo/storage/trevni/TrevniScanner.java    |   48 +-
 .../src/main/proto/CatalogProtos.proto             |  185 ++++----
 .../src/test/java/tajo/storage/TestFrameTuple.java |   44 +-
 .../test/java/tajo/storage/TestMergeScanner.java   |   30 +-
 .../test/java/tajo/storage/TestStorageManager.java |   18 +-
 .../src/test/java/tajo/storage/TestStorages.java   |   83 ++--
 .../java/tajo/storage/TestTupleComparator.java     |   32 +-
 .../src/test/java/tajo/storage/TestVTuple.java     |   75 ++--
 .../test/java/tajo/storage/hcfile/IndexTest.java   |   66 ---
 .../test/java/tajo/storage/index/TestBSTIndex.java |  276 ++++++------
 .../storage/index/TestSingleCSVFileBSTIndex.java   |   68 ++--
 .../test/java/tajo/storage/rcfile/TestRCFile.java  |   40 +-
 254 files changed, 3404 insertions(+), 11057 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-catalog/tajo-catalog-client/src/main/java/tajo/catalog/AbstractCatalogClient.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-client/src/main/java/tajo/catalog/AbstractCatalogClient.java b/tajo-catalog/tajo-catalog-client/src/main/java/tajo/catalog/AbstractCatalogClient.java
index ce0efcb..f9dd78c 100644
--- a/tajo-catalog/tajo-catalog-client/src/main/java/tajo/catalog/AbstractCatalogClient.java
+++ b/tajo-catalog/tajo-catalog-client/src/main/java/tajo/catalog/AbstractCatalogClient.java
@@ -23,6 +23,7 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import tajo.catalog.CatalogProtocol.CatalogProtocolService;
 import tajo.catalog.proto.CatalogProtos.*;
+import tajo.common.TajoDataTypes.DataType;
 import tajo.rpc.protocolrecords.PrimitiveProtos.NullProto;
 import tajo.rpc.protocolrecords.PrimitiveProtos.StringProto;
 
@@ -48,7 +49,7 @@ public abstract class AbstractCatalogClient implements CatalogService {
   @Override
   public final TableDesc getTableDesc(final String name) {
     try {
-      return TCatUtil.newTableDesc(stub.getTableDesc(null, StringProto.newBuilder()
+      return CatalogUtil.newTableDesc(stub.getTableDesc(null, StringProto.newBuilder()
           .setValue(name).build()));
     } catch (ServiceException e) {
       LOG.error(e);
@@ -210,9 +211,8 @@ public abstract class AbstractCatalogClient implements CatalogService {
     UnregisterFunctionRequest.Builder builder =
         UnregisterFunctionRequest.newBuilder();
     builder.setSignature(signature);
-    int size = paramTypes.length;
-    for (int i = 0; i < size; i++) {
-      builder.addParameterTypes(paramTypes[i]);
+    for (DataType type : paramTypes) {
+      builder.addParameterTypes(type);
     }
     try {
       return stub.unregisterFunction(null, builder.build()).getValue();
@@ -228,9 +228,8 @@ public abstract class AbstractCatalogClient implements CatalogService {
     GetFunctionMetaRequest.Builder builder =
         GetFunctionMetaRequest.newBuilder();
     builder.setSignature(signature);
-    int size = paramTypes.length;
-    for (int i = 0; i < size; i++) {
-      builder.addParameterTypes(paramTypes[i]);
+    for (DataType type : paramTypes) {
+      builder.addParameterTypes(type);
     }
 
     FunctionDescProto descProto;
@@ -249,9 +248,8 @@ public abstract class AbstractCatalogClient implements CatalogService {
     ContainFunctionRequest.Builder builder =
         ContainFunctionRequest.newBuilder();
     builder.setSignature(signature);
-    int size = paramTypes.length;
-    for (int i = 0; i < size; i++) {
-      builder.addParameterTypes(paramTypes[i]);
+    for (DataType type : paramTypes) {
+      builder.addParameterTypes(type);
     }
     try {
       return stub.containFunction(null, builder.build()).getValue();

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-catalog/tajo-catalog-client/src/main/java/tajo/catalog/AlreadyRegisteredURIException.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-client/src/main/java/tajo/catalog/AlreadyRegisteredURIException.java b/tajo-catalog/tajo-catalog-client/src/main/java/tajo/catalog/AlreadyRegisteredURIException.java
deleted file mode 100644
index 69df366..0000000
--- a/tajo-catalog/tajo-catalog-client/src/main/java/tajo/catalog/AlreadyRegisteredURIException.java
+++ /dev/null
@@ -1,40 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package tajo.catalog;
-
-import java.net.URI;
-
-public class AlreadyRegisteredURIException extends RuntimeException {
-
-	private static final long serialVersionUID = 747390434221048348L;
-
-	public AlreadyRegisteredURIException() {
-	}
-
-	/**
-	 * @param uri
-	 */
-	public AlreadyRegisteredURIException(String uri) {
-		super("Already registered TRID: "+uri);
-	}
-	
-	public AlreadyRegisteredURIException(URI uri) {
-		this("Already registered TRID: "+uri);
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-catalog/tajo-catalog-client/src/main/java/tajo/catalog/AttributeType.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-client/src/main/java/tajo/catalog/AttributeType.java b/tajo-catalog/tajo-catalog-client/src/main/java/tajo/catalog/AttributeType.java
deleted file mode 100644
index 49e757b..0000000
--- a/tajo-catalog/tajo-catalog-client/src/main/java/tajo/catalog/AttributeType.java
+++ /dev/null
@@ -1,25 +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 tajo.catalog;
-
-public enum AttributeType {
-	GROUPBY,
-	AGGREGATION,
-	NORMAL
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-catalog/tajo-catalog-client/src/main/java/tajo/catalog/CatalogReader.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-client/src/main/java/tajo/catalog/CatalogReader.java b/tajo-catalog/tajo-catalog-client/src/main/java/tajo/catalog/CatalogReader.java
deleted file mode 100644
index 3c61772..0000000
--- a/tajo-catalog/tajo-catalog-client/src/main/java/tajo/catalog/CatalogReader.java
+++ /dev/null
@@ -1,31 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package tajo.catalog;
-
-import tajo.catalog.proto.CatalogProtos.DataType;
-
-public interface CatalogReader {
-  boolean existsTable(String tableId);
-  
-  TableDesc getTableDesc(String tableId);
-  
-  FunctionDesc getFunctionMeta(String signature, DataType... paramTypes);
-  
-  boolean containFunction(String signature, DataType... paramTypes);
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-catalog/tajo-catalog-client/src/main/java/tajo/catalog/CatalogService.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-client/src/main/java/tajo/catalog/CatalogService.java b/tajo-catalog/tajo-catalog-client/src/main/java/tajo/catalog/CatalogService.java
index 290daa0..823ade1 100644
--- a/tajo-catalog/tajo-catalog-client/src/main/java/tajo/catalog/CatalogService.java
+++ b/tajo-catalog/tajo-catalog-client/src/main/java/tajo/catalog/CatalogService.java
@@ -18,7 +18,7 @@
 
 package tajo.catalog;
 
-import tajo.catalog.proto.CatalogProtos.DataType;
+import tajo.common.TajoDataTypes.DataType;
 
 import java.util.Collection;
 

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-catalog/tajo-catalog-client/src/main/java/tajo/catalog/CatalogServiceProtocol.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-client/src/main/java/tajo/catalog/CatalogServiceProtocol.java b/tajo-catalog/tajo-catalog-client/src/main/java/tajo/catalog/CatalogServiceProtocol.java
deleted file mode 100644
index bdc7e1a..0000000
--- a/tajo-catalog/tajo-catalog-client/src/main/java/tajo/catalog/CatalogServiceProtocol.java
+++ /dev/null
@@ -1,105 +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 tajo.catalog;
-
-import tajo.catalog.proto.CatalogProtos.*;
-import tajo.rpc.protocolrecords.PrimitiveProtos.BoolProto;
-import tajo.rpc.protocolrecords.PrimitiveProtos.NullProto;
-import tajo.rpc.protocolrecords.PrimitiveProtos.StringProto;
-
-public interface CatalogServiceProtocol {
-  
-  /**
-   * Get a table description by name
-   * @param name table name
-   * @return a table description
-   * @see TableDescImpl
-   * @throws Throwable
-   */
-  TableDescProto getTableDesc(StringProto name);
-  
-  /**
-   * 
-   * @return
-   * @throws tajo.catalog.exception.CatalogException
-   */
-  GetAllTableNamesResponse getAllTableNames(NullProto request);
-  
-  /**
-   * 
-   * @return
-   * @throws tajo.catalog.exception.CatalogException
-   */
-  GetFunctionsResponse getFunctions(NullProto request);
-  
-  /**
-   * Add a table via table description
-   * @param meta table meta
-   * @see TableDescImpl
-   * @throws Throwable
-   */
-  void addTable(TableDescProto desc);
-  
-  /**
-   * Drop a table by name
-   * @param name table name
-   * @throws Throwable
-   */
-  void deleteTable(StringProto name);
-  
-  BoolProto existsTable(StringProto tableId);
-  
-  void addIndex(IndexDescProto proto);
-  
-  BoolProto existIndex(StringProto indexName);
-  
-  BoolProto existIndex(GetIndexRequest req);
-  
-  IndexDescProto getIndex(StringProto indexName);
-  
-  IndexDescProto getIndex(GetIndexRequest req);
-  
-  void delIndex(StringProto indexName);
-  
-  /**
-   * 
-   * @param funcDesc
-   */
-  void registerFunction(FunctionDescProto funcDesc);
-  
-  /**
-   * 
-   * @param signature
-   */
-  void unregisterFunction(UnregisterFunctionRequest request);
-  
-  /**
-   * 
-   * @param signature
-   * @return
-   */
-  FunctionDescProto getFunctionMeta(GetFunctionMetaRequest request);
-  
-  /**
-   * 
-   * @param signature
-   * @return
-   */
-  BoolProto containFunction(ContainFunctionRequest request);
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-catalog/tajo-catalog-client/src/main/proto/CatalogProtos.proto
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-client/src/main/proto/CatalogProtos.proto b/tajo-catalog/tajo-catalog-client/src/main/proto/CatalogProtos.proto
index dd1bf69..acb89fe 100644
--- a/tajo-catalog/tajo-catalog-client/src/main/proto/CatalogProtos.proto
+++ b/tajo-catalog/tajo-catalog-client/src/main/proto/CatalogProtos.proto
@@ -22,28 +22,7 @@ option optimize_for = SPEED;
 option java_generic_services = false;
 option java_generate_equals_and_hash = true;
 
-enum DataType {
-    BOOLEAN = 0;
-    BYTE = 1;
-    CHAR = 2;
-    BYTES = 3;
-    SHORT = 4;
-    INT = 5;
-    BIGINT = 6;
-    LONG = 7;
-    FLOAT = 8;
-    DOUBLE = 9;
-    BIGDECIMAL = 10;
-    STRING = 11;
-    DATE = 12;
-    IPv4 = 13;
-    IPv6 = 14;
-    NULL = 15;
-    ANY = 16;
-    ALL = 17;
-    ARRAY = 18;
-    STRING2 = 19;
-}
+import "DataTypes.proto";
 
 enum StoreType {
 	MEM = 0;

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-catalog/tajo-catalog-common/src/main/java/tajo/catalog/CatalogUtil.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-common/src/main/java/tajo/catalog/CatalogUtil.java b/tajo-catalog/tajo-catalog-common/src/main/java/tajo/catalog/CatalogUtil.java
index de193f3..41a72de 100644
--- a/tajo-catalog/tajo-catalog-common/src/main/java/tajo/catalog/CatalogUtil.java
+++ b/tajo-catalog/tajo-catalog-common/src/main/java/tajo/catalog/CatalogUtil.java
@@ -18,10 +18,13 @@
 
 package tajo.catalog;
 
+import org.apache.hadoop.fs.Path;
 import tajo.catalog.proto.CatalogProtos;
 import tajo.catalog.proto.CatalogProtos.ColumnProto;
-import tajo.catalog.proto.CatalogProtos.DataType;
 import tajo.catalog.proto.CatalogProtos.SchemaProto;
+import tajo.catalog.proto.CatalogProtos.TableDescProto;
+import tajo.catalog.statistics.TableStat;
+import tajo.common.TajoDataTypes.DataType;
 import tajo.util.FileUtil;
 
 import java.io.File;
@@ -29,12 +32,12 @@ import java.io.IOException;
 import java.util.Collection;
 
 import static tajo.catalog.proto.CatalogProtos.StoreType;
+import static tajo.common.TajoDataTypes.Type;
 
 public class CatalogUtil {
   public static String getCanonicalName(String signature,
       Collection<DataType> paramTypes) {
-    DataType [] types = paramTypes.toArray(
-        new DataType[paramTypes.size()]);
+    DataType [] types = paramTypes.toArray(new DataType[paramTypes.size()]);
     return getCanonicalName(signature, types);
   }
   public static String getCanonicalName(String signature,
@@ -43,7 +46,7 @@ public class CatalogUtil {
     sb.append("(");
     int i = 0;
     for (DataType type : paramTypes) {
-      sb.append(type);
+      sb.append(type.getType());
       if(i < paramTypes.length - 1) {
         sb.append(",");
       }
@@ -53,58 +56,6 @@ public class CatalogUtil {
     sb.append(")");
     return sb.toString();
   }
-  
-  public static char getTypeCode(DataType type) {
-    switch(type) {
-    case BOOLEAN: return 'Z';
-    case BYTE: return 'B';
-    case SHORT: return 'S';
-    case INT: return 'I';
-    case LONG: return 'J';
-    case FLOAT: return 'F';
-    case DOUBLE: return 'D';
-    case BYTES: return 'N';
-    case IPv4: return '4';
-    case IPv6: return '6';
-    default: throw new InternalError("Unsupported type exception");
-    }
-  }
-
-    /**
-   * This method transforms the unqualified names of a given schema into
-   * the qualified names.
-   * 
-   * @param tableName a table name to be prefixed
-   * @param schema a schema to be transformed
-   * 
-   * @return
-   */
-
-  public static SchemaProto getQualfiedSchema(String tableName,
-      SchemaProto schema) {
-    SchemaProto.Builder revisedSchema = SchemaProto.newBuilder(schema);
-    revisedSchema.clearFields();
-    String[] split;
-    for (ColumnProto col : schema.getFieldsList()) {
-      split = col.getColumnName().split("\\.");
-      if (split.length == 1) { // if not qualified name
-        // rewrite the column
-        ColumnProto.Builder builder = ColumnProto.newBuilder(col);
-        builder.setColumnName(tableName + "." + col.getColumnName());
-        col = builder.build();
-      } else if (split.length == 2) {
-        ColumnProto.Builder builder = ColumnProto.newBuilder(col);
-        builder.setColumnName(tableName + "." + split[1]);
-        col = builder.build();
-      } else {
-        throw new InternalError("Unaccetable field name "
-            + col.getColumnName());
-      }
-      revisedSchema.addFields(col);
-    }
-
-    return revisedSchema.build();
-  }
 
   public static String prettyPrint(TableMeta meta) {
     StringBuilder sb = new StringBuilder();
@@ -152,4 +103,79 @@ public class CatalogUtil {
       return null;
     }
   }
+
+  public static TableMeta newTableMeta(Schema schema, StoreType type) {
+    return new TableMetaImpl(schema, type, new Options());
+  }
+
+  public static TableMeta newTableMeta(Schema schema, StoreType type,
+      Options options) {
+    return new TableMetaImpl(schema, type, options);
+  }
+
+  public static TableMeta newTableMeta(Schema schema, StoreType type, Options options,
+      TableStat stat) {
+    return new TableMetaImpl(schema, type, options, stat);
+  }
+
+  public static TableDesc newTableDesc(String tableName, TableMeta meta,
+      Path path) {
+    return new TableDescImpl(tableName, meta, path);
+  }
+
+  public static TableDesc newTableDesc(TableDescProto proto) {
+    return new TableDescImpl(proto);
+  }
+
+  public static TableDesc newTableDesc(String tableName,
+      Schema schema, StoreType type, Options options, Path path) {
+    return new TableDescImpl(tableName, schema, type, options, path);
+  }
+
+  /**
+  * This method transforms the unqualified names of a given schema into
+  * the qualified names.
+  *
+  * @param tableName a table name to be prefixed
+  * @param schema a schema to be transformed
+  *
+  * @return
+  */
+  public static SchemaProto getQualfiedSchema(String tableName,
+      SchemaProto schema) {
+    SchemaProto.Builder revisedSchema = SchemaProto.newBuilder(schema);
+    revisedSchema.clearFields();
+    String[] split;
+    for (ColumnProto col : schema.getFieldsList()) {
+      split = col.getColumnName().split("\\.");
+      if (split.length == 1) { // if not qualified name
+        // rewrite the column
+        ColumnProto.Builder builder = ColumnProto.newBuilder(col);
+        builder.setColumnName(tableName + "." + col.getColumnName());
+        col = builder.build();
+      } else if (split.length == 2) {
+        ColumnProto.Builder builder = ColumnProto.newBuilder(col);
+        builder.setColumnName(tableName + "." + split[1]);
+        col = builder.build();
+      } else {
+        throw new InternalError("Unaccetable field name "
+            + col.getColumnName());
+      }
+      revisedSchema.addFields(col);
+    }
+
+    return revisedSchema.build();
+  }
+
+  public static DataType newDataTypeWithoutLen(Type type) {
+    return DataType.newBuilder().setType(type).build();
+  }
+
+  public static DataType [] newDataTypesWithoutLen(Type... types) {
+    DataType [] dataTypes = new DataType[types.length];
+    for (int i = 0; i < types.length; i++) {
+      dataTypes[i] = DataType.newBuilder().setType(types[i]).build();
+    }
+    return dataTypes;
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-catalog/tajo-catalog-common/src/main/java/tajo/catalog/Column.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-common/src/main/java/tajo/catalog/Column.java b/tajo-catalog/tajo-catalog-common/src/main/java/tajo/catalog/Column.java
index 0deff1f..ecdc4c3 100644
--- a/tajo-catalog/tajo-catalog-common/src/main/java/tajo/catalog/Column.java
+++ b/tajo-catalog/tajo-catalog-common/src/main/java/tajo/catalog/Column.java
@@ -22,8 +22,9 @@ import com.google.gson.annotations.Expose;
 import tajo.catalog.json.GsonCreator;
 import tajo.catalog.proto.CatalogProtos.ColumnProto;
 import tajo.catalog.proto.CatalogProtos.ColumnProtoOrBuilder;
-import tajo.catalog.proto.CatalogProtos.DataType;
 import tajo.common.ProtoObject;
+import tajo.common.TajoDataTypes;
+import tajo.common.TajoDataTypes.DataType;
 
 public class Column implements ProtoObject<ColumnProto>, Cloneable {
 	private ColumnProto proto = ColumnProto.getDefaultInstance();
@@ -42,6 +43,10 @@ public class Column implements ProtoObject<ColumnProto>, Cloneable {
 		this.name = columnName.toLowerCase();
 		this.dataType = dataType;
 	}
+
+  public Column(String columnName, TajoDataTypes.Type type) {
+    this(columnName, CatalogUtil.newDataTypeWithoutLen(type));
+  }
 	
 	public Column(ColumnProto proto) {
 		this.proto = proto;
@@ -108,11 +113,8 @@ public class Column implements ProtoObject<ColumnProto>, Cloneable {
 	public boolean equals(Object o) {
 		if (o instanceof Column) {
 			Column cd = (Column)o;
-			if (this.getQualifiedName().equals(cd.getQualifiedName()) &&
-					this.getDataType() == cd.getDataType()
-					) {
-				return true;
-			}
+			return this.getQualifiedName().equals(cd.getQualifiedName()) &&
+					this.getDataType().equals(cd.getDataType());
 		}
 		return false;
 	}
@@ -164,7 +166,7 @@ public class Column implements ProtoObject<ColumnProto>, Cloneable {
 	}
 	
 	public String toString() {
-	  return getQualifiedName() +" (" + getDataType()+")";
+	  return getQualifiedName() +" (" + getDataType().getType() +")";
 	}
 	
 	public String toJSON() {

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-catalog/tajo-catalog-common/src/main/java/tajo/catalog/FunctionDesc.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-common/src/main/java/tajo/catalog/FunctionDesc.java b/tajo-catalog/tajo-catalog-common/src/main/java/tajo/catalog/FunctionDesc.java
index f11da36..7fdf336 100644
--- a/tajo-catalog/tajo-catalog-common/src/main/java/tajo/catalog/FunctionDesc.java
+++ b/tajo-catalog/tajo-catalog-common/src/main/java/tajo/catalog/FunctionDesc.java
@@ -23,11 +23,11 @@ import com.google.gson.annotations.Expose;
 import tajo.catalog.function.Function;
 import tajo.catalog.function.GeneralFunction;
 import tajo.catalog.json.GsonCreator;
-import tajo.catalog.proto.CatalogProtos.DataType;
 import tajo.catalog.proto.CatalogProtos.FunctionDescProto;
 import tajo.catalog.proto.CatalogProtos.FunctionDescProtoOrBuilder;
 import tajo.catalog.proto.CatalogProtos.FunctionType;
 import tajo.common.ProtoObject;
+import tajo.common.TajoDataTypes.DataType;
 import tajo.exception.InternalException;
 
 import java.lang.reflect.Constructor;
@@ -65,7 +65,7 @@ public class FunctionDesc implements ProtoObject<FunctionDescProto>, Cloneable {
 
   @SuppressWarnings("unchecked")
   public FunctionDesc(String signature, String className, FunctionType type,
-      DataType [] retType, DataType... argTypes) throws ClassNotFoundException {
+                      DataType [] retType, DataType... argTypes) throws ClassNotFoundException {
     this(signature, (Class<? extends Function>) Class.forName(className), type,
         retType, argTypes);
   }
@@ -151,7 +151,7 @@ public class FunctionDesc implements ProtoObject<FunctionDescProto>, Cloneable {
     
   }
 
-  public static DataType[] newNoNameSchema(DataType... types) {
+  public static DataType [] newNoNameSchema(DataType ... types) {
     DataType [] dataTypes = types.clone();
     return dataTypes;
   }

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-catalog/tajo-catalog-common/src/main/java/tajo/catalog/Schema.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-common/src/main/java/tajo/catalog/Schema.java b/tajo-catalog/tajo-catalog-common/src/main/java/tajo/catalog/Schema.java
index 846bb36..d11170d 100644
--- a/tajo-catalog/tajo-catalog-common/src/main/java/tajo/catalog/Schema.java
+++ b/tajo-catalog/tajo-catalog-common/src/main/java/tajo/catalog/Schema.java
@@ -25,10 +25,11 @@ import org.apache.commons.logging.LogFactory;
 import tajo.catalog.exception.AlreadyExistsFieldException;
 import tajo.catalog.json.GsonCreator;
 import tajo.catalog.proto.CatalogProtos.ColumnProto;
-import tajo.catalog.proto.CatalogProtos.DataType;
 import tajo.catalog.proto.CatalogProtos.SchemaProto;
 import tajo.catalog.proto.CatalogProtos.SchemaProtoOrBuilder;
 import tajo.common.ProtoObject;
+import tajo.common.TajoDataTypes.DataType;
+import tajo.common.TajoDataTypes.Type;
 
 import java.util.*;
 
@@ -142,7 +143,11 @@ public class Schema implements ProtoObject<SchemaProto>, Cloneable {
 		}
 	}
 
-	public synchronized Schema addColumn(String name, DataType dataType) {
+  public synchronized Schema addColumn(String name, Type type) {
+    return addColumn(name, CatalogUtil.newDataTypeWithoutLen(type));
+  }
+
+  public synchronized Schema addColumn(String name, DataType dataType) {
 		initColumns();
 		setModified();
 		String lowcased = name.toLowerCase();

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-catalog/tajo-catalog-common/src/main/java/tajo/catalog/TCatUtil.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-common/src/main/java/tajo/catalog/TCatUtil.java b/tajo-catalog/tajo-catalog-common/src/main/java/tajo/catalog/TCatUtil.java
deleted file mode 100644
index 5ce0e50..0000000
--- a/tajo-catalog/tajo-catalog-common/src/main/java/tajo/catalog/TCatUtil.java
+++ /dev/null
@@ -1,130 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package tajo.catalog;
-
-import org.apache.hadoop.fs.Path;
-import tajo.catalog.proto.CatalogProtos.*;
-import tajo.catalog.statistics.TableStat;
-
-import java.util.Collection;
-
-public class TCatUtil {
-  public static String getCanonicalName(String signature,
-      Collection<DataType> paramTypes) {
-    DataType [] types = paramTypes.toArray(
-        new DataType[paramTypes.size()]);
-    return getCanonicalName(signature, types);
-  }
-  public static String getCanonicalName(String signature,
-      DataType...paramTypes) {
-    StringBuilder sb = new StringBuilder(signature);
-    sb.append("(");
-    int i = 0;
-    for (DataType type : paramTypes) {
-      sb.append(type);
-      if(i < paramTypes.length - 1) {
-        sb.append(",");
-      }
-      
-      i++;
-    }
-    sb.append(")");
-    return sb.toString();
-  }
-  
-  public static char getTypeCode(DataType type) {
-    switch(type) {
-    case BOOLEAN: return 'Z';
-    case BYTE: return 'B';
-    case SHORT: return 'S';
-    case INT: return 'I';
-    case LONG: return 'J';
-    case FLOAT: return 'F';
-    case DOUBLE: return 'D';
-    case BYTES: return 'N';
-    case IPv4: return '4';
-    case IPv6: return '6';
-    default: throw new InternalError("Unsupported type exception");
-    }
-  }
-
-    /**
-   * This method transforms the unqualified names of a given schema into
-   * the qualified names.
-   * 
-   * @param tableName a table name to be prefixed
-   * @param schema a schema to be transformed
-   * 
-   * @return
-   */
-
-  public static SchemaProto getQualfiedSchema(String tableName,
-      SchemaProto schema) {
-    SchemaProto.Builder revisedSchema = SchemaProto.newBuilder(schema);
-    revisedSchema.clearFields();
-    String[] split;
-    for (ColumnProto col : schema.getFieldsList()) {
-      split = col.getColumnName().split("\\.");
-      if (split.length == 1) { // if not qualified name
-        // rewrite the column
-        ColumnProto.Builder builder = ColumnProto.newBuilder(col);
-        builder.setColumnName(tableName + "." + col.getColumnName());
-        col = builder.build();
-      } else if (split.length == 2) {
-        ColumnProto.Builder builder = ColumnProto.newBuilder(col);
-        builder.setColumnName(tableName + "." + split[1]);
-        col = builder.build();
-      } else {
-        throw new InternalError("Unaccetable field name "
-            + col.getColumnName());
-      }
-      revisedSchema.addFields(col);
-    }
-
-    return revisedSchema.build();
-  }
-  
-  public static TableMeta newTableMeta(Schema schema, StoreType type) {
-    return new TableMetaImpl(schema, type, new Options());
-  }
-  
-  public static TableMeta newTableMeta(Schema schema, StoreType type, 
-      Options options) {
-    return new TableMetaImpl(schema, type, options);
-  }
-  
-  public static TableMeta newTableMeta(Schema schema, StoreType type, Options options, 
-      TableStat stat) {
-    return new TableMetaImpl(schema, type, options, stat);
-  }
-  
-  public static TableDesc newTableDesc(String tableName, TableMeta meta, 
-      Path path) {
-    return new TableDescImpl(tableName, meta, path);
-  }
-  
-  public static TableDesc newTableDesc(TableDescProto proto) {
-    return new TableDescImpl(proto);
-  }
-  
-  public static TableDesc newTableDesc(String tableName, 
-      Schema schema, StoreType type, Options options, Path path) {
-    return new TableDescImpl(tableName, schema, type, options, path);
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-catalog/tajo-catalog-common/src/main/java/tajo/catalog/function/AggFunction.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-common/src/main/java/tajo/catalog/function/AggFunction.java b/tajo-catalog/tajo-catalog-common/src/main/java/tajo/catalog/function/AggFunction.java
index a337afb..38d1060 100644
--- a/tajo-catalog/tajo-catalog-common/src/main/java/tajo/catalog/function/AggFunction.java
+++ b/tajo-catalog/tajo-catalog-common/src/main/java/tajo/catalog/function/AggFunction.java
@@ -21,7 +21,7 @@ package tajo.catalog.function;
 import com.google.gson.Gson;
 import tajo.catalog.Column;
 import tajo.catalog.json.GsonCreator;
-import tajo.catalog.proto.CatalogProtos;
+import tajo.common.TajoDataTypes.DataType;
 import tajo.datum.Datum;
 import tajo.storage.Tuple;
 
@@ -41,7 +41,7 @@ public abstract class AggFunction<T extends Datum> extends Function<T> {
 
   public abstract Datum getPartialResult(FunctionContext ctx);
 
-  public abstract CatalogProtos.DataType [] getPartialResultType();
+  public abstract DataType [] getPartialResultType();
 
   public abstract T terminate(FunctionContext ctx);
 

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-catalog/tajo-catalog-common/src/main/java/tajo/catalog/json/GsonCreator.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-common/src/main/java/tajo/catalog/json/GsonCreator.java b/tajo-catalog/tajo-catalog-common/src/main/java/tajo/catalog/json/GsonCreator.java
index 24a817f..ee515ff 100644
--- a/tajo-catalog/tajo-catalog-common/src/main/java/tajo/catalog/json/GsonCreator.java
+++ b/tajo-catalog/tajo-catalog-common/src/main/java/tajo/catalog/json/GsonCreator.java
@@ -26,10 +26,12 @@ import tajo.catalog.TableMeta;
 import tajo.catalog.function.AggFunction;
 import tajo.catalog.function.Function;
 import tajo.catalog.function.GeneralFunction;
+import tajo.common.TajoDataTypes.DataType;
 import tajo.datum.Datum;
 import tajo.datum.json.DatumAdapter;
 import tajo.gson.ClassNameDeserializer;
 import tajo.gson.ClassNameSerializer;
+import tajo.gson.DataTypeAdapter;
 
 public class GsonCreator {
 	private static GsonBuilder builder;
@@ -48,6 +50,7 @@ public class GsonCreator {
       builder.registerTypeAdapter(GeneralFunction.class, new FunctionAdapter());
       builder.registerTypeAdapter(AggFunction.class, new FunctionAdapter());
 			builder.registerTypeAdapter(Datum.class, new DatumAdapter());
+      builder.registerTypeAdapter(DataType.class, new DataTypeAdapter());
 		}
 	}
 

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-catalog/tajo-catalog-common/src/main/java/tajo/catalog/statistics/TupleUtil.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-common/src/main/java/tajo/catalog/statistics/TupleUtil.java b/tajo-catalog/tajo-catalog-common/src/main/java/tajo/catalog/statistics/TupleUtil.java
index ab3038c..5663005 100644
--- a/tajo-catalog/tajo-catalog-common/src/main/java/tajo/catalog/statistics/TupleUtil.java
+++ b/tajo-catalog/tajo-catalog-common/src/main/java/tajo/catalog/statistics/TupleUtil.java
@@ -18,37 +18,32 @@
 
 package tajo.catalog.statistics;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import tajo.catalog.proto.CatalogProtos;
+import tajo.common.TajoDataTypes.DataType;
 import tajo.datum.*;
 
 public class TupleUtil {
-  /** class logger **/
-  private static final Log LOG = LogFactory.getLog(TupleUtil.class);
-
-  public static Datum createFromBytes(CatalogProtos.DataType type, byte [] bytes) {
-    switch (type) {
+  public static Datum createFromBytes(DataType type, byte [] bytes) {
+    switch (type.getType()) {
       case BOOLEAN:
-        return new BoolDatum(bytes);
-      case BYTE:
-        return new ByteDatum(bytes);
+        return new BooleanDatum(bytes);
+      case BLOB:
+        return new BlobDatum(bytes);
       case CHAR:
         return new CharDatum(bytes);
-      case SHORT:
-        return new ShortDatum(bytes);
-      case INT:
-        return new IntDatum(bytes);
-      case LONG:
-        return new LongDatum(bytes);
-      case FLOAT:
-        return new FloatDatum(bytes);
-      case DOUBLE:
-        return new DoubleDatum(bytes);
-      case STRING:
-        return new StringDatum(bytes);
-      case IPv4:
-        return new IPv4Datum(bytes);
+      case INT2:
+        return new Int2Datum(bytes);
+      case INT4:
+        return new Int4Datum(bytes);
+      case INT8:
+        return new Int8Datum(bytes);
+      case FLOAT4:
+        return new Float4Datum(bytes);
+      case FLOAT8:
+        return new Float8Datum(bytes);
+      case TEXT:
+        return new TextDatum(bytes);
+      case INET4:
+        return new Inet4Datum(bytes);
       default: throw new UnsupportedOperationException(type + " is not supported yet");
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/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 dd1bf69..36b33c1 100644
--- a/tajo-catalog/tajo-catalog-common/src/main/proto/CatalogProtos.proto
+++ b/tajo-catalog/tajo-catalog-common/src/main/proto/CatalogProtos.proto
@@ -22,33 +22,12 @@ option optimize_for = SPEED;
 option java_generic_services = false;
 option java_generate_equals_and_hash = true;
 
-enum DataType {
-    BOOLEAN = 0;
-    BYTE = 1;
-    CHAR = 2;
-    BYTES = 3;
-    SHORT = 4;
-    INT = 5;
-    BIGINT = 6;
-    LONG = 7;
-    FLOAT = 8;
-    DOUBLE = 9;
-    BIGDECIMAL = 10;
-    STRING = 11;
-    DATE = 12;
-    IPv4 = 13;
-    IPv6 = 14;
-    NULL = 15;
-    ANY = 16;
-    ALL = 17;
-    ARRAY = 18;
-    STRING2 = 19;
-}
+import "DataTypes.proto";
 
 enum StoreType {
-	MEM = 0;
-	CSV = 1;
-	RAW = 2;
+  MEM = 0;
+  CSV = 1;
+  RAW = 2;
   RCFILE = 3;
   ROWFILE = 4;
   HCFILE = 5;
@@ -56,147 +35,147 @@ enum StoreType {
 }
 
 enum OrderType {
-    ORDER_NONE = 0;
-    ASC = 1;
-    DSC = 2;
+  ORDER_NONE = 0;
+  ASC = 1;
+  DSC = 2;
 }
 
 enum CompressType {
-    COMP_NONE = 0;
-    NULL_SUPPRESS = 1;
-    RUN_LENGTH = 2;
-    BIT_VECTOR = 3;
-    DICTIONARY = 4;
-    SNAPPY = 5;
-    LZ = 6;
+  COMP_NONE = 0;
+  NULL_SUPPRESS = 1;
+  RUN_LENGTH = 2;
+  BIT_VECTOR = 3;
+  DICTIONARY = 4;
+  SNAPPY = 5;
+  LZ = 6;
 }
 
 message ColumnMetaProto {
-    required DataType dataType = 1;
-    required bool compressed = 2;
-    required bool sorted = 3;
-    required bool contiguous = 4;
-    required StoreType storeType = 5;
-    required CompressType compType = 6;
-    required int64 startRid = 7;
-    required int32 recordNum = 8;
-    required int32 offsetToIndex = 9;
+  required DataType dataType = 1;
+  required bool compressed = 2;
+  required bool sorted = 3;
+  required bool contiguous = 4;
+  required StoreType storeType = 5;
+  required CompressType compType = 6;
+  required int64 startRid = 7;
+  required int32 recordNum = 8;
+  required int32 offsetToIndex = 9;
 }
 
 message ColumnProto {
-	required string columnName = 1;
-	required DataType dataType = 2;
+  required string columnName = 1;
+  required DataType dataType = 2;
 }
 
 message SchemaProto {
-	repeated ColumnProto fields = 1;
+  repeated ColumnProto fields = 1;
 }
 
 message KeyValueProto {
-	required string key = 1;
-	required string value = 2;
+  required string key = 1;
+  required string value = 2;
 }
 
 message KeyValueSetProto {
-	repeated KeyValueProto keyval = 1;
+  repeated KeyValueProto keyval = 1;
 }
 
 message TabletProto {
-	required string id = 1;
-	required string path = 2;
-	required int64 startOffset = 3;
-	required int64 length = 4;
-	required TableProto meta = 5;
-	optional TableStatProto stat = 6;
+  required string id = 1;
+  required string path = 2;
+  required int64 startOffset = 3;
+  required int64 length = 4;
+  required TableProto meta = 5;
+  optional TableStatProto stat = 6;
   optional bool distCached = 7 [default = false];
 }
 
 message TableProto {
-    required SchemaProto schema = 1;
-    required StoreType storeType = 2;
-    required KeyValueSetProto params = 3;
-    optional TableStatProto stat = 4;
+  required SchemaProto schema = 1;
+  required StoreType storeType = 2;
+  required KeyValueSetProto params = 3;
+  optional TableStatProto stat = 4;
 }
 
 message TableDescProto {
-	required string id = 1;
-	required string path = 2;
-	required TableProto meta = 3;
+  required string id = 1;
+  required string path = 2;
+  required TableProto meta = 3;
 }
 
 enum FunctionType {
-	GENERAL = 0;
-	AGGREGATION = 1;
+  GENERAL = 0;
+  AGGREGATION = 1;
 }
 
 message FunctionDescProto {
-	required string signature = 1;
-	required string className = 2;
-	required FunctionType type = 3;
-	repeated DataType parameterTypes = 4;
-	required DataType returnType = 5;
+  required string signature = 1;
+  required string className = 2;
+  required FunctionType type = 3;
+  repeated DataType parameterTypes = 4;
+  required DataType returnType = 5;
 }
 
 message IndexDescProto {
-    required string name = 1;
-    required string tableId = 2;
-    required ColumnProto column = 3;
-    required IndexMethod indexMethod = 4;
-    optional bool isUnique = 5 [default = false];
-    optional bool isClustered = 6 [default = false];
-    optional bool isAscending = 7 [default = false];
+  required string name = 1;
+  required string tableId = 2;
+  required ColumnProto column = 3;
+  required IndexMethod indexMethod = 4;
+  optional bool isUnique = 5 [default = false];
+  optional bool isClustered = 6 [default = false];
+  optional bool isAscending = 7 [default = false];
 }
 
 enum IndexMethod {
-    TWO_LEVEL_BIN_TREE = 0;
-    BTREE = 1;
-    HASH = 2;
-    BITMAP = 3;
+  TWO_LEVEL_BIN_TREE = 0;
+  BTREE = 1;
+  HASH = 2;
+  BITMAP = 3;
 }
 
 message GetAllTableNamesResponse {
-    repeated string tableName = 1;
+  repeated string tableName = 1;
 }
 
 message GetIndexRequest {
-    required string tableName = 1;
-    required string columnName = 2;
+  required string tableName = 1;
+  required string columnName = 2;
 }
 
 message GetFunctionsResponse {
-	repeated FunctionDescProto functionDesc = 1;
+  repeated FunctionDescProto functionDesc = 1;
 }
 
 message UnregisterFunctionRequest {
-	required string signature = 1;
-	repeated DataType parameterTypes = 2;
+  required string signature = 1;
+  repeated DataType parameterTypes = 2;
 }
 
 message GetFunctionMetaRequest {
-	required string signature = 1;
-	repeated DataType parameterTypes = 2;
+  required string signature = 1;
+  repeated DataType parameterTypes = 2;
 }
 
 message ContainFunctionRequest {
-	required string signature = 1;
-	repeated DataType parameterTypes = 2;
+  required string signature = 1;
+  repeated DataType parameterTypes = 2;
 }
 
 message TableStatProto {
-	required int64 numRows = 1;
-	required int64 numBytes = 2;
-	optional int32 numBlocks = 3;
-	optional int32 numPartitions = 4;
-	optional int64 avgRows = 5;
-	repeated ColumnStatProto colStat = 6;
+  required int64 numRows = 1;
+  required int64 numBytes = 2;
+  optional int32 numBlocks = 3;
+  optional int32 numPartitions = 4;
+  optional int64 avgRows = 5;
+  repeated ColumnStatProto colStat = 6;
 }
 
 message ColumnStatProto {
-    required ColumnProto column = 1;
-    optional int64 numDistVal = 2;
-    optional int64 numNulls = 3;
-    optional bytes minValue = 4;
-    optional bytes maxValue = 5;
+  required ColumnProto column = 1;
+  optional int64 numDistVal = 2;
+  optional int64 numNulls = 3;
+  optional bytes minValue = 4;
+  optional bytes maxValue = 5;
 }
 
 enum StatType {

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-catalog/tajo-catalog-common/src/test/java/tajo/catalog/TestCatalogUtil.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-common/src/test/java/tajo/catalog/TestCatalogUtil.java b/tajo-catalog/tajo-catalog-common/src/test/java/tajo/catalog/TestCatalogUtil.java
index 5d85461..a5a2279 100644
--- a/tajo-catalog/tajo-catalog-common/src/test/java/tajo/catalog/TestCatalogUtil.java
+++ b/tajo-catalog/tajo-catalog-common/src/test/java/tajo/catalog/TestCatalogUtil.java
@@ -19,15 +19,15 @@
 package tajo.catalog;
 
 import org.junit.Test;
-import tajo.catalog.proto.CatalogProtos.DataType;
+import tajo.common.TajoDataTypes.Type;
 
 import static org.junit.Assert.assertEquals;
 
 public class TestCatalogUtil {
   @Test
   public final void testGetCanonicalName() {
-    String canonical = TCatUtil.getCanonicalName("sum",
-        new DataType[]{DataType.INT, DataType.LONG});
-    assertEquals("sum(INT,LONG)", canonical);
+    String canonical = CatalogUtil.getCanonicalName("sum",
+        CatalogUtil.newDataTypesWithoutLen(Type.INT4,  Type.INT8));
+    assertEquals("sum(INT4,INT8)", canonical);
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-catalog/tajo-catalog-common/src/test/java/tajo/catalog/TestColumn.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-common/src/test/java/tajo/catalog/TestColumn.java b/tajo-catalog/tajo-catalog-common/src/test/java/tajo/catalog/TestColumn.java
index 2502b82..f434573 100644
--- a/tajo-catalog/tajo-catalog-common/src/test/java/tajo/catalog/TestColumn.java
+++ b/tajo-catalog/tajo-catalog-common/src/test/java/tajo/catalog/TestColumn.java
@@ -22,7 +22,8 @@ import com.google.gson.Gson;
 import org.junit.Before;
 import org.junit.Test;
 import tajo.catalog.json.GsonCreator;
-import tajo.catalog.proto.CatalogProtos.DataType;
+import tajo.common.TajoDataTypes.DataType;
+import tajo.common.TajoDataTypes.Type;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
@@ -32,12 +33,9 @@ public class TestColumn {
 	static final String FieldName2="f2";
 	static final String FieldName3="f3";	
 	
-	static final DataType Type1 = DataType.BYTE;
-	static final DataType Type2 = DataType.INT;
-	static final DataType Type3 = DataType.LONG;
-	
-	static final int Len2 = 10;
-	static final int Len3 = 12;
+	static final DataType Type1 = CatalogUtil.newDataTypeWithoutLen(Type.BLOB);
+	static final DataType Type2 = CatalogUtil.newDataTypeWithoutLen(Type.INT4);
+	static final DataType Type3 = CatalogUtil.newDataTypeWithoutLen(Type.INT8);
 	
 	Column field1;
 	Column field2;
@@ -45,9 +43,9 @@ public class TestColumn {
 	
 	@Before
 	public void setUp() {
-		field1 = new Column(FieldName1, DataType.BYTE);
-		field2 = new Column(FieldName2, DataType.INT );
-		field3 = new Column(FieldName3, DataType.LONG);
+		field1 = new Column(FieldName1, Type.BLOB);
+		field2 = new Column(FieldName2, Type.INT4);
+		field3 = new Column(FieldName3, Type.INT8);
 	}
 	
 	@Test
@@ -77,7 +75,7 @@ public class TestColumn {
 	
 	@Test
 	public final void testQualifiedName() {
-	  Column col = new Column("table_1.id", DataType.INT);
+	  Column col = new Column("table_1.id", Type.INT4);
 	  
 	  assertTrue(col.isQualified());
 	  assertEquals("id", col.getColumnName());
@@ -86,7 +84,7 @@ public class TestColumn {
 	}
 
 	@Test
-	public final void testToSon() {
+	public final void testToJson() {
 		Column col = new Column(field1.getProto());
 		String json = col.toJSON();
 		System.out.println(json);

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-catalog/tajo-catalog-common/src/test/java/tajo/catalog/TestFunctionDesc.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-common/src/test/java/tajo/catalog/TestFunctionDesc.java b/tajo-catalog/tajo-catalog-common/src/test/java/tajo/catalog/TestFunctionDesc.java
index 49fc09f..33b1de1 100644
--- a/tajo-catalog/tajo-catalog-common/src/test/java/tajo/catalog/TestFunctionDesc.java
+++ b/tajo-catalog/tajo-catalog-common/src/test/java/tajo/catalog/TestFunctionDesc.java
@@ -22,9 +22,10 @@ import com.google.gson.Gson;
 import org.junit.Test;
 import tajo.catalog.function.GeneralFunction;
 import tajo.catalog.json.GsonCreator;
-import tajo.catalog.proto.CatalogProtos.DataType;
 import tajo.catalog.proto.CatalogProtos.FunctionDescProto;
 import tajo.catalog.proto.CatalogProtos.FunctionType;
+import tajo.common.TajoDataTypes;
+import tajo.common.TajoDataTypes.Type;
 import tajo.datum.Datum;
 import tajo.datum.DatumFactory;
 import tajo.exception.InternalException;
@@ -45,15 +46,15 @@ public class TestFunctionDesc {
     private Integer y;
 
     public TestSum() {
-      super(new Column[] { new Column("arg1", DataType.INT),
-          new Column("arg2", DataType.INT) });
+      super(new Column[] { new Column("arg1", TajoDataTypes.Type.INT4),
+          new Column("arg2", TajoDataTypes.Type.INT4) });
     }
 
     @Override
     public Datum eval(Tuple params) {
-      x =  params.get(0).asInt();
-      y =  params.get(1).asInt();
-      return DatumFactory.createInt(x + y);
+      x =  params.get(0).asInt4();
+      y =  params.get(1).asInt4();
+      return DatumFactory.createInt4(x + y);
     }
 
     public String toJSON() {
@@ -65,13 +66,13 @@ public class TestFunctionDesc {
   @Test
   public void testGetSignature() throws IOException {
     FunctionDesc desc = new FunctionDesc("sum", TestSum.class, FunctionType.GENERAL,
-        new DataType [] {DataType.INT},
-        new DataType [] {DataType.INT,DataType.LONG});
+        CatalogUtil.newDataTypesWithoutLen(Type.INT4),
+        CatalogUtil.newDataTypesWithoutLen(Type.INT4, Type.INT8));
     assertEquals("sum", desc.getSignature());
     assertEquals(TestSum.class, desc.getFuncClass());
     assertEquals(FunctionType.GENERAL, desc.getFuncType());
-    assertEquals(DataType.INT, desc.getReturnType()[0]);
-    assertArrayEquals(new DataType[] { DataType.INT, DataType.LONG },
+    assertEquals(Type.INT4, desc.getReturnType()[0].getType());
+    assertArrayEquals(CatalogUtil.newDataTypesWithoutLen(Type.INT4, Type.INT8),
         desc.getParamTypes());
 
     CommonTestingUtil.getTestDir(TEST_PATH);
@@ -85,8 +86,8 @@ public class TestFunctionDesc {
     assertEquals("sum", newDesc.getSignature());
     assertEquals(TestSum.class, newDesc.getFuncClass());
     assertEquals(FunctionType.GENERAL, newDesc.getFuncType());
-    assertEquals(DataType.INT, newDesc.getReturnType()[0]);
-    assertArrayEquals(new DataType[] { DataType.INT, DataType.LONG },
+    assertEquals(Type.INT4, newDesc.getReturnType()[0].getType());
+    assertArrayEquals(CatalogUtil.newDataTypesWithoutLen(Type.INT4, Type.INT8),
         newDesc.getParamTypes());
 
     assertEquals(desc.getProto(), newDesc.getProto());
@@ -95,8 +96,8 @@ public class TestFunctionDesc {
   @Test
   public void testJson() throws InternalException {
 	  FunctionDesc desc = new FunctionDesc("sum", TestSum.class, FunctionType.GENERAL,
-        new DataType [] {DataType.INT},
-        new DataType [] {DataType.INT,DataType.LONG});
+        CatalogUtil.newDataTypesWithoutLen(Type.INT4),
+        CatalogUtil.newDataTypesWithoutLen(Type.INT4, Type.INT8));
 	  String json = desc.toJSON();
 	  System.out.println(json);
 	  Gson gson = GsonCreator.getInstance();
@@ -105,8 +106,8 @@ public class TestFunctionDesc {
 	  assertEquals("sum", fromJson.getSignature());
 	    assertEquals(TestSum.class, fromJson.getFuncClass());
 	    assertEquals(FunctionType.GENERAL, fromJson.getFuncType());
-	    assertEquals(DataType.INT, fromJson.getReturnType()[0]);
-	    assertArrayEquals(new DataType[] { DataType.INT, DataType.LONG },
+	    assertEquals(Type.INT4, fromJson.getReturnType()[0].getType());
+	    assertArrayEquals(CatalogUtil.newDataTypesWithoutLen(Type.INT4, Type.INT8),
 	    		fromJson.getParamTypes());
 
 	    assertEquals(desc.getProto(), fromJson.getProto());
@@ -115,8 +116,8 @@ public class TestFunctionDesc {
   @Test
   public void testClone() throws CloneNotSupportedException {
     FunctionDesc desc = new FunctionDesc("sum", TestSum.class, FunctionType.GENERAL,
-        new DataType [] {DataType.INT},
-        new DataType [] {DataType.INT,DataType.LONG});
+        CatalogUtil.newDataTypesWithoutLen(Type.INT4),
+        CatalogUtil.newDataTypesWithoutLen(Type.INT4, Type.INT8));
     FunctionDesc cloned = (FunctionDesc)desc.clone();
     assertTrue("reference chk" , !(desc == cloned));
     assertTrue("getClass() chk", desc.getClass() == cloned.getClass());

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-catalog/tajo-catalog-common/src/test/java/tajo/catalog/TestIndexDesc.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-common/src/test/java/tajo/catalog/TestIndexDesc.java b/tajo-catalog/tajo-catalog-common/src/test/java/tajo/catalog/TestIndexDesc.java
index afbfe5a..c8eb166 100644
--- a/tajo-catalog/tajo-catalog-common/src/test/java/tajo/catalog/TestIndexDesc.java
+++ b/tajo-catalog/tajo-catalog-common/src/test/java/tajo/catalog/TestIndexDesc.java
@@ -21,9 +21,9 @@ package tajo.catalog;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
-import tajo.catalog.proto.CatalogProtos.DataType;
 import tajo.catalog.proto.CatalogProtos.IndexDescProto;
 import tajo.catalog.proto.CatalogProtos.IndexMethod;
+import tajo.common.TajoDataTypes.Type;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotSame;
@@ -35,15 +35,15 @@ public class TestIndexDesc {
   
   static {
     desc1 = new IndexDesc(
-        "idx_test", "indexed", new Column("id", DataType.INT),
+        "idx_test", "indexed", new Column("id", Type.INT4),
         IndexMethod.TWO_LEVEL_BIN_TREE, true, true, true);
     
     desc2 = new IndexDesc(
-        "idx_test2", "indexed", new Column("score", DataType.DOUBLE), 
+        "idx_test2", "indexed", new Column("score", Type.FLOAT8),
         IndexMethod.TWO_LEVEL_BIN_TREE, false, false, false);
     
     desc3 = new IndexDesc(
-        "idx_test", "indexed", new Column("id", DataType.INT), 
+        "idx_test", "indexed", new Column("id", Type.INT4),
         IndexMethod.TWO_LEVEL_BIN_TREE, true, true, true);
   }
 
@@ -66,7 +66,7 @@ public class TestIndexDesc {
   public void testGetFields() {
     assertEquals("idx_test", desc1.getName());
     assertEquals("indexed", desc1.getTableId());
-    assertEquals(new Column("id", DataType.INT), desc1.getColumn());
+    assertEquals(new Column("id", Type.INT4), desc1.getColumn());
     assertEquals(IndexMethod.TWO_LEVEL_BIN_TREE, desc1.getIndexMethod());
     assertEquals(true, desc1.isUnique());
     assertEquals(true, desc1.isClustered());
@@ -74,7 +74,7 @@ public class TestIndexDesc {
     
     assertEquals("idx_test2", desc2.getName());
     assertEquals("indexed", desc2.getTableId());
-    assertEquals(new Column("score", DataType.DOUBLE), desc2.getColumn());
+    assertEquals(new Column("score", Type.FLOAT8), desc2.getColumn());
     assertEquals(IndexMethod.TWO_LEVEL_BIN_TREE, desc2.getIndexMethod());
     assertEquals(false, desc2.isUnique());
     assertEquals(false, desc2.isClustered());

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-catalog/tajo-catalog-common/src/test/java/tajo/catalog/TestSchema.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-common/src/test/java/tajo/catalog/TestSchema.java b/tajo-catalog/tajo-catalog-common/src/test/java/tajo/catalog/TestSchema.java
index 8b5bee3..e23790b 100644
--- a/tajo-catalog/tajo-catalog-common/src/test/java/tajo/catalog/TestSchema.java
+++ b/tajo-catalog/tajo-catalog-common/src/test/java/tajo/catalog/TestSchema.java
@@ -23,8 +23,8 @@ import org.junit.Before;
 import org.junit.Test;
 import tajo.catalog.exception.AlreadyExistsFieldException;
 import tajo.catalog.json.GsonCreator;
-import tajo.catalog.proto.CatalogProtos.DataType;
 import tajo.catalog.proto.CatalogProtos.SchemaProto;
+import tajo.common.TajoDataTypes.Type;
 
 import static org.junit.Assert.*;
 
@@ -38,11 +38,11 @@ public class TestSchema {
 	@Before
 	public void setUp() throws Exception {
 		schema = new Schema();
-		col1 = new Column("name", DataType.STRING);
+		col1 = new Column("name", Type.TEXT);
 		schema.addColumn(col1);
-		col2 = new Column("age", DataType.INT);
+		col2 = new Column("age", Type.INT4);
 		schema.addColumn(col2);
-		col3 = new Column("addr", DataType.STRING);
+		col3 = new Column("addr", Type.TEXT);
 		schema.addColumn(col3);
 	}
 
@@ -71,16 +71,16 @@ public class TestSchema {
 	public final void testAddField() {
 		Schema schema = new Schema();
 		assertFalse(schema.contains("studentId"));
-		schema.addColumn("studentId", DataType.INT);
+		schema.addColumn("studentId", Type.INT4);
 		assertTrue(schema.contains("studentId"));
 	}
 
 	@Test
 	public final void testEqualsObject() {
 		Schema schema2 = new Schema();
-		schema2.addColumn("name", DataType.STRING);
-		schema2.addColumn("age", DataType.INT);
-		schema2.addColumn("addr", DataType.STRING);
+		schema2.addColumn("name", Type.TEXT);
+		schema2.addColumn("age", Type.INT4);
+		schema2.addColumn("addr", Type.TEXT);
 		
 		assertEquals(schema, schema2);
 	}
@@ -97,8 +97,8 @@ public class TestSchema {
 	@Test
 	public final void testClone() throws CloneNotSupportedException {
 	  Schema schema = new Schema();
-	  schema.addColumn("abc", DataType.DOUBLE);
-	  schema.addColumn("bbc", DataType.DOUBLE);
+	  schema.addColumn("abc", Type.FLOAT8);
+	  schema.addColumn("bbc", Type.FLOAT8);
 	  
 	  Schema schema2 = new Schema(schema.getProto());
 	  assertEquals(schema.getProto(), schema2.getProto());
@@ -114,9 +114,9 @@ public class TestSchema {
 	@Test(expected = AlreadyExistsFieldException.class)
 	public final void testAddExistColumn() {
     Schema schema = new Schema();
-    schema.addColumn("abc", DataType.DOUBLE);
-    schema.addColumn("bbc", DataType.DOUBLE);
-    schema.addColumn("abc", DataType.INT);
+    schema.addColumn("abc", Type.FLOAT8);
+    schema.addColumn("bbc", Type.FLOAT8);
+    schema.addColumn("abc", Type.INT4);
 	}
 
 	@Test

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-catalog/tajo-catalog-common/src/test/java/tajo/catalog/TestTableDesc.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-common/src/test/java/tajo/catalog/TestTableDesc.java b/tajo-catalog/tajo-catalog-common/src/test/java/tajo/catalog/TestTableDesc.java
index f3c60cf..23e6dbc 100644
--- a/tajo-catalog/tajo-catalog-common/src/test/java/tajo/catalog/TestTableDesc.java
+++ b/tajo-catalog/tajo-catalog-common/src/test/java/tajo/catalog/TestTableDesc.java
@@ -23,8 +23,8 @@ import org.apache.hadoop.fs.Path;
 import org.junit.Before;
 import org.junit.Test;
 import tajo.catalog.json.GsonCreator;
-import tajo.catalog.proto.CatalogProtos.DataType;
 import tajo.catalog.proto.CatalogProtos.StoreType;
+import tajo.common.TajoDataTypes.Type;
 
 import static org.junit.Assert.assertEquals;
 
@@ -35,9 +35,9 @@ public class TestTableDesc {
 	@Before
 	public void setup() {
 	  Schema schema = new Schema();
-    schema.addColumn("name", DataType.BYTE);
-    schema.addColumn("addr", DataType.STRING);
-    info = TCatUtil.newTableMeta(schema, StoreType.CSV);
+    schema.addColumn("name", Type.BLOB);
+    schema.addColumn("addr", Type.TEXT);
+    info = CatalogUtil.newTableMeta(schema, StoreType.CSV);
 
     desc = new TableDescImpl("table1", info, new Path("/nta/data"));
 	}
@@ -45,9 +45,9 @@ public class TestTableDesc {
   @Test
   public void test() throws CloneNotSupportedException {
     Schema schema = new Schema();
-    schema.addColumn("name", DataType.BYTE);
-    schema.addColumn("addr", DataType.STRING);
-    TableMeta info = TCatUtil.newTableMeta(schema, StoreType.CSV);
+    schema.addColumn("name", Type.BLOB);
+    schema.addColumn("addr", Type.TEXT);
+    TableMeta info = CatalogUtil.newTableMeta(schema, StoreType.CSV);
     testClone(info);
 
     TableDesc desc = new TableDescImpl("table1", info, new Path("/nta/data"));

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-catalog/tajo-catalog-common/src/test/java/tajo/catalog/TestTableInfo.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-common/src/test/java/tajo/catalog/TestTableInfo.java b/tajo-catalog/tajo-catalog-common/src/test/java/tajo/catalog/TestTableInfo.java
index 052bbfe..6b0e1e5 100644
--- a/tajo-catalog/tajo-catalog-common/src/test/java/tajo/catalog/TestTableInfo.java
+++ b/tajo-catalog/tajo-catalog-common/src/test/java/tajo/catalog/TestTableInfo.java
@@ -20,9 +20,9 @@ package tajo.catalog;
 
 import org.junit.Before;
 import org.junit.Test;
-import tajo.catalog.proto.CatalogProtos.DataType;
 import tajo.catalog.proto.CatalogProtos.StoreType;
 import tajo.catalog.proto.CatalogProtos.TableProto;
+import tajo.common.TajoDataTypes.Type;
 
 import static org.junit.Assert.*;
 
@@ -33,17 +33,17 @@ public class TestTableInfo {
   @Before
   public void setUp() {
     schema = new Schema();
-    schema.addColumn("name", DataType.BYTE);
-    schema.addColumn("addr", DataType.STRING);
-    meta = TCatUtil.newTableMeta(schema, StoreType.CSV);
+    schema.addColumn("name", Type.BLOB);
+    schema.addColumn("addr", Type.TEXT);
+    meta = CatalogUtil.newTableMeta(schema, StoreType.CSV);
   }
   
   @Test
   public void testTableMetaTableProto() {    
     Schema schema1 = new Schema();
-    schema1.addColumn("name", DataType.BYTE);
-    schema1.addColumn("addr", DataType.STRING);    
-    TableMeta meta1 = TCatUtil.newTableMeta(schema1, StoreType.CSV);
+    schema1.addColumn("name", Type.BLOB);
+    schema1.addColumn("addr", Type.TEXT);
+    TableMeta meta1 = CatalogUtil.newTableMeta(schema1, StoreType.CSV);
     
     TableMeta meta2 = new TableMetaImpl(meta1.getProto());
     assertEquals(meta1, meta2);
@@ -52,9 +52,9 @@ public class TestTableInfo {
   @Test
   public final void testClone() throws CloneNotSupportedException {    
     Schema schema1 = new Schema();
-    schema1.addColumn("name", DataType.BYTE);
-    schema1.addColumn("addr", DataType.STRING);
-    TableMeta meta1 = TCatUtil.newTableMeta(schema1, StoreType.CSV);
+    schema1.addColumn("name", Type.BLOB);
+    schema1.addColumn("addr", Type.TEXT);
+    TableMeta meta1 = CatalogUtil.newTableMeta(schema1, StoreType.CSV);
     
     TableMetaImpl info = (TableMetaImpl) meta1;
     
@@ -67,9 +67,9 @@ public class TestTableInfo {
   @Test
   public void testSchema() throws CloneNotSupportedException {    
     Schema schema1 = new Schema();
-    schema1.addColumn("name", DataType.BYTE);
-    schema1.addColumn("addr", DataType.STRING);
-    TableMeta meta1 = TCatUtil.newTableMeta(schema1,  StoreType.CSV);
+    schema1.addColumn("name", Type.BLOB);
+    schema1.addColumn("addr", Type.TEXT);
+    TableMeta meta1 = CatalogUtil.newTableMeta(schema1, StoreType.CSV);
     
     TableMeta meta2 = (TableMeta) meta1.clone();
     
@@ -84,8 +84,8 @@ public class TestTableInfo {
   @Test
   public void testGetSchema() {
     Schema schema2 = new Schema();
-    schema2.addColumn("name", DataType.BYTE);
-    schema2.addColumn("addr", DataType.STRING);
+    schema2.addColumn("name", Type.BLOB);
+    schema2.addColumn("addr", Type.TEXT);
     
     assertEquals(schema, schema2);
   }
@@ -93,9 +93,9 @@ public class TestTableInfo {
   @Test
   public void testSetSchema() {
     Schema schema2 = new Schema();
-    schema2.addColumn("name", DataType.BYTE);
-    schema2.addColumn("addr", DataType.STRING);
-    schema2.addColumn("age", DataType.INT);
+    schema2.addColumn("name", Type.BLOB);
+    schema2.addColumn("addr", Type.TEXT);
+    schema2.addColumn("age", Type.INT4);
     
     assertNotSame(meta.getSchema(), schema2);
     meta.setSchema(schema2);
@@ -105,9 +105,9 @@ public class TestTableInfo {
   @Test
   public void testEqualsObject() {    
     Schema schema2 = new Schema();
-    schema2.addColumn("name", DataType.BYTE);
-    schema2.addColumn("addr", DataType.STRING);
-    TableMeta meta2 = TCatUtil.newTableMeta(schema2, StoreType.CSV);
+    schema2.addColumn("name", Type.BLOB);
+    schema2.addColumn("addr", Type.TEXT);
+    TableMeta meta2 = CatalogUtil.newTableMeta(schema2, StoreType.CSV);
     
     assertTrue(meta.equals(meta2));
     
@@ -117,9 +117,9 @@ public class TestTableInfo {
   @Test
   public void testGetProto() {
     Schema schema1 = new Schema();
-    schema1.addColumn("name", DataType.BYTE);
-    schema1.addColumn("addr", DataType.STRING);
-    TableMeta meta1 = TCatUtil.newTableMeta(schema1, StoreType.CSV);
+    schema1.addColumn("name", Type.BLOB);
+    schema1.addColumn("addr", Type.TEXT);
+    TableMeta meta1 = CatalogUtil.newTableMeta(schema1, StoreType.CSV);
     
     TableProto proto = meta1.getProto();
     TableMeta newMeta = new TableMetaImpl(proto);

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-catalog/tajo-catalog-common/src/test/java/tajo/catalog/TestTableMeta.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-common/src/test/java/tajo/catalog/TestTableMeta.java b/tajo-catalog/tajo-catalog-common/src/test/java/tajo/catalog/TestTableMeta.java
index e388375..d393f32 100644
--- a/tajo-catalog/tajo-catalog-common/src/test/java/tajo/catalog/TestTableMeta.java
+++ b/tajo-catalog/tajo-catalog-common/src/test/java/tajo/catalog/TestTableMeta.java
@@ -20,9 +20,9 @@ package tajo.catalog;
 
 import org.junit.Before;
 import org.junit.Test;
-import tajo.catalog.proto.CatalogProtos.DataType;
 import tajo.catalog.proto.CatalogProtos.StoreType;
 import tajo.catalog.proto.CatalogProtos.TableProto;
+import tajo.common.TajoDataTypes.Type;
 
 import static org.junit.Assert.*;
 
@@ -33,17 +33,17 @@ public class TestTableMeta {
   @Before
   public void setUp() {    
     schema = new Schema();
-    schema.addColumn("name", DataType.BYTE);
-    schema.addColumn("addr", DataType.STRING);
-    meta = TCatUtil.newTableMeta(schema, StoreType.CSV);
+    schema.addColumn("name", Type.BLOB);
+    schema.addColumn("addr", Type.TEXT);
+    meta = CatalogUtil.newTableMeta(schema, StoreType.CSV);
   }
   
   @Test
   public void testTableMetaTableProto() {    
     Schema schema1 = new Schema();
-    schema1.addColumn("name", DataType.BYTE);
-    schema1.addColumn("addr", DataType.STRING);
-    TableMeta meta1 = TCatUtil.newTableMeta(schema1, StoreType.CSV);
+    schema1.addColumn("name", Type.BLOB);
+    schema1.addColumn("addr", Type.TEXT);
+    TableMeta meta1 = CatalogUtil.newTableMeta(schema1, StoreType.CSV);
     
     TableMeta meta2 = new TableMetaImpl(meta1.getProto());
     assertEquals(meta1, meta2);
@@ -52,9 +52,9 @@ public class TestTableMeta {
   @Test
   public final void testClone() throws CloneNotSupportedException {
     Schema schema1 = new Schema();
-    schema1.addColumn("name", DataType.BYTE);
-    schema1.addColumn("addr", DataType.STRING);
-    TableMeta meta1 = TCatUtil.newTableMeta(schema1, StoreType.CSV);
+    schema1.addColumn("name", Type.BLOB);
+    schema1.addColumn("addr", Type.TEXT);
+    TableMeta meta1 = CatalogUtil.newTableMeta(schema1, StoreType.CSV);
     
     TableMetaImpl meta2 = (TableMetaImpl) meta1.clone();
     assertEquals(meta1.getSchema(), meta2.getSchema());
@@ -65,9 +65,9 @@ public class TestTableMeta {
   @Test
   public void testSchema() throws CloneNotSupportedException {
     Schema schema1 = new Schema();
-    schema1.addColumn("name", DataType.BYTE);
-    schema1.addColumn("addr", DataType.STRING);
-    TableMeta meta1 = TCatUtil.newTableMeta(schema1, StoreType.CSV);
+    schema1.addColumn("name", Type.BLOB);
+    schema1.addColumn("addr", Type.TEXT);
+    TableMeta meta1 = CatalogUtil.newTableMeta(schema1, StoreType.CSV);
     
     TableMeta meta2 = (TableMeta) meta1.clone();
     
@@ -82,8 +82,8 @@ public class TestTableMeta {
   @Test
   public void testGetSchema() {
     Schema schema2 = new Schema();
-    schema2.addColumn("name", DataType.BYTE);
-    schema2.addColumn("addr", DataType.STRING);
+    schema2.addColumn("name", Type.BLOB);
+    schema2.addColumn("addr", Type.TEXT);
     
     assertEquals(schema, schema2);
   }
@@ -91,9 +91,9 @@ public class TestTableMeta {
   @Test
   public void testSetSchema() {
     Schema schema2 = new Schema();
-    schema2.addColumn("name", DataType.BYTE);
-    schema2.addColumn("addr", DataType.STRING);
-    schema2.addColumn("age", DataType.INT);
+    schema2.addColumn("name", Type.BLOB);
+    schema2.addColumn("addr", Type.TEXT);
+    schema2.addColumn("age", Type.INT4);
     
     assertNotSame(meta.getSchema(), schema2);
     meta.setSchema(schema2);
@@ -103,9 +103,9 @@ public class TestTableMeta {
   @Test
   public void testEqualsObject() {   
     Schema schema2 = new Schema();
-    schema2.addColumn("name", DataType.BYTE);
-    schema2.addColumn("addr", DataType.STRING);
-    TableMeta meta2 = TCatUtil.newTableMeta(schema2, StoreType.CSV);
+    schema2.addColumn("name", Type.BLOB);
+    schema2.addColumn("addr", Type.TEXT);
+    TableMeta meta2 = CatalogUtil.newTableMeta(schema2, StoreType.CSV);
     
     assertTrue(meta.equals(meta2));
     
@@ -115,9 +115,9 @@ public class TestTableMeta {
   @Test
   public void testGetProto() {    
     Schema schema1 = new Schema();
-    schema1.addColumn("name", DataType.BYTE);
-    schema1.addColumn("addr", DataType.STRING);
-    TableMeta meta1 = TCatUtil.newTableMeta(schema1, StoreType.CSV);
+    schema1.addColumn("name", Type.BLOB);
+    schema1.addColumn("addr", Type.TEXT);
+    TableMeta meta1 = CatalogUtil.newTableMeta(schema1, StoreType.CSV);
     
     TableProto proto = meta1.getProto();
     TableMeta newMeta = new TableMetaImpl(proto);

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-catalog/tajo-catalog-common/src/test/java/tajo/catalog/statistics/TestColumnStat.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-common/src/test/java/tajo/catalog/statistics/TestColumnStat.java b/tajo-catalog/tajo-catalog-common/src/test/java/tajo/catalog/statistics/TestColumnStat.java
index 9af4453..cffd42f 100644
--- a/tajo-catalog/tajo-catalog-common/src/test/java/tajo/catalog/statistics/TestColumnStat.java
+++ b/tajo-catalog/tajo-catalog-common/src/test/java/tajo/catalog/statistics/TestColumnStat.java
@@ -20,7 +20,7 @@ package tajo.catalog.statistics;
 
 import org.junit.Test;
 import tajo.catalog.Column;
-import tajo.catalog.proto.CatalogProtos;
+import tajo.common.TajoDataTypes.Type;
 import tajo.datum.DatumFactory;
 
 import static org.junit.Assert.assertEquals;
@@ -30,7 +30,7 @@ public class TestColumnStat {
 
   @Test
   public final void testColumnStat() {
-    ColumnStat stat = new ColumnStat(new Column("test", CatalogProtos.DataType.LONG));
+    ColumnStat stat = new ColumnStat(new Column("test", Type.INT8));
     stat.setNumDistVals(1000);
     stat.setNumNulls(999);
     
@@ -44,11 +44,11 @@ public class TestColumnStat {
 
   @Test
   public final void testEqualsObject() {
-    ColumnStat stat = new ColumnStat(new Column("test", CatalogProtos.DataType.LONG));
+    ColumnStat stat = new ColumnStat(new Column("test", Type.INT8));
     stat.setNumDistVals(1000);
     stat.setNumNulls(999);
-    stat.setMinValue(DatumFactory.createLong(5));
-    stat.setMaxValue(DatumFactory.createLong(10));
+    stat.setMinValue(DatumFactory.createInt8(5));
+    stat.setMaxValue(DatumFactory.createInt8(10));
     
     ColumnStat stat2 = new ColumnStat(stat.getProto());
     assertEquals(stat, stat2);
@@ -56,11 +56,11 @@ public class TestColumnStat {
 
   @Test
   public final void testClone() throws CloneNotSupportedException {
-    ColumnStat stat = new ColumnStat(new Column("test", CatalogProtos.DataType.LONG));
+    ColumnStat stat = new ColumnStat(new Column("test", Type.INT8));
     stat.setNumDistVals(1000);
     stat.setNumNulls(999);
-    stat.setMinValue(DatumFactory.createLong(5));
-    stat.setMaxValue(DatumFactory.createLong(10));
+    stat.setMinValue(DatumFactory.createInt8(5));
+    stat.setMaxValue(DatumFactory.createInt8(10));
     
     ColumnStat stat2 = (ColumnStat) stat.clone();
     assertEquals(stat, stat2);

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-catalog/tajo-catalog-common/src/test/java/tajo/catalog/statistics/TestTableStat.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-common/src/test/java/tajo/catalog/statistics/TestTableStat.java b/tajo-catalog/tajo-catalog-common/src/test/java/tajo/catalog/statistics/TestTableStat.java
index 3237d25..d679c69 100644
--- a/tajo-catalog/tajo-catalog-common/src/test/java/tajo/catalog/statistics/TestTableStat.java
+++ b/tajo-catalog/tajo-catalog-common/src/test/java/tajo/catalog/statistics/TestTableStat.java
@@ -20,7 +20,7 @@ package tajo.catalog.statistics;
 
 import org.junit.Test;
 import tajo.catalog.Column;
-import tajo.catalog.proto.CatalogProtos;
+import tajo.common.TajoDataTypes.Type;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
@@ -38,7 +38,7 @@ public class TestTableStat {
     int numCols = 3;
     ColumnStat[] cols = new ColumnStat[numCols];
     for (int i = 0; i < numCols; i++) {
-      cols[i] = new ColumnStat(new Column("col_" + i, CatalogProtos.DataType.LONG));
+      cols[i] = new ColumnStat(new Column("col_" + i, Type.INT8));
       cols[i].setNumDistVals(1024 * i);
       cols[i].setNumNulls(100 * i);
       stat.addColumnStat(cols[i]);


[09/11] TAJO-57: Recognize Parser and Catalog Standard SQL data types. (hyunsik)

Posted by hy...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-common/src/main/java/tajo/datum/ShortDatum.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/tajo/datum/ShortDatum.java b/tajo-common/src/main/java/tajo/datum/ShortDatum.java
deleted file mode 100644
index ff2f79c..0000000
--- a/tajo-common/src/main/java/tajo/datum/ShortDatum.java
+++ /dev/null
@@ -1,268 +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 tajo.datum;
-
-import com.google.gson.annotations.Expose;
-import tajo.datum.exception.InvalidOperationException;
-import tajo.datum.json.GsonCreator;
-
-import java.nio.ByteBuffer;
-
-public class ShortDatum extends NumericDatum {
-  private static final int size = 2;  
-  @Expose private short val;
-
-  public ShortDatum() {
-    super(DatumType.SHORT);
-  }
-
-	public ShortDatum(short val) {
-		this();
-		this.val = val;		
-	}
-
-  public ShortDatum(byte [] bytes) {
-    this();
-    ByteBuffer bb = ByteBuffer.wrap(bytes);
-    this.val = bb.getShort();
-  }
-	
-	@Override
-	public short asShort() {	
-		return val;
-	}
-
-	@Override
-	public int asInt() {
-		return val;
-	}
-
-	@Override
-	public long asLong() {
-		return val;
-	}
-
-	@Override
-	public byte [] asByteArray() {
-		ByteBuffer bb = ByteBuffer.allocate(2);
-		bb.putShort(val);
-		return bb.array();
-	}
-
-	@Override
-	public float asFloat() {
-		return val;
-	}
-
-	@Override
-	public double asDouble() {
-		return val;
-	}
-
-	@Override
-	public String asChars() {
-		return ""+val;
-	}
-
-  @Override
-	public String toJSON() {
-		return GsonCreator.getInstance().toJson(this, Datum.class);
-	}
-
-  @Override
-  public int size() {
-    return size;
-  }
-
-  @Override
-  public int hashCode() {
-    return val;
-  }
-
-  @Override
-  public boolean equals(Object obj) {
-    if (obj instanceof ShortDatum) {
-      ShortDatum other = (ShortDatum) obj;
-      return val == other.val;
-    }
-    
-    return false;
-  }
-
-  @Override
-  public BoolDatum equalsTo(Datum datum) {
-    switch (datum.type()) {
-    case SHORT:
-      return DatumFactory.createBool(val == datum.asShort());
-    case INT:
-      return DatumFactory.createBool(val == datum.asInt());
-    case LONG:
-      return DatumFactory.createBool(val == datum.asLong());
-    case FLOAT:
-      return DatumFactory.createBool(val == datum.asFloat());
-    case DOUBLE:
-      return DatumFactory.createBool(val == datum.asDouble());
-    default:
-      throw new InvalidOperationException(datum.type());
-    }
-  }
-
-  @Override
-  public int compareTo(Datum datum) {
-    switch (datum.type()) {
-      case SHORT:
-        if (val < datum.asShort()) {
-          return -1;
-        } else if (datum.asShort() < val) {
-          return 1;
-        } else {
-          return 0;
-        }
-      case INT:
-        if (val < datum.asInt()) {
-          return -1;
-        } else if (datum.asInt() < val) {
-          return 1;
-        } else {
-          return 0;
-        }
-      case LONG:
-        if (val < datum.asLong()) {
-          return -1;
-        } else if (datum.asLong() < val) {
-          return 1;
-        } else {
-          return 0;
-        }
-      case FLOAT:
-        if (val < datum.asFloat()) {
-          return -1;
-        } else if (datum.asFloat() < val) {
-          return 1;
-        } else {
-          return 0;
-        }
-      case DOUBLE:
-        if (val < datum.asDouble()) {
-          return -1;
-        } else if (datum.asDouble() < val) {
-          return 1;
-        } else {
-          return 0;
-        }
-      default:
-        throw new InvalidOperationException(datum.type());
-    }
-  }
-
-  @Override
-  public Datum plus(Datum datum) {
-    switch (datum.type()) {
-    case SHORT:
-      return DatumFactory.createShort((short) (val + datum.asShort()));
-    case INT:
-      return DatumFactory.createInt(val + datum.asInt());
-    case LONG:
-      return DatumFactory.createLong(val + datum.asLong());
-    case FLOAT:
-      return DatumFactory.createFloat(val + datum.asFloat());
-    case DOUBLE:
-      return DatumFactory.createDouble(val + datum.asDouble());
-    default:
-      throw new InvalidOperationException(datum.type());
-    }
-  }
-
-  @Override
-  public Datum minus(Datum datum) {
-    switch (datum.type()) {
-    case SHORT:
-      return DatumFactory.createShort((short) (val - datum.asShort()));
-    case INT:
-      return DatumFactory.createInt(val - datum.asInt());
-    case LONG:
-      return DatumFactory.createLong(val - datum.asLong());
-    case FLOAT:
-      return DatumFactory.createFloat(val - datum.asFloat());
-    case DOUBLE:
-      return DatumFactory.createDouble(val - datum.asDouble());
-    default:
-      throw new InvalidOperationException(datum.type());
-    }
-  }
-
-  @Override
-  public Datum multiply(Datum datum) {
-    switch (datum.type()) {
-    case SHORT:
-      return DatumFactory.createInt(val * datum.asShort());
-    case INT:
-      return DatumFactory.createInt(val * datum.asInt());
-    case LONG:
-      return DatumFactory.createLong(val * datum.asLong());
-    case FLOAT:
-      return DatumFactory.createFloat(val * datum.asFloat());
-    case DOUBLE:
-      return DatumFactory.createDouble(val * datum.asDouble());
-    default:
-      throw new InvalidOperationException(datum.type());
-    }
-  }
-
-  @Override
-  public Datum divide(Datum datum) {
-    switch (datum.type()) {
-    case SHORT:
-      return DatumFactory.createShort((short) (val / datum.asShort()));
-    case INT:
-      return DatumFactory.createInt(val / datum.asInt());
-    case LONG:
-      return DatumFactory.createLong(val / datum.asLong());
-    case FLOAT:
-      return DatumFactory.createFloat(val / datum.asFloat());
-    case DOUBLE:
-      return DatumFactory.createDouble(val / datum.asDouble());
-    default:
-      throw new InvalidOperationException(datum.type());
-    }
-  }
-
-  @Override
-  public Datum modular(Datum datum) {
-    switch (datum.type()) {
-      case SHORT:
-        return DatumFactory.createShort((short) (val % datum.asShort()));
-      case INT:
-        return DatumFactory.createInt(val % datum.asInt());
-      case LONG:
-        return DatumFactory.createLong(val % datum.asLong());
-      case FLOAT:
-        return DatumFactory.createFloat(val % datum.asFloat());
-      case DOUBLE:
-        return DatumFactory.createDouble(val % datum.asDouble());
-      default:
-        throw new InvalidOperationException(datum.type());
-    }
-  }
-
-  @Override
-  public void inverseSign() {
-    this.val = (short) -val;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-common/src/main/java/tajo/datum/StringDatum.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/tajo/datum/StringDatum.java b/tajo-common/src/main/java/tajo/datum/StringDatum.java
deleted file mode 100644
index 8f26c54..0000000
--- a/tajo-common/src/main/java/tajo/datum/StringDatum.java
+++ /dev/null
@@ -1,157 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package tajo.datum;
-
-import com.google.gson.annotations.Expose;
-import tajo.datum.exception.InvalidCastException;
-import tajo.datum.exception.InvalidOperationException;
-import tajo.datum.json.GsonCreator;
-
-
-public class StringDatum extends Datum { 
-	@Expose String val;
-	
-	public StringDatum() {
-    super(DatumType.STRING);
-  }
-
-	public StringDatum(String val) {
-		this();
-		this.val = val;
-	}
-
-  public StringDatum(byte [] bytes) {
-    this();
-    this.val = new String(bytes);
-  }
-
-	@Override
-	public boolean asBool() {	
-		throw new InvalidCastException();
-	}
-
-	@Override
-	public byte asByte() {
-		throw new InvalidCastException();
-	}
-	
-	@Override
-	public short asShort() {	
-		throw new InvalidCastException();
-	}
-
-	@Override
-	public int asInt() {
-		int res;
-		try {
-			res = Integer.valueOf(val);
-		} catch (Exception e) {
-			throw new InvalidCastException();
-		}
-		return res;
-	}
-
-	@Override
-	public long asLong() {
-		long res;
-		try {
-			res = Long.valueOf(val);
-		} catch (Exception e) {
-			throw new InvalidCastException();
-		}
-		return res;
-	}
-
-	@Override
-	public byte[] asByteArray() {		
-		return val.getBytes();
-	}
-
-	@Override
-	public float asFloat() {
-		float res;
-		try {
-			res = Float.valueOf(val);
-		} catch (Exception e) {
-			throw new InvalidCastException();
-		}
-		return res;
-	}
-
-	@Override
-	public double asDouble() {
-		double res;
-		try {
-			res = Double.valueOf(val);
-		} catch (Exception e) {
-			throw new InvalidCastException();
-		}
-		return res;
-	}
-
-	@Override
-	public String asChars() {
-		return val;
-	}
-
-  @Override
-  public boolean equals(Object obj) {
-    if (obj instanceof StringDatum && this.val.equals(((StringDatum) obj).val)) {
-      return true;
-    } else {
-      return false;
-    }
-  }
-
-  @Override
-  public int size() {
-    return val.getBytes().length;
-  }
-  
-  @Override
-  public int hashCode() {
-    return val.hashCode();
-  }
-
-  @Override
-  public BoolDatum equalsTo(Datum datum) {
-    switch (datum.type()) {
-    case STRING:
-      return DatumFactory
-          .createBool(this.val.equals(((StringDatum) datum).val));
-    default:
-      throw new InvalidOperationException(datum.type());
-    }
-  }
-
-  @Override
-  public int compareTo(Datum datum) {
-    switch (datum.type()) {
-    case STRING:
-      return this.val.compareTo(((StringDatum) datum).val);
-    default:
-      throw new InvalidOperationException(datum.type());
-    }
-  }
-
-  @Override
-  public String toJSON() {
-		return GsonCreator.getInstance().toJson(this, Datum.class);
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-common/src/main/java/tajo/datum/StringDatum2.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/tajo/datum/StringDatum2.java b/tajo-common/src/main/java/tajo/datum/StringDatum2.java
deleted file mode 100644
index 8dd7c14..0000000
--- a/tajo-common/src/main/java/tajo/datum/StringDatum2.java
+++ /dev/null
@@ -1,140 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package tajo.datum;
-
-import com.google.gson.annotations.Expose;
-import org.apache.hadoop.io.WritableComparator;
-import tajo.datum.exception.InvalidCastException;
-import tajo.datum.exception.InvalidOperationException;
-import tajo.datum.json.GsonCreator;
-
-import java.util.Arrays;
-
-public class StringDatum2 extends Datum {
-  @Expose
-  private int size;
-  @Expose
-  private byte[] bytes;
-
-  public StringDatum2() {
-    super(DatumType.STRING2);
-  }
-
-  public StringDatum2(byte[] bytes) {
-    this();
-    this.bytes = bytes;
-    this.size = bytes.length;
-  }
-
-  public StringDatum2(String string) {
-    this(string.getBytes());
-  }
-
-  @Override
-  public boolean asBool() {
-    throw new InvalidCastException();
-  }
-
-  @Override
-  public byte asByte() {
-    throw new InvalidCastException();
-  }
-
-  @Override
-  public short asShort() {
-    throw new InvalidCastException();
-  }
-
-  @Override
-  public int asInt() {
-    throw new InvalidCastException();
-  }
-
-  @Override
-  public long asLong() {
-    throw new InvalidCastException();
-  }
-
-  @Override
-  public float asFloat() {
-    throw new InvalidCastException();
-  }
-
-  @Override
-  public double asDouble() {
-    throw new InvalidCastException();
-  }
-
-  @Override
-  public byte[] asByteArray() {
-    return this.bytes;
-  }
-
-  public String asChars() {
-    return new String(this.bytes);
-  }
-
-  @Override
-  public int size() {
-    return size;
-  }
-
-  @Override
-  public int compareTo(Datum datum) {
-    switch (datum.type()) {
-      case STRING2:
-        byte[] o = datum.asByteArray();
-        return WritableComparator.compareBytes(this.bytes, 0, this.bytes.length,
-            o, 0, o.length);
-      default:
-        throw new InvalidOperationException(datum.type());
-    }
-  }
-
-  @Override
-  public boolean equals(Object obj) {
-    if (obj instanceof StringDatum2) {
-      StringDatum2 o = (StringDatum2) obj;
-      return Arrays.equals(this.bytes, o.bytes);
-    }
-
-    return false;
-  }
-
-  @Override
-  public BoolDatum equalsTo(Datum datum) {
-    switch (datum.type()) {
-      case STRING2:
-        return DatumFactory.createBool(
-            Arrays.equals(this.bytes, datum.asByteArray()));
-      default:
-        throw new InvalidOperationException(datum.type());
-    }
-  }
-
-  @Override
-  public String toJSON() {
-    return GsonCreator.getInstance().toJson(this, Datum.class);
-  }
-
-  @Override
-  public int hashCode() {
-    return bytes.hashCode();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-common/src/main/java/tajo/datum/exception/InvalidOperationException.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/tajo/datum/exception/InvalidOperationException.java b/tajo-common/src/main/java/tajo/datum/exception/InvalidOperationException.java
index 5eaa8fe..15a52bd 100644
--- a/tajo-common/src/main/java/tajo/datum/exception/InvalidOperationException.java
+++ b/tajo-common/src/main/java/tajo/datum/exception/InvalidOperationException.java
@@ -18,7 +18,7 @@
 
 package tajo.datum.exception;
 
-import tajo.datum.DatumType;
+import static tajo.common.TajoDataTypes.Type;
 
 public class InvalidOperationException extends RuntimeException {
 	private static final long serialVersionUID = -7689027447969916148L;
@@ -36,7 +36,7 @@ public class InvalidOperationException extends RuntimeException {
 		super(message);
 	}
 	
-	public InvalidOperationException(DatumType type) {
+	public InvalidOperationException(Type type) {
 	  super("Cannot compare to " + type + " type datum");
 	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-common/src/main/java/tajo/storage/Tuple.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/tajo/storage/Tuple.java b/tajo-common/src/main/java/tajo/storage/Tuple.java
index d13d388..403e1bc 100644
--- a/tajo-common/src/main/java/tajo/storage/Tuple.java
+++ b/tajo-common/src/main/java/tajo/storage/Tuple.java
@@ -22,58 +22,57 @@ import tajo.datum.*;
 
 import java.net.InetAddress;
 
-// TODO - to be unified in tajo-core-storage
 public interface Tuple {
-  
-	public int size();
-	
-	public boolean contains(int fieldid);
+
+  public int size();
+
+  public boolean contains(int fieldid);
 
   public boolean isNull(int fieldid);
-	
-	public void clear();
-	
-	public void put(int fieldId, Datum value);
 
-  public void put(int fieldId, Datum[] values);
+  public void clear();
+
+  public void put(int fieldId, Datum value);
+
+  public void put(int fieldId, Datum [] values);
 
   public void put(int fieldId, Tuple tuple);
-	
-	public void put(Datum[] values);
-	
-	public Datum get(int fieldId);
-	
-	public void setOffset(long offset);
-	
-	public long getOffset();
-
-	public BoolDatum getBoolean(int fieldId);
-	
-	public ByteDatum getByte(int fieldId);
+
+  public void put(Datum [] values);
+
+  public Datum get(int fieldId);
+
+  public void setOffset(long offset);
+
+  public long getOffset();
+
+  public BooleanDatum getBoolean(int fieldId);
+
+  public BitDatum getByte(int fieldId);
 
   public CharDatum getChar(int fieldId);
-	
-	public BytesDatum getBytes(int fieldId);
-	
-	public ShortDatum getShort(int fieldId);
-	
-	public IntDatum getInt(int fieldId);
-	
-	public LongDatum getLong(int fieldId);
-	
-	public FloatDatum getFloat(int fieldId);
-	
-	public DoubleDatum getDouble(int fieldId);
-	
-	public IPv4Datum getIPv4(int fieldId);
-	
-	public byte [] getIPv4Bytes(int fieldId);
-	
-	public InetAddress getIPv6(int fieldId);
-	
-	public byte [] getIPv6Bytes(int fieldId);
-	
-	public StringDatum getString(int fieldId);
-
-  public StringDatum2 getString2(int fieldId);
+
+  public BlobDatum getBytes(int fieldId);
+
+  public Int2Datum getShort(int fieldId);
+
+  public Int4Datum getInt(int fieldId);
+
+  public Int8Datum getLong(int fieldId);
+
+  public Float4Datum getFloat(int fieldId);
+
+  public Float8Datum getDouble(int fieldId);
+
+  public Inet4Datum getIPv4(int fieldId);
+
+  public byte [] getIPv4Bytes(int fieldId);
+
+  public InetAddress getIPv6(int fieldId);
+
+  public byte [] getIPv6Bytes(int fieldId);
+
+  public TextDatum getString(int fieldId);
+
+  public TextDatum getText(int fieldId);
 }

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-common/src/test/java/tajo/datum/TestBoolDatum.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/test/java/tajo/datum/TestBoolDatum.java b/tajo-common/src/test/java/tajo/datum/TestBoolDatum.java
index f73fb98..f7cfaaf 100644
--- a/tajo-common/src/test/java/tajo/datum/TestBoolDatum.java
+++ b/tajo-common/src/test/java/tajo/datum/TestBoolDatum.java
@@ -19,6 +19,7 @@
 package tajo.datum;
 
 import org.junit.Test;
+import tajo.common.TajoDataTypes.Type;
 
 import static org.junit.Assert.assertEquals;
 
@@ -27,7 +28,7 @@ public class TestBoolDatum {
 	@Test
 	public final void testType() {
 		Datum d = DatumFactory.createBool(true);
-		assertEquals(DatumType.BOOLEAN, d.type());
+		assertEquals(Type.BOOLEAN, d.type());
 	}
 	
 	@Test
@@ -39,19 +40,19 @@ public class TestBoolDatum {
 	@Test
 	public final void testAsShort() {
 		Datum d = DatumFactory.createBool(true);
-		assertEquals(1, d.asShort());
+		assertEquals(1, d.asInt2());
 	}
 	
 	@Test
 	public final void testAsInt() {
 		Datum d = DatumFactory.createBool(true);
-		assertEquals(1, d.asInt());
+		assertEquals(1, d.asInt4());
 	}
 	
 	@Test
 	public final void testAsLong() {
 		Datum d = DatumFactory.createBool(false);
-		assertEquals(0, d.asLong());
+		assertEquals(0, d.asInt8());
 	}
 	
 	@Test

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-common/src/test/java/tajo/datum/TestByteDatum.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/test/java/tajo/datum/TestByteDatum.java b/tajo-common/src/test/java/tajo/datum/TestByteDatum.java
deleted file mode 100644
index 9c8dd6a..0000000
--- a/tajo-common/src/test/java/tajo/datum/TestByteDatum.java
+++ /dev/null
@@ -1,75 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package tajo.datum;
-
-import org.junit.Test;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
-public class TestByteDatum {
-
-	@Test
-	public final void testType() {
-		Datum d = DatumFactory.createByte((byte) 1);
-		assertEquals(DatumType.BYTE, d.type());
-	}
-
-	@Test
-	public final void testAsInt() {
-		Datum d = DatumFactory.createByte((byte)5);
-		assertEquals(5,d.asInt());
-	}
-	
-	@Test
-	public final void testAsLong() {
-		Datum d = DatumFactory.createByte((byte)5);
-		assertEquals(5l,d.asLong());
-	}
-	
-	@Test
-	public final void testAsByte() {
-		Datum d = DatumFactory.createByte((byte)5);
-		assertEquals(5,d.asLong());
-	}
-
-	@Test
-	public final void testAsFloat() {
-		Datum d = DatumFactory.createByte((byte)5);
-		assertTrue(5.0f == d.asFloat());
-	}
-
-	@Test
-	public final void testAsDouble() {
-		Datum d = DatumFactory.createByte((byte)5);
-		assertTrue(5.0d == d.asDouble());
-	}
-	
-	@Test
-	public final void testAsChars() {
-		Datum d = DatumFactory.createByte((byte)5);
-		System.out.println(d.asChars());
-	}
-	
-	@Test
-  public final void testSize() {
-    Datum d = DatumFactory.createByte((byte) 1);
-    assertEquals(1, d.size());
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-common/src/test/java/tajo/datum/TestBytesDatum.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/test/java/tajo/datum/TestBytesDatum.java b/tajo-common/src/test/java/tajo/datum/TestBytesDatum.java
index 962a793..b718f23 100644
--- a/tajo-common/src/test/java/tajo/datum/TestBytesDatum.java
+++ b/tajo-common/src/test/java/tajo/datum/TestBytesDatum.java
@@ -19,6 +19,7 @@
 package tajo.datum;
 
 import org.junit.Test;
+import tajo.common.TajoDataTypes.Type;
 import tajo.datum.json.GsonCreator;
 
 import static org.junit.Assert.assertEquals;
@@ -28,25 +29,25 @@ public class TestBytesDatum {
 
   @Test
   public final void testType() {
-    Datum d = DatumFactory.createBytes("12345".getBytes());
-    assertEquals(DatumType.BYTES, d.type());
+    Datum d = DatumFactory.createBlob("12345".getBytes());
+    assertEquals(Type.BLOB, d.type());
   }
   
   @Test
   public final void testAsChars() {
-    Datum d = DatumFactory.createBytes("12345".getBytes());
+    Datum d = DatumFactory.createBlob("12345".getBytes());
     assertEquals("12345", d.asChars());
   }
   
   @Test
   public final void testSize() {
-    Datum d = DatumFactory.createBytes("12345".getBytes());
+    Datum d = DatumFactory.createBlob("12345".getBytes());
     assertEquals(5, d.size());
   }
   
   @Test
   public final void testJson() {
-	  Datum d = DatumFactory.createBytes("12345".getBytes());
+	  Datum d = DatumFactory.createBlob("12345".getBytes());
 	  String json = d.toJSON();
 	  Datum fromJson = GsonCreator.getInstance().fromJson(json, Datum.class);
 	  assertTrue(d.equalsTo(fromJson).asBool());

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-common/src/test/java/tajo/datum/TestCharDatum.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/test/java/tajo/datum/TestCharDatum.java b/tajo-common/src/test/java/tajo/datum/TestCharDatum.java
index c9272dd..c193142 100644
--- a/tajo-common/src/test/java/tajo/datum/TestCharDatum.java
+++ b/tajo-common/src/test/java/tajo/datum/TestCharDatum.java
@@ -22,43 +22,44 @@ import org.junit.Test;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
+import static tajo.common.TajoDataTypes.Type;
 
 public class TestCharDatum {
 
 	@Test
 	public final void testType() {
 		Datum d = DatumFactory.createChar((char) 1);
-		assertEquals(DatumType.CHAR, d.type());
+		assertEquals(Type.CHAR, d.type());
 	}
 
 	@Test
 	public final void testAsInt() {
 		Datum d = DatumFactory.createChar((char)5);
-		assertEquals(5,d.asInt());
+		assertEquals(5,d.asInt4());
 	}
 	
 	@Test
 	public final void testAsLong() {
 		Datum d = DatumFactory.createChar((char)5);
-		assertEquals(5l,d.asLong());
+		assertEquals(5l,d.asInt8());
 	}
 	
 	@Test
 	public final void testAsByte() {
 		Datum d = DatumFactory.createChar((char)5);
-		assertEquals(5,d.asLong());
+		assertEquals(5,d.asInt8());
 	}
 
 	@Test
 	public final void testAsFloat() {
 		Datum d = DatumFactory.createChar((char)5);
-		assertTrue(5.0f == d.asFloat());
+		assertTrue(5.0f == d.asFloat4());
 	}
 
 	@Test
 	public final void testAsDouble() {
 		Datum d = DatumFactory.createChar((char)5);
-		assertTrue(5.0d == d.asDouble());
+		assertTrue(5.0d == d.asFloat8());
 	}
 	
 	@Test

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-common/src/test/java/tajo/datum/TestDatum.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/test/java/tajo/datum/TestDatum.java b/tajo-common/src/test/java/tajo/datum/TestDatum.java
index 361ab9d..73776e8 100644
--- a/tajo-common/src/test/java/tajo/datum/TestDatum.java
+++ b/tajo-common/src/test/java/tajo/datum/TestDatum.java
@@ -19,6 +19,7 @@
 package tajo.datum;
 
 import org.junit.Test;
+import tajo.common.TajoDataTypes.Type;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
@@ -31,39 +32,39 @@ public class TestDatum {
 		Datum y;
 		Datum z;
 		
-		x = DatumFactory.createInt(1);
-		y = DatumFactory.createInt(2);
+		x = DatumFactory.createInt4(1);
+		y = DatumFactory.createInt4(2);
 		z = x.plus(y);
-		assertEquals(z.type(), DatumType.INT);
-		assertEquals(z.asInt(),3);		
+		assertEquals(z.type(), Type.INT4);
+		assertEquals(z.asInt4(),3);
 		z = y.plus(x);
-		assertEquals(z.type(),DatumType.INT);
-		assertEquals(z.asInt(),3);
+		assertEquals(z.type(),Type.INT4);
+		assertEquals(z.asInt4(),3);
 		
-		x = DatumFactory.createInt(1);
-		y = DatumFactory.createLong(2l);
+		x = DatumFactory.createInt4(1);
+		y = DatumFactory.createInt8(2l);
 		z = x.plus(y);
-		assertEquals(z.type(),DatumType.LONG);
-		assertEquals(z.asLong(),3l);		
+		assertEquals(z.type(),Type.INT8);
+		assertEquals(z.asInt8(),3l);
 		z = y.plus(x);
-		assertEquals(z.type(),DatumType.LONG);
-		assertEquals(z.asLong(),3l);
+		assertEquals(z.type(),Type.INT8);
+		assertEquals(z.asInt8(),3l);
 		
-		y = DatumFactory.createFloat(2.5f);
+		y = DatumFactory.createFloat4(2.5f);
 		z = x.plus(y);
-		assertEquals(z.type(),DatumType.FLOAT);
-		assertTrue(z.asFloat() == 3.5f);
+		assertEquals(z.type(),Type.FLOAT4);
+		assertTrue(z.asFloat4() == 3.5f);
 		z = y.plus(x);
-		assertEquals(z.type(),DatumType.FLOAT);
-		assertEquals(z.asInt(),3);
+		assertEquals(z.type(),Type.FLOAT4);
+		assertEquals(z.asInt4(),3);
 		
-		y = DatumFactory.createDouble(4.5d);
+		y = DatumFactory.createFloat8(4.5d);
 		z = x.plus(y);
-		assertEquals(z.type(),DatumType.DOUBLE);
-		assertTrue(z.asDouble() == 5.5d);
+		assertEquals(z.type(),Type.FLOAT8);
+		assertTrue(z.asFloat8() == 5.5d);
 		z = y.plus(x);
-		assertEquals(z.type(),DatumType.DOUBLE);
-		assertTrue(z.asDouble() == 5.5d);
+		assertEquals(z.type(),Type.FLOAT8);
+		assertTrue(z.asFloat8() == 5.5d);
 	}
 
 	@Test
@@ -72,38 +73,38 @@ public class TestDatum {
 		Datum y;
 		Datum z;
 		
-		x = DatumFactory.createInt(5);
-		y = DatumFactory.createInt(2);
+		x = DatumFactory.createInt4(5);
+		y = DatumFactory.createInt4(2);
 		z = x.minus(y);
-		assertEquals(z.type(),DatumType.INT);
-		assertEquals(z.asInt(),3);		
+		assertEquals(z.type(),Type.INT4);
+		assertEquals(z.asInt4(),3);
 		z = y.minus(x);
-		assertEquals(z.type(),DatumType.INT);
-		assertEquals(z.asInt(),-3);
+		assertEquals(z.type(),Type.INT4);
+		assertEquals(z.asInt4(),-3);
 		
-		y = DatumFactory.createLong(2l);
+		y = DatumFactory.createInt8(2l);
 		z = x.minus(y);
-		assertEquals(z.type(),DatumType.LONG);
-		assertEquals(z.asLong(),3l);		
+		assertEquals(z.type(),Type.INT8);
+		assertEquals(z.asInt8(),3l);
 		z = y.minus(x);
-		assertEquals(z.type(),DatumType.LONG);
-		assertEquals(z.asLong(),-3l);
+		assertEquals(z.type(),Type.INT8);
+		assertEquals(z.asInt8(),-3l);
 		
-		y = DatumFactory.createFloat(2.5f);
+		y = DatumFactory.createFloat4(2.5f);
 		z = x.minus(y);
-		assertEquals(z.type(),DatumType.FLOAT);
-		assertTrue(z.asFloat() == 2.5f);
+		assertEquals(z.type(),Type.FLOAT4);
+		assertTrue(z.asFloat4() == 2.5f);
 		z = y.minus(x);
-		assertEquals(z.type(),DatumType.FLOAT);
-		assertTrue(z.asFloat() == -2.5f);
+		assertEquals(z.type(),Type.FLOAT4);
+		assertTrue(z.asFloat4() == -2.5f);
 		
-		y = DatumFactory.createDouble(4.5d);
+		y = DatumFactory.createFloat8(4.5d);
 		z = x.minus(y);
-		assertEquals(z.type(),DatumType.DOUBLE);
-		assertTrue(z.asDouble() == 0.5d);
+		assertEquals(z.type(),Type.FLOAT8);
+		assertTrue(z.asFloat8() == 0.5d);
 		z = y.minus(x);
-		assertEquals(z.type(),DatumType.DOUBLE);
-		assertTrue(z.asDouble() == -0.5d);
+		assertEquals(z.type(),Type.FLOAT8);
+		assertTrue(z.asFloat8() == -0.5d);
 	}
 
 	@Test
@@ -112,38 +113,38 @@ public class TestDatum {
 		Datum y;
 		Datum z;
 		
-		x = DatumFactory.createInt(5);
-		y = DatumFactory.createInt(2);
+		x = DatumFactory.createInt4(5);
+		y = DatumFactory.createInt4(2);
 		z = x.multiply(y);
-		assertEquals(z.type(),DatumType.INT);
-		assertEquals(z.asInt(),10);		
+		assertEquals(z.type(),Type.INT4);
+		assertEquals(z.asInt4(),10);
 		z = y.multiply(x);
-		assertEquals(z.type(),DatumType.INT);
-		assertEquals(z.asInt(),10);
+		assertEquals(z.type(),Type.INT4);
+		assertEquals(z.asInt4(),10);
 		
-		y = DatumFactory.createLong(2l);
+		y = DatumFactory.createInt8(2l);
 		z = x.multiply(y);
-		assertEquals(z.type(),DatumType.LONG);
-		assertEquals(z.asLong(),10l);		
+		assertEquals(z.type(),Type.INT8);
+		assertEquals(z.asInt8(),10l);
 		z = y.multiply(x);
-		assertEquals(z.type(),DatumType.LONG);
-		assertEquals(z.asLong(),10l);
+		assertEquals(z.type(),Type.INT8);
+		assertEquals(z.asInt8(),10l);
 		
-		y = DatumFactory.createFloat(2.5f);
+		y = DatumFactory.createFloat4(2.5f);
 		z = x.multiply(y);
-		assertEquals(z.type(),DatumType.FLOAT);
-		assertTrue(z.asFloat() == 12.5f);
+		assertEquals(z.type(),Type.FLOAT4);
+		assertTrue(z.asFloat4() == 12.5f);
 		z = y.multiply(x);
-		assertEquals(z.type(),DatumType.FLOAT);
-		assertTrue(z.asFloat() == 12.5f);
+		assertEquals(z.type(),Type.FLOAT4);
+		assertTrue(z.asFloat4() == 12.5f);
 		
-		y = DatumFactory.createDouble(4.5d);
+		y = DatumFactory.createFloat8(4.5d);
 		z = x.multiply(y);
-		assertEquals(z.type(),DatumType.DOUBLE);
-		assertTrue(z.asDouble() == 22.5d);
+		assertEquals(z.type(),Type.FLOAT8);
+		assertTrue(z.asFloat8() == 22.5d);
 		z = y.multiply(x);
-		assertEquals(z.type(),DatumType.DOUBLE);
-		assertTrue(z.asDouble() == 22.5d);
+		assertEquals(z.type(),Type.FLOAT8);
+		assertTrue(z.asFloat8() == 22.5d);
 	}
 
 	@Test
@@ -152,38 +153,38 @@ public class TestDatum {
 		Datum y;
 		Datum z;
 		
-		x = DatumFactory.createInt(6);
-		y = DatumFactory.createInt(3);
+		x = DatumFactory.createInt4(6);
+		y = DatumFactory.createInt4(3);
 		z = x.divide(y);
-		assertEquals(z.type(),DatumType.INT);
-		assertEquals(z.asInt(),2);		
+		assertEquals(z.type(), Type.INT4);
+		assertEquals(z.asInt4(),2);
 		z = y.divide(x);
-		assertEquals(z.type(),DatumType.INT);
-		assertTrue(z.asInt() == 0);
+		assertEquals(z.type(),Type.INT4);
+		assertTrue(z.asInt4() == 0);
 		
-		y = DatumFactory.createLong(3l);
+		y = DatumFactory.createInt8(3l);
 		z = x.divide(y);
-		assertEquals(z.type(),DatumType.LONG);
-		assertEquals(z.asLong(),2l);		
+		assertEquals(z.type(),Type.INT8);
+		assertEquals(z.asInt8(),2l);
 		z = y.divide(x);
-		assertEquals(z.type(),DatumType.LONG);
-		assertEquals(z.asLong(),0l);
+		assertEquals(z.type(),Type.INT8);
+		assertEquals(z.asInt8(),0l);
 		
-		y = DatumFactory.createFloat(3f);
+		y = DatumFactory.createFloat4(3f);
 		z = x.divide(y);
-		assertEquals(z.type(),DatumType.FLOAT);
-		assertTrue(z.asFloat() == 2.0f);
+		assertEquals(z.type(),Type.FLOAT4);
+		assertTrue(z.asFloat4() == 2.0f);
 		z = y.divide(x);
-		assertEquals(z.type(),DatumType.FLOAT);
-		assertTrue(z.asFloat() == 0.5f);
+		assertEquals(z.type(),Type.FLOAT4);
+		assertTrue(z.asFloat4() == 0.5f);
 		
-		y = DatumFactory.createDouble(3d);
+		y = DatumFactory.createFloat8(3d);
 		z = x.divide(y);
-		assertEquals(z.type(),DatumType.DOUBLE);
-		assertTrue(z.asDouble() == 2.0d);
+		assertEquals(z.type(),Type.FLOAT8);
+		assertTrue(z.asFloat8() == 2.0d);
 		z = y.divide(x);
-		assertEquals(z.type(),DatumType.DOUBLE);
-		assertTrue(z.asDouble() == 0.5d);
+		assertEquals(z.type(),Type.FLOAT8);
+		assertTrue(z.asFloat8() == 0.5d);
 	}
 	
 	@Test
@@ -192,31 +193,31 @@ public class TestDatum {
 		Datum y;
 		Datum z;
 		
-		x = DatumFactory.createInt(6);
-		y = DatumFactory.createInt(3);
+		x = DatumFactory.createInt4(6);
+		y = DatumFactory.createInt4(3);
 		z = x.equalsTo(y);
-		assertEquals(z.type(),DatumType.BOOLEAN);
+		assertEquals(z.type(),Type.BOOLEAN);
 		assertEquals(z.asBool(),false);		
 		z = y.equalsTo(x);
-		assertEquals(z.type(),DatumType.BOOLEAN);
+		assertEquals(z.type(),Type.BOOLEAN);
 		assertEquals(z.asBool(), false);
 		
-		x = DatumFactory.createFloat(3.27f);
-		y = DatumFactory.createFloat(3.27f);
+		x = DatumFactory.createFloat4(3.27f);
+		y = DatumFactory.createFloat4(3.27f);
 		z = x.equalsTo(y);
-		assertEquals(z.type(),DatumType.BOOLEAN);
+		assertEquals(z.type(),Type.BOOLEAN);
 		assertEquals(z.asBool(),true);		
 		z = y.equalsTo(x);
-		assertEquals(z.type(),DatumType.BOOLEAN);
+		assertEquals(z.type(),Type.BOOLEAN);
 		assertEquals(z.asBool(), true);
 		
-		x = DatumFactory.createLong(123456789012345l);
-		y = DatumFactory.createLong(123456789012345l);
+		x = DatumFactory.createInt8(123456789012345l);
+		y = DatumFactory.createInt8(123456789012345l);
 		z = x.equalsTo(y);
-		assertEquals(z.type(),DatumType.BOOLEAN);
+		assertEquals(z.type(),Type.BOOLEAN);
 		assertEquals(z.asBool(),true);		
 		z = y.equalsTo(x);
-		assertEquals(z.type(),DatumType.BOOLEAN);
+		assertEquals(z.type(),Type.BOOLEAN);
 		assertEquals(z.asBool(), true);
 	}
 	
@@ -226,13 +227,13 @@ public class TestDatum {
 		Datum y;
 		Datum z;
 		
-		x = DatumFactory.createInt(6);
-		y = DatumFactory.createInt(3);
+		x = DatumFactory.createInt4(6);
+		y = DatumFactory.createInt4(3);
 		z = x.lessThan(y);
-		assertEquals(z.type(),DatumType.BOOLEAN);
+		assertEquals(z.type(),Type.BOOLEAN);
 		assertEquals(z.asBool(),false);		
 		z = y.lessThan(x);
-		assertEquals(z.type(),DatumType.BOOLEAN);
+		assertEquals(z.type(),Type.BOOLEAN);
 		assertEquals(z.asBool(), true);
 	}
 	
@@ -242,22 +243,22 @@ public class TestDatum {
 		Datum y;
 		Datum z;
 		
-		x = DatumFactory.createInt(6);
-		y = DatumFactory.createInt(3);
+		x = DatumFactory.createInt4(6);
+		y = DatumFactory.createInt4(3);
 		z = x.lessThanEqual(y);
-		assertEquals(z.type(),DatumType.BOOLEAN);
+		assertEquals(z.type(),Type.BOOLEAN);
 		assertEquals(z.asBool(),false);		
 		z = y.lessThanEqual(x);
-		assertEquals(z.type(),DatumType.BOOLEAN);
+		assertEquals(z.type(),Type.BOOLEAN);
 		assertEquals(z.asBool(), true);
 		
-		x = DatumFactory.createInt(6);
-		y = DatumFactory.createInt(6);
+		x = DatumFactory.createInt4(6);
+		y = DatumFactory.createInt4(6);
 		z = x.lessThanEqual(y);
-		assertEquals(z.type(),DatumType.BOOLEAN);
+		assertEquals(z.type(),Type.BOOLEAN);
 		assertEquals(z.asBool(),true);		
 		z = y.lessThanEqual(x);
-		assertEquals(z.type(),DatumType.BOOLEAN);
+		assertEquals(z.type(),Type.BOOLEAN);
 		assertEquals(z.asBool(), true);
 	}
 	
@@ -267,22 +268,22 @@ public class TestDatum {
 		Datum y;
 		Datum z;
 		
-		x = DatumFactory.createInt(6);
-		y = DatumFactory.createInt(3);
+		x = DatumFactory.createInt4(6);
+		y = DatumFactory.createInt4(3);
 		z = x.greaterThan(y);
-		assertEquals(z.type(),DatumType.BOOLEAN);
+		assertEquals(z.type(),Type.BOOLEAN);
 		assertEquals(z.asBool(),true);		
 		z = y.greaterThan(x);
-		assertEquals(z.type(),DatumType.BOOLEAN);
+		assertEquals(z.type(),Type.BOOLEAN);
 		assertEquals(z.asBool(), false);
 		
-		x = DatumFactory.createInt(6);
-		y = DatumFactory.createInt(6);
+		x = DatumFactory.createInt4(6);
+		y = DatumFactory.createInt4(6);
 		z = x.greaterThan(y);
-		assertEquals(z.type(),DatumType.BOOLEAN);
+		assertEquals(z.type(),Type.BOOLEAN);
 		assertEquals(z.asBool(),false);		
 		z = y.greaterThan(x);
-		assertEquals(z.type(),DatumType.BOOLEAN);
+		assertEquals(z.type(),Type.BOOLEAN);
 		assertEquals(z.asBool(), false);
 	}
 	
@@ -292,22 +293,22 @@ public class TestDatum {
 		Datum y;
 		Datum z;
 		
-		x = DatumFactory.createInt(6);
-		y = DatumFactory.createInt(3);
+		x = DatumFactory.createInt4(6);
+		y = DatumFactory.createInt4(3);
 		z = x.greaterThanEqual(y);
-		assertEquals(z.type(),DatumType.BOOLEAN);
+		assertEquals(z.type(),Type.BOOLEAN);
 		assertEquals(z.asBool(),true);		
 		z = y.greaterThanEqual(x);
-		assertEquals(z.type(),DatumType.BOOLEAN);
+		assertEquals(z.type(),Type.BOOLEAN);
 		assertEquals(z.asBool(), false);
 		
-		x = DatumFactory.createInt(6);
-		y = DatumFactory.createInt(6);
+		x = DatumFactory.createInt4(6);
+		y = DatumFactory.createInt4(6);
 		z = x.greaterThanEqual(y);
-		assertEquals(z.type(),DatumType.BOOLEAN);
+		assertEquals(z.type(),Type.BOOLEAN);
 		assertEquals(z.asBool(),true);		
 		z = y.greaterThanEqual(x);
-		assertEquals(z.type(),DatumType.BOOLEAN);
+		assertEquals(z.type(),Type.BOOLEAN);
 		assertEquals(z.asBool(), true);
 	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-common/src/test/java/tajo/datum/TestDatumFactory.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/test/java/tajo/datum/TestDatumFactory.java b/tajo-common/src/test/java/tajo/datum/TestDatumFactory.java
index 4370bf8..2b59f46 100644
--- a/tajo-common/src/test/java/tajo/datum/TestDatumFactory.java
+++ b/tajo-common/src/test/java/tajo/datum/TestDatumFactory.java
@@ -19,56 +19,57 @@
 package tajo.datum;
 
 import org.junit.Test;
+import tajo.common.TajoDataTypes.Type;
 
 import static org.junit.Assert.assertEquals;
 
 public class TestDatumFactory {
 	
 	@Test
-	public final void testCreateByte() {
-		Datum d = DatumFactory.createByte((byte) 5);
-		assertEquals(DatumType.BYTE, d.type());
+	public final void testCreateBit() {
+		Datum d = DatumFactory.createBit((byte) 5);
+		assertEquals(Type.BIT, d.type());
 	}
 
 	@Test
-	public final void testCreateShort() {
-		Datum d = DatumFactory.createShort((short)5);
-		assertEquals(DatumType.SHORT, d.type());
+	public final void testCreateInt2() {
+		Datum d = DatumFactory.createInt2((short) 5);
+		assertEquals(Type.INT2, d.type());
 	}
 	
 	@Test
-	public final void testCreateInt() {
-		Datum d = DatumFactory.createInt(5);
-		assertEquals(DatumType.INT, d.type());
+	public final void testCreateInt4() {
+		Datum d = DatumFactory.createInt4(5);
+		assertEquals(Type.INT4, d.type());
 	}
 	
 	@Test
-	public final void testCreateLong() {
-		Datum d = DatumFactory.createLong((long)5);
-		assertEquals(DatumType.LONG, d.type());
+	public final void testCreateInt8() {
+		Datum d = DatumFactory.createInt8((long) 5);
+		assertEquals(Type.INT8, d.type());
 	}
 
 	@Test
-	public final void testCreateFloat() {
-		Datum d = DatumFactory.createFloat(5.0f);
-		assertEquals(DatumType.FLOAT, d.type());
+	public final void testCreateFloat4() {
+		Datum d = DatumFactory.createFloat4(5.0f);
+		assertEquals(Type.FLOAT4, d.type());
 	}
 
 	@Test
-	public final void testCreateDouble() {
-		Datum d = DatumFactory.createDouble(5.0d);
-		assertEquals(DatumType.DOUBLE, d.type());
+	public final void testCreateFloat8() {
+		Datum d = DatumFactory.createFloat8(5.0d);
+		assertEquals(Type.FLOAT8, d.type());
 	}
 
 	@Test
 	public final void testCreateBoolean() {
 		Datum d = DatumFactory.createBool(true);
-		assertEquals(DatumType.BOOLEAN, d.type());
+		assertEquals(Type.BOOLEAN, d.type());
 	}
 
 	@Test
 	public final void testCreateString() {
-		Datum d = DatumFactory.createString("12345a");
-		assertEquals(DatumType.STRING, d.type());
+		Datum d = DatumFactory.createText("12345a");
+		assertEquals(Type.TEXT, d.type());
 	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-common/src/test/java/tajo/datum/TestDoubleDatum.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/test/java/tajo/datum/TestDoubleDatum.java b/tajo-common/src/test/java/tajo/datum/TestDoubleDatum.java
deleted file mode 100644
index 178ce01..0000000
--- a/tajo-common/src/test/java/tajo/datum/TestDoubleDatum.java
+++ /dev/null
@@ -1,69 +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 tajo.datum;
-
-import org.junit.Test;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
-public class TestDoubleDatum {
-
-	@Test
-	public final void testType() {
-		Datum d = DatumFactory.createDouble(1d);
-		assertEquals(DatumType.DOUBLE, d.type());
-	}
-
-	@Test
-	public final void testAsInt() {
-		Datum d = DatumFactory.createDouble(5d);
-		assertEquals(5,d.asInt());
-	}
-
-	@Test
-	public final void testAsLong() {
-		Datum d = DatumFactory.createDouble(5d);
-		assertEquals(5l,d.asLong());		
-	}
-
-	@Test
-	public final void testAsFloat() {
-		Datum d = DatumFactory.createDouble(5d);
-		assertTrue(5.0f == d.asFloat());
-	}
-
-	@Test
-	public final void testAsDouble() {
-		Datum d = DatumFactory.createDouble(5d);
-		assertTrue(5.0d == d.asDouble());
-	}
-
-	@Test
-	public final void testAsChars() {
-		Datum d = DatumFactory.createDouble(5d);
-		assertEquals("5.0", d.asChars());
-	}
-	
-	@Test
-  public final void testSize() {
-	  Datum d = DatumFactory.createDouble(5d);
-	  assertEquals(8, d.size());
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-common/src/test/java/tajo/datum/TestFloatDatum.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/test/java/tajo/datum/TestFloatDatum.java b/tajo-common/src/test/java/tajo/datum/TestFloatDatum.java
index bfc45bc..b207666 100644
--- a/tajo-common/src/test/java/tajo/datum/TestFloatDatum.java
+++ b/tajo-common/src/test/java/tajo/datum/TestFloatDatum.java
@@ -19,6 +19,7 @@
 package tajo.datum;
 
 import org.junit.Test;
+import tajo.common.TajoDataTypes.Type;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
@@ -27,43 +28,43 @@ public class TestFloatDatum {
 
 	@Test
 	public final void testType() {
-		Datum d = DatumFactory.createFloat(1f);
-		assertEquals(DatumType.FLOAT, d.type());
+		Datum d = DatumFactory.createFloat4(1f);
+		assertEquals(Type.FLOAT4, d.type());
 	}
 
 	@Test
 	public final void testAsInt() {
-		Datum d = DatumFactory.createFloat(5f);
-		assertEquals(5,d.asInt());
+		Datum d = DatumFactory.createFloat4(5f);
+		assertEquals(5,d.asInt4());
 	}
 
 	@Test
 	public final void testAsLong() {
-		Datum d = DatumFactory.createFloat(5f);
-		assertEquals(5l,d.asLong());		
+		Datum d = DatumFactory.createFloat4(5f);
+		assertEquals(5l,d.asInt8());
 	}
 
 	@Test
 	public final void testAsFloat() {
-		Datum d = DatumFactory.createFloat(5f);
-		assertTrue(5.0f == d.asFloat());
+		Datum d = DatumFactory.createFloat4(5f);
+		assertTrue(5.0f == d.asFloat4());
 	}
 
 	@Test
 	public final void testAsDouble() {
-		Datum d = DatumFactory.createFloat(5f);
-		assertTrue(5.0d == d.asDouble());
+		Datum d = DatumFactory.createFloat4(5f);
+		assertTrue(5.0d == d.asFloat8());
 	}
 
 	@Test
 	public final void testAsChars() {
-		Datum d = DatumFactory.createFloat(5f);
+		Datum d = DatumFactory.createFloat4(5f);
 		assertEquals("5.0", d.asChars());
 	}
 	
 	@Test
   public final void testSize() {
-    Datum d = DatumFactory.createFloat(5f);
+    Datum d = DatumFactory.createFloat4(5f);
     assertEquals(4, d.size());
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-common/src/test/java/tajo/datum/TestIPv4Datum.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/test/java/tajo/datum/TestIPv4Datum.java b/tajo-common/src/test/java/tajo/datum/TestIPv4Datum.java
deleted file mode 100644
index e60a33b..0000000
--- a/tajo-common/src/test/java/tajo/datum/TestIPv4Datum.java
+++ /dev/null
@@ -1,75 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package tajo.datum;
-
-import org.junit.Before;
-import org.junit.Test;
-import tajo.datum.json.GsonCreator;
-
-import java.util.Arrays;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
-public class TestIPv4Datum {
-
-	@Before
-	public void setUp() throws Exception {
-	}
-	
-	@Test
-	public final void testEquals() {
-	  IPv4Datum ip1 = new IPv4Datum("192.168.0.1");
-	  IPv4Datum ip2 = new IPv4Datum("192.168.0.1");
-	  
-	  assertEquals(ip1, ip2);
-	  
-	  IPv4Datum ip3 = new IPv4Datum(ip1.asByteArray());
-	  assertEquals(ip1, ip3);
-	  IPv4Datum ip4 = DatumFactory.createIPv4(ip1.asByteArray());
-	  assertEquals(ip1, ip4);
-	}
-
-	@Test
-	public final void testAsByteArray() {
-		byte[] bytes = {(byte) 0xA3, (byte) 0x98, 0x17, (byte) 0xDE};
-		IPv4Datum ip = new IPv4Datum(bytes);
-		assertTrue(Arrays.equals(bytes, ip.asByteArray()));
-	}
-
-	@Test
-	public final void testAsChars() {
-		IPv4Datum ip = new IPv4Datum("163.152.23.222");
-		assertEquals("163.152.23.222", ip.asChars());
-	}
-	
-	@Test
-  public final void testSize() {
-    Datum d = DatumFactory.createIPv4("163.152.23.222");
-    assertEquals(4, d.size());
-  }
-	
-	@Test
-	public final void testJson() {
-		Datum d = DatumFactory.createIPv4("163.152.163.152");
-		String json = d.toJSON();
-		Datum fromJson = GsonCreator.getInstance().fromJson(json, Datum.class);
-		assertTrue(d.equalsTo(fromJson).asBool());
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-common/src/test/java/tajo/datum/TestIntDatum.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/test/java/tajo/datum/TestIntDatum.java b/tajo-common/src/test/java/tajo/datum/TestIntDatum.java
deleted file mode 100644
index faca9d8..0000000
--- a/tajo-common/src/test/java/tajo/datum/TestIntDatum.java
+++ /dev/null
@@ -1,69 +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 tajo.datum;
-
-import org.junit.Test;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
-public class TestIntDatum {
-
-  @Test
-  public final void testType() {
-    Datum d = DatumFactory.createInt(1);
-    assertEquals(DatumType.INT, d.type());
-  }
-
-  @Test
-  public final void testAsInt() {
-    Datum d = DatumFactory.createInt(5);
-    assertEquals(5, d.asInt());
-  }
-
-  @Test
-  public final void testAsLong() {
-    Datum d = DatumFactory.createInt(5);
-    assertEquals(5, d.asLong());
-  }
-
-  @Test
-  public final void testAsFloat() {
-    Datum d = DatumFactory.createInt(5);
-    assertTrue(5.0f == d.asFloat());
-  }
-
-  @Test
-  public final void testAsDouble() {
-    Datum d = DatumFactory.createInt(5);
-    assertTrue(5.0d == d.asDouble());
-  }
-
-  @Test
-  public final void testAsChars() {
-    Datum d = DatumFactory.createInt(5);
-    assertEquals("5", d.asChars());
-  }
-
-  @Test
-  public final void testSize() {
-    Datum d = DatumFactory.createInt(5);
-    assertEquals(4, d.size());
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-common/src/test/java/tajo/datum/TestLongDatum.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/test/java/tajo/datum/TestLongDatum.java b/tajo-common/src/test/java/tajo/datum/TestLongDatum.java
deleted file mode 100644
index 00170f7..0000000
--- a/tajo-common/src/test/java/tajo/datum/TestLongDatum.java
+++ /dev/null
@@ -1,69 +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 tajo.datum;
-
-import org.junit.Test;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
-public class TestLongDatum {
-
-	@Test
-	public final void testType() {
-		Datum d = DatumFactory.createLong(1l);
-		assertEquals(d.type(), DatumType.LONG);
-	}
-
-	@Test
-	public final void testAsInt() {
-		Datum d = DatumFactory.createLong(5l);
-		assertEquals(5,d.asInt());
-	}
-
-	@Test
-	public final void testAsLong() {
-		Datum d = DatumFactory.createLong(5l);
-		assertEquals(5l,d.asLong());		
-	}
-
-	@Test
-	public final void testAsFloat() {
-		Datum d = DatumFactory.createLong(5l);
-		assertTrue(5.0f == d.asFloat());
-	}
-
-	@Test
-	public final void testAsDouble() {
-		Datum d = DatumFactory.createLong(5l);
-		assertTrue(5.0d == d.asDouble());
-	}
-
-	@Test
-	public final void testAsChars() {
-		Datum d = DatumFactory.createLong(5l);
-		assertEquals("5", d.asChars());
-	}
-	
-	@Test
-  public final void testSize() {
-    Datum d = DatumFactory.createLong(5l);
-    assertEquals(8, d.size());
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-common/src/test/java/tajo/datum/TestShortDatum.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/test/java/tajo/datum/TestShortDatum.java b/tajo-common/src/test/java/tajo/datum/TestShortDatum.java
deleted file mode 100644
index 83da6a5..0000000
--- a/tajo-common/src/test/java/tajo/datum/TestShortDatum.java
+++ /dev/null
@@ -1,69 +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 tajo.datum;
-
-import org.junit.Test;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
-public class TestShortDatum {
-
-	@Test
-	public final void testType() {
-		Datum d = DatumFactory.createShort((short) 1);
-		assertEquals(d.type(), DatumType.SHORT);
-	}
-
-	@Test
-	public final void testAsInt() {
-		Datum d = DatumFactory.createShort((short)5);
-		assertEquals(5,d.asInt());
-	}
-
-	@Test
-	public final void testAsLong() {
-		Datum d = DatumFactory.createShort((short) 5);
-		assertEquals(5,d.asLong());
-	}
-
-	@Test
-	public final void testAsFloat() {
-		Datum d = DatumFactory.createShort((short) 5);
-		assertTrue(5.0f == d.asFloat());
-	}
-
-	@Test
-	public final void testAsDouble() {
-		Datum d = DatumFactory.createShort((short) 5);
-		assertTrue(5.0d == d.asDouble());
-	}
-
-	@Test
-	public final void testAsChars() {
-		Datum d = DatumFactory.createShort((short) 5);
-		assertEquals("5", d.asChars());
-	}
-	
-	@Test
-  public final void testSize() {
-    Datum d = DatumFactory.createShort((short) 5);
-    assertEquals(2, d.size());
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-common/src/test/java/tajo/datum/TestStringDatum.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/test/java/tajo/datum/TestStringDatum.java b/tajo-common/src/test/java/tajo/datum/TestStringDatum.java
deleted file mode 100644
index a5f975b..0000000
--- a/tajo-common/src/test/java/tajo/datum/TestStringDatum.java
+++ /dev/null
@@ -1,69 +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 tajo.datum;
-
-import org.junit.Test;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
-public class TestStringDatum {
-	
-	@Test
-	public final void testType() {
-		Datum d = DatumFactory.createString("12345");
-		assertEquals(d.type(), DatumType.STRING);
-	}
-	
-	@Test
-	public final void testAsInt() {
-		Datum d = DatumFactory.createString("12345");
-		assertEquals(12345,d.asInt());
-	}
-
-	@Test
-	public final void testAsLong() {
-		Datum d = DatumFactory.createString("12345");
-		assertEquals(12345l,d.asLong());
-	}
-
-	@Test
-	public final void testAsFloat() {
-		Datum d = DatumFactory.createString("12345");
-		assertTrue(12345.0f == d.asFloat());
-	}
-
-	@Test
-	public final void testAsDouble() {
-		Datum d = DatumFactory.createString("12345");
-		assertTrue(12345.0d == d.asDouble());
-	}
-
-	@Test
-	public final void testAsChars() {
-		Datum d = DatumFactory.createString("12345");
-		assertEquals("12345", d.asChars());
-	}
-	
-	@Test
-  public final void testSize() {
-	  Datum d = DatumFactory.createString("12345");
-    assertEquals(5, d.size());
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-backend/src/main/antlr3/tajo/engine/parser/NQL.g
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/antlr3/tajo/engine/parser/NQL.g b/tajo-core/tajo-core-backend/src/main/antlr3/tajo/engine/parser/NQL.g
index 725941c..49eb91a 100644
--- a/tajo-core/tajo-core-backend/src/main/antlr3/tajo/engine/parser/NQL.g
+++ b/tajo-core/tajo-core-backend/src/main/antlr3/tajo/engine/parser/NQL.g
@@ -36,9 +36,9 @@ tokens {
   DROP_TABLE;
   DESC_TABLE;
   EMPTY_GROUPING_SET;
-  FIELD_NAME;  
+  FIELD_NAME;
   FIELD_DEF;
-  FUNCTION;    
+  FUNCTION;
   FUNC_ARGS;
   GROUP_BY;
   NULL_ORDER;
@@ -56,7 +56,7 @@ tokens {
   STORE;
   STORE_TYPE;
   TABLE_DEF;
-  TARGET_FIELDS;  
+  TARGET_FIELDS;
   VALUES;
 }
 
@@ -112,32 +112,32 @@ statement
 sessionStatement
   : 'session' 'clear' -> ^(SESSION_CLEAR)
   ;
-  
+
 controlStatement
   : '\\' 't' (table)? -> ^(SHOW_TABLE table?)
   | '\\' 'd' table -> ^(DESC_TABLE table)
   | '\\' 'f' -> ^(SHOW_FUNCTION)
   ;
-  
+
 dataStatement
   : query_expression
   | set_stmt
   | copyStatement
   ;
-  
+
 dataChangeStatement
   : insertStmt
   ;
-  
+
 schemaStatement
   : createTableStatement
   | DROP TABLE table -> ^(DROP_TABLE table)
   ;
-  
+
 indexStatement
   : CREATE (u=UNIQUE)? INDEX n=ID ON t=table (m=method_specifier)? LEFT_PAREN s=sort_specifier_list RIGHT_PAREN p=param_clause?-> ^(CREATE_INDEX $u? $m? $p? $n $t $s)
   ;
-  
+
 createTableStatement
   : CREATE EXTERNAL TABLE t=table def=tableElements USING f=ID p=param_clause? (LOCATION path=STRING)
       -> ^(CREATE_TABLE $t EXTERNAL ^(TABLE_DEF $def) ^(USING $f) $p? ^(LOCATION $path))
@@ -148,61 +148,165 @@ createTableStatement
 copyStatement
   : COPY t=table FROM path=string_value_expr FORMAT s=ID (p=param_clause)? -> ^(COPY $t $path $s $p?)
   ;
-  
+
 tableElements
   : LEFT_PAREN fieldElement (COMMA fieldElement)* RIGHT_PAREN -> fieldElement+
   ;
-  
+
 fieldElement
   : ID fieldType -> ^(FIELD_DEF ID fieldType)
   ;
-  
+
 fieldType
-  : BOOL
-  | BYTE
-  | CHAR
-  | INT
-  | LONG
-  | FLOAT
-  | DOUBLE
-  | TEXT
+  : dataType
+  ;
+
+precision_param
+  : LEFT_PAREN! DIGIT RIGHT_PAREN!
+  | LEFT_PAREN! DIGIT COMMA! DIGIT RIGHT_PAREN!
+  ;
+type_length
+  : LEFT_PAREN! DIGIT RIGHT_PAREN!
+  ;
+
+boolean_type
+  : BOOLEAN
+  | BOOL -> BOOLEAN
+  ;
+bit_type
+  : BIT type_length? -> BIT
+  ;
+varbit_type
+  : VARBIT type_length? -> VARBIT
+  | BIT VARYING type_length? -> VARBIT
+  ;
+int1_type
+  : INT1
+  | TINYINT -> INT1
+  ;
+int2_type
+  : INT2
+  | SMALLINT -> INT2
+  ;
+int4_type
+  : INT4
+  | INT -> INT4
+  | INTEGER -> INT4
+  ;
+int8_type
+  : INT8
+  | BIGINT -> INT8
+  ;
+float4_type
+  : FLOAT4
+  | REAL -> FLOAT4
+  ;
+float_type : FLOAT type_length? -> ^(FLOAT type_length?);
+float8_type
+  : FLOAT8
+  | DOUBLE -> FLOAT8
+  | DOUBLE PRECISION -> FLOAT8
+  ;
+number_type
+  : NUMERIC (precision_param)? -> NUMERIC
+  | DECIMAL (precision_param)? -> NUMERIC
+  ;
+char_type
+  : CHAR type_length? -> CHAR
+  | CHARACTER type_length? -> CHAR
+  ;
+varchar_type
+  : VARCHAR type_length? -> VARCHAR
+  | CHARACTER VARYING type_length? -> VARCHAR
+  ;
+nchar_type
+  : NCHAR type_length? -> NCHAR
+  | NATIONAL CHARACTER type_length? -> NCHAR
+  ;
+nvarchar_type
+  : NVARCHAR type_length? -> NVARCHAR
+  | NATIONAL CHARACTER VARYING type_length? -> NVARCHAR
+  ;
+timetz_type
+  : TIMETZ
+  | TIME WITH TIME ZONE -> TIMETZ
+  ;
+timestamptz_type
+  : TIMESTAMPTZ
+  | TIMESTAMP WITH TIME ZONE -> TIMESTAMPTZ
+  ;
+binary_type
+  : BINARY type_length?
+  ;
+varbinary_type
+  : VARBINARY type_length?
+  | BINARY VARYING type_length?
+  ;
+blob_type
+  : BLOB
+  | BYTEA -> BLOB
+  ;
+
+dataType
+  : boolean_type
+  | bit_type
+  | varbit_type
+  | int1_type
+  | int2_type
+  | int4_type
+  | int8_type
+  | float4_type
+  | float_type
+  | float8_type
+  | number_type
+  | char_type
+  | varchar_type
+  | nchar_type
+  | nvarchar_type
   | DATE
-  | BYTES
-  | IPv4
+  | TIME
+  | timetz_type
+  | TIMESTAMP
+  | timestamptz_type
+  | TEXT
+  | binary_type
+  | varbinary_type
+  | blob_type
+  | INET4
   ;
 
 query_expression
   : query_expression_body
   ;
-  
+
 query_expression_body
   : non_join_query_expression
   | joined_table
   ;
-  
+
 non_join_query_expression
-  : (non_join_query_term | joined_table (UNION | EXCEPT)^ (ALL|DISTINCT)? query_term) ((UNION | EXCEPT)^ (ALL|DISTINCT)? query_term)*  
+  : (non_join_query_term | joined_table (UNION | EXCEPT)^ (ALL|DISTINCT)? query_term) ((UNION | EXCEPT)^ (ALL|DISTINCT)? query_term)*
   ;
-  
+
 query_term
   : non_join_query_term
   | joined_table
   ;
-  
+
 non_join_query_term
   : ( non_join_query_primary | joined_table INTERSECT^ (ALL|DISTINCT)? query_primary) (INTERSECT^ (ALL|DISTINCT)? query_primary)*
   ;
-  
+
 query_primary
   : non_join_query_primary
   | joined_table
   ;
-  
+
 non_join_query_primary
   : simple_table
   | LEFT_PAREN non_join_query_expression RIGHT_PAREN
   ;
-  
+
 simple_table
 options {k=1;}
   : query_specification
@@ -212,39 +316,39 @@ query_specification
   : SELECT setQualifier? selectList from_clause? where_clause? groupby_clause? having_clause? orderby_clause? limit_clause?
   -> ^(SELECT from_clause? setQualifier? selectList where_clause? groupby_clause? having_clause? orderby_clause? limit_clause?)
   ;
-  
+
 insertStmt
   : INSERT 'into' table (LEFT_PAREN column_reference RIGHT_PAREN)? 'values' array
   -> ^(INSERT ^(TABLE table) ^(VALUES array) ^(TARGET_FIELDS column_reference)?)
   ;
-	
+
 selectList
   : MULTIPLY -> ^(SEL_LIST ALL)
   | derivedColumn (COMMA derivedColumn)* -> ^(SEL_LIST derivedColumn+)
   ;
-  
+
 setQualifier
-  : DISTINCT -> ^(SET_QUALIFIER DISTINCT) 
+  : DISTINCT -> ^(SET_QUALIFIER DISTINCT)
   | ALL -> ^(SET_QUALIFIER ALL)
   ;
-  
+
 derivedColumn
   : bool_expr asClause? -> ^(COLUMN bool_expr asClause?)
   ;
-	
+
 fieldName
 	:	(t=ID DOT)? b=ID -> ^(FIELD_NAME $b $t?)
 	;
-	
+
 asClause
   : (AS)? fieldName
   ;
-	
-column_reference  
+
+column_reference
 	:	fieldName (COMMA fieldName)* -> fieldName+
 	;
-  
-table  
+
+table
   : ID
   ;
 
@@ -254,19 +358,19 @@ funcCall
 	| COUNT LEFT_PAREN funcArgs RIGHT_PAREN -> ^(COUNT_VAL funcArgs)
 	| COUNT LEFT_PAREN MULTIPLY RIGHT_PAREN -> ^(COUNT_ROWS)
 	;
-	
+
 funcArgs
-  : bool_expr (COMMA bool_expr)* -> bool_expr+ 
+  : bool_expr (COMMA bool_expr)* -> bool_expr+
   ;
-	
+
 from_clause
   : FROM^ table_reference_list
   ;
-  
+
 table_reference_list
   :table_reference (COMMA table_reference)* -> table_reference+
   ;
-  
+
 table_reference
   : table_primary
   | joined_table
@@ -303,26 +407,26 @@ join_type
   : INNER
   | t=outer_join_type ('outer')? -> ^(OUTER $t)
   ;
-  
+
 outer_join_type
   : LEFT
   | RIGHT
   | FULL
   ;
-  
+
 join_specification
   : join_condition
   | named_columns_join
   ;
-  
+
 join_condition
   : ON^ search_condition
   ;
-  
+
 named_columns_join
   : USING LEFT_PAREN f=column_reference RIGHT_PAREN -> ^(USING $f)
   ;
-  
+
 table_primary
   : table ((AS)? a=ID)? -> ^(TABLE table ($a)?)
   ;
@@ -330,22 +434,22 @@ table_primary
 where_clause
   : WHERE^ search_condition
   ;
-  
+
 groupby_clause
   : GROUP BY g=grouping_element_list -> ^(GROUP_BY $g)
   ;
-  
+
 grouping_element_list
   : grouping_element (COMMA grouping_element)* -> grouping_element+
   ;
-  
+
 grouping_element
   : ordinary_grouping_set
   | rollup_list
   | cube_list
   | empty_grouping_set
   ;
-  
+
 ordinary_grouping_set
   : column_reference
   | LEFT_PAREN! column_reference RIGHT_PAREN!
@@ -354,7 +458,7 @@ ordinary_grouping_set
 rollup_list
   : ROLLUP LEFT_PAREN c=ordinary_grouping_set RIGHT_PAREN -> ^(ROLLUP $c)
   ;
-  
+
 cube_list
   : CUBE LEFT_PAREN c=ordinary_grouping_set RIGHT_PAREN -> ^(CUBE $c)
   ;
@@ -362,23 +466,23 @@ cube_list
 empty_grouping_set
   : LEFT_PAREN RIGHT_PAREN -> ^(EMPTY_GROUPING_SET)
   ;
-  
+
 having_clause
   : HAVING^ bool_expr
   ;
-  
+
 orderby_clause
   : ORDER BY sort_specifier_list -> ^(ORDER_BY sort_specifier_list)
   ;
-  
+
 sort_specifier_list
   : sort_specifier (COMMA sort_specifier)* -> ^(SORT_SPECIFIERS sort_specifier+)
   ;
-  
+
 sort_specifier
   : fn=fieldName a=order_specification? o=null_ordering? -> ^(SORT_KEY $fn $a? $o?)
   ;
-  
+
 order_specification
   : ASC -> ^(ORDER ASC)
   | DESC -> ^(ORDER DESC)
@@ -392,23 +496,23 @@ null_ordering
   : NULL FIRST -> ^(NULL_ORDER FIRST)
   | NULL LAST -> ^(NULL_ORDER LAST)
   ;
-	
+
 set_stmt
 	:	'set' ('union'|'intersect'|'diff') table
 	;
-	
+
 search_condition
 	:	bool_expr
-	; 
-	
+	;
+
 param_clause
   : WITH LEFT_PAREN param (COMMA param)* RIGHT_PAREN -> ^(PARAMS param+)
   ;
-  
+
 param
   : k=STRING EQUAL v=bool_expr -> ^(PARAM $k $v)
   ;
-  
+
 method_specifier
   : USING m=ID -> ^(USING[$m.text])
   ;
@@ -420,21 +524,21 @@ bool_expr
 and_predicate
   :	boolean_factor (AND^ boolean_factor)*
 	;
-	
+
 boolean_factor
   : boolean_test
   | NOT boolean_test -> ^(NOT boolean_test)
   ;
-  
+
 boolean_test
   : boolean_primary is_clause?
   ;
-  
+
 is_clause
   : IS NOT? t=truth_value -> ^(IS NOT? $t)
   ;
 
-  
+
 truth_value
   : TRUE | FALSE | UNKNOWN
   ;
@@ -446,7 +550,7 @@ boolean_primary
   | case_expression
   ;
 
-predicate 
+predicate
   : comparison_predicate
   | in_predicate
   | like_predicate
@@ -456,7 +560,7 @@ predicate
 in_predicate
 	:	expr NOT? IN array -> ^(IN expr array NOT?)
 	;
-	
+
 like_predicate
   : f=fieldName NOT? LIKE s=string_value_expr -> ^(LIKE NOT? $f $s)
   ;
@@ -464,7 +568,7 @@ like_predicate
 null_predicate
   : f=expr IS (n=NOT)? NULL -> ^(IS $f NULL $n?)
   ;
-	
+
 comparison_predicate
 	:	expr EQUAL^ expr
 	|	expr NOT_EQUAL^ expr
@@ -477,7 +581,7 @@ comparison_predicate
 expr
 	:	multExpr ((PLUS|MINUS)^ multExpr)*
 	;
-	
+
 multExpr
   :	atom ((MULTIPLY|DIVIDE|MODULAR)^ atom)*
 	;
@@ -485,33 +589,33 @@ multExpr
 array
   : LEFT_PAREN literal (COMMA literal )* RIGHT_PAREN -> literal+
   ;
-	
+
 atom
   :	literal
 	| fieldName
 	|	LEFT_PAREN! expr RIGHT_PAREN!
 	| funcCall
 	;
-	
+
 literal
   : string_value_expr
   | signed_numerical_literal
   | NULL
   ;
-	
+
 string_value_expr
   : STRING
   ;
-  
+
 signed_numerical_literal
   : sign? unsigned_numerical_literal
   ;
-  
+
 unsigned_numerical_literal
-  : DIGIT 
-  | REAL
+  : DIGIT
+  | REAL_NUMBER
   ;
-  
+
 sign
   : PLUS | MINUS
   ;
@@ -542,9 +646,9 @@ else_clause
 result
   : bool_expr
   ;
-	
-////////////////////////////////	
-// Lexer Section  
+
+////////////////////////////////
+// Lexer Section
 ////////////////////////////////
 // Keywords
 AS : 'as';
@@ -553,6 +657,7 @@ AND : 'and';
 ASC : 'asc';
 BY : 'by';
 CASE : 'case';
+CHARACTER : 'character';
 COUNT : 'count';
 COPY : 'copy';
 CREATE : 'create';
@@ -571,8 +676,6 @@ FORMAT : 'format';
 FULL : 'full';
 FROM : 'from';
 GROUP : 'group';
-NATURAL : 'natural';
-NULL : 'null';
 HAVING : 'having';
 IN : 'in';
 INDEX : 'index';
@@ -587,11 +690,15 @@ LEFT : 'left';
 LIKE : 'like';
 LIMIT : 'limit';
 LOCATION : 'location';
+NATIONAL : 'national';
+NATURAL : 'natural';
 NOT : 'not';
+NULL : 'null';
 ON : 'on';
 OUTER : 'outer';
 OR : 'or';
 ORDER : 'order';
+PRECISION : 'precision';
 RIGHT : 'right';
 ROLLUP : 'rollup';
 SELECT : 'select';
@@ -603,22 +710,61 @@ UNIQUE : 'unique';
 UNKNOWN: 'unknown';
 USING : 'using';
 VALUES : 'values';
+VARYING : 'varying';
 WHEN : 'when';
 WHERE : 'where';
 WITH : 'with';
+ZONE : 'zone';
 
-// column types
+///////////////////////////////////////////////////
+// Data Types
+///////////////////////////////////////////////////
+BOOLEAN : 'boolean';
 BOOL : 'bool';
-BYTE : 'byte';
+BIT : 'bit';
+VARBIT : 'varbit';
+
+// Numeric Types
+INT1 : 'int1';
+INT2 : 'int2';
+INT4 : 'int4';
+INT8 : 'int8';
+
+TINYINT : 'tinyint'; // alias for INT1
+SMALLINT : 'smallint'; // alias for INT2
+INT : 'int'; // alias for INT4
+INTEGER : 'integer'; // alias - INT4
+BIGINT : 'bigint'; // alias for INT8
+
+FLOAT4 : 'float4';
+FLOAT8 : 'float8';
+
+REAL : 'real'; // alias for FLOAT4
+FLOAT : 'float'; // alias for FLOAT8
+DOUBLE : 'double'; // alias for FLOAT8
+
+NUMERIC : 'numeric';
+DECIMAL : 'decimal'; // alias for number
+
 CHAR : 'char';
-INT : 'int';
-LONG : 'long';
-FLOAT : 'float';
-DOUBLE : 'double';
-TEXT : 'string';
+VARCHAR : 'varchar';
+NCHAR : 'nchar';
+NVARCHAR : 'nvarchar';
+
 DATE : 'date';
-BYTES : 'bytes';
-IPv4 : 'ipv4';
+TIME : 'time';
+TIMETZ : 'timetz';
+TIMESTAMP : 'timestamp';
+TIMESTAMPTZ : 'timestamptz';
+
+TEXT : 'text';
+
+BINARY : 'binary';
+VARBINARY : 'varbinary';
+BLOB : 'blob';
+BYTEA : 'bytea'; // alias for BLOB
+
+INET4 : 'inet4';
 
 // Operators
 ASSIGN  : ':=';
@@ -646,7 +792,7 @@ ID  : ('a'..'z'|'A'..'Z'|'_') ('a'..'z'|'A'..'Z'|'0'..'9'|'_'|':')*
 DIGIT : '0'..'9'+
     ;
 
-REAL
+REAL_NUMBER
     :   ('0'..'9')+ '.' ('0'..'9')* EXPONENT?
     |   '.' ('0'..'9')+ EXPONENT?
     |   ('0'..'9')+ EXPONENT

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-backend/src/main/java/tajo/benchmark/TPCH.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/tajo/benchmark/TPCH.java b/tajo-core/tajo-core-backend/src/main/java/tajo/benchmark/TPCH.java
index c4ce6d4..ae68bb0 100644
--- a/tajo-core/tajo-core-backend/src/main/java/tajo/benchmark/TPCH.java
+++ b/tajo-core/tajo-core-backend/src/main/java/tajo/benchmark/TPCH.java
@@ -22,11 +22,11 @@ import com.google.protobuf.ServiceException;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.fs.Path;
+import tajo.catalog.CatalogUtil;
 import tajo.catalog.Schema;
-import tajo.catalog.TCatUtil;
 import tajo.catalog.TableMeta;
-import tajo.catalog.proto.CatalogProtos.DataType;
 import tajo.catalog.proto.CatalogProtos.StoreType;
+import tajo.common.TajoDataTypes.Type;
 import tajo.storage.CSVFile;
 
 import java.io.IOException;
@@ -47,104 +47,104 @@ public class TPCH extends BenchmarkSet {
   @Override
   public void loadSchemas() {
     Schema lineitem = new Schema()
-        .addColumn("l_orderkey", DataType.LONG) // 0
-        .addColumn("l_partkey", DataType.INT) // 1
-        .addColumn("l_suppkey", DataType.INT) // 2
-        .addColumn("l_linenumber", DataType.INT) // 3
-        .addColumn("l_quantity", DataType.FLOAT) // 4
-        .addColumn("l_extendedprice", DataType.FLOAT) // 5
-        .addColumn("l_discount", DataType.FLOAT) // 6
-        .addColumn("l_tax", DataType.FLOAT) // 7
+        .addColumn("l_orderkey", Type.INT8) // 0
+        .addColumn("l_partkey", Type.INT4) // 1
+        .addColumn("l_suppkey", Type.INT4) // 2
+        .addColumn("l_linenumber", Type.INT4) // 3
+        .addColumn("l_quantity", Type.FLOAT4) // 4
+        .addColumn("l_extendedprice", Type.FLOAT4) // 5
+        .addColumn("l_discount", Type.FLOAT4) // 6
+        .addColumn("l_tax", Type.FLOAT4) // 7
             // TODO - This is temporal solution. 8 and 9 are actually Char type.
-        .addColumn("l_returnflag", DataType.STRING) // 8
-        .addColumn("l_linestatus", DataType.STRING) // 9
+        .addColumn("l_returnflag", Type.TEXT) // 8
+        .addColumn("l_linestatus", Type.TEXT) // 9
             // TODO - This is temporal solution. 10,11, and 12 are actually Date type.
-        .addColumn("l_shipdate", DataType.STRING) // 10
-        .addColumn("l_commitdate", DataType.STRING) // 11
-        .addColumn("l_receiptdate", DataType.STRING) // 12
-        .addColumn("l_shipinstruct", DataType.STRING) // 13
-        .addColumn("l_shipmode", DataType.STRING) // 14
-        .addColumn("l_comment", DataType.STRING); // 15
+        .addColumn("l_shipdate", Type.TEXT) // 10
+        .addColumn("l_commitdate", Type.TEXT) // 11
+        .addColumn("l_receiptdate", Type.TEXT) // 12
+        .addColumn("l_shipinstruct", Type.TEXT) // 13
+        .addColumn("l_shipmode", Type.TEXT) // 14
+        .addColumn("l_comment", Type.TEXT); // 15
     schemas.put(LINEITEM, lineitem);
 
     Schema customer = new Schema()
-        .addColumn("c_custkey", DataType.INT) // 0
-        .addColumn("c_name", DataType.STRING) // 1
-        .addColumn("c_address", DataType.STRING) // 2
-        .addColumn("c_nationkey", DataType.INT) // 3
-        .addColumn("c_phone", DataType.STRING) // 4
-        .addColumn("c_acctbal", DataType.FLOAT) // 5
-        .addColumn("c_mktsegment", DataType.STRING) // 6
-        .addColumn("c_comment", DataType.STRING); // 7
+        .addColumn("c_custkey", Type.INT4) // 0
+        .addColumn("c_name", Type.TEXT) // 1
+        .addColumn("c_address", Type.TEXT) // 2
+        .addColumn("c_nationkey", Type.INT4) // 3
+        .addColumn("c_phone", Type.TEXT) // 4
+        .addColumn("c_acctbal", Type.FLOAT4) // 5
+        .addColumn("c_mktsegment", Type.TEXT) // 6
+        .addColumn("c_comment", Type.TEXT); // 7
     schemas.put(CUSTOMER, customer);
 
     Schema nation = new Schema()
-        .addColumn("n_nationkey", DataType.INT) // 0
-        .addColumn("n_name", DataType.STRING) // 1
-        .addColumn("n_regionkey", DataType.INT) // 2
-        .addColumn("n_comment", DataType.STRING); // 3
+        .addColumn("n_nationkey", Type.INT4) // 0
+        .addColumn("n_name", Type.TEXT) // 1
+        .addColumn("n_regionkey", Type.INT4) // 2
+        .addColumn("n_comment", Type.TEXT); // 3
     schemas.put(NATION, nation);
 
     Schema part = new Schema()
-        .addColumn("p_partkey", DataType.INT) // 0
-        .addColumn("p_name", DataType.STRING) // 1
-        .addColumn("p_mfgr", DataType.STRING) // 2
-        .addColumn("p_brand", DataType.STRING) // 3
-        .addColumn("p_type", DataType.STRING) // 4
-        .addColumn("p_size", DataType.INT) // 5
-        .addColumn("p_container", DataType.STRING) // 6
-        .addColumn("p_retailprice", DataType.FLOAT) // 7
-        .addColumn("p_comment", DataType.STRING); // 8
+        .addColumn("p_partkey", Type.INT4) // 0
+        .addColumn("p_name", Type.TEXT) // 1
+        .addColumn("p_mfgr", Type.TEXT) // 2
+        .addColumn("p_brand", Type.TEXT) // 3
+        .addColumn("p_type", Type.TEXT) // 4
+        .addColumn("p_size", Type.INT4) // 5
+        .addColumn("p_container", Type.TEXT) // 6
+        .addColumn("p_retailprice", Type.FLOAT4) // 7
+        .addColumn("p_comment", Type.TEXT); // 8
     schemas.put(PART, part);
 
     Schema region = new Schema()
-        .addColumn("r_regionkey", DataType.INT) // 0
-        .addColumn("r_name", DataType.STRING) // 1
-        .addColumn("r_comment", DataType.STRING); // 2
+        .addColumn("r_regionkey", Type.INT4) // 0
+        .addColumn("r_name", Type.TEXT) // 1
+        .addColumn("r_comment", Type.TEXT); // 2
     schemas.put(REGION, region);
 
     Schema orders = new Schema()
-        .addColumn("o_orderkey", DataType.INT) // 0
-        .addColumn("o_custkey", DataType.INT) // 1
-        .addColumn("o_orderstatus", DataType.STRING) // 2
-        .addColumn("o_totalprice", DataType.FLOAT) // 3
+        .addColumn("o_orderkey", Type.INT4) // 0
+        .addColumn("o_custkey", Type.INT4) // 1
+        .addColumn("o_orderstatus", Type.TEXT) // 2
+        .addColumn("o_totalprice", Type.FLOAT4) // 3
             // TODO - This is temporal solution. o_orderdate is actually Date type.
-        .addColumn("o_orderdate", DataType.STRING) // 4
-        .addColumn("o_orderpriority", DataType.STRING) // 5
-        .addColumn("o_clerk", DataType.STRING) // 6
-        .addColumn("o_shippriority", DataType.INT) // 7
-        .addColumn("o_comment", DataType.STRING); // 8
+        .addColumn("o_orderdate", Type.TEXT) // 4
+        .addColumn("o_orderpriority", Type.TEXT) // 5
+        .addColumn("o_clerk", Type.TEXT) // 6
+        .addColumn("o_shippriority", Type.INT4) // 7
+        .addColumn("o_comment", Type.TEXT); // 8
     schemas.put(ORDERS, orders);
 
     Schema partsupp = new Schema()
-        .addColumn("ps_partkey", DataType.INT) // 0
-        .addColumn("ps_suppkey", DataType.INT) // 1
-        .addColumn("ps_availqty", DataType.INT) // 2
-        .addColumn("ps_supplycost", DataType.FLOAT) // 3
-        .addColumn("ps_comment", DataType.STRING); // 4
+        .addColumn("ps_partkey", Type.INT4) // 0
+        .addColumn("ps_suppkey", Type.INT4) // 1
+        .addColumn("ps_availqty", Type.INT4) // 2
+        .addColumn("ps_supplycost", Type.FLOAT4) // 3
+        .addColumn("ps_comment", Type.TEXT); // 4
     schemas.put(PARTSUPP, partsupp);
 
     Schema supplier = new Schema()
-        .addColumn("s_suppkey", DataType.INT) // 0
-        .addColumn("s_name", DataType.STRING) // 1
-        .addColumn("s_address", DataType.STRING) // 2
-        .addColumn("s_nationkey", DataType.INT) // 3
-        .addColumn("s_phone", DataType.STRING) // 4
-        .addColumn("s_acctbal", DataType.FLOAT) // 5
-        .addColumn("s_comment", DataType.STRING); // 6
+        .addColumn("s_suppkey", Type.INT4) // 0
+        .addColumn("s_name", Type.TEXT) // 1
+        .addColumn("s_address", Type.TEXT) // 2
+        .addColumn("s_nationkey", Type.INT4) // 3
+        .addColumn("s_phone", Type.TEXT) // 4
+        .addColumn("s_acctbal", Type.FLOAT4) // 5
+        .addColumn("s_comment", Type.TEXT); // 6
     schemas.put(SUPPLIER, supplier);
   }
 
   public void loadOutSchema() {
     Schema q2 = new Schema()
-        .addColumn("s_acctbal", DataType.FLOAT)
-        .addColumn("s_name", DataType.STRING)
-        .addColumn("n_name", DataType.STRING)
-        .addColumn("p_partkey", DataType.INT)
-        .addColumn("p_mfgr", DataType.STRING)
-        .addColumn("s_address", DataType.STRING)
-        .addColumn("s_phone", DataType.STRING)
-        .addColumn("s_comment", DataType.STRING);
+        .addColumn("s_acctbal", Type.FLOAT4)
+        .addColumn("s_name", Type.TEXT)
+        .addColumn("n_name", Type.TEXT)
+        .addColumn("p_partkey", Type.INT4)
+        .addColumn("p_mfgr", Type.TEXT)
+        .addColumn("s_address", Type.TEXT)
+        .addColumn("s_phone", Type.TEXT)
+        .addColumn("s_comment", Type.TEXT);
     outSchemas.put("q2", q2);
   }
 
@@ -164,7 +164,7 @@ public class TPCH extends BenchmarkSet {
   }
 
   private void loadTable(String tableName) throws ServiceException {
-    TableMeta meta = TCatUtil.newTableMeta(getSchema(tableName), StoreType.CSV);
+    TableMeta meta = CatalogUtil.newTableMeta(getSchema(tableName), StoreType.CSV);
     meta.putOption(CSVFile.DELIMITER, "|");
     tajo.createTable(tableName, new Path(dataDir, tableName), meta);
   }

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-backend/src/main/java/tajo/client/TajoClient.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/tajo/client/TajoClient.java b/tajo-core/tajo-core-backend/src/main/java/tajo/client/TajoClient.java
index 32eed96..a022541 100644
--- a/tajo-core/tajo-core-backend/src/main/java/tajo/client/TajoClient.java
+++ b/tajo-core/tajo-core-backend/src/main/java/tajo/client/TajoClient.java
@@ -25,7 +25,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.net.NetUtils;
 import tajo.QueryId;
 import tajo.TajoProtos.QueryState;
-import tajo.catalog.TCatUtil;
+import tajo.catalog.CatalogUtil;
 import tajo.catalog.TableDesc;
 import tajo.catalog.TableMeta;
 import tajo.client.ClientProtocol.*;
@@ -187,7 +187,7 @@ public class TajoClient {
     GetQueryResultResponse response = service.getQueryResult(null,
         builder.build());
 
-    return TCatUtil.newTableDesc(response.getTableDesc());
+    return CatalogUtil.newTableDesc(response.getTableDesc());
   }
 
   public boolean updateQuery(String tql) throws ServiceException {
@@ -211,7 +211,7 @@ public class TajoClient {
     builder.setName(name);
     builder.setPath(path);
     TableResponse res = service.attachTable(null, builder.build());
-    return TCatUtil.newTableDesc(res.getTableDesc());
+    return CatalogUtil.newTableDesc(res.getTableDesc());
   }
 
   public TableDesc attachTable(String name, Path path)
@@ -232,7 +232,7 @@ public class TajoClient {
     builder.setPath(path.toString());
     builder.setMeta(meta.getProto());
     TableResponse res = service.createTable(null, builder.build());
-    return TCatUtil.newTableDesc(res.getTableDesc());
+    return CatalogUtil.newTableDesc(res.getTableDesc());
   }
 
   public boolean dropTable(String name) throws ServiceException {
@@ -262,7 +262,7 @@ public class TajoClient {
     if (res == null) {
       return null;
     } else {
-      return TCatUtil.newTableDesc(res.getTableDesc());
+      return CatalogUtil.newTableDesc(res.getTableDesc());
     }
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-backend/src/main/java/tajo/engine/eval/AggFuncCallEval.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/eval/AggFuncCallEval.java b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/eval/AggFuncCallEval.java
index 55237e7..805239e 100644
--- a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/eval/AggFuncCallEval.java
+++ b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/eval/AggFuncCallEval.java
@@ -23,7 +23,7 @@ import tajo.catalog.FunctionDesc;
 import tajo.catalog.Schema;
 import tajo.catalog.function.AggFunction;
 import tajo.catalog.function.FunctionContext;
-import tajo.catalog.proto.CatalogProtos.DataType;
+import tajo.common.TajoDataTypes.DataType;
 import tajo.datum.Datum;
 import tajo.engine.json.GsonCreator;
 import tajo.storage.Tuple;
@@ -79,7 +79,7 @@ public class AggFuncCallEval extends FuncEval implements Cloneable {
   }
 
   @Override
-  public DataType [] getValueType() {
+  public DataType[] getValueType() {
     if (firstPhase) {
       return instance.getPartialResultType();
     } else {

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-backend/src/main/java/tajo/engine/eval/BinaryEval.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/eval/BinaryEval.java b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/eval/BinaryEval.java
index 7468eec..4d900ca 100644
--- a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/eval/BinaryEval.java
+++ b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/eval/BinaryEval.java
@@ -22,8 +22,10 @@ import com.google.common.base.Objects;
 import com.google.common.base.Preconditions;
 import com.google.gson.Gson;
 import com.google.gson.annotations.Expose;
+import tajo.catalog.CatalogUtil;
 import tajo.catalog.Schema;
-import tajo.catalog.proto.CatalogProtos.DataType;
+import tajo.common.TajoDataTypes;
+import tajo.common.TajoDataTypes.DataType;
 import tajo.datum.Datum;
 import tajo.datum.DatumFactory;
 import tajo.engine.json.GsonCreator;
@@ -31,7 +33,7 @@ import tajo.engine.utils.SchemaUtil;
 import tajo.storage.Tuple;
 
 public class BinaryEval extends EvalNode implements Cloneable {
-	@Expose private DataType [] returnType = null;
+	@Expose private DataType[] returnType = null;
 
   private class BinaryEvalCtx implements EvalContext {
     EvalContext left;
@@ -56,7 +58,7 @@ public class BinaryEval extends EvalNode implements Cloneable {
 			type == Type.LEQ ||
 			type == Type.GEQ
 		) {
-			this.returnType = SchemaUtil.newNoNameSchema(DataType.BOOLEAN);
+			this.returnType = CatalogUtil.newDataTypesWithoutLen(TajoDataTypes.Type.BOOLEAN);
 		} else if (
 			type == Type.PLUS ||
 			type == Type.MINUS ||
@@ -83,47 +85,47 @@ public class BinaryEval extends EvalNode implements Cloneable {
   }
 
   private DataType determineType(DataType left, DataType right) {
-    switch (left) {
-      case INT: {
-        switch(right) {
-          case SHORT:
-          case INT: return DataType.INT;
-          case LONG: return DataType.LONG;
-          case FLOAT: return DataType.FLOAT;
-          case DOUBLE: return DataType.DOUBLE;
+    switch (left.getType()) {
+      case INT4: {
+        switch(right.getType()) {
+          case INT2:
+          case INT4: return CatalogUtil.newDataTypeWithoutLen(TajoDataTypes.Type.INT4);
+          case INT8: return CatalogUtil.newDataTypeWithoutLen(TajoDataTypes.Type.INT8);
+          case FLOAT4: return CatalogUtil.newDataTypeWithoutLen(TajoDataTypes.Type.FLOAT4);
+          case FLOAT8: return CatalogUtil.newDataTypeWithoutLen(TajoDataTypes.Type.FLOAT8);
           default: throw new InvalidEvalException();
         }
       }
 
-      case LONG: {
-        switch(right) {
-          case SHORT:
-          case INT:
-          case LONG: return DataType.LONG;
-          case FLOAT:
-          case DOUBLE: return DataType.DOUBLE;
+      case INT8: {
+        switch(right.getType()) {
+          case INT2:
+          case INT4:
+          case INT8: return CatalogUtil.newDataTypeWithoutLen(TajoDataTypes.Type.INT8);
+          case FLOAT4:
+          case FLOAT8: return CatalogUtil.newDataTypeWithoutLen(TajoDataTypes.Type.FLOAT8);
           default: throw new InvalidEvalException();
         }
       }
 
-      case FLOAT: {
-        switch(right) {
-          case SHORT:
-          case INT:
-          case LONG:
-          case FLOAT:
-          case DOUBLE: return DataType.DOUBLE;
+      case FLOAT4: {
+        switch(right.getType()) {
+          case INT2:
+          case INT4:
+          case INT8:
+          case FLOAT4:
+          case FLOAT8: return CatalogUtil.newDataTypeWithoutLen(TajoDataTypes.Type.FLOAT8);
           default: throw new InvalidEvalException();
         }
       }
 
-      case DOUBLE: {
-        switch(right) {
-          case SHORT:
-          case INT:
-          case LONG:
-          case FLOAT:
-          case DOUBLE: return DataType.DOUBLE;
+      case FLOAT8: {
+        switch(right.getType()) {
+          case INT2:
+          case INT4:
+          case INT8:
+          case FLOAT4:
+          case FLOAT8: return CatalogUtil.newDataTypeWithoutLen(TajoDataTypes.Type.FLOAT8);
           default: throw new InvalidEvalException();
         }
       }


[06/11] TAJO-57: Recognize Parser and Catalog Standard SQL data types. (hyunsik)

Posted by hy...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-backend/src/test/java/tajo/engine/eval/TestEvalTree.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/tajo/engine/eval/TestEvalTree.java b/tajo-core/tajo-core-backend/src/test/java/tajo/engine/eval/TestEvalTree.java
index 6e811b6..e9a3bff 100644
--- a/tajo-core/tajo-core-backend/src/test/java/tajo/engine/eval/TestEvalTree.java
+++ b/tajo-core/tajo-core-backend/src/test/java/tajo/engine/eval/TestEvalTree.java
@@ -25,9 +25,9 @@ import org.junit.Test;
 import tajo.TajoTestingCluster;
 import tajo.catalog.*;
 import tajo.catalog.function.GeneralFunction;
-import tajo.catalog.proto.CatalogProtos.DataType;
 import tajo.catalog.proto.CatalogProtos.FunctionType;
 import tajo.catalog.proto.CatalogProtos.StoreType;
+import tajo.common.TajoDataTypes.DataType;
 import tajo.datum.Datum;
 import tajo.datum.DatumFactory;
 import tajo.engine.eval.EvalNode.Type;
@@ -39,6 +39,7 @@ import tajo.storage.Tuple;
 import tajo.storage.VTuple;
 
 import static org.junit.Assert.*;
+import static tajo.common.TajoDataTypes.Type.*;
 
 public class TestEvalTree {
   private static TajoTestingCluster util;
@@ -56,36 +57,36 @@ public class TestEvalTree {
     }
 
     Schema schema = new Schema();
-    schema.addColumn("name", DataType.STRING);
-    schema.addColumn("score", DataType.INT);
-    schema.addColumn("age", DataType.INT);
+    schema.addColumn("name", TEXT);
+    schema.addColumn("score", INT4);
+    schema.addColumn("age", INT4);
 
-    TableMeta meta = TCatUtil.newTableMeta(schema, StoreType.CSV);
+    TableMeta meta = CatalogUtil.newTableMeta(schema, StoreType.CSV);
     TableDesc desc = new TableDescImpl("people", meta, new Path("file:///"));
     cat.addTable(desc);
 
     FunctionDesc funcMeta = new FunctionDesc("sum", TestSum.class, FunctionType.GENERAL,
-        new DataType [] {DataType.INT},
-        new DataType [] {DataType.INT, DataType.INT});
+        CatalogUtil.newDataTypesWithoutLen(INT4),
+        CatalogUtil.newDataTypesWithoutLen(INT4, INT4));
     cat.registerFunction(funcMeta);
 
     analyzer = new QueryAnalyzer(cat);
     
     tuples[0] = new VTuple(3);
     tuples[0].put(new Datum[] {
-        DatumFactory.createString("aabc"),
-        DatumFactory.createInt(100), 
-        DatumFactory.createInt(10)});
+        DatumFactory.createText("aabc"),
+        DatumFactory.createInt4(100),
+        DatumFactory.createInt4(10)});
     tuples[1] = new VTuple(3);
     tuples[1].put(new Datum[] {
-        DatumFactory.createString("aaba"),
-        DatumFactory.createInt(200), 
-        DatumFactory.createInt(20)});
+        DatumFactory.createText("aaba"),
+        DatumFactory.createInt4(200),
+        DatumFactory.createInt4(20)});
     tuples[2] = new VTuple(3);
     tuples[2].put(new Datum[] {
-        DatumFactory.createString("kabc"),
-        DatumFactory.createInt(300), 
-        DatumFactory.createInt(30)});
+        DatumFactory.createText("kabc"),
+        DatumFactory.createInt4(300),
+        DatumFactory.createInt4(30)});
   }
 
   @AfterClass
@@ -98,15 +99,15 @@ public class TestEvalTree {
     private Integer y;
 
     public TestSum() {
-      super(new Column[] { new Column("arg1", DataType.INT),
-          new Column("arg2", DataType.INT) });
+      super(new Column[] { new Column("arg1", INT4),
+          new Column("arg2", INT4) });
     }
 
     @Override
     public Datum eval(Tuple params) {
-      x =  params.get(0).asInt();
-      y =  params.get(1).asInt();
-      return DatumFactory.createInt(x + y);
+      x =  params.get(0).asInt4();
+      y =  params.get(1).asInt4();
+      return DatumFactory.createInt4(x + y);
     }
     
     public String toJSON() {
@@ -128,9 +129,9 @@ public class TestEvalTree {
     Tuple tuple = new VTuple(3);
     tuple.put(
         new Datum[] {
-          DatumFactory.createString("hyunsik"),
-          DatumFactory.createInt(500),
-          DatumFactory.createInt(30)});
+          DatumFactory.createText("hyunsik"),
+          DatumFactory.createInt4(500),
+          DatumFactory.createInt4(30)});
 
     QueryBlock block;
     EvalNode expr;
@@ -147,21 +148,21 @@ public class TestEvalTree {
     expr = block.getWhereCondition();
     evalCtx = expr.newContext();
     expr.eval(evalCtx, peopleSchema, tuple);
-    assertEquals(15000, expr.terminate(evalCtx).asInt());
+    assertEquals(15000, expr.terminate(evalCtx).asInt4());
     assertCloneEqual(expr);
 
     block = (QueryBlock) analyzer.parse(QUERIES[2]).getParseTree();
     expr = block.getWhereCondition();
     evalCtx = expr.newContext();
     expr.eval(evalCtx, peopleSchema, tuple);
-    assertEquals(15050, expr.terminate(evalCtx).asInt());
+    assertEquals(15050, expr.terminate(evalCtx).asInt4());
     assertCloneEqual(expr);
     
     block = (QueryBlock) analyzer.parse(QUERIES[2]).getParseTree();
     expr = block.getWhereCondition();
     evalCtx = expr.newContext();
     expr.eval(evalCtx, peopleSchema, tuple);
-    assertEquals(15050, expr.terminate(evalCtx).asInt());
+    assertEquals(15050, expr.terminate(evalCtx).asInt4());
     assertCloneEqual(expr);
     
     // Aggregation function test
@@ -173,41 +174,41 @@ public class TestEvalTree {
     Tuple [] tuples = new Tuple[tuplenum];
     for (int i=0; i < tuplenum; i++) {
       tuples[i] = new VTuple(3);
-      tuples[i].put(0, DatumFactory.createString("hyunsik"));
-      tuples[i].put(1, DatumFactory.createInt(i+1));
-      tuples[i].put(2, DatumFactory.createInt(30));
+      tuples[i].put(0, DatumFactory.createText("hyunsik"));
+      tuples[i].put(1, DatumFactory.createInt4(i + 1));
+      tuples[i].put(2, DatumFactory.createInt4(30));
     }
     
     int sum = 0;
     for (int i=0; i < tuplenum; i++) {
       expr.eval(evalCtx, peopleSchema, tuples[i]);
       sum = sum + (i+1);
-      assertEquals(sum, expr.terminate(evalCtx).asInt());
+      assertEquals(sum, expr.terminate(evalCtx).asInt4());
     }
   }
   
   
   @Test
   public void testTupleEval() throws CloneNotSupportedException {
-    ConstEval e1 = new ConstEval(DatumFactory.createInt(1));
+    ConstEval e1 = new ConstEval(DatumFactory.createInt4(1));
     assertCloneEqual(e1);
-    FieldEval e2 = new FieldEval("table1.score", DataType.INT); // it indicates
+    FieldEval e2 = new FieldEval("table1.score", CatalogUtil.newDataTypeWithoutLen(INT4)); // it indicates
     assertCloneEqual(e2);
 
     Schema schema1 = new Schema();
-    schema1.addColumn("table1.id", DataType.INT);
-    schema1.addColumn("table1.score", DataType.INT);
+    schema1.addColumn("table1.id", INT4);
+    schema1.addColumn("table1.score", INT4);
     
     BinaryEval expr = new BinaryEval(Type.PLUS, e1, e2);
     EvalContext evalCtx = expr.newContext();
     assertCloneEqual(expr);
     VTuple tuple = new VTuple(2);
-    tuple.put(0, DatumFactory.createInt(1)); // put 0th field
-    tuple.put(1, DatumFactory.createInt(99)); // put 0th field
+    tuple.put(0, DatumFactory.createInt4(1)); // put 0th field
+    tuple.put(1, DatumFactory.createInt4(99)); // put 0th field
 
     // the result of evaluation must be 100.
     expr.eval(evalCtx, schema1, tuple);
-    assertEquals(expr.terminate(evalCtx).asInt(), 100);
+    assertEquals(expr.terminate(evalCtx).asInt4(), 100);
   }
 
   public static class MockTrueEval extends EvalNode {
@@ -238,7 +239,7 @@ public class TestEvalTree {
 
     @Override
     public DataType [] getValueType() {
-      return new DataType [] {DataType.BOOLEAN};
+      return CatalogUtil.newDataTypesWithoutLen(BOOLEAN);
     }
 
   }
@@ -271,7 +272,7 @@ public class TestEvalTree {
 
     @Override
     public DataType [] getValueType() {
-      return new DataType [] {DataType.BOOLEAN};
+      return CatalogUtil.newDataTypesWithoutLen(BOOLEAN);
     }
   }
 
@@ -334,8 +335,8 @@ public class TestEvalTree {
     BinaryEval expr;
 
     // Constant
-    e1 = new ConstEval(DatumFactory.createInt(9));
-    e2 = new ConstEval(DatumFactory.createInt(34));
+    e1 = new ConstEval(DatumFactory.createInt4(9));
+    e2 = new ConstEval(DatumFactory.createInt4(34));
     expr = new BinaryEval(Type.LTH, e1, e2);
     EvalContext evalCtx = expr.newContext();
     expr.eval(evalCtx, null, null);
@@ -414,39 +415,39 @@ public class TestEvalTree {
     ConstEval e2;
 
     // PLUS
-    e1 = new ConstEval(DatumFactory.createInt(9));
-    e2 = new ConstEval(DatumFactory.createInt(34));
+    e1 = new ConstEval(DatumFactory.createInt4(9));
+    e2 = new ConstEval(DatumFactory.createInt4(34));
     BinaryEval expr = new BinaryEval(Type.PLUS, e1, e2);
     EvalContext evalCtx = expr.newContext();
     expr.eval(evalCtx, null, null);
-    assertEquals(expr.terminate(evalCtx).asInt(), 43);
+    assertEquals(expr.terminate(evalCtx).asInt4(), 43);
     assertCloneEqual(expr);
     
     // MINUS
-    e1 = new ConstEval(DatumFactory.createInt(5));
-    e2 = new ConstEval(DatumFactory.createInt(2));
+    e1 = new ConstEval(DatumFactory.createInt4(5));
+    e2 = new ConstEval(DatumFactory.createInt4(2));
     expr = new BinaryEval(Type.MINUS, e1, e2);
     evalCtx = expr.newContext();
     expr.eval(evalCtx, null, null);
-    assertEquals(expr.terminate(evalCtx).asInt(), 3);
+    assertEquals(expr.terminate(evalCtx).asInt4(), 3);
     assertCloneEqual(expr);
     
     // MULTIPLY
-    e1 = new ConstEval(DatumFactory.createInt(5));
-    e2 = new ConstEval(DatumFactory.createInt(2));
+    e1 = new ConstEval(DatumFactory.createInt4(5));
+    e2 = new ConstEval(DatumFactory.createInt4(2));
     expr = new BinaryEval(Type.MULTIPLY, e1, e2);
     evalCtx = expr.newContext();
     expr.eval(evalCtx, null, null);
-    assertEquals(expr.terminate(evalCtx).asInt(), 10);
+    assertEquals(expr.terminate(evalCtx).asInt4(), 10);
     assertCloneEqual(expr);
     
     // DIVIDE
-    e1 = new ConstEval(DatumFactory.createInt(10));
-    e2 = new ConstEval(DatumFactory.createInt(5));
+    e1 = new ConstEval(DatumFactory.createInt4(10));
+    e2 = new ConstEval(DatumFactory.createInt4(5));
     expr = new BinaryEval(Type.DIVIDE, e1, e2);
     evalCtx = expr.newContext();
     expr.eval(evalCtx, null, null);
-    assertEquals(expr.terminate(evalCtx).asInt(), 2);
+    assertEquals(expr.terminate(evalCtx).asInt4(), 2);
     assertCloneEqual(expr);
   }
 
@@ -456,21 +457,21 @@ public class TestEvalTree {
     ConstEval e2;
 
     // PLUS
-    e1 = new ConstEval(DatumFactory.createInt(9));
-    e2 = new ConstEval(DatumFactory.createInt(34));
+    e1 = new ConstEval(DatumFactory.createInt4(9));
+    e2 = new ConstEval(DatumFactory.createInt4(34));
     BinaryEval expr = new BinaryEval(Type.PLUS, e1, e2);
-    assertEquals(DataType.INT, expr.getValueType()[0]);
+    assertEquals(CatalogUtil.newDataTypeWithoutLen(INT4), expr.getValueType()[0]);
 
     expr = new BinaryEval(Type.LTH, e1, e2);
     EvalContext evalCtx = expr.newContext();
     expr.eval(evalCtx, null, null);
     assertTrue(expr.terminate(evalCtx).asBool());
-    assertEquals(DataType.BOOLEAN, expr.getValueType()[0]);
+    assertEquals(CatalogUtil.newDataTypeWithoutLen(BOOLEAN), expr.getValueType()[0]);
 
-    e1 = new ConstEval(DatumFactory.createDouble(9.3));
-    e2 = new ConstEval(DatumFactory.createDouble(34.2));
+    e1 = new ConstEval(DatumFactory.createFloat8(9.3));
+    e2 = new ConstEval(DatumFactory.createFloat8(34.2));
     expr = new BinaryEval(Type.PLUS, e1, e2);
-    assertEquals(DataType.DOUBLE, expr.getValueType()[0]);
+    assertEquals(CatalogUtil.newDataTypeWithoutLen(FLOAT8), expr.getValueType()[0]);
   }
   
   @Test
@@ -479,26 +480,26 @@ public class TestEvalTree {
     ConstEval e2;
 
     // PLUS
-    e1 = new ConstEval(DatumFactory.createInt(34));
-    e2 = new ConstEval(DatumFactory.createInt(34));
+    e1 = new ConstEval(DatumFactory.createInt4(34));
+    e2 = new ConstEval(DatumFactory.createInt4(34));
     assertEquals(e1, e2);
     
     BinaryEval plus1 = new BinaryEval(Type.PLUS, e1, e2);
     BinaryEval plus2 = new BinaryEval(Type.PLUS, e2, e1);
     assertEquals(plus1, plus2);
     
-    ConstEval e3 = new ConstEval(DatumFactory.createInt(29));
+    ConstEval e3 = new ConstEval(DatumFactory.createInt4(29));
     BinaryEval plus3 = new BinaryEval(Type.PLUS, e1, e3);
     assertFalse(plus1.equals(plus3));
     
     // LTH
-    ConstEval e4 = new ConstEval(DatumFactory.createInt(9));
-    ConstEval e5 = new ConstEval(DatumFactory.createInt(34));
+    ConstEval e4 = new ConstEval(DatumFactory.createInt4(9));
+    ConstEval e5 = new ConstEval(DatumFactory.createInt4(34));
     BinaryEval compExpr1 = new BinaryEval(Type.LTH, e4, e5);
     assertCloneEqual(compExpr1);
     
-    ConstEval e6 = new ConstEval(DatumFactory.createInt(9));
-    ConstEval e7 = new ConstEval(DatumFactory.createInt(34));
+    ConstEval e6 = new ConstEval(DatumFactory.createInt4(9));
+    ConstEval e7 = new ConstEval(DatumFactory.createInt4(34));
     BinaryEval compExpr2 = new BinaryEval(Type.LTH, e6, e7);
     assertCloneEqual(compExpr2);
     
@@ -511,8 +512,8 @@ public class TestEvalTree {
     ConstEval e2;
 
     // 29 > (34 + 5) + (5 + 34)
-    e1 = new ConstEval(DatumFactory.createInt(34));
-    e2 = new ConstEval(DatumFactory.createInt(5));
+    e1 = new ConstEval(DatumFactory.createInt4(34));
+    e2 = new ConstEval(DatumFactory.createInt4(5));
     assertCloneEqual(e1); 
     
     BinaryEval plus1 = new BinaryEval(Type.PLUS, e1, e2);
@@ -522,7 +523,7 @@ public class TestEvalTree {
     BinaryEval plus3 = new BinaryEval(Type.PLUS, plus2, plus1);
     assertCloneEqual(plus3);
     
-    ConstEval e3 = new ConstEval(DatumFactory.createInt(29));
+    ConstEval e3 = new ConstEval(DatumFactory.createInt4(29));
     BinaryEval gth = new BinaryEval(Type.GTH, e3, plus3);
     assertCloneEqual(gth);
     
@@ -558,8 +559,8 @@ public class TestEvalTree {
     EvalNode expr;
 
     // Constant
-    e1 = new ConstEval(DatumFactory.createInt(9));
-    e2 = new ConstEval(DatumFactory.createInt(34));
+    e1 = new ConstEval(DatumFactory.createInt4(9));
+    e2 = new ConstEval(DatumFactory.createInt4(34));
     expr = new BinaryEval(Type.LTH, e1, e2);
     EvalContext evalCtx = expr.newContext();
     expr.eval(evalCtx, null, null);

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-backend/src/test/java/tajo/engine/eval/TestEvalTreeUtil.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/tajo/engine/eval/TestEvalTreeUtil.java b/tajo-core/tajo-core-backend/src/test/java/tajo/engine/eval/TestEvalTreeUtil.java
index 0bb9b32..2eae279 100644
--- a/tajo-core/tajo-core-backend/src/test/java/tajo/engine/eval/TestEvalTreeUtil.java
+++ b/tajo-core/tajo-core-backend/src/test/java/tajo/engine/eval/TestEvalTreeUtil.java
@@ -25,9 +25,9 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 import tajo.TajoTestingCluster;
 import tajo.catalog.*;
-import tajo.catalog.proto.CatalogProtos.DataType;
 import tajo.catalog.proto.CatalogProtos.FunctionType;
 import tajo.catalog.proto.CatalogProtos.StoreType;
+import tajo.common.TajoDataTypes;
 import tajo.datum.DatumFactory;
 import tajo.engine.eval.EvalNode.Type;
 import tajo.engine.eval.TestEvalTree.TestSum;
@@ -65,18 +65,18 @@ public class TestEvalTreeUtil {
     }
 
     Schema schema = new Schema();
-    schema.addColumn("name", DataType.STRING);
-    schema.addColumn("score", DataType.INT);
-    schema.addColumn("age", DataType.INT);
+    schema.addColumn("name", TajoDataTypes.Type.TEXT);
+    schema.addColumn("score", TajoDataTypes.Type.INT4);
+    schema.addColumn("age", TajoDataTypes.Type.INT4);
 
-    TableMeta meta = TCatUtil.newTableMeta(schema, StoreType.CSV);
+    TableMeta meta = CatalogUtil.newTableMeta(schema, StoreType.CSV);
     TableDesc desc = new TableDescImpl("people", meta, new Path("file:///"));
     catalog.addTable(desc);
 
     FunctionDesc funcMeta = new FunctionDesc("sum", TestSum.class,
         FunctionType.GENERAL,
-        new DataType [] {DataType.INT},
-        new DataType [] {DataType.INT,DataType.INT});
+        CatalogUtil.newDataTypesWithoutLen(TajoDataTypes.Type.INT4),
+        CatalogUtil.newDataTypesWithoutLen(TajoDataTypes.Type.INT4, TajoDataTypes.Type.INT4));
     catalog.registerFunction(funcMeta);
 
     analyzer = new QueryAnalyzer(catalog);
@@ -105,38 +105,38 @@ public class TestEvalTreeUtil {
     EvalTreeUtil.changeColumnRef(copy, "people.score", "newscore");
     Set<Column> set = EvalTreeUtil.findDistinctRefColumns(copy);
     assertEquals(1, set.size());
-    assertTrue(set.contains(new Column("newscore", DataType.INT)));
+    assertTrue(set.contains(new Column("newscore", TajoDataTypes.Type.INT4)));
     
     copy = (EvalNode)expr2.clone();
     EvalTreeUtil.changeColumnRef(copy, "people.age", "sum_age");
     set = EvalTreeUtil.findDistinctRefColumns(copy);
     assertEquals(2, set.size());
-    assertTrue(set.contains(new Column("people.score", DataType.INT)));
-    assertTrue(set.contains(new Column("sum_age", DataType.INT)));
+    assertTrue(set.contains(new Column("people.score", TajoDataTypes.Type.INT4)));
+    assertTrue(set.contains(new Column("sum_age", TajoDataTypes.Type.INT4)));
     
     copy = (EvalNode)expr3.clone();
     EvalTreeUtil.changeColumnRef(copy, "people.age", "sum_age");
     set = EvalTreeUtil.findDistinctRefColumns(copy);
     assertEquals(2, set.size());
-    assertTrue(set.contains(new Column("people.score", DataType.INT)));
-    assertTrue(set.contains(new Column("sum_age", DataType.INT)));
+    assertTrue(set.contains(new Column("people.score", TajoDataTypes.Type.INT4)));
+    assertTrue(set.contains(new Column("sum_age", TajoDataTypes.Type.INT4)));
   }
 
   @Test
   public final void testFindAllRefColumns() {    
     Set<Column> set = EvalTreeUtil.findDistinctRefColumns(expr1);
     assertEquals(1, set.size());
-    assertTrue(set.contains(new Column("people.score", DataType.INT)));
+    assertTrue(set.contains(new Column("people.score", TajoDataTypes.Type.INT4)));
     
     set = EvalTreeUtil.findDistinctRefColumns(expr2);
     assertEquals(2, set.size());
-    assertTrue(set.contains(new Column("people.score", DataType.INT)));
-    assertTrue(set.contains(new Column("people.age", DataType.INT)));
+    assertTrue(set.contains(new Column("people.score", TajoDataTypes.Type.INT4)));
+    assertTrue(set.contains(new Column("people.age", TajoDataTypes.Type.INT4)));
     
     set = EvalTreeUtil.findDistinctRefColumns(expr3);
     assertEquals(2, set.size());
-    assertTrue(set.contains(new Column("people.score", DataType.INT)));
-    assertTrue(set.contains(new Column("people.age", DataType.INT)));
+    assertTrue(set.contains(new Column("people.score", TajoDataTypes.Type.INT4)));
+    assertTrue(set.contains(new Column("people.age", TajoDataTypes.Type.INT4)));
   }
   
   public static final String [] QUERIES = {
@@ -157,9 +157,9 @@ public class TestEvalTreeUtil {
     Column col1 = schema.getColumn(0);
     Column col2 = schema.getColumn(1);
     assertEquals("plus", col1.getColumnName());
-    assertEquals(DataType.INT, col1.getDataType());
+    assertEquals(TajoDataTypes.Type.INT4, col1.getDataType().getType());
     assertEquals("mul", col2.getColumnName());
-    assertEquals(DataType.DOUBLE, col2.getDataType());
+    assertEquals(TajoDataTypes.Type.FLOAT8, col2.getDataType().getType());
   }
   
   @Test
@@ -167,20 +167,20 @@ public class TestEvalTreeUtil {
     QueryBlock block = (QueryBlock) analyzer.parse(QUERIES[1]).getParseTree();
     Target [] targets = block.getTargetList();
     
-    Column col1 = new Column("people.score", DataType.INT);
+    Column col1 = new Column("people.score", TajoDataTypes.Type.INT4);
     Collection<EvalNode> exprs =
         EvalTreeUtil.getContainExpr(targets[0].getEvalTree(), col1);
     EvalNode node = exprs.iterator().next();
     assertEquals(Type.LTH, node.getType());
     assertEquals(Type.PLUS, node.getLeftExpr().getType());
-    assertEquals(new ConstEval(DatumFactory.createInt(4)), node.getRightExpr());
+    assertEquals(new ConstEval(DatumFactory.createInt4(4)), node.getRightExpr());
     
-    Column col2 = new Column("people.age", DataType.INT);
+    Column col2 = new Column("people.age", TajoDataTypes.Type.INT4);
     exprs = EvalTreeUtil.getContainExpr(targets[1].getEvalTree(), col2);
     node = exprs.iterator().next();
     assertEquals(Type.GTH, node.getType());
     assertEquals("people.age", node.getLeftExpr().getName());
-    assertEquals(new ConstEval(DatumFactory.createInt(5)), node.getRightExpr());
+    assertEquals(new ConstEval(DatumFactory.createInt4(5)), node.getRightExpr());
   }
   
   @Test
@@ -190,7 +190,7 @@ public class TestEvalTreeUtil {
     EvalNode node = block.getWhereCondition();
     EvalNode [] cnf = EvalTreeUtil.getConjNormalForm(node);
     
-    Column col1 = new Column("people.score", DataType.INT);
+    Column col1 = new Column("people.score", TajoDataTypes.Type.INT4);
     
     assertEquals(2, cnf.length);
     EvalNode first = cnf[0];
@@ -201,14 +201,14 @@ public class TestEvalTreeUtil {
     assertEquals(Type.LTH, first.getType());
     EvalContext firstRCtx = first.getRightExpr().newContext();
     first.getRightExpr().eval(firstRCtx, null,  null);
-    assertEquals(10, first.getRightExpr().terminate(firstRCtx).asInt());
+    assertEquals(10, first.getRightExpr().terminate(firstRCtx).asInt4());
     
     field = (FieldEval) second.getRightExpr();
     assertEquals(col1, field.getColumnRef());
     assertEquals(Type.LTH, second.getType());
     EvalContext secondLCtx = second.getLeftExpr().newContext();
     second.getLeftExpr().eval(secondLCtx, null,  null);
-    assertEquals(4, second.getLeftExpr().terminate(secondLCtx).asInt());
+    assertEquals(4, second.getLeftExpr().terminate(secondLCtx).asInt4());
   }
   
   @Test
@@ -235,16 +235,16 @@ public class TestEvalTreeUtil {
     EvalContext nodeCtx = node.newContext();
     assertEquals(Type.CONST, node.getType());
     node.eval(nodeCtx, null, null);
-    assertEquals(7, node.terminate(nodeCtx).asInt());
+    assertEquals(7, node.terminate(nodeCtx).asInt4());
     node = AlgebraicUtil.simplify(targets[1].getEvalTree());
     assertEquals(Type.CONST, node.getType());
     nodeCtx = node.newContext();
     node.eval(nodeCtx, null, null);
-    assertTrue(7.0d == node.terminate(nodeCtx).asDouble());
+    assertTrue(7.0d == node.terminate(nodeCtx).asFloat8());
 
     block = (QueryBlock) analyzer.parse(QUERIES[1]).getParseTree();
     targets = block.getTargetList();
-    Column col1 = new Column("people.score", DataType.INT);
+    Column col1 = new Column("people.score", TajoDataTypes.Type.INT4);
     Collection<EvalNode> exprs =
         EvalTreeUtil.getContainExpr(targets[0].getEvalTree(), col1);
     node = exprs.iterator().next();
@@ -268,7 +268,7 @@ public class TestEvalTreeUtil {
     EvalNode node = block.getWhereCondition();
     assertEquals(true, AlgebraicUtil.containSingleVar(node));
     
-    Column col1 = new Column("people.score", DataType.INT);
+    Column col1 = new Column("people.score", TajoDataTypes.Type.INT4);
     block = (QueryBlock) analyzer.parse(QUERIES[3]).getParseTree();
     node = block.getWhereCondition();    
     // we expect that score < 3
@@ -278,7 +278,7 @@ public class TestEvalTreeUtil {
     assertEquals(col1, field.getColumnRef());
     EvalContext evalCtx = transposed.getRightExpr().newContext();
     transposed.getRightExpr().eval(evalCtx, null, null);
-    assertEquals(1, transposed.getRightExpr().terminate(evalCtx).asInt());
+    assertEquals(1, transposed.getRightExpr().terminate(evalCtx).asInt4());
 
     block = (QueryBlock) analyzer.parse(QUERIES[4]).getParseTree();
     node = block.getWhereCondition();    
@@ -289,7 +289,7 @@ public class TestEvalTreeUtil {
     assertEquals(col1, field.getColumnRef());
     evalCtx = transposed.getRightExpr().newContext();
     transposed.getRightExpr().eval(evalCtx, null, null);
-    assertEquals(2, transposed.getRightExpr().terminate(evalCtx).asInt());
+    assertEquals(2, transposed.getRightExpr().terminate(evalCtx).asInt4());
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-backend/src/test/java/tajo/engine/function/TestAggFunction.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/tajo/engine/function/TestAggFunction.java b/tajo-core/tajo-core-backend/src/test/java/tajo/engine/function/TestAggFunction.java
index 35c8859..3544bb9 100644
--- a/tajo-core/tajo-core-backend/src/test/java/tajo/engine/function/TestAggFunction.java
+++ b/tajo-core/tajo-core-backend/src/test/java/tajo/engine/function/TestAggFunction.java
@@ -36,7 +36,7 @@ public class TestAggFunction {
 
     for (int i = 1; i <= 5; i++) {
       tuples[i-1] = new VTuple(1);
-      tuples[i-1].put(0, DatumFactory.createInt(i));
+      tuples[i-1].put(0, DatumFactory.createInt4(i));
     }
 
     AvgLong avg = new AvgLong();
@@ -45,7 +45,7 @@ public class TestAggFunction {
       avg.eval(ctx, tuples[i-1]);
     }
 
-    assertTrue(15 / 5 == avg.terminate(ctx).asDouble());
+    assertTrue(15 / 5 == avg.terminate(ctx).asFloat8());
 
 
     Tuple [] tuples2 = new Tuple[10];
@@ -53,13 +53,13 @@ public class TestAggFunction {
     FunctionContext ctx2 = avg.newContext();
     for (int i = 1; i <= 10; i++) {
       tuples2[i-1] = new VTuple(1);
-      tuples2[i-1].put(0, DatumFactory.createInt(i));
+      tuples2[i-1].put(0, DatumFactory.createInt4(i));
       avg.eval(ctx2, tuples2[i-1]);
     }
-    assertTrue((double)55 / 10 == avg.terminate(ctx2).asDouble());
+    assertTrue((double)55 / 10 == avg.terminate(ctx2).asFloat8());
 
 
     avg.merge(ctx, new VTuple(new Datum[] {avg.getPartialResult(ctx2)}));
-    assertTrue((double)(15 + 55) / (5 + 10) == avg.terminate(ctx).asDouble());
+    assertTrue((double)(15 + 55) / (5 + 10) == avg.terminate(ctx).asFloat8());
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-backend/src/test/java/tajo/engine/function/TestGeneralFunction.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/tajo/engine/function/TestGeneralFunction.java b/tajo-core/tajo-core-backend/src/test/java/tajo/engine/function/TestGeneralFunction.java
index 5a85eeb..889d57b 100644
--- a/tajo-core/tajo-core-backend/src/test/java/tajo/engine/function/TestGeneralFunction.java
+++ b/tajo-core/tajo-core-backend/src/test/java/tajo/engine/function/TestGeneralFunction.java
@@ -20,8 +20,8 @@ package tajo.engine.function;
 
 import org.junit.Test;
 import tajo.datum.Datum;
-import tajo.datum.LongDatum;
-import tajo.datum.StringDatum;
+import tajo.datum.Int8Datum;
+import tajo.datum.TextDatum;
 import tajo.engine.function.builtin.Date;
 import tajo.storage.Tuple;
 import tajo.storage.VTuple;
@@ -35,10 +35,10 @@ public class TestGeneralFunction {
   @Test
   public void testDate() {
     Date date = new Date();
-    Tuple tuple = new VTuple(new Datum[] {new StringDatum("25/12/2012 00:00:00")});
-    LongDatum unixtime = (LongDatum) date.eval(tuple);
+    Tuple tuple = new VTuple(new Datum[] {new TextDatum("25/12/2012 00:00:00")});
+    Int8Datum unixtime = (Int8Datum) date.eval(tuple);
     Calendar c = Calendar.getInstance();
-    c.setTimeInMillis(unixtime.asLong());
+    c.setTimeInMillis(unixtime.asInt8());
     assertEquals(2012, c.get(Calendar.YEAR));
     assertEquals(11, c.get(Calendar.MONTH));
     assertEquals(25, c.get(Calendar.DAY_OF_MONTH));

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-backend/src/test/java/tajo/engine/parser/TestNQLParser.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/tajo/engine/parser/TestNQLParser.java b/tajo-core/tajo-core-backend/src/test/java/tajo/engine/parser/TestNQLParser.java
index ae72ee4..29d815a 100644
--- a/tajo-core/tajo-core-backend/src/test/java/tajo/engine/parser/TestNQLParser.java
+++ b/tajo-core/tajo-core-backend/src/test/java/tajo/engine/parser/TestNQLParser.java
@@ -93,13 +93,13 @@ public class TestNQLParser {
     int cubeIdx = 0;
     Tree cube = groupby.getChild(grpIdx);
     assertEquals(NQLParser.FIELD_NAME, cube.getChild(cubeIdx++).getType());
-    assertEquals(NQLParser.FIELD_NAME, cube.getChild(cubeIdx++).getType());
+    assertEquals(NQLParser.FIELD_NAME, cube.getChild(cubeIdx).getType());
     grpIdx++;
     assertEquals(NQLParser.ROLLUP, groupby.getChild(grpIdx).getType());
     
     int rollupIdx = 0;
     Tree rollup = groupby.getChild(grpIdx);
-    assertEquals(NQLParser.FIELD_NAME, rollup.getChild(rollupIdx++).getType());
+    assertEquals(NQLParser.FIELD_NAME, rollup.getChild(rollupIdx).getType());
     
     idx++;
     assertEquals(NQLParser.HAVING, ast.getChild(idx).getType());
@@ -443,15 +443,64 @@ public class TestNQLParser {
   }
 
   static String[] schemaStmts = { 
-    "drop table abc",
-    "create table name (name string, age int)",
-    "create table name (name string, age int) using rcfile",
-    "create table name (name string, age int) using rcfile with ('rcfile.buffer'=4096)",
+    "drop table abc", // 0
+    "create table name (name text, age int)", // 1
+    "create table name (name text, age int) using rcfile", // 2
+    "create table name (name text, age int) using rcfile with ('rcfile.buffer'=4096)", // 3
     "create table name as select * from test", // 4
-    "create table name (name string, age int) as select * from test", // 5
-    "create table name (name string, age int) using rcfile as select * from test", // 6
-    "create table name (name string, age int) using rcfile with ('rcfile.buffer'= 4096) as select * from test", // 7
-    "create external table table1 (name string, age int, earn long, score float) using csv location '/tmp/data'", // 8
+    "create table name (name text, age int) as select * from test", // 5
+    "create table name (name text, age int) using rcfile as select * from test", // 6
+    "create table name (name text, age int) using rcfile with ('rcfile.buffer'= 4096) as select * from test", // 7
+    "create table widetable (" +
+        "col0 bit," +
+        "col0 bit(10)," +
+        "col0 bit varying," +
+        "col0 bit varying(10)," +
+        "col1 tinyint, " +
+        "col2 smallint, " +
+        "col3 integer, " +
+        "col4 bigint, " +
+        "col5 real, " +
+        "col5 float, " +
+        "col5 float(53), " +
+        "col6 double, " +
+        "col6 double precision, " +
+        "col7 numeric, " +
+        "col7 numeric(10), " +
+        "col7 numeric(10,2), " +
+        "col8 decimal," +
+        "col8 decimal(10)," +
+        "col8 decimal(10,2)," +
+        "col9 char," +
+        "col9 character," +
+        "col10 char(10)," +
+        "col10 character(10)," +
+        "col11 varchar," +
+        "col11 character varying," +
+        "col12 varchar(255)," +
+        "col11 character varying (255)," +
+        "col11 nchar," +
+        "col11 nchar(255)," +
+        "col11 national character," +
+        "col11 national character(255)," +
+        "col11 nvarchar," +
+        "col11 nvarchar(255)," +
+        "col11 national character varying," +
+        "col11 national character varying (255)," +
+        "col11 date," +
+        "col11 time," +
+        "col11 timetz," +
+        "col11 time with time zone," +
+        "col11 timestamptz," +
+        "col11 timestamp with time zone," +
+        "col11 binary," +
+        "col11 binary(10)," +
+        "col11 varbinary(10)," +
+        "col11 binary varying(10)," +
+        "col11 blob" +
+        ") as select * from test", // 8
+    "create table widetable (col1 float(10), col2 float) as select * from test", // 9
+    "create external table table1 (name text, age int, earn bigint, score float) using csv location '/tmp/data'", // 10
   };
 
   @Test
@@ -524,10 +573,31 @@ public class TestNQLParser {
     assertEquals(NQLParser.AS, ast.getChild(4).getType());
     assertEquals(NQLParser.SELECT, ast.getChild(4).getChild(0).getType());
   }
+
+  @Test
+  public void testCreateTableWithVariousDataType1() throws RecognitionException, TQLSyntaxError {
+    Tree ast = parseQuery(schemaStmts[8]);
+    assertEquals(ast.getType(), NQLParser.CREATE_TABLE);
+    assertEquals(NQLParser.ID, ast.getChild(0).getType());
+    assertEquals(NQLParser.TABLE_DEF, ast.getChild(1).getType());
+    assertEquals(NQLParser.AS, ast.getChild(2).getType());
+    assertEquals(NQLParser.SELECT, ast.getChild(2).getChild(0).getType());
+  }
+
+  @Test
+  public void testCreateTableWithVariousDataType2() throws RecognitionException, TQLSyntaxError {
+    Tree ast = parseQuery(schemaStmts[9]);
+    assertEquals(ast.getType(), NQLParser.CREATE_TABLE);
+    assertEquals(NQLParser.ID, ast.getChild(0).getType());
+    assertEquals(NQLParser.TABLE_DEF, ast.getChild(1).getType());
+    assertEquals("10", ast.getChild(1).getChild(0).getChild(1).getChild(0).getText());
+    assertEquals(NQLParser.AS, ast.getChild(2).getType());
+    assertEquals(NQLParser.SELECT, ast.getChild(2).getChild(0).getType());
+  }
   
   @Test
   public void testCreateTableLocation1() throws RecognitionException, TQLSyntaxError {
-    Tree ast = parseQuery(schemaStmts[8]);
+    Tree ast = parseQuery(schemaStmts[10]);
     assertEquals(ast.getType(), NQLParser.CREATE_TABLE);
     assertEquals(NQLParser.ID, ast.getChild(0).getType());
     assertEquals(NQLParser.EXTERNAL, ast.getChild(1).getType());

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-backend/src/test/java/tajo/engine/parser/TestQueryAnalyzer.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/tajo/engine/parser/TestQueryAnalyzer.java b/tajo-core/tajo-core-backend/src/test/java/tajo/engine/parser/TestQueryAnalyzer.java
index dbfe71b..99865dc 100644
--- a/tajo-core/tajo-core-backend/src/test/java/tajo/engine/parser/TestQueryAnalyzer.java
+++ b/tajo-core/tajo-core-backend/src/test/java/tajo/engine/parser/TestQueryAnalyzer.java
@@ -25,10 +25,10 @@ import org.junit.Test;
 import tajo.TajoTestingCluster;
 import tajo.benchmark.TPCH;
 import tajo.catalog.*;
-import tajo.catalog.proto.CatalogProtos.DataType;
 import tajo.catalog.proto.CatalogProtos.FunctionType;
 import tajo.catalog.proto.CatalogProtos.IndexMethod;
 import tajo.catalog.proto.CatalogProtos.StoreType;
+import tajo.common.TajoDataTypes;
 import tajo.datum.DatumFactory;
 import tajo.engine.eval.ConstEval;
 import tajo.engine.eval.EvalNode;
@@ -65,47 +65,47 @@ public class TestQueryAnalyzer {
     cat = util.getMiniCatalogCluster().getCatalog();
     
     schema1 = new Schema();
-    schema1.addColumn("id", DataType.INT);
-    schema1.addColumn("name", DataType.STRING);
-    schema1.addColumn("score", DataType.INT);
-    schema1.addColumn("age", DataType.INT);
+    schema1.addColumn("id", TajoDataTypes.Type.INT4);
+    schema1.addColumn("name", TajoDataTypes.Type.TEXT);
+    schema1.addColumn("score", TajoDataTypes.Type.INT4);
+    schema1.addColumn("age", TajoDataTypes.Type.INT4);
     
     Schema schema2 = new Schema();
-    schema2.addColumn("id", DataType.INT);
-    schema2.addColumn("people_id", DataType.INT);
-    schema2.addColumn("dept", DataType.STRING);
-    schema2.addColumn("year", DataType.INT);
+    schema2.addColumn("id", TajoDataTypes.Type.INT4);
+    schema2.addColumn("people_id", TajoDataTypes.Type.INT4);
+    schema2.addColumn("dept", TajoDataTypes.Type.TEXT);
+    schema2.addColumn("year", TajoDataTypes.Type.INT4);
     
     Schema schema3 = new Schema();
-    schema3.addColumn("id", DataType.INT);
-    schema3.addColumn("people_id", DataType.INT);
-    schema3.addColumn("class", DataType.STRING);
-    schema3.addColumn("branch_name", DataType.STRING);
+    schema3.addColumn("id", TajoDataTypes.Type.INT4);
+    schema3.addColumn("people_id", TajoDataTypes.Type.INT4);
+    schema3.addColumn("class", TajoDataTypes.Type.TEXT);
+    schema3.addColumn("branch_name", TajoDataTypes.Type.TEXT);
 
     Schema schema4 = new Schema();
-    schema4.addColumn("char_col", DataType.CHAR);
-    schema4.addColumn("short_col", DataType.SHORT);
-    schema4.addColumn("int_col", DataType.INT);
-    schema4.addColumn("long_col", DataType.LONG);
-    schema4.addColumn("float_col", DataType.FLOAT);
-    schema4.addColumn("double_col", DataType.DOUBLE);
-    schema4.addColumn("string_col", DataType.STRING);
-
-    TableMeta meta = TCatUtil.newTableMeta(schema1, StoreType.CSV);
+    schema4.addColumn("char_col", TajoDataTypes.Type.CHAR);
+    schema4.addColumn("short_col", TajoDataTypes.Type.INT2);
+    schema4.addColumn("int_col", TajoDataTypes.Type.INT4);
+    schema4.addColumn("long_col", TajoDataTypes.Type.INT8);
+    schema4.addColumn("float_col", TajoDataTypes.Type.FLOAT4);
+    schema4.addColumn("double_col", TajoDataTypes.Type.FLOAT8);
+    schema4.addColumn("string_col", TajoDataTypes.Type.TEXT);
+
+    TableMeta meta = CatalogUtil.newTableMeta(schema1, StoreType.CSV);
     TableDesc people = new TableDescImpl("people", meta, new Path("file:///"));
     cat.addTable(people);
     
-    TableDesc student = TCatUtil.newTableDesc("student", schema2, StoreType.CSV,
+    TableDesc student = CatalogUtil.newTableDesc("student", schema2, StoreType.CSV,
         new Options(),
         new Path("file:///"));
     cat.addTable(student);
     
-    TableDesc branch = TCatUtil.newTableDesc("branch", schema3, StoreType.CSV,
+    TableDesc branch = CatalogUtil.newTableDesc("branch", schema3, StoreType.CSV,
         new Options(),
         new Path("file:///"));
     cat.addTable(branch);
 
-    TableDesc allType = TCatUtil.newTableDesc("alltype", schema4, StoreType.CSV,
+    TableDesc allType = CatalogUtil.newTableDesc("alltype", schema4, StoreType.CSV,
         new Options(),
         new Path("file:///"));
     cat.addTable(allType);
@@ -114,18 +114,18 @@ public class TestQueryAnalyzer {
     tpch.loadSchemas();
     Schema lineitemSchema = tpch.getSchema("lineitem");
     Schema partSchema = tpch.getSchema("part");
-    TableDesc lineitem = TCatUtil.newTableDesc("lineitem", lineitemSchema, StoreType.CSV,
+    TableDesc lineitem = CatalogUtil.newTableDesc("lineitem", lineitemSchema, StoreType.CSV,
         new Options(),
         new Path("file:///"));
-    TableDesc part = TCatUtil.newTableDesc("part", partSchema, StoreType.CSV,
+    TableDesc part = CatalogUtil.newTableDesc("part", partSchema, StoreType.CSV,
         new Options(),
         new Path("file:///"));
     cat.addTable(lineitem);
     cat.addTable(part);
     
     FunctionDesc funcMeta = new FunctionDesc("sumtest", TestSum.class, FunctionType.GENERAL,
-        new DataType [] {DataType.INT},
-        new DataType [] {DataType.INT});
+        CatalogUtil.newDataTypesWithoutLen(TajoDataTypes.Type.INT4),
+        CatalogUtil.newDataTypesWithoutLen(TajoDataTypes.Type.INT4));
 
     cat.registerFunction(funcMeta);
     
@@ -273,19 +273,19 @@ public class TestQueryAnalyzer {
   }
 
   static final String [] createTableStmts = {
-    "create table table1 (name string, age int)",
-    "create table table1 (name string, age int) using rcfile",
-    "create table table1 (name string, age int) using rcfile with ('rcfile.buffer'=4096)",
+    "create table table1 (name text, age int)", // 0
+    "create table table1 (name text, age int) using rcfile", // 1
+    "create table table1 (name text, age int) using rcfile with ('rcfile.buffer'=4096)", // 2
     // create table test
-    "create table store1 as select name, score from people order by score asc, age desc null first",// 0
+    "create table store1 as select name, score from people order by score asc, age desc null first",// 3
     // create table test
-    "create table store1 (c1 string, c2 long) as select name, score from people order by score asc, age desc null first",// 1
+    "create table store1 (c1 text, c2 bigint) as select name, score from people order by score asc, age desc null first",// 4
     // create table test
-    "create table store2 using rcfile with ('rcfile.buffer' = 4096) as select name, score from people order by score asc, age desc null first", // 2
+    "create table store2 using rcfile with ('rcfile.buffer' = 4096) as select name, score from people order by score asc, age desc null first", // 5
     // create table def
-    "create table table1 (name string, age int, earn long, score float) using rcfile with ('rcfile.buffer' = 4096)", // 4
+    "create table table1 (name text, age int, earn float(10), score float(30)) using rcfile with ('rcfile.buffer' = 4096)", // 6
     // create table def with location
-    "create external table table1 (name string, age int, earn long, score float) using csv with ('csv.delimiter'='|') location '/tmp/data'" // 5
+    "create external table table1 (name text, age int, earn bigint, score float) using csv with ('csv.delimiter'='|') location '/tmp/data'" // 7
   };
 
   @Test
@@ -333,13 +333,13 @@ public class TestQueryAnalyzer {
     assertEquals("table1", stmt.getTableName());
     Schema def = stmt.getTableDef();
     assertEquals("name", def.getColumn(0).getColumnName());
-    assertEquals(DataType.STRING, def.getColumn(0).getDataType());
+    assertEquals(TajoDataTypes.Type.TEXT, def.getColumn(0).getDataType().getType());
     assertEquals("age", def.getColumn(1).getColumnName());
-    assertEquals(DataType.INT, def.getColumn(1).getDataType());
+    assertEquals(TajoDataTypes.Type.INT4, def.getColumn(1).getDataType().getType());
     assertEquals("earn", def.getColumn(2).getColumnName());
-    assertEquals(DataType.LONG, def.getColumn(2).getDataType());
+    assertEquals(TajoDataTypes.Type.FLOAT4, def.getColumn(2).getDataType().getType()); // float(10)
     assertEquals("score", def.getColumn(3).getColumnName());
-    assertEquals(DataType.FLOAT, def.getColumn(3).getDataType());
+    assertEquals(TajoDataTypes.Type.FLOAT8, def.getColumn(3).getDataType().getType()); // float(30)
     assertEquals(StoreType.RCFILE, stmt.getStoreType());
     assertFalse(stmt.hasPath());
     assertTrue(stmt.hasOptions());
@@ -352,13 +352,13 @@ public class TestQueryAnalyzer {
     assertEquals("table1", stmt.getTableName());
     Schema def = stmt.getTableDef();
     assertEquals("name", def.getColumn(0).getColumnName());
-    assertEquals(DataType.STRING, def.getColumn(0).getDataType());
+    assertEquals(TajoDataTypes.Type.TEXT, def.getColumn(0).getDataType().getType());
     assertEquals("age", def.getColumn(1).getColumnName());
-    assertEquals(DataType.INT, def.getColumn(1).getDataType());
+    assertEquals(TajoDataTypes.Type.INT4, def.getColumn(1).getDataType().getType());
     assertEquals("earn", def.getColumn(2).getColumnName());
-    assertEquals(DataType.LONG, def.getColumn(2).getDataType());
+    assertEquals(TajoDataTypes.Type.INT8, def.getColumn(2).getDataType().getType());
     assertEquals("score", def.getColumn(3).getColumnName());
-    assertEquals(DataType.FLOAT, def.getColumn(3).getDataType());    
+    assertEquals(TajoDataTypes.Type.FLOAT8, def.getColumn(3).getDataType().getType()); // float
     assertEquals(StoreType.CSV, stmt.getStoreType());    
     assertEquals("/tmp/data", stmt.getPath().toString());
     assertTrue(stmt.hasOptions());
@@ -380,9 +380,9 @@ public class TestQueryAnalyzer {
     SortSpec [] sortKeys = stmt.getSortSpecs();
     assertEquals(2, sortKeys.length);
     assertEquals("score", sortKeys[0].getSortKey().getColumnName());
-    assertEquals(DataType.INT, sortKeys[0].getSortKey().getDataType());
+    assertEquals(TajoDataTypes.Type.INT4, sortKeys[0].getSortKey().getDataType().getType());
     assertEquals("age", sortKeys[1].getSortKey().getColumnName());
-    assertEquals(DataType.INT, sortKeys[1].getSortKey().getDataType());
+    assertEquals(TajoDataTypes.Type.INT4, sortKeys[1].getSortKey().getDataType().getType());
     assertEquals(false, sortKeys[1].isAscending());
     assertEquals(true, sortKeys[1].isNullFirst());
     
@@ -605,31 +605,31 @@ public class TestQueryAnalyzer {
   @Test
   public final void testTypeInferring() {
     QueryBlock block = (QueryBlock) analyzer.parse("select 1 from alltype where char_col = 'a'").getParseTree();
-    assertEquals(DataType.CHAR, block.getWhereCondition().getRightExpr().getValueType()[0]);
+    assertEquals(TajoDataTypes.Type.CHAR, block.getWhereCondition().getRightExpr().getValueType()[0].getType());
 
     block = (QueryBlock) analyzer.parse("select 1 from alltype where short_col = 1").getParseTree();
-    assertEquals(DataType.SHORT, block.getWhereCondition().getRightExpr().getValueType()[0]);
+    assertEquals(TajoDataTypes.Type.INT2, block.getWhereCondition().getRightExpr().getValueType()[0].getType());
 
     block = (QueryBlock) analyzer.parse("select 1 from alltype where int_col = 1").getParseTree();
-    assertEquals(DataType.INT, block.getWhereCondition().getRightExpr().getValueType()[0]);
+    assertEquals(TajoDataTypes.Type.INT4, block.getWhereCondition().getRightExpr().getValueType()[0].getType());
 
     block = (QueryBlock) analyzer.parse("select 1 from alltype where long_col = 1").getParseTree();
-    assertEquals(DataType.LONG, block.getWhereCondition().getRightExpr().getValueType()[0]);
+    assertEquals(TajoDataTypes.Type.INT8, block.getWhereCondition().getRightExpr().getValueType()[0].getType());
 
     block = (QueryBlock) analyzer.parse("select 1 from alltype where float_col = 1").getParseTree();
-    assertEquals(DataType.INT, block.getWhereCondition().getRightExpr().getValueType()[0]);
+    assertEquals(TajoDataTypes.Type.INT4, block.getWhereCondition().getRightExpr().getValueType()[0].getType());
 
     block = (QueryBlock) analyzer.parse("select 1 from alltype where float_col = 1.0").getParseTree();
-    assertEquals(DataType.FLOAT, block.getWhereCondition().getRightExpr().getValueType()[0]);
+    assertEquals(TajoDataTypes.Type.FLOAT4, block.getWhereCondition().getRightExpr().getValueType()[0].getType());
 
     block = (QueryBlock) analyzer.parse("select 1 from alltype where int_col = 1.0").getParseTree();
-    assertEquals(DataType.DOUBLE, block.getWhereCondition().getRightExpr().getValueType()[0]);
+    assertEquals(TajoDataTypes.Type.FLOAT4, block.getWhereCondition().getRightExpr().getValueType()[0].getType());
 
     block = (QueryBlock) analyzer.parse("select 1 from alltype where double_col = 1.0").getParseTree();
-    assertEquals(DataType.DOUBLE, block.getWhereCondition().getRightExpr().getValueType()[0]);
+    assertEquals(TajoDataTypes.Type.FLOAT4, block.getWhereCondition().getRightExpr().getValueType()[0].getType());
 
     block = (QueryBlock) analyzer.parse("select 1 from alltype where string_col = 'a'").getParseTree();
-    assertEquals(DataType.STRING, block.getWhereCondition().getRightExpr().getValueType()[0]);
+    assertEquals(TajoDataTypes.Type.TEXT, block.getWhereCondition().getRightExpr().getValueType()[0].getType());
   }
 
   @Test
@@ -641,12 +641,12 @@ public class TestQueryAnalyzer {
     QueryBlock block = (QueryBlock) tree;
     assertTrue(block.getTargetList()[0].hasAlias());
     assertEquals("cond", block.getTargetList()[0].getAlias());
-    assertEquals(DataType.DOUBLE, block.getTargetList()[0].getEvalTree().getValueType()[0]);
+    assertEquals(TajoDataTypes.Type.FLOAT8, block.getTargetList()[0].getEvalTree().getValueType()[0].getType());
   }
 
   @Test
   public void testTarget() throws CloneNotSupportedException {
-    QueryBlock.Target t1 = new QueryBlock.Target(new ConstEval(DatumFactory.createInt(5)), 0);
+    QueryBlock.Target t1 = new QueryBlock.Target(new ConstEval(DatumFactory.createInt4(5)), 0);
     QueryBlock.Target t2 = (QueryBlock.Target) t1.clone();
     assertEquals(t1,t2);
   }

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-backend/src/test/java/tajo/engine/plan/global/TestGlobalQueryPlanner.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/tajo/engine/plan/global/TestGlobalQueryPlanner.java b/tajo-core/tajo-core-backend/src/test/java/tajo/engine/plan/global/TestGlobalQueryPlanner.java
index be8a7f1..ebd9d07 100644
--- a/tajo-core/tajo-core-backend/src/test/java/tajo/engine/plan/global/TestGlobalQueryPlanner.java
+++ b/tajo-core/tajo-core-backend/src/test/java/tajo/engine/plan/global/TestGlobalQueryPlanner.java
@@ -32,9 +32,9 @@ import tajo.QueryId;
 import tajo.QueryIdFactory;
 import tajo.TajoTestingCluster;
 import tajo.catalog.*;
-import tajo.catalog.proto.CatalogProtos.DataType;
 import tajo.catalog.proto.CatalogProtos.FunctionType;
 import tajo.catalog.proto.CatalogProtos.StoreType;
+import tajo.common.TajoDataTypes.Type;
 import tajo.conf.TajoConf;
 import tajo.datum.Datum;
 import tajo.datum.DatumFactory;
@@ -80,10 +80,10 @@ public class TestGlobalQueryPlanner {
     int i, j;
 
     schema = new Schema();
-    schema.addColumn("id", DataType.INT);
-    schema.addColumn("age", DataType.INT);
-    schema.addColumn("name", DataType.STRING);
-    schema.addColumn("salary", DataType.INT);
+    schema.addColumn("id", Type.INT4);
+    schema.addColumn("age", Type.INT4);
+    schema.addColumn("name", Type.TEXT);
+    schema.addColumn("salary", Type.INT4);
 
     TableMeta meta;
 
@@ -95,8 +95,8 @@ public class TestGlobalQueryPlanner {
 
     sm = new StorageManager(util.getConfiguration());
     FunctionDesc funcDesc = new FunctionDesc("sumtest", TestSum.class, FunctionType.GENERAL,
-        new DataType[] {DataType.INT},
-        new DataType[] {DataType.INT});
+        CatalogUtil.newDataTypesWithoutLen(Type.INT4),
+        CatalogUtil.newDataTypesWithoutLen(Type.INT4));
     catalog.registerFunction(funcDesc);
     FileSystem fs = sm.getFileSystem();
 
@@ -114,11 +114,11 @@ public class TestGlobalQueryPlanner {
     Appender appender;
     Tuple t = new VTuple(4);
     t.put(new Datum[] {
-        DatumFactory.createInt(1), DatumFactory.createInt(32),
-        DatumFactory.createString("h"), DatumFactory.createInt(10)});
+        DatumFactory.createInt4(1), DatumFactory.createInt4(32),
+        DatumFactory.createText("h"), DatumFactory.createInt4(10)});
 
     for (i = 0; i < tbNum; i++) {
-      meta = TCatUtil.newTableMeta((Schema)schema.clone(), StoreType.CSV);
+      meta = CatalogUtil.newTableMeta((Schema) schema.clone(), StoreType.CSV);
       meta.putOption(CSVFile.DELIMITER, ",");
 
       Path dataRoot = sm.getBaseDir();
@@ -135,7 +135,7 @@ public class TestGlobalQueryPlanner {
       }
       appender.close();
 
-      TableDesc desc = TCatUtil.newTableDesc("table" + i, (TableMeta)meta.clone(), tablePath);
+      TableDesc desc = CatalogUtil.newTableDesc("table" + i, (TableMeta) meta.clone(), tablePath);
       catalog.addTable(desc);
     }
 
@@ -436,7 +436,7 @@ public class TestGlobalQueryPlanner {
     };
 
     Schema schema1 = new Schema();
-    schema1.addColumn("col1", DataType.INT);
+    schema1.addColumn("col1", Type.INT4);
     TableMeta meta1 = new TableMetaImpl(schema1, StoreType.CSV, Options.create());
     TableDesc desc1 = new TableDescImpl("table1", meta1, new Path("/"));
     TableDesc desc2 = new TableDescImpl("table2", meta1, new Path("/"));
@@ -499,8 +499,8 @@ public class TestGlobalQueryPlanner {
 
     Column[] originKeys = secondGroupby.getGroupingColumns();
     Column[] newKeys = new Column[2];
-    newKeys[0] = new Column("age", DataType.INT);
-    newKeys[1] = new Column("name", DataType.STRING);
+    newKeys[0] = new Column("age", Type.INT4);
+    newKeys[1] = new Column("name", Type.TEXT);
 
     mid = planner.createMultilevelGroupby(first, newKeys);
     midGroupby = (GroupbyNode) mid.getStoreTableNode().getSubNode();

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-backend/src/test/java/tajo/engine/planner/TestLogicalOptimizer.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/tajo/engine/planner/TestLogicalOptimizer.java b/tajo-core/tajo-core-backend/src/test/java/tajo/engine/planner/TestLogicalOptimizer.java
index 1b5fd19..2333177 100644
--- a/tajo-core/tajo-core-backend/src/test/java/tajo/engine/planner/TestLogicalOptimizer.java
+++ b/tajo-core/tajo-core-backend/src/test/java/tajo/engine/planner/TestLogicalOptimizer.java
@@ -24,9 +24,9 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 import tajo.TajoTestingCluster;
 import tajo.catalog.*;
-import tajo.catalog.proto.CatalogProtos.DataType;
 import tajo.catalog.proto.CatalogProtos.FunctionType;
 import tajo.catalog.proto.CatalogProtos.StoreType;
+import tajo.common.TajoDataTypes.Type;
 import tajo.engine.function.builtin.SumInt;
 import tajo.engine.parser.QueryAnalyzer;
 import tajo.engine.planner.logical.*;
@@ -51,20 +51,20 @@ public class TestLogicalOptimizer {
     }
     
     Schema schema = new Schema();
-    schema.addColumn("name", DataType.STRING);
-    schema.addColumn("empId", DataType.INT);
-    schema.addColumn("deptName", DataType.STRING);
+    schema.addColumn("name", Type.TEXT);
+    schema.addColumn("empId", Type.INT4);
+    schema.addColumn("deptName", Type.TEXT);
 
     Schema schema2 = new Schema();
-    schema2.addColumn("deptName", DataType.STRING);
-    schema2.addColumn("manager", DataType.STRING);
+    schema2.addColumn("deptName", Type.TEXT);
+    schema2.addColumn("manager", Type.TEXT);
 
     Schema schema3 = new Schema();
-    schema3.addColumn("deptName", DataType.STRING);
-    schema3.addColumn("score", DataType.INT);
-    schema3.addColumn("phone", DataType.INT);
+    schema3.addColumn("deptName", Type.TEXT);
+    schema3.addColumn("score", Type.INT4);
+    schema3.addColumn("phone", Type.INT4);
 
-    TableMeta meta = TCatUtil.newTableMeta(schema, StoreType.CSV);
+    TableMeta meta = CatalogUtil.newTableMeta(schema, StoreType.CSV);
     TableDesc people = new TableDescImpl("employee", meta,
         new Path("file:///"));
     catalog.addTable(people);
@@ -80,8 +80,8 @@ public class TestLogicalOptimizer {
     catalog.addTable(score);
 
     FunctionDesc funcDesc = new FunctionDesc("sumtest", SumInt.class, FunctionType.GENERAL,
-        new DataType[] {DataType.INT},
-        new DataType[] {DataType.INT});
+        CatalogUtil.newDataTypesWithoutLen(Type.INT4),
+        CatalogUtil.newDataTypesWithoutLen(Type.INT4));
 
     catalog.registerFunction(funcDesc);
     analyzer = new QueryAnalyzer(catalog);

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-backend/src/test/java/tajo/engine/planner/TestLogicalPlanner.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/tajo/engine/planner/TestLogicalPlanner.java b/tajo-core/tajo-core-backend/src/test/java/tajo/engine/planner/TestLogicalPlanner.java
index 0c8ec4f..b5fba81 100644
--- a/tajo-core/tajo-core-backend/src/test/java/tajo/engine/planner/TestLogicalPlanner.java
+++ b/tajo-core/tajo-core-backend/src/test/java/tajo/engine/planner/TestLogicalPlanner.java
@@ -28,10 +28,10 @@ import org.junit.Test;
 import tajo.TajoTestingCluster;
 import tajo.benchmark.TPCH;
 import tajo.catalog.*;
-import tajo.catalog.proto.CatalogProtos.DataType;
 import tajo.catalog.proto.CatalogProtos.FunctionType;
 import tajo.catalog.proto.CatalogProtos.IndexMethod;
 import tajo.catalog.proto.CatalogProtos.StoreType;
+import tajo.common.TajoDataTypes.Type;
 import tajo.engine.eval.EvalNode;
 import tajo.engine.function.builtin.SumInt;
 import tajo.engine.json.GsonCreator;
@@ -64,19 +64,19 @@ public class TestLogicalPlanner {
     }
     
     Schema schema = new Schema();
-    schema.addColumn("name", DataType.STRING);
-    schema.addColumn("empId", DataType.INT);
-    schema.addColumn("deptName", DataType.STRING);
+    schema.addColumn("name", Type.TEXT);
+    schema.addColumn("empId", Type.INT4);
+    schema.addColumn("deptName", Type.TEXT);
 
     Schema schema2 = new Schema();
-    schema2.addColumn("deptName", DataType.STRING);
-    schema2.addColumn("manager", DataType.STRING);
+    schema2.addColumn("deptName", Type.TEXT);
+    schema2.addColumn("manager", Type.TEXT);
 
     Schema schema3 = new Schema();
-    schema3.addColumn("deptName", DataType.STRING);
-    schema3.addColumn("score", DataType.INT);
+    schema3.addColumn("deptName", Type.TEXT);
+    schema3.addColumn("score", Type.INT4);
 
-    TableMeta meta = TCatUtil.newTableMeta(schema, StoreType.CSV);
+    TableMeta meta = CatalogUtil.newTableMeta(schema, StoreType.CSV);
     TableDesc people = new TableDescImpl("employee", meta,
         new Path("file:///"));
     catalog.addTable(people);
@@ -92,8 +92,8 @@ public class TestLogicalPlanner {
     catalog.addTable(score);
 
     FunctionDesc funcDesc = new FunctionDesc("sumtest", SumInt.class, FunctionType.AGGREGATION,
-        new DataType [] {DataType.INT},
-        new DataType [] {DataType.INT});
+        CatalogUtil.newDataTypesWithoutLen(Type.INT4),
+        CatalogUtil.newDataTypesWithoutLen(Type.INT4));
 
 
     // TPC-H Schema for Complex Queries
@@ -104,8 +104,8 @@ public class TestLogicalPlanner {
     tpch.loadSchemas();
     tpch.loadOutSchema();
     for (String table : tpchTables) {
-      TableMeta m = TCatUtil.newTableMeta(tpch.getSchema(table), StoreType.CSV);
-      TableDesc d = TCatUtil.newTableDesc(table, m, new Path("file:///"));
+      TableMeta m = CatalogUtil.newTableMeta(tpch.getSchema(table), StoreType.CSV);
+      TableDesc d = CatalogUtil.newTableDesc(table, m, new Path("file:///"));
       catalog.addTable(d);
     }
 
@@ -177,10 +177,10 @@ public class TestLogicalPlanner {
     TestLogicalNode.testCloneLogicalNode(root);
 
     Schema expectedSchema = new Schema();
-    expectedSchema.addColumn("name", DataType.STRING);
-    expectedSchema.addColumn("empId", DataType.INT);
-    expectedSchema.addColumn("deptName", DataType.STRING);
-    expectedSchema.addColumn("manager", DataType.STRING);
+    expectedSchema.addColumn("name", Type.TEXT);
+    expectedSchema.addColumn("empId", Type.INT4);
+    expectedSchema.addColumn("deptName", Type.TEXT);
+    expectedSchema.addColumn("manager", Type.TEXT);
     assertSchema(expectedSchema, root.getOutSchema());
 
     assertEquals(ExprType.PROJECTION, root.getSubNode().getType());
@@ -204,7 +204,7 @@ public class TestLogicalPlanner {
     plan = planner.createPlan(context);
     TestLogicalNode.testCloneLogicalNode(plan);
 
-    expectedSchema.addColumn("score", DataType.INT);
+    expectedSchema.addColumn("score", Type.INT4);
     assertSchema(expectedSchema, plan.getOutSchema());
 
     assertEquals(ExprType.ROOT, plan.getType());
@@ -246,9 +246,9 @@ public class TestLogicalPlanner {
   static Schema expectedJoinSchema;
   static {
     expectedJoinSchema = new Schema();
-    expectedJoinSchema.addColumn("name", DataType.STRING);
-    expectedJoinSchema.addColumn("deptName", DataType.STRING);
-    expectedJoinSchema.addColumn("score", DataType.INT);
+    expectedJoinSchema.addColumn("name", Type.TEXT);
+    expectedJoinSchema.addColumn("deptName", Type.TEXT);
+    expectedJoinSchema.addColumn("score", Type.INT4);
   }
   
   @Test
@@ -586,10 +586,10 @@ public class TestLogicalPlanner {
     assertEquals(false, indexNode.isUnique());
     assertEquals(2, indexNode.getSortSpecs().length);
     assertEquals("name", indexNode.getSortSpecs()[0].getSortKey().getColumnName());
-    assertEquals(DataType.STRING, indexNode.getSortSpecs()[0].getSortKey().getDataType());
+    assertEquals(Type.TEXT, indexNode.getSortSpecs()[0].getSortKey().getDataType().getType());
     assertEquals(true, indexNode.getSortSpecs()[0].isNullFirst());
     assertEquals("empid", indexNode.getSortSpecs()[1].getSortKey().getColumnName());
-    assertEquals(DataType.INT, indexNode.getSortSpecs()[1].getSortKey().getDataType());
+    assertEquals(Type.INT4, indexNode.getSortSpecs()[1].getSortKey().getDataType().getType());
     assertEquals(false, indexNode.getSortSpecs()[1].isAscending());
     assertEquals(false, indexNode.getSortSpecs()[1].isNullFirst());
     assertEquals(IndexMethod.BITMAP, indexNode.getMethod());
@@ -628,7 +628,7 @@ public class TestLogicalPlanner {
   }
   
   static final String CREATE_TABLE [] = {
-    "create external table table1 (name string, age int, earn long, score float) using csv with ('csv.delimiter'='|') location '/tmp/data'"
+    "create external table table1 (name text, age int, earn bigint, score real) using csv with ('csv.delimiter'='|') location '/tmp/data'"
   };
   
   @Test
@@ -642,13 +642,13 @@ public class TestLogicalPlanner {
     
     Schema def = createTable.getSchema();
     assertEquals("name", def.getColumn(0).getColumnName());
-    assertEquals(DataType.STRING, def.getColumn(0).getDataType());
+    assertEquals(Type.TEXT, def.getColumn(0).getDataType().getType());
     assertEquals("age", def.getColumn(1).getColumnName());
-    assertEquals(DataType.INT, def.getColumn(1).getDataType());
+    assertEquals(Type.INT4, def.getColumn(1).getDataType().getType());
     assertEquals("earn", def.getColumn(2).getColumnName());
-    assertEquals(DataType.LONG, def.getColumn(2).getDataType());
+    assertEquals(Type.INT8, def.getColumn(2).getDataType().getType());
     assertEquals("score", def.getColumn(3).getColumnName());
-    assertEquals(DataType.FLOAT, def.getColumn(3).getDataType());    
+    assertEquals(Type.FLOAT4, def.getColumn(3).getDataType().getType());
     assertEquals(StoreType.CSV, createTable.getStorageType());
     assertEquals("/tmp/data", createTable.getPath().toString());
     assertTrue(createTable.hasOptions());
@@ -664,9 +664,9 @@ public class TestLogicalPlanner {
     = Lists.newArrayList();
   private static final Column [] testCubeByCuboids = new Column[2];
   static {
-    testGenerateCuboids[0] = new Column("col1", DataType.INT);
-    testGenerateCuboids[1] = new Column("col2", DataType.LONG);
-    testGenerateCuboids[2] = new Column("col3", DataType.FLOAT);
+    testGenerateCuboids[0] = new Column("col1", Type.INT4);
+    testGenerateCuboids[1] = new Column("col2", Type.INT8);
+    testGenerateCuboids[2] = new Column("col3", Type.FLOAT4);
     
     testGenerateCuboidsResult.add(new HashSet<Column>());
     testGenerateCuboidsResult.add(Sets.newHashSet(testGenerateCuboids[0]));
@@ -681,8 +681,8 @@ public class TestLogicalPlanner {
     testGenerateCuboidsResult.add(Sets.newHashSet(testGenerateCuboids[0], 
         testGenerateCuboids[1], testGenerateCuboids[2]));
     
-    testCubeByCuboids[0] = new Column("employee.name", DataType.STRING);
-    testCubeByCuboids[1] = new Column("employee.empid", DataType.INT);
+    testCubeByCuboids[0] = new Column("employee.name", Type.TEXT);
+    testCubeByCuboids[1] = new Column("employee.empid", Type.INT4);
     testCubeByResult.add(new HashSet<Column>());
     testCubeByResult.add(Sets.newHashSet(testCubeByCuboids[0]));
     testCubeByResult.add(Sets.newHashSet(testCubeByCuboids[1]));
@@ -694,11 +694,11 @@ public class TestLogicalPlanner {
   public final void testGenerateCuboids() {
     Column [] columns = new Column[3];
     
-    columns[0] = new Column("col1", DataType.INT);
-    columns[1] = new Column("col2", DataType.LONG);
-    columns[2] = new Column("col3", DataType.FLOAT);
+    columns[0] = new Column("col1", Type.INT4);
+    columns[1] = new Column("col2", Type.INT8);
+    columns[2] = new Column("col3", Type.FLOAT4);
     
-    List<Column[]> cube = planner.generateCuboids(columns);
+    List<Column[]> cube = LogicalPlanner.generateCuboids(columns);
     assertEquals(((int)Math.pow(2, numCubeColumns)), cube.size());    
     
     Set<Set<Column>> cuboids = Sets.newHashSet();

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-backend/src/test/java/tajo/engine/planner/TestPlannerUtil.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/tajo/engine/planner/TestPlannerUtil.java b/tajo-core/tajo-core-backend/src/test/java/tajo/engine/planner/TestPlannerUtil.java
index 1b311ab..9b8efc0 100644
--- a/tajo-core/tajo-core-backend/src/test/java/tajo/engine/planner/TestPlannerUtil.java
+++ b/tajo-core/tajo-core-backend/src/test/java/tajo/engine/planner/TestPlannerUtil.java
@@ -24,9 +24,9 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 import tajo.TajoTestingCluster;
 import tajo.catalog.*;
-import tajo.catalog.proto.CatalogProtos.DataType;
 import tajo.catalog.proto.CatalogProtos.FunctionType;
 import tajo.catalog.proto.CatalogProtos.StoreType;
+import tajo.common.TajoDataTypes.Type;
 import tajo.datum.DatumFactory;
 import tajo.engine.eval.BinaryEval;
 import tajo.engine.eval.ConstEval;
@@ -56,19 +56,19 @@ public class TestPlannerUtil {
     catalog = util.getMiniCatalogCluster().getCatalog();
 
     Schema schema = new Schema();
-    schema.addColumn("name", DataType.STRING);
-    schema.addColumn("empId", DataType.INT);
-    schema.addColumn("deptName", DataType.STRING);
+    schema.addColumn("name", Type.TEXT);
+    schema.addColumn("empId", CatalogUtil.newDataTypeWithoutLen(Type.INT4));
+    schema.addColumn("deptName", Type.TEXT);
 
     Schema schema2 = new Schema();
-    schema2.addColumn("deptName", DataType.STRING);
-    schema2.addColumn("manager", DataType.STRING);
+    schema2.addColumn("deptName", Type.TEXT);
+    schema2.addColumn("manager", Type.TEXT);
 
     Schema schema3 = new Schema();
-    schema3.addColumn("deptName", DataType.STRING);
-    schema3.addColumn("score", DataType.INT);
+    schema3.addColumn("deptName", Type.TEXT);
+    schema3.addColumn("score", CatalogUtil.newDataTypeWithoutLen(Type.INT4));
 
-    TableMeta meta = TCatUtil.newTableMeta(schema, StoreType.CSV);
+    TableMeta meta = CatalogUtil.newTableMeta(schema, StoreType.CSV);
     TableDesc people = new TableDescImpl("employee", meta,
         new Path("file:///"));
     catalog.addTable(people);
@@ -84,8 +84,8 @@ public class TestPlannerUtil {
     catalog.addTable(score);
 
     FunctionDesc funcDesc = new FunctionDesc("sumtest", SumInt.class, FunctionType.AGGREGATION,
-        new DataType [] {DataType.INT},
-        new DataType [] {DataType.INT});
+        CatalogUtil.newDataTypesWithoutLen(Type.INT4),
+        CatalogUtil.newDataTypesWithoutLen(Type.INT4));
 
     catalog.registerFunction(funcDesc);
     analyzer = new QueryAnalyzer(catalog);
@@ -180,8 +180,9 @@ public class TestPlannerUtil {
 
   @Test
   public final void testIsJoinQual() {
-    FieldEval f1 = new FieldEval("part.p_partkey", DataType.INT);
-    FieldEval f2 = new FieldEval("partsupp.ps_partkey", DataType.INT);
+    FieldEval f1 = new FieldEval("part.p_partkey", CatalogUtil.newDataTypeWithoutLen(Type.INT4));
+    FieldEval f2 = new FieldEval("partsupp.ps_partkey",
+        CatalogUtil.newDataTypeWithoutLen(Type.INT4));
 
 
     BinaryEval [] joinQuals = new BinaryEval[5];
@@ -201,7 +202,7 @@ public class TestPlannerUtil {
     wrongJoinQuals[idx++] = new BinaryEval(EvalNode.Type.PLUS, f1, f2);
     wrongJoinQuals[idx++] = new BinaryEval(EvalNode.Type.LIKE, f1, f2);
 
-    ConstEval f3 = new ConstEval(DatumFactory.createInt(1));
+    ConstEval f3 = new ConstEval(DatumFactory.createInt4(1));
     wrongJoinQuals[idx] = new BinaryEval(EvalNode.Type.EQUAL, f1, f3);
 
     for (int i = 0; i < idx; i++) {
@@ -212,16 +213,16 @@ public class TestPlannerUtil {
   @Test
   public final void testGetJoinKeyPairs() {
     Schema outerSchema = new Schema();
-    outerSchema.addColumn("employee.id1", DataType.INT);
-    outerSchema.addColumn("employee.id2", DataType.INT);
+    outerSchema.addColumn("employee.id1", CatalogUtil.newDataTypeWithoutLen(Type.INT4));
+    outerSchema.addColumn("employee.id2", CatalogUtil.newDataTypeWithoutLen(Type.INT4));
     Schema innerSchema = new Schema();
-    innerSchema.addColumn("people.fid1", DataType.INT);
-    innerSchema.addColumn("people.fid2", DataType.INT);
+    innerSchema.addColumn("people.fid1", CatalogUtil.newDataTypeWithoutLen(Type.INT4));
+    innerSchema.addColumn("people.fid2", CatalogUtil.newDataTypeWithoutLen(Type.INT4));
 
-    FieldEval f1 = new FieldEval("employee.id1", DataType.INT);
-    FieldEval f2 = new FieldEval("people.fid1", DataType.INT);
-    FieldEval f3 = new FieldEval("employee.id2", DataType.INT);
-    FieldEval f4 = new FieldEval("people.fid2", DataType.INT);
+    FieldEval f1 = new FieldEval("employee.id1", CatalogUtil.newDataTypeWithoutLen(Type.INT4));
+    FieldEval f2 = new FieldEval("people.fid1", CatalogUtil.newDataTypeWithoutLen(Type.INT4));
+    FieldEval f3 = new FieldEval("employee.id2", CatalogUtil.newDataTypeWithoutLen(Type.INT4));
+    FieldEval f4 = new FieldEval("people.fid2", CatalogUtil.newDataTypeWithoutLen(Type.INT4));
 
     EvalNode joinQual = new BinaryEval(EvalNode.Type.EQUAL, f1, f2);
 
@@ -258,16 +259,16 @@ public class TestPlannerUtil {
   @Test
   public final void testGetSortKeysFromJoinQual() {
     Schema outerSchema = new Schema();
-    outerSchema.addColumn("employee.id1", DataType.INT);
-    outerSchema.addColumn("employee.id2", DataType.INT);
+    outerSchema.addColumn("employee.id1", CatalogUtil.newDataTypeWithoutLen(Type.INT4));
+    outerSchema.addColumn("employee.id2", CatalogUtil.newDataTypeWithoutLen(Type.INT4));
     Schema innerSchema = new Schema();
-    innerSchema.addColumn("people.fid1", DataType.INT);
-    innerSchema.addColumn("people.fid2", DataType.INT);
+    innerSchema.addColumn("people.fid1", CatalogUtil.newDataTypeWithoutLen(Type.INT4));
+    innerSchema.addColumn("people.fid2", CatalogUtil.newDataTypeWithoutLen(Type.INT4));
 
-    FieldEval f1 = new FieldEval("employee.id1", DataType.INT);
-    FieldEval f2 = new FieldEval("people.fid1", DataType.INT);
-    FieldEval f3 = new FieldEval("employee.id2", DataType.INT);
-    FieldEval f4 = new FieldEval("people.fid2", DataType.INT);
+    FieldEval f1 = new FieldEval("employee.id1", CatalogUtil.newDataTypeWithoutLen(Type.INT4));
+    FieldEval f2 = new FieldEval("people.fid1", CatalogUtil.newDataTypeWithoutLen(Type.INT4));
+    FieldEval f3 = new FieldEval("employee.id2", CatalogUtil.newDataTypeWithoutLen(Type.INT4));
+    FieldEval f4 = new FieldEval("people.fid2", CatalogUtil.newDataTypeWithoutLen(Type.INT4));
 
     EvalNode joinQual = new BinaryEval(EvalNode.Type.EQUAL, f1, f2);
     SortSpec[][] sortSpecs = PlannerUtil.getSortKeysFromJoinQual(joinQual, outerSchema, innerSchema);
@@ -294,27 +295,27 @@ public class TestPlannerUtil {
   @Test
   public final void testComparatorsFromJoinQual() {
     Schema outerSchema = new Schema();
-    outerSchema.addColumn("employee.id1", DataType.INT);
-    outerSchema.addColumn("employee.id2", DataType.INT);
+    outerSchema.addColumn("employee.id1", CatalogUtil.newDataTypeWithoutLen(Type.INT4));
+    outerSchema.addColumn("employee.id2", CatalogUtil.newDataTypeWithoutLen(Type.INT4));
     Schema innerSchema = new Schema();
-    innerSchema.addColumn("people.fid1", DataType.INT);
-    innerSchema.addColumn("people.fid2", DataType.INT);
+    innerSchema.addColumn("people.fid1", CatalogUtil.newDataTypeWithoutLen(Type.INT4));
+    innerSchema.addColumn("people.fid2", CatalogUtil.newDataTypeWithoutLen(Type.INT4));
 
-    FieldEval f1 = new FieldEval("employee.id1", DataType.INT);
-    FieldEval f2 = new FieldEval("people.fid1", DataType.INT);
-    FieldEval f3 = new FieldEval("employee.id2", DataType.INT);
-    FieldEval f4 = new FieldEval("people.fid2", DataType.INT);
+    FieldEval f1 = new FieldEval("employee.id1", CatalogUtil.newDataTypeWithoutLen(Type.INT4));
+    FieldEval f2 = new FieldEval("people.fid1", CatalogUtil.newDataTypeWithoutLen(Type.INT4));
+    FieldEval f3 = new FieldEval("employee.id2", CatalogUtil.newDataTypeWithoutLen(Type.INT4));
+    FieldEval f4 = new FieldEval("people.fid2", CatalogUtil.newDataTypeWithoutLen(Type.INT4));
 
     EvalNode joinQual = new BinaryEval(EvalNode.Type.EQUAL, f1, f2);
     TupleComparator [] comparators = PlannerUtil.getComparatorsFromJoinQual(joinQual, outerSchema, innerSchema);
 
     Tuple t1 = new VTuple(2);
-    t1.put(0, DatumFactory.createInt(1));
-    t1.put(1, DatumFactory.createInt(2));
+    t1.put(0, DatumFactory.createInt4(1));
+    t1.put(1, DatumFactory.createInt4(2));
 
     Tuple t2 = new VTuple(2);
-    t2.put(0, DatumFactory.createInt(2));
-    t2.put(1, DatumFactory.createInt(3));
+    t2.put(0, DatumFactory.createInt4(2));
+    t2.put(1, DatumFactory.createInt4(3));
 
     TupleComparator outerComparator = comparators[0];
     assertTrue(outerComparator.compare(t1, t2) < 0);

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-backend/src/test/java/tajo/engine/planner/TestUniformRangePartition.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/tajo/engine/planner/TestUniformRangePartition.java b/tajo-core/tajo-core-backend/src/test/java/tajo/engine/planner/TestUniformRangePartition.java
index e436c7b..521bc55 100644
--- a/tajo-core/tajo-core-backend/src/test/java/tajo/engine/planner/TestUniformRangePartition.java
+++ b/tajo-core/tajo-core-backend/src/test/java/tajo/engine/planner/TestUniformRangePartition.java
@@ -20,7 +20,7 @@ package tajo.engine.planner;
 
 import org.junit.Test;
 import tajo.catalog.Schema;
-import tajo.catalog.proto.CatalogProtos.DataType;
+import tajo.common.TajoDataTypes.Type;
 import tajo.datum.DatumFactory;
 import tajo.engine.utils.TupleUtil;
 import tajo.storage.Tuple;
@@ -37,14 +37,14 @@ public class TestUniformRangePartition {
   @Test
   public void testIncrement1() {
     Schema schema = new Schema()
-    .addColumn("l_returnflag", DataType.STRING)
-    .addColumn("l_linestatus", DataType.STRING);
+    .addColumn("l_returnflag", Type.TEXT)
+    .addColumn("l_linestatus", Type.TEXT);
     Tuple s = new VTuple(2);
-    s.put(0, DatumFactory.createString("A"));
-    s.put(1, DatumFactory.createString("A"));
+    s.put(0, DatumFactory.createText("A"));
+    s.put(1, DatumFactory.createText("A"));
     Tuple e = new VTuple(2);
-    e.put(0, DatumFactory.createString("D"));
-    e.put(1, DatumFactory.createString("C"));
+    e.put(0, DatumFactory.createText("D"));
+    e.put(1, DatumFactory.createText("C"));
 
     TupleRange expected = new TupleRange(schema, s, e);
 
@@ -82,14 +82,14 @@ public class TestUniformRangePartition {
   @Test
   public void testIncrement2() {
     Schema schema = new Schema()
-    .addColumn("l_returnflag", DataType.STRING)
-    .addColumn("l_linestatus", DataType.STRING);
+    .addColumn("l_returnflag", Type.TEXT)
+    .addColumn("l_linestatus", Type.TEXT);
     Tuple s = new VTuple(2);
-    s.put(0, DatumFactory.createString("A"));
-    s.put(1, DatumFactory.createString("A"));
+    s.put(0, DatumFactory.createText("A"));
+    s.put(1, DatumFactory.createText("A"));
     Tuple e = new VTuple(2);
-    e.put(0, DatumFactory.createString("D"));
-    e.put(1, DatumFactory.createString("C"));
+    e.put(0, DatumFactory.createText("D"));
+    e.put(1, DatumFactory.createText("C"));
 
     TupleRange expected = new TupleRange(schema, s, e);
 
@@ -125,18 +125,18 @@ public class TestUniformRangePartition {
   @Test
   public void testIncrement3() {
     Schema schema = new Schema()
-    .addColumn("l_returnflag", DataType.STRING)
-    .addColumn("l_linestatus", DataType.STRING)
-    .addColumn("final", DataType.STRING);
+    .addColumn("l_returnflag", Type.TEXT)
+    .addColumn("l_linestatus", Type.TEXT)
+    .addColumn("final", Type.TEXT);
 
     Tuple s = new VTuple(3);
-    s.put(0, DatumFactory.createString("A"));
-    s.put(1, DatumFactory.createString("A"));
-    s.put(2, DatumFactory.createString("A"));
+    s.put(0, DatumFactory.createText("A"));
+    s.put(1, DatumFactory.createText("A"));
+    s.put(2, DatumFactory.createText("A"));
     Tuple e = new VTuple(3);
-    e.put(0, DatumFactory.createString("D")); //  4
-    e.put(1, DatumFactory.createString("B")); //  2
-    e.put(2, DatumFactory.createString("C")); // x3 = 24
+    e.put(0, DatumFactory.createText("D")); //  4
+    e.put(1, DatumFactory.createText("B")); //  2
+    e.put(2, DatumFactory.createText("C")); // x3 = 24
 
     TupleRange expected = new TupleRange(schema, s, e);
 
@@ -157,14 +157,14 @@ public class TestUniformRangePartition {
   @Test
   public void testIncrement4() {
     Schema schema = new Schema()
-    .addColumn("l_orderkey", DataType.LONG)
-    .addColumn("l_linenumber", DataType.LONG);
+    .addColumn("l_orderkey", Type.INT8)
+    .addColumn("l_linenumber", Type.INT8);
     Tuple s = new VTuple(2);
-    s.put(0, DatumFactory.createLong(10));
-    s.put(1, DatumFactory.createLong(20));
+    s.put(0, DatumFactory.createInt8(10));
+    s.put(1, DatumFactory.createInt8(20));
     Tuple e = new VTuple(2);
-    e.put(0, DatumFactory.createLong(19));
-    e.put(1, DatumFactory.createLong(39));
+    e.put(0, DatumFactory.createInt8(19));
+    e.put(1, DatumFactory.createInt8(39));
 
     TupleRange expected = new TupleRange(schema, s, e);
 
@@ -173,26 +173,26 @@ public class TestUniformRangePartition {
     assertEquals(200, partitioner.getTotalCardinality().longValue());
 
     Tuple range2 = partitioner.increment(s, 100, 1);
-    assertEquals(15, range2.get(0).asInt());
-    assertEquals(20, range2.get(1).asInt());
+    assertEquals(15, range2.get(0).asInt4());
+    assertEquals(20, range2.get(1).asInt4());
     Tuple range3 = partitioner.increment(range2, 99, 1);
-    assertEquals(19, range3.get(0).asInt());
-    assertEquals(39, range3.get(1).asInt());
+    assertEquals(19, range3.get(0).asInt4());
+    assertEquals(39, range3.get(1).asInt4());
   }
 
   @Test public void testIncrement5() {
     Schema schema = new Schema()
-    .addColumn("l_orderkey", DataType.LONG)
-    .addColumn("l_linenumber", DataType.LONG)
-    .addColumn("final", DataType.LONG);
+    .addColumn("l_orderkey", Type.INT8)
+    .addColumn("l_linenumber", Type.INT8)
+    .addColumn("final", Type.INT8);
     Tuple s = new VTuple(3);
-    s.put(0, DatumFactory.createLong(1));
-    s.put(1, DatumFactory.createLong(1));
-    s.put(2, DatumFactory.createLong(1));
+    s.put(0, DatumFactory.createInt8(1));
+    s.put(1, DatumFactory.createInt8(1));
+    s.put(2, DatumFactory.createInt8(1));
     Tuple e = new VTuple(3);
-    e.put(0, DatumFactory.createLong(4)); // 4
-    e.put(1, DatumFactory.createLong(2)); // 2
-    e.put(2, DatumFactory.createLong(3)); //x3 = 24
+    e.put(0, DatumFactory.createInt8(4)); // 4
+    e.put(1, DatumFactory.createInt8(2)); // 2
+    e.put(2, DatumFactory.createInt8(3)); //x3 = 24
 
     TupleRange expected = new TupleRange(schema, s, e);
 
@@ -201,29 +201,29 @@ public class TestUniformRangePartition {
     assertEquals(24, partitioner.getTotalCardinality().longValue());
 
     Tuple beforeOverflow = partitioner.increment(s, 5, 2);
-    assertEquals(1, beforeOverflow.get(0).asLong());
-    assertEquals(2, beforeOverflow.get(1).asLong());
-    assertEquals(3, beforeOverflow.get(2).asLong());
+    assertEquals(1, beforeOverflow.get(0).asInt8());
+    assertEquals(2, beforeOverflow.get(1).asInt8());
+    assertEquals(3, beforeOverflow.get(2).asInt8());
     Tuple overflow = partitioner.increment(beforeOverflow, 1, 2);
-    assertEquals(2, overflow.get(0).asLong());
-    assertEquals(1, overflow.get(1).asLong());
-    assertEquals(1, overflow.get(2).asLong());
+    assertEquals(2, overflow.get(0).asInt8());
+    assertEquals(1, overflow.get(1).asInt8());
+    assertEquals(1, overflow.get(2).asInt8());
   }
 
   @Test
   public void testIncrement6() {
     Schema schema = new Schema()
-      .addColumn("l_orderkey", DataType.DOUBLE)
-      .addColumn("l_linenumber", DataType.DOUBLE)
-      .addColumn("final", DataType.DOUBLE);
+      .addColumn("l_orderkey", Type.FLOAT8)
+      .addColumn("l_linenumber", Type.FLOAT8)
+      .addColumn("final", Type.FLOAT8);
     Tuple s = new VTuple(3);
-    s.put(0, DatumFactory.createDouble(1.1d));
-    s.put(1, DatumFactory.createDouble(1.1d));
-    s.put(2, DatumFactory.createDouble(1.1d));
+    s.put(0, DatumFactory.createFloat8(1.1d));
+    s.put(1, DatumFactory.createFloat8(1.1d));
+    s.put(2, DatumFactory.createFloat8(1.1d));
     Tuple e = new VTuple(3);
-    e.put(0, DatumFactory.createDouble(4.1d)); // 4
-    e.put(1, DatumFactory.createDouble(2.1d)); // 2
-    e.put(2, DatumFactory.createDouble(3.1d)); //x3 = 24
+    e.put(0, DatumFactory.createFloat8(4.1d)); // 4
+    e.put(1, DatumFactory.createFloat8(2.1d)); // 2
+    e.put(2, DatumFactory.createFloat8(3.1d)); //x3 = 24
 
     TupleRange expected = new TupleRange(schema, s, e);
 
@@ -232,26 +232,26 @@ public class TestUniformRangePartition {
     assertEquals(24, partitioner.getTotalCardinality().longValue());
 
     Tuple beforeOverflow = partitioner.increment(s, 5, 2);
-    assertTrue(1.1d == beforeOverflow.get(0).asDouble());
-    assertTrue(2.1d == beforeOverflow.get(1).asDouble());
-    assertTrue(3.1d == beforeOverflow.get(2).asDouble());
+    assertTrue(1.1d == beforeOverflow.get(0).asFloat8());
+    assertTrue(2.1d == beforeOverflow.get(1).asFloat8());
+    assertTrue(3.1d == beforeOverflow.get(2).asFloat8());
     Tuple overflow = partitioner.increment(beforeOverflow, 1, 2);
-    assertTrue(2.1d == overflow.get(0).asDouble());
-    assertTrue(1.1d == overflow.get(1).asDouble());
-    assertTrue(1.1d == overflow.get(2).asDouble());
+    assertTrue(2.1d == overflow.get(0).asFloat8());
+    assertTrue(1.1d == overflow.get(1).asFloat8());
+    assertTrue(1.1d == overflow.get(2).asFloat8());
   }
 
   @Test
   public void testPartition() {
     Schema schema = new Schema();
-    schema.addColumn("l_returnflag", DataType.STRING);
-    schema.addColumn("l_linestatus", DataType.STRING);
+    schema.addColumn("l_returnflag", Type.TEXT);
+    schema.addColumn("l_linestatus", Type.TEXT);
     Tuple s = new VTuple(2);
-    s.put(0, DatumFactory.createString("A"));
-    s.put(1, DatumFactory.createString("F"));
+    s.put(0, DatumFactory.createText("A"));
+    s.put(1, DatumFactory.createText("F"));
     Tuple e = new VTuple(2);
-    e.put(0, DatumFactory.createString("R"));
-    e.put(1, DatumFactory.createString("O"));
+    e.put(0, DatumFactory.createText("R"));
+    e.put(1, DatumFactory.createText("O"));
     TupleRange expected = new TupleRange(schema, s, e);
     RangePartitionAlgorithm partitioner
         = new UniformRangePartition(schema, expected, true);
@@ -271,14 +271,14 @@ public class TestUniformRangePartition {
   @Test
   public void testPartitionForOnePartNum() {
     Schema schema = new Schema()
-      .addColumn("l_returnflag", DataType.STRING)
-      .addColumn("l_linestatus", DataType.STRING);
+      .addColumn("l_returnflag", Type.TEXT)
+      .addColumn("l_linestatus", Type.TEXT);
     Tuple s = new VTuple(2);
-    s.put(0, DatumFactory.createString("A"));
-    s.put(1, DatumFactory.createString("F"));
+    s.put(0, DatumFactory.createText("A"));
+    s.put(1, DatumFactory.createText("F"));
     Tuple e = new VTuple(2);
-    e.put(0, DatumFactory.createString("R"));
-    e.put(1, DatumFactory.createString("O"));
+    e.put(0, DatumFactory.createText("R"));
+    e.put(1, DatumFactory.createText("O"));
     TupleRange expected = new TupleRange(schema, s, e);
     RangePartitionAlgorithm partitioner =
         new UniformRangePartition(schema, expected, true);

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-backend/src/test/java/tajo/engine/planner/global/TestGlobalQueryOptimizer.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/tajo/engine/planner/global/TestGlobalQueryOptimizer.java b/tajo-core/tajo-core-backend/src/test/java/tajo/engine/planner/global/TestGlobalQueryOptimizer.java
index c0d0dd2..465038c 100644
--- a/tajo-core/tajo-core-backend/src/test/java/tajo/engine/planner/global/TestGlobalQueryOptimizer.java
+++ b/tajo-core/tajo-core-backend/src/test/java/tajo/engine/planner/global/TestGlobalQueryOptimizer.java
@@ -31,9 +31,9 @@ import tajo.QueryId;
 import tajo.QueryIdFactory;
 import tajo.TajoTestingCluster;
 import tajo.catalog.*;
-import tajo.catalog.proto.CatalogProtos.DataType;
 import tajo.catalog.proto.CatalogProtos.FunctionType;
 import tajo.catalog.proto.CatalogProtos.StoreType;
+import tajo.common.TajoDataTypes.Type;
 import tajo.conf.TajoConf;
 import tajo.datum.Datum;
 import tajo.datum.DatumFactory;
@@ -70,10 +70,10 @@ public class TestGlobalQueryOptimizer {
     int i, j;
 
     schema = new Schema();
-    schema.addColumn("id", DataType.INT);
-    schema.addColumn("age", DataType.INT);
-    schema.addColumn("name", DataType.STRING);
-    schema.addColumn("salary", DataType.INT);
+    schema.addColumn("id", Type.INT4);
+    schema.addColumn("age", Type.INT4);
+    schema.addColumn("name", Type.TEXT);
+    schema.addColumn("salary", Type.INT4);
 
     TableMeta meta;
 
@@ -81,8 +81,8 @@ public class TestGlobalQueryOptimizer {
     catalog = util.getMiniCatalogCluster().getCatalog();
     StorageManager sm = new StorageManager(util.getConfiguration());
     FunctionDesc funcDesc = new FunctionDesc("sumtest", TestSum.class, FunctionType.GENERAL,
-        new DataType [] {DataType.INT},
-        new DataType [] {DataType.INT});
+        CatalogUtil.newDataTypesWithoutLen(Type.INT4),
+        CatalogUtil.newDataTypesWithoutLen(Type.INT4));
     catalog.registerFunction(funcDesc);
     FileSystem fs = sm.getFileSystem();
 
@@ -98,11 +98,11 @@ public class TestGlobalQueryOptimizer {
     Appender appender;
     Tuple t = new VTuple(4);
     t.put(new Datum[] {
-        DatumFactory.createInt(1), DatumFactory.createInt(32),
-        DatumFactory.createString("h"), DatumFactory.createInt(10)});
+        DatumFactory.createInt4(1), DatumFactory.createInt4(32),
+        DatumFactory.createText("h"), DatumFactory.createInt4(10)});
 
     for (i = 0; i < tbNum; i++) {
-      meta = TCatUtil.newTableMeta((Schema)schema.clone(), StoreType.CSV);
+      meta = CatalogUtil.newTableMeta((Schema) schema.clone(), StoreType.CSV);
       meta.putOption(CSVFile.DELIMITER, ",");
 
       Path dataRoot = sm.getBaseDir();
@@ -119,7 +119,8 @@ public class TestGlobalQueryOptimizer {
       }
       appender.close();
 
-      TableDesc desc = TCatUtil.newTableDesc("table" + i, (TableMeta)meta.clone(), sm.getTablePath("table"+i));
+      TableDesc desc = CatalogUtil
+          .newTableDesc("table" + i, (TableMeta) meta.clone(), sm.getTablePath("table" + i));
       catalog.addTable(desc);
     }
 


[08/11] TAJO-57: Recognize Parser and Catalog Standard SQL data types. (hyunsik)

Posted by hy...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-backend/src/main/java/tajo/engine/eval/CaseWhenEval.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/eval/CaseWhenEval.java b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/eval/CaseWhenEval.java
index 07fec9c..31a1a02 100644
--- a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/eval/CaseWhenEval.java
+++ b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/eval/CaseWhenEval.java
@@ -20,12 +20,12 @@ package tajo.engine.eval;
 
 import com.google.common.collect.Lists;
 import com.google.gson.annotations.Expose;
+import tajo.catalog.CatalogUtil;
 import tajo.catalog.Schema;
-import tajo.catalog.proto.CatalogProtos;
-import tajo.catalog.proto.CatalogProtos.DataType;
+import tajo.common.TajoDataTypes;
+import tajo.common.TajoDataTypes.DataType;
 import tajo.datum.Datum;
 import tajo.datum.DatumFactory;
-import tajo.engine.utils.SchemaUtil;
 import tajo.storage.Tuple;
 
 import java.util.List;
@@ -52,7 +52,7 @@ public class CaseWhenEval extends EvalNode {
   }
 
   @Override
-  public CatalogProtos.DataType[] getValueType() {
+  public DataType [] getValueType() {
     return whens.get(0).getResultExpr().getValueType();
   }
 
@@ -153,7 +153,7 @@ public class CaseWhenEval extends EvalNode {
 
     @Override
     public DataType [] getValueType() {
-      return SchemaUtil.newNoNameSchema(CatalogProtos.DataType.BOOLEAN);
+      return CatalogUtil.newDataTypesWithoutLen(TajoDataTypes.Type.BOOLEAN);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-backend/src/main/java/tajo/engine/eval/ConstEval.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/eval/ConstEval.java b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/eval/ConstEval.java
index 3e61632..a8917bc 100644
--- a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/eval/ConstEval.java
+++ b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/eval/ConstEval.java
@@ -21,7 +21,9 @@ package tajo.engine.eval;
 import com.google.common.base.Objects;
 import com.google.gson.Gson;
 import com.google.gson.annotations.Expose;
-import tajo.catalog.proto.CatalogProtos.DataType;
+import tajo.catalog.CatalogUtil;
+import tajo.common.TajoDataTypes;
+import tajo.common.TajoDataTypes.DataType;
 import tajo.datum.Datum;
 import tajo.engine.json.GsonCreator;
 
@@ -60,18 +62,19 @@ public class ConstEval extends EvalNode implements Comparable<ConstEval>, Clonea
   @Override
 	public DataType [] getValueType() {
 		switch(this.datum.type()) {
-      case CHAR: return new DataType[] {DataType.CHAR};
-      case BOOLEAN: return new DataType[] {DataType.BOOLEAN};
-      case BYTE: return new DataType[] {DataType.BYTE};
-      case BYTES : return new DataType[] {DataType.BYTES};
-      case DOUBLE : return new DataType[] {DataType.DOUBLE};
-      case FLOAT: return new DataType[] {DataType.FLOAT};
-      case INT: return new DataType[] {DataType.INT};
-      case IPv4: return new DataType[] {DataType.IPv4};
-      case LONG: return new DataType[] {DataType.LONG};
-      case SHORT: return new DataType[] {DataType.SHORT};
-      case STRING: return new DataType[] {DataType.STRING};
-      default: return new DataType[] {DataType.ANY};
+      case BOOLEAN: return CatalogUtil.newDataTypesWithoutLen(TajoDataTypes.Type.BOOLEAN);
+      case BIT: return CatalogUtil.newDataTypesWithoutLen(TajoDataTypes.Type.BIT);
+      case CHAR: return CatalogUtil.newDataTypesWithoutLen(TajoDataTypes.Type.CHAR);
+      case INT1: return CatalogUtil.newDataTypesWithoutLen(TajoDataTypes.Type.INT1);
+      case INT2: return CatalogUtil.newDataTypesWithoutLen(TajoDataTypes.Type.INT2);
+      case INT4: return CatalogUtil.newDataTypesWithoutLen(TajoDataTypes.Type.INT4);
+      case INT8: return CatalogUtil.newDataTypesWithoutLen(TajoDataTypes.Type.INT8);
+      case FLOAT4: return CatalogUtil.newDataTypesWithoutLen(TajoDataTypes.Type.FLOAT4);
+      case FLOAT8 : return CatalogUtil.newDataTypesWithoutLen(TajoDataTypes.Type.FLOAT8);
+      case BLOB : return CatalogUtil.newDataTypesWithoutLen(TajoDataTypes.Type.BLOB);
+      case TEXT: return CatalogUtil.newDataTypesWithoutLen(TajoDataTypes.Type.TEXT);
+      case INET4: return CatalogUtil.newDataTypesWithoutLen(TajoDataTypes.Type.INET4);
+      default: return CatalogUtil.newDataTypesWithoutLen(TajoDataTypes.Type.ANY);
 		}
 	}
 

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-backend/src/main/java/tajo/engine/eval/EvalNode.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/eval/EvalNode.java b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/eval/EvalNode.java
index 8c31add..14664b0 100644
--- a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/eval/EvalNode.java
+++ b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/eval/EvalNode.java
@@ -21,7 +21,7 @@ package tajo.engine.eval;
 import com.google.gson.Gson;
 import com.google.gson.annotations.Expose;
 import tajo.catalog.Schema;
-import tajo.catalog.proto.CatalogProtos.DataType;
+import tajo.common.TajoDataTypes.DataType;
 import tajo.datum.Datum;
 import tajo.engine.json.GsonCreator;
 import tajo.storage.Tuple;

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-backend/src/main/java/tajo/engine/eval/EvalTreeUtil.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/eval/EvalTreeUtil.java b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/eval/EvalTreeUtil.java
index e8afb65..be3b607 100644
--- a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/eval/EvalTreeUtil.java
+++ b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/eval/EvalTreeUtil.java
@@ -23,7 +23,7 @@ import com.google.common.collect.Maps;
 import com.google.common.collect.Sets;
 import tajo.catalog.Column;
 import tajo.catalog.Schema;
-import tajo.catalog.proto.CatalogProtos.DataType;
+import tajo.common.TajoDataTypes.DataType;
 import tajo.engine.eval.EvalNode.Type;
 import tajo.engine.parser.QueryBlock.Target;
 import tajo.engine.utils.SchemaUtil;
@@ -131,7 +131,7 @@ public class EvalTreeUtil {
     return schema;
   }
   
-  public static DataType [] getDomainByExpr(Schema inputSchema, EvalNode expr)
+  public static DataType[] getDomainByExpr(Schema inputSchema, EvalNode expr)
       throws InternalException {
     switch (expr.getType()) {
     case AND:      

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-backend/src/main/java/tajo/engine/eval/FieldEval.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/eval/FieldEval.java b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/eval/FieldEval.java
index e258b93..c0d856a 100644
--- a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/eval/FieldEval.java
+++ b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/eval/FieldEval.java
@@ -22,7 +22,7 @@ import com.google.gson.Gson;
 import com.google.gson.annotations.Expose;
 import tajo.catalog.Column;
 import tajo.catalog.Schema;
-import tajo.catalog.proto.CatalogProtos.DataType;
+import tajo.common.TajoDataTypes.DataType;
 import tajo.datum.Datum;
 import tajo.engine.json.GsonCreator;
 import tajo.engine.utils.SchemaUtil;
@@ -79,7 +79,7 @@ public class FieldEval extends EvalNode implements Cloneable {
   }
 
   @Override
-	public DataType [] getValueType() {
+	public DataType[] getValueType() {
 		return SchemaUtil.newNoNameSchema(column.getDataType());
 	}
 	

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-backend/src/main/java/tajo/engine/eval/FuncEval.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/eval/FuncEval.java b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/eval/FuncEval.java
index da7df36..4debee8 100644
--- a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/eval/FuncEval.java
+++ b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/eval/FuncEval.java
@@ -23,7 +23,7 @@ import com.google.gson.Gson;
 import com.google.gson.annotations.Expose;
 import tajo.catalog.FunctionDesc;
 import tajo.catalog.Schema;
-import tajo.catalog.proto.CatalogProtos.DataType;
+import tajo.common.TajoDataTypes.DataType;
 import tajo.datum.Datum;
 import tajo.engine.json.GsonCreator;
 import tajo.storage.Tuple;

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-backend/src/main/java/tajo/engine/eval/IsNullEval.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/eval/IsNullEval.java b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/eval/IsNullEval.java
index a236426..20c2161 100644
--- a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/eval/IsNullEval.java
+++ b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/eval/IsNullEval.java
@@ -19,18 +19,21 @@
 package tajo.engine.eval;
 
 import com.google.gson.annotations.Expose;
+import tajo.catalog.CatalogUtil;
 import tajo.catalog.Column;
 import tajo.catalog.Schema;
-import tajo.catalog.proto.CatalogProtos;
-import tajo.datum.BoolDatum;
+import tajo.common.TajoDataTypes;
+import tajo.common.TajoDataTypes.DataType;
+import tajo.datum.BooleanDatum;
 import tajo.datum.Datum;
 import tajo.datum.DatumFactory;
-import tajo.engine.utils.SchemaUtil;
+import tajo.datum.NullDatum;
 import tajo.storage.Tuple;
 
 public class IsNullEval extends BinaryEval {
   private final static ConstEval NULL_EVAL = new ConstEval(DatumFactory.createNullDatum());
-  private static final CatalogProtos.DataType[] RES_TYPE = SchemaUtil.newNoNameSchema(CatalogProtos.DataType.BOOLEAN);
+  private static final DataType [] RES_TYPE =
+      CatalogUtil.newDataTypesWithoutLen(TajoDataTypes.Type.BOOLEAN);
 
   // persistent variables
   @Expose private boolean isNot;
@@ -49,7 +52,7 @@ public class IsNullEval extends BinaryEval {
   }
 
   @Override
-  public CatalogProtos.DataType[] getValueType() {
+  public DataType[] getValueType() {
     return RES_TYPE;
   }
 
@@ -65,9 +68,9 @@ public class IsNullEval extends BinaryEval {
       fieldId = schema.getColumnId(columnRef.getQualifiedName());
     }
     if (isNot) {
-      isNullCtx.result.setValue(!tuple.get(fieldId).isNull());
+      isNullCtx.result.setValue(!(tuple.get(fieldId) instanceof NullDatum));
     } else {
-      isNullCtx.result.setValue(tuple.get(fieldId).isNull());
+      isNullCtx.result.setValue(tuple.get(fieldId) instanceof NullDatum);
     }
   }
 
@@ -101,7 +104,7 @@ public class IsNullEval extends BinaryEval {
   }
 
   private class IsNullEvalCtx implements EvalContext {
-    BoolDatum result;
+    BooleanDatum result;
 
     IsNullEvalCtx() {
       this.result = DatumFactory.createBool(false);

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-backend/src/main/java/tajo/engine/eval/LikeEval.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/eval/LikeEval.java b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/eval/LikeEval.java
index a963335..57c571a 100644
--- a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/eval/LikeEval.java
+++ b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/eval/LikeEval.java
@@ -19,14 +19,15 @@
 package tajo.engine.eval;
 
 import com.google.gson.annotations.Expose;
+import tajo.catalog.CatalogUtil;
 import tajo.catalog.Column;
 import tajo.catalog.Schema;
-import tajo.catalog.proto.CatalogProtos.DataType;
-import tajo.datum.BoolDatum;
+import tajo.common.TajoDataTypes;
+import tajo.common.TajoDataTypes.DataType;
+import tajo.datum.BooleanDatum;
 import tajo.datum.Datum;
 import tajo.datum.DatumFactory;
-import tajo.datum.StringDatum;
-import tajo.engine.utils.SchemaUtil;
+import tajo.datum.TextDatum;
 import tajo.storage.Tuple;
 
 import java.util.regex.Pattern;
@@ -35,12 +36,13 @@ public class LikeEval extends BinaryEval {
   @Expose private boolean not;
   @Expose private Column column;
   @Expose private String pattern;
-  private static final DataType [] RES_TYPE = SchemaUtil.newNoNameSchema(DataType.BOOLEAN);
+  private static final DataType [] RES_TYPE =
+      CatalogUtil.newDataTypesWithoutLen(TajoDataTypes.Type.BOOLEAN);
 
   // temporal variables
   private Integer fieldId = null;
   private Pattern compiled;
-  private BoolDatum result;
+  private BooleanDatum result;
 
   
   public LikeEval(boolean not, FieldEval field, ConstEval pattern) {
@@ -75,7 +77,7 @@ public class LikeEval extends BinaryEval {
       fieldId = schema.getColumnId(column.getQualifiedName());
       compile(this.pattern);
     }    
-    StringDatum str = tuple.getString(fieldId);
+    TextDatum str = tuple.getString(fieldId);
     if (not) {
       result.setValue(!compiled.matcher(str.asChars()).matches());      
     } else {

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-backend/src/main/java/tajo/engine/eval/NotEval.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/eval/NotEval.java b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/eval/NotEval.java
index aaa1fee..fbef406 100644
--- a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/eval/NotEval.java
+++ b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/eval/NotEval.java
@@ -20,17 +20,18 @@ package tajo.engine.eval;
 
 import com.google.common.base.Preconditions;
 import com.google.gson.annotations.Expose;
+import tajo.catalog.CatalogUtil;
 import tajo.catalog.Schema;
-import tajo.catalog.proto.CatalogProtos.DataType;
+import tajo.common.TajoDataTypes;
+import tajo.common.TajoDataTypes.DataType;
 import tajo.datum.Datum;
 import tajo.datum.DatumFactory;
-import tajo.engine.utils.SchemaUtil;
 import tajo.storage.Tuple;
 
 public class NotEval extends EvalNode implements Cloneable {
   @Expose private EvalNode subExpr;
-  private static final DataType [] RES_TYPE = SchemaUtil
-      .newNoNameSchema(DataType.BOOLEAN);
+  private static final DataType[] RES_TYPE =
+      CatalogUtil.newDataTypesWithoutLen(TajoDataTypes.Type.BOOLEAN);
 
   public NotEval(EvalNode subExpr) {
     super(Type.NOT);

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-backend/src/main/java/tajo/engine/eval/PartialBinaryExpr.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/eval/PartialBinaryExpr.java b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/eval/PartialBinaryExpr.java
index 7e92274..10a3707 100644
--- a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/eval/PartialBinaryExpr.java
+++ b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/eval/PartialBinaryExpr.java
@@ -19,7 +19,7 @@
 package tajo.engine.eval;
 
 import tajo.catalog.Schema;
-import tajo.catalog.proto.CatalogProtos.DataType;
+import tajo.common.TajoDataTypes.DataType;
 import tajo.datum.Datum;
 import tajo.storage.Tuple;
 
@@ -39,7 +39,7 @@ public class PartialBinaryExpr extends EvalNode {
   }
 
   @Override
-  public DataType [] getValueType() {
+  public DataType[] getValueType() {
     return null;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/Country.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/Country.java b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/Country.java
index 8992f98..50593b3 100644
--- a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/Country.java
+++ b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/Country.java
@@ -20,20 +20,21 @@ package tajo.engine.function;
 
 import tajo.catalog.Column;
 import tajo.catalog.function.GeneralFunction;
-import tajo.catalog.proto.CatalogProtos.DataType;
 import tajo.datum.Datum;
-import tajo.datum.StringDatum;
+import tajo.datum.TextDatum;
 import tajo.storage.Tuple;
 import tajo.util.GeoUtil;
 
-public class Country extends GeneralFunction<StringDatum> {
+import static tajo.common.TajoDataTypes.Type.TEXT;
+
+public class Country extends GeneralFunction<TextDatum> {
 
   public Country() {
-    super(new Column[] {new Column("addr", DataType.STRING)});
+    super(new Column[] {new Column("addr", TEXT)});
   }
 
   @Override
   public Datum eval(Tuple params) {
-    return new StringDatum(GeoUtil.getCountryCode(params.get(0).asChars()));
+    return new TextDatum(GeoUtil.getCountryCode(params.get(0).asChars()));
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/InCountry.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/InCountry.java b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/InCountry.java
index a7468fd..c2e86a3 100644
--- a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/InCountry.java
+++ b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/InCountry.java
@@ -20,17 +20,17 @@ package tajo.engine.function;
 
 import tajo.catalog.Column;
 import tajo.catalog.function.GeneralFunction;
-import tajo.catalog.proto.CatalogProtos.DataType;
-import tajo.datum.BoolDatum;
+import tajo.common.TajoDataTypes;
+import tajo.datum.BooleanDatum;
 import tajo.datum.Datum;
 import tajo.storage.Tuple;
 import tajo.util.GeoUtil;
 
-public class InCountry extends GeneralFunction<BoolDatum> {
+public class InCountry extends GeneralFunction<BooleanDatum> {
 
   public InCountry() {
-    super(new Column[] {new Column("addr", DataType.STRING),
-        new Column("code", DataType.STRING)});
+    super(new Column[] {new Column("addr", TajoDataTypes.Type.TEXT),
+        new Column("code", TajoDataTypes.Type.TEXT)});
   }
 
   @Override
@@ -39,6 +39,6 @@ public class InCountry extends GeneralFunction<BoolDatum> {
     String otherCode = params.get(1).asChars();
     String thisCode = GeoUtil.getCountryCode(addr);
 
-    return new BoolDatum(thisCode.equals(otherCode));
+    return new BooleanDatum(thisCode.equals(otherCode));
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/AvgDouble.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/AvgDouble.java b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/AvgDouble.java
index 39d0c28..af18676 100644
--- a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/AvgDouble.java
+++ b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/AvgDouble.java
@@ -18,11 +18,12 @@
 
 package tajo.engine.function.builtin;
 
+import tajo.catalog.CatalogUtil;
 import tajo.catalog.Column;
 import tajo.catalog.function.AggFunction;
 import tajo.catalog.function.FunctionContext;
-import tajo.catalog.proto.CatalogProtos;
-import tajo.catalog.proto.CatalogProtos.DataType;
+import tajo.common.TajoDataTypes.DataType;
+import tajo.common.TajoDataTypes.Type;
 import tajo.datum.ArrayDatum;
 import tajo.datum.Datum;
 import tajo.datum.DatumFactory;
@@ -31,7 +32,7 @@ import tajo.storage.Tuple;
 public class AvgDouble extends AggFunction {
   public AvgDouble() {
     super(new Column[] {
-        new Column("val", CatalogProtos.DataType.DOUBLE)
+        new Column("val", Type.FLOAT8)
     });
   }
 
@@ -45,7 +46,7 @@ public class AvgDouble extends AggFunction {
   @Override
   public void eval(FunctionContext ctx, Tuple params) {
     AvgContext avgCtx = (AvgContext) ctx;
-    avgCtx.sum += params.get(0).asDouble();
+    avgCtx.sum += params.get(0).asFloat8();
     avgCtx.count++;
   }
 
@@ -53,29 +54,29 @@ public class AvgDouble extends AggFunction {
   public void merge(FunctionContext ctx, Tuple part) {
     AvgContext avgCtx = (AvgContext) ctx;
     ArrayDatum array = (ArrayDatum) part.get(0);
-    avgCtx.sum += array.get(0).asDouble();
-    avgCtx.count += array.get(1).asLong();
+    avgCtx.sum += array.get(0).asFloat8();
+    avgCtx.count += array.get(1).asInt8();
   }
 
   @Override
   public Datum getPartialResult(FunctionContext ctx) {
     AvgContext avgCtx = (AvgContext) ctx;
     ArrayDatum part = new ArrayDatum(2);
-    part.put(0, DatumFactory.createDouble(avgCtx.sum));
-    part.put(1, DatumFactory.createLong(avgCtx.count));
+    part.put(0, DatumFactory.createFloat8(avgCtx.sum));
+    part.put(1, DatumFactory.createInt8(avgCtx.count));
 
     return part;
   }
 
   @Override
-  public CatalogProtos.DataType[] getPartialResultType() {
-    return new DataType[] {DataType.DOUBLE,DataType.LONG};
+  public DataType[] getPartialResultType() {
+    return CatalogUtil.newDataTypesWithoutLen(Type.FLOAT8, Type.INT8);
   }
 
   @Override
   public Datum terminate(FunctionContext ctx) {
     AvgContext avgCtx = (AvgContext) ctx;
-    return DatumFactory.createDouble(avgCtx.sum / avgCtx.count);
+    return DatumFactory.createFloat8(avgCtx.sum / avgCtx.count);
   }
 
   private class AvgContext implements FunctionContext {

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/AvgFloat.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/AvgFloat.java b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/AvgFloat.java
index babedfd..d13b0ae 100644
--- a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/AvgFloat.java
+++ b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/AvgFloat.java
@@ -18,21 +18,23 @@
 
 package tajo.engine.function.builtin;
 
+import tajo.catalog.CatalogUtil;
 import tajo.catalog.Column;
 import tajo.catalog.function.AggFunction;
 import tajo.catalog.function.FunctionContext;
-import tajo.catalog.proto.CatalogProtos.DataType;
+import tajo.common.TajoDataTypes.DataType;
+import tajo.common.TajoDataTypes.Type;
 import tajo.datum.ArrayDatum;
 import tajo.datum.Datum;
 import tajo.datum.DatumFactory;
-import tajo.datum.FloatDatum;
+import tajo.datum.Float4Datum;
 import tajo.storage.Tuple;
 
-public class AvgFloat extends AggFunction<FloatDatum> {
+public class AvgFloat extends AggFunction<Float4Datum> {
 
   public AvgFloat() {
     super(new Column[] {
-        new Column("val", DataType.DOUBLE)
+        new Column("val", Type.FLOAT8)
     });
   }
 
@@ -43,7 +45,7 @@ public class AvgFloat extends AggFunction<FloatDatum> {
   @Override
   public void eval(FunctionContext ctx, Tuple params) {
     AvgContext avgCtx = (AvgContext) ctx;
-    avgCtx.sum += params.get(0).asFloat();
+    avgCtx.sum += params.get(0).asFloat4();
     avgCtx.count++;
   }
 
@@ -51,29 +53,29 @@ public class AvgFloat extends AggFunction<FloatDatum> {
   public void merge(FunctionContext ctx, Tuple part) {
     AvgContext avgCtx = (AvgContext) ctx;
     ArrayDatum array = (ArrayDatum) part.get(0);
-    avgCtx.sum += array.get(0).asDouble();
-    avgCtx.count += array.get(1).asLong();
+    avgCtx.sum += array.get(0).asFloat8();
+    avgCtx.count += array.get(1).asInt8();
   }
 
   @Override
   public Datum getPartialResult(FunctionContext ctx) {
     AvgContext avgCtx = (AvgContext) ctx;
     ArrayDatum part = new ArrayDatum(2);
-    part.put(0, DatumFactory.createDouble(avgCtx.sum));
-    part.put(1, DatumFactory.createLong(avgCtx.count));
+    part.put(0, DatumFactory.createFloat8(avgCtx.sum));
+    part.put(1, DatumFactory.createInt8(avgCtx.count));
 
     return part;
   }
 
   @Override
   public DataType[] getPartialResultType() {
-    return new DataType[] {DataType.DOUBLE, DataType.LONG};
+    return CatalogUtil.newDataTypesWithoutLen(Type.FLOAT8, Type.INT8);
   }
 
   @Override
-  public FloatDatum terminate(FunctionContext ctx) {
+  public Float4Datum terminate(FunctionContext ctx) {
     AvgContext avgCtx = (AvgContext) ctx;
-    return DatumFactory.createFloat((float) (avgCtx.sum / avgCtx.count));
+    return DatumFactory.createFloat4((float) (avgCtx.sum / avgCtx.count));
   }
 
   private class AvgContext implements FunctionContext {

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/AvgInt.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/AvgInt.java b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/AvgInt.java
index 2f41cf4..5adb2fb 100644
--- a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/AvgInt.java
+++ b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/AvgInt.java
@@ -18,21 +18,23 @@
 
 package tajo.engine.function.builtin;
 
+import tajo.catalog.CatalogUtil;
 import tajo.catalog.Column;
 import tajo.catalog.function.AggFunction;
 import tajo.catalog.function.FunctionContext;
-import tajo.catalog.proto.CatalogProtos.DataType;
+import tajo.common.TajoDataTypes.DataType;
+import tajo.common.TajoDataTypes.Type;
 import tajo.datum.ArrayDatum;
 import tajo.datum.Datum;
 import tajo.datum.DatumFactory;
-import tajo.datum.FloatDatum;
+import tajo.datum.Float4Datum;
 import tajo.storage.Tuple;
 
-public class AvgInt extends AggFunction<FloatDatum> {
+public class AvgInt extends AggFunction<Float4Datum> {
 
   public AvgInt() {
     super(new Column[] {
-        new Column("val", DataType.DOUBLE)
+        new Column("val", Type.FLOAT8)
     });
   }
 
@@ -43,7 +45,7 @@ public class AvgInt extends AggFunction<FloatDatum> {
   @Override
   public void eval(FunctionContext ctx, Tuple params) {
     AvgContext avgCtx = (AvgContext) ctx;
-    avgCtx.sum += params.get(0).asInt();
+    avgCtx.sum += params.get(0).asInt4();
     avgCtx.count++;
   }
 
@@ -51,29 +53,29 @@ public class AvgInt extends AggFunction<FloatDatum> {
   public void merge(FunctionContext ctx, Tuple part) {
     AvgContext avgCtx = (AvgContext) ctx;
     ArrayDatum array = (ArrayDatum) part.get(0);
-    avgCtx.sum += array.get(0).asLong();
-    avgCtx.count += array.get(1).asLong();
+    avgCtx.sum += array.get(0).asInt8();
+    avgCtx.count += array.get(1).asInt8();
   }
 
   @Override
   public Datum getPartialResult(FunctionContext ctx) {
     AvgContext avgCtx = (AvgContext) ctx;
     ArrayDatum part = new ArrayDatum(2);
-    part.put(0, DatumFactory.createLong(avgCtx.sum));
-    part.put(1, DatumFactory.createLong(avgCtx.count));
+    part.put(0, DatumFactory.createInt8(avgCtx.sum));
+    part.put(1, DatumFactory.createInt8(avgCtx.count));
 
     return part;
   }
 
   @Override
-  public DataType[] getPartialResultType() {
-    return new DataType[] {DataType.LONG, DataType.LONG};
+  public DataType [] getPartialResultType() {
+    return CatalogUtil.newDataTypesWithoutLen(Type.INT8, Type.INT8);
   }
 
   @Override
-  public FloatDatum terminate(FunctionContext ctx) {
+  public Float4Datum terminate(FunctionContext ctx) {
     AvgContext avgCtx = (AvgContext) ctx;
-    return DatumFactory.createFloat((float)avgCtx.sum / avgCtx.count);
+    return DatumFactory.createFloat4((float) avgCtx.sum / avgCtx.count);
   }
 
   private class AvgContext implements FunctionContext {

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/AvgLong.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/AvgLong.java b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/AvgLong.java
index 85aefa9..3e76133 100644
--- a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/AvgLong.java
+++ b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/AvgLong.java
@@ -18,22 +18,23 @@
 
 package tajo.engine.function.builtin;
 
+import tajo.catalog.CatalogUtil;
 import tajo.catalog.Column;
 import tajo.catalog.function.AggFunction;
 import tajo.catalog.function.FunctionContext;
-import tajo.catalog.proto.CatalogProtos;
-import tajo.catalog.proto.CatalogProtos.DataType;
+import tajo.common.TajoDataTypes.DataType;
+import tajo.common.TajoDataTypes.Type;
 import tajo.datum.ArrayDatum;
 import tajo.datum.Datum;
 import tajo.datum.DatumFactory;
-import tajo.datum.DoubleDatum;
+import tajo.datum.Float8Datum;
 import tajo.storage.Tuple;
 
-public class AvgLong extends AggFunction<DoubleDatum> {
+public class AvgLong extends AggFunction<Float8Datum> {
 
   public AvgLong() {
     super(new Column[] {
-        new Column("val", CatalogProtos.DataType.DOUBLE)
+        new Column("val", Type.FLOAT8)
     });
   }
 
@@ -44,7 +45,7 @@ public class AvgLong extends AggFunction<DoubleDatum> {
   @Override
   public void eval(FunctionContext ctx, Tuple params) {
     AvgContext avgCtx = (AvgContext) ctx;
-    avgCtx.sum += params.get(0).asLong();
+    avgCtx.sum += params.get(0).asInt8();
     avgCtx.count++;
   }
 
@@ -52,29 +53,29 @@ public class AvgLong extends AggFunction<DoubleDatum> {
   public void merge(FunctionContext ctx, Tuple part) {
     AvgContext avgCtx = (AvgContext) ctx;
     ArrayDatum array = (ArrayDatum) part.get(0);
-    avgCtx.sum += array.get(0).asLong();
-    avgCtx.count += array.get(1).asLong();
+    avgCtx.sum += array.get(0).asInt8();
+    avgCtx.count += array.get(1).asInt8();
   }
 
   @Override
   public Datum getPartialResult(FunctionContext ctx) {
     AvgContext avgCtx = (AvgContext) ctx;
     ArrayDatum part = new ArrayDatum(2);
-    part.put(0, DatumFactory.createLong(avgCtx.sum));
-    part.put(1, DatumFactory.createLong(avgCtx.count));
+    part.put(0, DatumFactory.createInt8(avgCtx.sum));
+    part.put(1, DatumFactory.createInt8(avgCtx.count));
 
     return part;
   }
 
   @Override
   public DataType[] getPartialResultType() {
-    return new DataType[] {DataType.LONG, DataType.LONG};
+    return CatalogUtil.newDataTypesWithoutLen(Type.INT8, Type.INT8);
   }
 
   @Override
-  public DoubleDatum terminate(FunctionContext ctx) {
+  public Float8Datum terminate(FunctionContext ctx) {
     AvgContext avgCtx = (AvgContext) ctx;
-    return DatumFactory.createDouble((double)avgCtx.sum / avgCtx.count);
+    return DatumFactory.createFloat8((double) avgCtx.sum / avgCtx.count);
   }
 
   private class AvgContext implements FunctionContext {

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/CountRows.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/CountRows.java b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/CountRows.java
index e776de9..14a76da 100644
--- a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/CountRows.java
+++ b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/CountRows.java
@@ -18,13 +18,15 @@
 
 package tajo.engine.function.builtin;
 
+import tajo.catalog.CatalogUtil;
 import tajo.catalog.Column;
 import tajo.catalog.function.AggFunction;
 import tajo.catalog.function.FunctionContext;
-import tajo.catalog.proto.CatalogProtos.DataType;
+import tajo.common.TajoDataTypes.DataType;
+import tajo.common.TajoDataTypes.Type;
 import tajo.datum.Datum;
 import tajo.datum.DatumFactory;
-import tajo.datum.LongDatum;
+import tajo.datum.Int8Datum;
 import tajo.storage.Tuple;
 
 public class CountRows extends AggFunction<Datum> {
@@ -49,22 +51,22 @@ public class CountRows extends AggFunction<Datum> {
 
   @Override
   public void merge(FunctionContext ctx, Tuple part) {
-    ((CountRowContext) ctx).count += part.get(0).asLong();
+    ((CountRowContext) ctx).count += part.get(0).asInt8();
   }
 
   @Override
   public Datum getPartialResult(FunctionContext ctx) {
-    return DatumFactory.createLong(((CountRowContext) ctx).count);
+    return DatumFactory.createInt8(((CountRowContext) ctx).count);
   }
 
   @Override
   public DataType[] getPartialResultType() {
-    return new DataType[] {DataType.LONG};
+    return CatalogUtil.newDataTypesWithoutLen(Type.INT8);
   }
 
   @Override
-  public LongDatum terminate(FunctionContext ctx) {
-    return DatumFactory.createLong(((CountRowContext) ctx).count);
+  public Int8Datum terminate(FunctionContext ctx) {
+    return DatumFactory.createInt8(((CountRowContext) ctx).count);
   }
 
   protected class CountRowContext implements FunctionContext {

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/CountValue.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/CountValue.java b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/CountValue.java
index 4eb18e5..01a7cef 100644
--- a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/CountValue.java
+++ b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/CountValue.java
@@ -20,20 +20,20 @@ package tajo.engine.function.builtin;
 
 import tajo.catalog.Column;
 import tajo.catalog.function.FunctionContext;
-import tajo.catalog.proto.CatalogProtos;
-import tajo.datum.DatumType;
+import tajo.common.TajoDataTypes.Type;
+import tajo.datum.NullDatum;
 import tajo.storage.Tuple;
 
 public final class CountValue extends CountRows {
 
   public CountValue() {
     super(new Column[] {
-        new Column("col", CatalogProtos.DataType.ANY)
+        new Column("col", Type.ANY)
     });
   }
   @Override
   public void eval(FunctionContext ctx, Tuple params) {
-    if (params.get(0).type() != DatumType.NULL) {
+    if (params.get(0) instanceof NullDatum) {
       ((CountRowContext) ctx).count++;
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/Date.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/Date.java b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/Date.java
index 1d5d64c..80b4124 100644
--- a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/Date.java
+++ b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/Date.java
@@ -22,27 +22,28 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import tajo.catalog.Column;
 import tajo.catalog.function.GeneralFunction;
-import tajo.catalog.proto.CatalogProtos.DataType;
 import tajo.datum.Datum;
 import tajo.datum.DatumFactory;
-import tajo.datum.LongDatum;
+import tajo.datum.Int8Datum;
 import tajo.storage.Tuple;
 
 import java.text.ParseException;
 import java.text.SimpleDateFormat;
 
-public class Date extends GeneralFunction<LongDatum> {
+import static tajo.common.TajoDataTypes.Type.TEXT;
+
+public class Date extends GeneralFunction<Int8Datum> {
   private final Log LOG = LogFactory.getLog(Date.class);
   private final static String dateFormat = "dd/MM/yyyy HH:mm:ss";
 
   public Date() {
-    super(new Column[] {new Column("val", DataType.STRING)});
+    super(new Column[] {new Column("val", TEXT)});
   }
 
   @Override
   public Datum eval(Tuple params) {
     try {
-      return DatumFactory.createLong(new SimpleDateFormat(dateFormat)
+      return DatumFactory.createInt8(new SimpleDateFormat(dateFormat)
           .parse(params.get(0).asChars()).getTime());
     } catch (ParseException e) {
       LOG.error(e);

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/MaxDouble.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/MaxDouble.java b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/MaxDouble.java
index 8b4db74..d895b08 100644
--- a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/MaxDouble.java
+++ b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/MaxDouble.java
@@ -18,21 +18,22 @@
 
 package tajo.engine.function.builtin;
 
+import tajo.catalog.CatalogUtil;
 import tajo.catalog.Column;
 import tajo.catalog.function.AggFunction;
 import tajo.catalog.function.FunctionContext;
-import tajo.catalog.proto.CatalogProtos;
-import tajo.catalog.proto.CatalogProtos.DataType;
+import tajo.common.TajoDataTypes.DataType;
+import tajo.common.TajoDataTypes.Type;
 import tajo.datum.Datum;
 import tajo.datum.DatumFactory;
-import tajo.datum.DoubleDatum;
+import tajo.datum.Float8Datum;
 import tajo.storage.Tuple;
 
-public class MaxDouble extends AggFunction<DoubleDatum> {
+public class MaxDouble extends AggFunction<Float8Datum> {
 
   public MaxDouble() {
     super(new Column[] {
-        new Column("val", CatalogProtos.DataType.DOUBLE)
+        new Column("val", Type.FLOAT8)
     });
   }
 
@@ -44,22 +45,22 @@ public class MaxDouble extends AggFunction<DoubleDatum> {
   @Override
   public void eval(FunctionContext ctx, Tuple params) {
     MaxContext maxCtx = (MaxContext) ctx;
-    maxCtx.max = Math.max(maxCtx.max, params.get(0).asDouble());
+    maxCtx.max = Math.max(maxCtx.max, params.get(0).asFloat8());
   }
 
   @Override
   public Datum getPartialResult(FunctionContext ctx) {
-    return DatumFactory.createDouble(((MaxContext)ctx).max);
+    return DatumFactory.createFloat8(((MaxContext) ctx).max);
   }
 
   @Override
   public DataType[] getPartialResultType() {
-    return new DataType[] {DataType.DOUBLE};
+    return CatalogUtil.newDataTypesWithoutLen(Type.FLOAT8);
   }
 
   @Override
-  public DoubleDatum terminate(FunctionContext ctx) {
-    return DatumFactory.createDouble(((MaxContext)ctx).max);
+  public Float8Datum terminate(FunctionContext ctx) {
+    return DatumFactory.createFloat8(((MaxContext) ctx).max);
   }
 
   private class MaxContext implements FunctionContext {

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/MaxFloat.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/MaxFloat.java b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/MaxFloat.java
index fdde18c..0a65904 100644
--- a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/MaxFloat.java
+++ b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/MaxFloat.java
@@ -18,11 +18,12 @@
 
 package tajo.engine.function.builtin;
 
+import tajo.catalog.CatalogUtil;
 import tajo.catalog.Column;
 import tajo.catalog.function.AggFunction;
 import tajo.catalog.function.FunctionContext;
-import tajo.catalog.proto.CatalogProtos;
-import tajo.catalog.proto.CatalogProtos.DataType;
+import tajo.common.TajoDataTypes.DataType;
+import tajo.common.TajoDataTypes.Type;
 import tajo.datum.Datum;
 import tajo.datum.DatumFactory;
 import tajo.storage.Tuple;
@@ -30,7 +31,7 @@ import tajo.storage.Tuple;
 public class MaxFloat extends AggFunction<Datum> {
   public MaxFloat() {
     super(new Column[] {
-        new Column("val", CatalogProtos.DataType.FLOAT)
+        new Column("val", Type.FLOAT8)
     });
   }
 
@@ -42,22 +43,22 @@ public class MaxFloat extends AggFunction<Datum> {
   @Override
   public void eval(FunctionContext ctx, Tuple params) {
     MaxContext maxCtx = (MaxContext) ctx;
-    maxCtx.max = Math.max(maxCtx.max, params.get(0).asFloat());
+    maxCtx.max = Math.max(maxCtx.max, params.get(0).asFloat4());
   }
 
   @Override
   public Datum getPartialResult(FunctionContext ctx) {
-    return DatumFactory.createFloat(((MaxContext)ctx).max);
+    return DatumFactory.createFloat4(((MaxContext) ctx).max);
   }
 
   @Override
   public DataType[] getPartialResultType() {
-    return new DataType[] {DataType.FLOAT};
+    return CatalogUtil.newDataTypesWithoutLen(Type.FLOAT4);
   }
 
   @Override
   public Datum terminate(FunctionContext ctx) {
-    return DatumFactory.createFloat(((MaxContext)ctx).max);
+    return DatumFactory.createFloat4(((MaxContext) ctx).max);
   }
 
   private class MaxContext implements FunctionContext {

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/MaxInt.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/MaxInt.java b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/MaxInt.java
index 25ea8d6..ad96b4f 100644
--- a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/MaxInt.java
+++ b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/MaxInt.java
@@ -18,11 +18,12 @@
 
 package tajo.engine.function.builtin;
 
+import tajo.catalog.CatalogUtil;
 import tajo.catalog.Column;
 import tajo.catalog.function.AggFunction;
 import tajo.catalog.function.FunctionContext;
-import tajo.catalog.proto.CatalogProtos;
-import tajo.catalog.proto.CatalogProtos.DataType;
+import tajo.common.TajoDataTypes.DataType;
+import tajo.common.TajoDataTypes.Type;
 import tajo.datum.Datum;
 import tajo.datum.DatumFactory;
 import tajo.storage.Tuple;
@@ -31,7 +32,7 @@ public class MaxInt extends AggFunction<Datum> {
 
   public MaxInt() {
     super(new Column[] {
-        new Column("val", CatalogProtos.DataType.LONG)
+        new Column("val", Type.INT8)
     });
   }
 
@@ -43,22 +44,22 @@ public class MaxInt extends AggFunction<Datum> {
   @Override
   public void eval(FunctionContext ctx, Tuple params) {
     MaxContext maxCtx = (MaxContext) ctx;
-    maxCtx.max = Math.max(maxCtx.max, params.get(0).asInt());
+    maxCtx.max = Math.max(maxCtx.max, params.get(0).asInt4());
   }
 
   @Override
   public Datum getPartialResult(FunctionContext ctx) {
-    return DatumFactory.createInt(((MaxContext)ctx).max);
+    return DatumFactory.createInt4(((MaxContext) ctx).max);
   }
 
   @Override
-  public DataType[] getPartialResultType() {
-    return new DataType[] {DataType.INT};
+  public DataType [] getPartialResultType() {
+    return CatalogUtil.newDataTypesWithoutLen(Type.INT4);
   }
 
   @Override
   public Datum terminate(FunctionContext ctx) {
-    return DatumFactory.createInt(((MaxContext)ctx).max);
+    return DatumFactory.createInt4(((MaxContext) ctx).max);
   }
 
   private class MaxContext implements FunctionContext {

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/MaxLong.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/MaxLong.java b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/MaxLong.java
index 24c10d5..eddf1c5 100644
--- a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/MaxLong.java
+++ b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/MaxLong.java
@@ -18,20 +18,21 @@
 
 package tajo.engine.function.builtin;
 
+import tajo.catalog.CatalogUtil;
 import tajo.catalog.Column;
 import tajo.catalog.function.AggFunction;
 import tajo.catalog.function.FunctionContext;
-import tajo.catalog.proto.CatalogProtos;
-import tajo.catalog.proto.CatalogProtos.DataType;
+import tajo.common.TajoDataTypes.DataType;
+import tajo.common.TajoDataTypes.Type;
 import tajo.datum.Datum;
 import tajo.datum.DatumFactory;
-import tajo.datum.LongDatum;
+import tajo.datum.Int8Datum;
 import tajo.storage.Tuple;
 
-public class MaxLong extends AggFunction<LongDatum> {
+public class MaxLong extends AggFunction<Int8Datum> {
   public MaxLong() {
     super(new Column[] {
-        new Column("val", CatalogProtos.DataType.LONG)
+        new Column("val", Type.INT8)
     });
   }
 
@@ -43,22 +44,22 @@ public class MaxLong extends AggFunction<LongDatum> {
   @Override
   public void eval(FunctionContext ctx, Tuple params) {
     MaxContext maxCtx = (MaxContext) ctx;
-    maxCtx.max = Math.max(maxCtx.max, params.get(0).asLong());
+    maxCtx.max = Math.max(maxCtx.max, params.get(0).asInt8());
   }
 
   @Override
   public Datum getPartialResult(FunctionContext ctx) {
-    return DatumFactory.createLong(((MaxContext)ctx).max);
+    return DatumFactory.createInt8(((MaxContext) ctx).max);
   }
 
   @Override
-  public DataType[] getPartialResultType() {
-    return new DataType[] {DataType.LONG};
+  public DataType [] getPartialResultType() {
+    return CatalogUtil.newDataTypesWithoutLen(Type.INT8);
   }
 
   @Override
-  public LongDatum terminate(FunctionContext ctx) {
-    return DatumFactory.createLong(((MaxContext)ctx).max);
+  public Int8Datum terminate(FunctionContext ctx) {
+    return DatumFactory.createInt8(((MaxContext) ctx).max);
   }
 
   private class MaxContext implements FunctionContext {

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/MinDouble.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/MinDouble.java b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/MinDouble.java
index 82b4db0..920a83a 100644
--- a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/MinDouble.java
+++ b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/MinDouble.java
@@ -18,11 +18,12 @@
 
 package tajo.engine.function.builtin;
 
+import tajo.catalog.CatalogUtil;
 import tajo.catalog.Column;
 import tajo.catalog.function.AggFunction;
 import tajo.catalog.function.FunctionContext;
-import tajo.catalog.proto.CatalogProtos;
-import tajo.catalog.proto.CatalogProtos.DataType;
+import tajo.common.TajoDataTypes.DataType;
+import tajo.common.TajoDataTypes.Type;
 import tajo.datum.Datum;
 import tajo.datum.DatumFactory;
 import tajo.storage.Tuple;
@@ -31,7 +32,7 @@ public class MinDouble extends AggFunction<Datum> {
 
   public MinDouble() {
     super(new Column[] {
-        new Column("val", CatalogProtos.DataType.DOUBLE)
+        new Column("val", Type.FLOAT8)
     });
   }
 
@@ -43,22 +44,22 @@ public class MinDouble extends AggFunction<Datum> {
   @Override
   public void eval(FunctionContext ctx, Tuple params) {
     MinContext minCtx = (MinContext) ctx;
-    minCtx.min = Math.min(minCtx.min, params.get(0).asDouble());
+    minCtx.min = Math.min(minCtx.min, params.get(0).asFloat8());
   }
 
   @Override
   public Datum getPartialResult(FunctionContext ctx) {
-    return DatumFactory.createDouble(((MinContext)ctx).min);
+    return DatumFactory.createFloat8(((MinContext) ctx).min);
   }
 
   @Override
-  public DataType[] getPartialResultType() {
-    return new DataType[] {DataType.DOUBLE};
+  public DataType [] getPartialResultType() {
+    return CatalogUtil.newDataTypesWithoutLen(Type.FLOAT8);
   }
 
   @Override
   public Datum terminate(FunctionContext ctx) {
-    return DatumFactory.createDouble(((MinContext)ctx).min);
+    return DatumFactory.createFloat8(((MinContext) ctx).min);
   }
 
   private class MinContext implements FunctionContext {

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/MinFloat.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/MinFloat.java b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/MinFloat.java
index 8c6cabd..3ddde2e 100644
--- a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/MinFloat.java
+++ b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/MinFloat.java
@@ -18,20 +18,22 @@
 
 package tajo.engine.function.builtin;
 
+import tajo.catalog.CatalogUtil;
 import tajo.catalog.Column;
 import tajo.catalog.function.AggFunction;
 import tajo.catalog.function.FunctionContext;
-import tajo.catalog.proto.CatalogProtos.DataType;
+import tajo.common.TajoDataTypes.DataType;
+import tajo.common.TajoDataTypes.Type;
 import tajo.datum.Datum;
 import tajo.datum.DatumFactory;
-import tajo.datum.FloatDatum;
+import tajo.datum.Float4Datum;
 import tajo.storage.Tuple;
 
-public class MinFloat extends AggFunction<FloatDatum> {
+public class MinFloat extends AggFunction<Float4Datum> {
 
   public MinFloat() {
     super(new Column[] {
-        new Column("val", DataType.FLOAT)
+        new Column("val", Type.FLOAT4)
     });
   }
 
@@ -43,22 +45,22 @@ public class MinFloat extends AggFunction<FloatDatum> {
   @Override
   public void eval(FunctionContext ctx, Tuple params) {
     MinContext minCtx = (MinContext) ctx;
-    minCtx.min = Math.min(minCtx.min, params.get(0).asFloat());
+    minCtx.min = Math.min(minCtx.min, params.get(0).asFloat4());
   }
 
   @Override
   public Datum getPartialResult(FunctionContext ctx) {
-    return DatumFactory.createFloat(((MinContext) ctx).min);
+    return DatumFactory.createFloat4(((MinContext) ctx).min);
   }
 
   @Override
-  public DataType[] getPartialResultType() {
-    return new DataType[] {DataType.FLOAT};
+  public DataType [] getPartialResultType() {
+    return CatalogUtil.newDataTypesWithoutLen(Type.FLOAT4);
   }
 
   @Override
-  public FloatDatum terminate(FunctionContext ctx) {
-    return DatumFactory.createFloat(((MinContext) ctx).min);
+  public Float4Datum terminate(FunctionContext ctx) {
+    return DatumFactory.createFloat4(((MinContext) ctx).min);
   }
 
   private class MinContext implements FunctionContext {

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/MinInt.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/MinInt.java b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/MinInt.java
index 90f9be8..98300c7 100644
--- a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/MinInt.java
+++ b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/MinInt.java
@@ -18,11 +18,12 @@
 
 package tajo.engine.function.builtin;
 
+import tajo.catalog.CatalogUtil;
 import tajo.catalog.Column;
 import tajo.catalog.function.AggFunction;
 import tajo.catalog.function.FunctionContext;
-import tajo.catalog.proto.CatalogProtos;
-import tajo.catalog.proto.CatalogProtos.DataType;
+import tajo.common.TajoDataTypes.DataType;
+import tajo.common.TajoDataTypes.Type;
 import tajo.datum.Datum;
 import tajo.datum.DatumFactory;
 import tajo.storage.Tuple;
@@ -31,7 +32,7 @@ public class MinInt extends AggFunction<Datum> {
 
   public MinInt() {
     super(new Column[] {
-        new Column("val", CatalogProtos.DataType.LONG)
+        new Column("val", Type.INT8)
     });
   }
 
@@ -43,22 +44,22 @@ public class MinInt extends AggFunction<Datum> {
   @Override
   public void eval(FunctionContext ctx, Tuple params) {
     MinContext minCtx = (MinContext) ctx;
-    minCtx.min = Math.min(minCtx.min, params.get(0).asInt());
+    minCtx.min = Math.min(minCtx.min, params.get(0).asInt4());
   }
 
   @Override
   public Datum getPartialResult(FunctionContext ctx) {
-    return DatumFactory.createInt(((MinContext)ctx).min);
+    return DatumFactory.createInt4(((MinContext) ctx).min);
   }
 
   @Override
   public DataType[] getPartialResultType() {
-    return new DataType[] {DataType.INT};
+    return CatalogUtil.newDataTypesWithoutLen(Type.INT4);
   }
 
   @Override
   public Datum terminate(FunctionContext ctx) {
-    return DatumFactory.createInt(((MinContext)ctx).min);
+    return DatumFactory.createInt4(((MinContext) ctx).min);
   }
 
   private class MinContext implements FunctionContext {

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/MinLong.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/MinLong.java b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/MinLong.java
index 85d484d..8813eb5 100644
--- a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/MinLong.java
+++ b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/MinLong.java
@@ -18,21 +18,22 @@
 
 package tajo.engine.function.builtin;
 
+import tajo.catalog.CatalogUtil;
 import tajo.catalog.Column;
 import tajo.catalog.function.AggFunction;
 import tajo.catalog.function.FunctionContext;
-import tajo.catalog.proto.CatalogProtos;
-import tajo.catalog.proto.CatalogProtos.DataType;
+import tajo.common.TajoDataTypes.DataType;
+import tajo.common.TajoDataTypes.Type;
 import tajo.datum.Datum;
 import tajo.datum.DatumFactory;
-import tajo.datum.LongDatum;
+import tajo.datum.Int8Datum;
 import tajo.storage.Tuple;
 
 public class MinLong extends AggFunction<Datum> {
 
   public MinLong() {
     super(new Column[] {
-        new Column("val", CatalogProtos.DataType.LONG)
+        new Column("val", Type.INT8)
     });
   }
 
@@ -44,22 +45,22 @@ public class MinLong extends AggFunction<Datum> {
   @Override
   public void eval(FunctionContext ctx, Tuple params) {
     MinContext minCtx = (MinContext)ctx;
-    minCtx.min = Math.min(minCtx.min, params.get(0).asLong());
+    minCtx.min = Math.min(minCtx.min, params.get(0).asInt8());
   }
 
   @Override
   public Datum getPartialResult(FunctionContext ctx) {
-    return DatumFactory.createLong(((MinContext)ctx).min);
+    return DatumFactory.createInt8(((MinContext) ctx).min);
   }
 
   @Override
   public DataType[] getPartialResultType() {
-    return new DataType[] {DataType.LONG};
+    return CatalogUtil.newDataTypesWithoutLen(Type.INT8);
   }
 
   @Override
-  public LongDatum terminate(FunctionContext ctx) {
-    return DatumFactory.createLong(((MinContext)ctx).min);
+  public Int8Datum terminate(FunctionContext ctx) {
+    return DatumFactory.createInt8(((MinContext) ctx).min);
   }
 
   private class MinContext implements FunctionContext {

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/MinString.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/MinString.java b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/MinString.java
index 333963e..e3b0b38 100644
--- a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/MinString.java
+++ b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/MinString.java
@@ -18,21 +18,22 @@
 
 package tajo.engine.function.builtin;
 
+import tajo.catalog.CatalogUtil;
 import tajo.catalog.Column;
 import tajo.catalog.function.AggFunction;
 import tajo.catalog.function.FunctionContext;
-import tajo.catalog.proto.CatalogProtos;
-import tajo.catalog.proto.CatalogProtos.DataType;
+import tajo.common.TajoDataTypes.DataType;
+import tajo.common.TajoDataTypes.Type;
 import tajo.datum.Datum;
 import tajo.datum.DatumFactory;
-import tajo.datum.StringDatum;
+import tajo.datum.TextDatum;
 import tajo.storage.Tuple;
 
 public class MinString extends AggFunction<Datum> {
 
   public MinString() {
     super(new Column[] {
-        new Column("val", CatalogProtos.DataType.STRING)
+        new Column("val", Type.TEXT)
     });
   }
 
@@ -53,17 +54,17 @@ public class MinString extends AggFunction<Datum> {
 
   @Override
   public Datum getPartialResult(FunctionContext ctx) {
-    return DatumFactory.createString(((MinContext)ctx).min);
+    return DatumFactory.createText(((MinContext) ctx).min);
   }
 
   @Override
-  public DataType[] getPartialResultType() {
-    return new DataType[] {DataType.STRING};
+  public DataType [] getPartialResultType() {
+    return CatalogUtil.newDataTypesWithoutLen(Type.TEXT);
   }
 
   @Override
-  public StringDatum terminate(FunctionContext ctx) {
-    return DatumFactory.createString(((MinContext)ctx).min);
+  public TextDatum terminate(FunctionContext ctx) {
+    return DatumFactory.createText(((MinContext) ctx).min);
   }
 
   private class MinContext implements FunctionContext {

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/RandomInt.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/RandomInt.java b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/RandomInt.java
index a38583d..7143af9 100644
--- a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/RandomInt.java
+++ b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/RandomInt.java
@@ -20,25 +20,26 @@ package tajo.engine.function.builtin;
 
 import tajo.catalog.Column;
 import tajo.catalog.function.GeneralFunction;
-import tajo.catalog.proto.CatalogProtos;
 import tajo.datum.Datum;
 import tajo.datum.DatumFactory;
 import tajo.storage.Tuple;
 
 import java.util.Random;
 
+import static tajo.common.TajoDataTypes.Type.INT4;
+
 public class RandomInt extends GeneralFunction<Datum> {
   private Random random;
 
   public RandomInt() {
     super(new Column[] {
-        new Column("val", CatalogProtos.DataType.INT)
+        new Column("val", INT4)
     });
     random = new Random(System.nanoTime());
   }
 
   @Override
   public Datum eval(Tuple params) {
-    return DatumFactory.createInt(random.nextInt(params.get(0).asInt()));
+    return DatumFactory.createInt4(random.nextInt(params.get(0).asInt4()));
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/SumDouble.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/SumDouble.java b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/SumDouble.java
index 01754a2..7089fb3 100644
--- a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/SumDouble.java
+++ b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/SumDouble.java
@@ -18,21 +18,22 @@
 
 package tajo.engine.function.builtin;
 
+import tajo.catalog.CatalogUtil;
 import tajo.catalog.Column;
 import tajo.catalog.function.AggFunction;
 import tajo.catalog.function.FunctionContext;
-import tajo.catalog.proto.CatalogProtos;
-import tajo.catalog.proto.CatalogProtos.DataType;
+import tajo.common.TajoDataTypes.DataType;
+import tajo.common.TajoDataTypes.Type;
 import tajo.datum.Datum;
 import tajo.datum.DatumFactory;
-import tajo.datum.DoubleDatum;
+import tajo.datum.Float8Datum;
 import tajo.storage.Tuple;
 
 public class SumDouble extends AggFunction<Datum> {
 
   public SumDouble() {
     super(new Column[] {
-        new Column("val", CatalogProtos.DataType.DOUBLE)
+        new Column("val", Type.FLOAT8)
     });
   }
 
@@ -43,22 +44,22 @@ public class SumDouble extends AggFunction<Datum> {
 
   @Override
   public void eval(FunctionContext ctx, Tuple params) {
-    ((SumContext)ctx).sum += params.get(0).asDouble();
+    ((SumContext)ctx).sum += params.get(0).asFloat8();
   }
 
   @Override
   public Datum getPartialResult(FunctionContext ctx) {
-    return DatumFactory.createDouble(((SumContext)ctx).sum);
+    return DatumFactory.createFloat8(((SumContext) ctx).sum);
   }
 
   @Override
-  public DataType[] getPartialResultType() {
-    return new DataType[] {DataType.DOUBLE};
+  public DataType [] getPartialResultType() {
+    return CatalogUtil.newDataTypesWithoutLen(Type.FLOAT8);
   }
 
   @Override
-  public DoubleDatum terminate(FunctionContext ctx) {
-    return DatumFactory.createDouble(((SumContext)ctx).sum);
+  public Float8Datum terminate(FunctionContext ctx) {
+    return DatumFactory.createFloat8(((SumContext) ctx).sum);
   }
 
   private class SumContext implements FunctionContext {

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/SumFloat.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/SumFloat.java b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/SumFloat.java
index 102679d..8c06c03 100644
--- a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/SumFloat.java
+++ b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/SumFloat.java
@@ -18,11 +18,12 @@
 
 package tajo.engine.function.builtin;
 
+import tajo.catalog.CatalogUtil;
 import tajo.catalog.Column;
 import tajo.catalog.function.AggFunction;
 import tajo.catalog.function.FunctionContext;
-import tajo.catalog.proto.CatalogProtos;
-import tajo.catalog.proto.CatalogProtos.DataType;
+import tajo.common.TajoDataTypes.DataType;
+import tajo.common.TajoDataTypes.Type;
 import tajo.datum.Datum;
 import tajo.datum.DatumFactory;
 import tajo.storage.Tuple;
@@ -30,7 +31,7 @@ import tajo.storage.Tuple;
 public class SumFloat extends AggFunction<Datum> {
   public SumFloat() {
     super(new Column[] {
-        new Column("val", CatalogProtos.DataType.FLOAT)
+        new Column("val", Type.FLOAT4)
     });
   }
 
@@ -41,22 +42,22 @@ public class SumFloat extends AggFunction<Datum> {
 
   @Override
   public void eval(FunctionContext ctx, Tuple params) {
-    ((SumContext)ctx).sum += params.get(0).asFloat();
+    ((SumContext)ctx).sum += params.get(0).asFloat4();
   }
 
   @Override
   public Datum getPartialResult(FunctionContext ctx) {
-    return DatumFactory.createFloat(((SumContext)ctx).sum);
+    return DatumFactory.createFloat4(((SumContext) ctx).sum);
   }
 
   @Override
-  public CatalogProtos.DataType[] getPartialResultType() {
-    return new CatalogProtos.DataType[] {DataType.FLOAT};
+  public DataType [] getPartialResultType() {
+    return CatalogUtil.newDataTypesWithoutLen(Type.FLOAT4);
   }
 
   @Override
   public Datum terminate(FunctionContext ctx) {
-    return DatumFactory.createFloat(((SumContext)ctx).sum);
+    return DatumFactory.createFloat4(((SumContext) ctx).sum);
   }
 
   private class SumContext implements FunctionContext {

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/SumInt.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/SumInt.java b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/SumInt.java
index b180f7a..f80f7e0 100644
--- a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/SumInt.java
+++ b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/SumInt.java
@@ -18,11 +18,12 @@
 
 package tajo.engine.function.builtin;
 
+import tajo.catalog.CatalogUtil;
 import tajo.catalog.Column;
 import tajo.catalog.function.AggFunction;
 import tajo.catalog.function.FunctionContext;
-import tajo.catalog.proto.CatalogProtos;
-import tajo.catalog.proto.CatalogProtos.DataType;
+import tajo.common.TajoDataTypes.DataType;
+import tajo.common.TajoDataTypes.Type;
 import tajo.datum.Datum;
 import tajo.datum.DatumFactory;
 import tajo.storage.Tuple;
@@ -31,7 +32,7 @@ public class SumInt extends AggFunction<Datum> {
 
   public SumInt() {
     super(new Column[] {
-        new Column("val", CatalogProtos.DataType.INT)
+        new Column("val", Type.INT4)
     });
   }
 
@@ -43,22 +44,22 @@ public class SumInt extends AggFunction<Datum> {
   @Override
   public void eval(FunctionContext ctx, Tuple params) {
     SumIntContext sumCtx = (SumIntContext) ctx;
-    sumCtx.sum += params.get(0).asLong();
+    sumCtx.sum += params.get(0).asInt8();
   }
 
   @Override
   public Datum getPartialResult(FunctionContext ctx) {
-    return DatumFactory.createInt(((SumIntContext)ctx).sum);
+    return DatumFactory.createInt4(((SumIntContext) ctx).sum);
   }
 
   @Override
-  public DataType[] getPartialResultType() {
-    return new DataType[] {DataType.INT};
+  public DataType [] getPartialResultType() {
+    return CatalogUtil.newDataTypesWithoutLen(Type.INT4);
   }
 
   @Override
   public Datum terminate(FunctionContext ctx) {
-    return DatumFactory.createInt(((SumIntContext)ctx).sum);
+    return DatumFactory.createInt4(((SumIntContext) ctx).sum);
   }
 
   private class SumIntContext implements FunctionContext {

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/SumLong.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/SumLong.java b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/SumLong.java
index 2e4d3c5..9dcceb2 100644
--- a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/SumLong.java
+++ b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/SumLong.java
@@ -18,21 +18,22 @@
 
 package tajo.engine.function.builtin;
 
+import tajo.catalog.CatalogUtil;
 import tajo.catalog.Column;
 import tajo.catalog.function.AggFunction;
 import tajo.catalog.function.FunctionContext;
-import tajo.catalog.proto.CatalogProtos;
-import tajo.catalog.proto.CatalogProtos.DataType;
+import tajo.common.TajoDataTypes.DataType;
+import tajo.common.TajoDataTypes.Type;
 import tajo.datum.Datum;
 import tajo.datum.DatumFactory;
-import tajo.datum.LongDatum;
+import tajo.datum.Int8Datum;
 import tajo.storage.Tuple;
 
 public class SumLong extends AggFunction<Datum> {
 
   public SumLong() {
     super(new Column[] {
-        new Column("val", CatalogProtos.DataType.LONG)
+        new Column("val", Type.INT8)
     });
   }
 
@@ -43,22 +44,22 @@ public class SumLong extends AggFunction<Datum> {
 
   @Override
   public void eval(FunctionContext ctx, Tuple params) {
-    ((SumContext)ctx).sum += params.get(0).asLong();
+    ((SumContext)ctx).sum += params.get(0).asInt8();
   }
 
   @Override
   public Datum getPartialResult(FunctionContext ctx) {
-    return DatumFactory.createLong(((SumContext)ctx).sum);
+    return DatumFactory.createInt8(((SumContext) ctx).sum);
   }
 
   @Override
-  public DataType[] getPartialResultType() {
-    return new DataType[] {DataType.LONG};
+  public DataType [] getPartialResultType() {
+    return CatalogUtil.newDataTypesWithoutLen(Type.INT8);
   }
 
   @Override
-  public LongDatum terminate(FunctionContext ctx) {
-    return DatumFactory.createLong(((SumContext)ctx).sum);
+  public Int8Datum terminate(FunctionContext ctx) {
+    return DatumFactory.createInt8(((SumContext) ctx).sum);
   }
 
   private class SumContext implements FunctionContext {

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/Today.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/Today.java b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/Today.java
index a4a334e..d9b1a92 100644
--- a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/Today.java
+++ b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/function/builtin/Today.java
@@ -22,10 +22,10 @@ import tajo.catalog.Column;
 import tajo.catalog.function.GeneralFunction;
 import tajo.datum.Datum;
 import tajo.datum.DatumFactory;
-import tajo.datum.LongDatum;
+import tajo.datum.Int8Datum;
 import tajo.storage.Tuple;
 
-public class Today extends GeneralFunction<LongDatum> {
+public class Today extends GeneralFunction<Int8Datum> {
 
   public Today() {
     super(new Column[] {});
@@ -33,6 +33,6 @@ public class Today extends GeneralFunction<LongDatum> {
 
   @Override
   public Datum eval(Tuple params) {
-    return DatumFactory.createLong(System.currentTimeMillis());
+    return DatumFactory.createInt8(System.currentTimeMillis());
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-backend/src/main/java/tajo/engine/json/GsonCreator.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/json/GsonCreator.java b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/json/GsonCreator.java
index 087a122..ebcd5eb 100644
--- a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/json/GsonCreator.java
+++ b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/json/GsonCreator.java
@@ -32,6 +32,7 @@ import tajo.catalog.function.GeneralFunction;
 import tajo.catalog.json.FunctionAdapter;
 import tajo.catalog.json.TableDescAdapter;
 import tajo.catalog.json.TableMetaAdapter;
+import tajo.common.TajoDataTypes.DataType;
 import tajo.datum.Datum;
 import tajo.datum.json.DatumAdapter;
 import tajo.engine.eval.EvalNode;
@@ -58,6 +59,7 @@ public class GsonCreator {
       builder.registerTypeAdapter(GeneralFunction.class, new FunctionAdapter());
       builder.registerTypeAdapter(AggFunction.class, new FunctionAdapter());
 			builder.registerTypeAdapter(Datum.class, new DatumAdapter());
+      builder.registerTypeAdapter(DataType.class, new DataTypeAdapter());
 		}
 	}
 

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-backend/src/main/java/tajo/engine/parser/EvalTreeBin.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/parser/EvalTreeBin.java b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/parser/EvalTreeBin.java
deleted file mode 100644
index 45556d3..0000000
--- a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/parser/EvalTreeBin.java
+++ /dev/null
@@ -1,31 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package tajo.engine.parser;
-
-public class EvalTreeBin {
-  private byte [] bytecode;
-  
-  public EvalTreeBin(final byte [] bytecode) {
-    this.bytecode = bytecode;
-  }
-  
-  public final byte [] getBinary() {
-    return bytecode;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-backend/src/main/java/tajo/engine/parser/ParseUtil.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/parser/ParseUtil.java b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/parser/ParseUtil.java
index b8f0b97..66e7a2e 100644
--- a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/parser/ParseUtil.java
+++ b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/parser/ParseUtil.java
@@ -21,55 +21,11 @@ package tajo.engine.parser;
 import org.antlr.runtime.tree.Tree;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import tajo.catalog.proto.CatalogProtos.DataType;
-import tajo.catalog.proto.CatalogProtos.IndexMethod;
 import tajo.engine.eval.EvalNode;
 import tajo.engine.eval.InvalidEvalException;
 
 public class ParseUtil {
   private static final Log LOG = LogFactory.getLog(ParseUtil.class);
-  
-  public static DataType getDataTypeFromEnum(final String typeStr) {
-    if (typeStr.equals(DataType.BOOLEAN.toString())) {
-      return DataType.BOOLEAN;
-    } else if (typeStr.equals(DataType.BYTE.toString())) {
-      return DataType.BYTE;
-    } else if (typeStr.equals(DataType.SHORT.toString())) {
-      return DataType.SHORT;
-    } else if (typeStr.equals(DataType.INT.toString())) {
-      return DataType.INT;
-    } else if (typeStr.equals(DataType.LONG.toString())) {
-      return DataType.LONG;
-    } else if (typeStr.equals(DataType.FLOAT.toString())) {
-      return DataType.FLOAT;
-    } else if (typeStr.equals(DataType.DOUBLE.toString())) {
-      return DataType.DOUBLE;
-    } else if (typeStr.equals(DataType.STRING.toString())) {
-      return DataType.STRING;
-    } else if (typeStr.equals(DataType.IPv4.toString())) {
-      return DataType.IPv4;
-    } else if (typeStr.equals(DataType.IPv6.toString())) {
-      return DataType.IPv6;
-    } else if (typeStr.equals(DataType.BYTES.toString())) {
-      return DataType.BYTES;
-    } else {
-      LOG.error("Cannot find a matched type aginst from '"
-          + typeStr + "'");
-      // TODO - needs exception handling
-      return null;
-    }
-  }
-  
-  public static IndexMethod getIndexMethod(final String typeStr) {
-    if (typeStr.equals(IndexMethod.TWO_LEVEL_BIN_TREE.toString())) {
-      return IndexMethod.TWO_LEVEL_BIN_TREE;
-    } else {
-      LOG.error("Cannot find a matched type aginst from '"
-          + typeStr + "'");
-      // TODO - needs exception handling
-      return null;
-    }
-  }
 
   public static boolean isConstant(final Tree tree) {
     switch (tree.getType()) {

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-backend/src/main/java/tajo/engine/parser/QueryAnalyzer.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/parser/QueryAnalyzer.java b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/parser/QueryAnalyzer.java
index 8ee1ad6..36d00d3 100644
--- a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/parser/QueryAnalyzer.java
+++ b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/parser/QueryAnalyzer.java
@@ -32,10 +32,12 @@ import tajo.catalog.*;
 import tajo.catalog.exception.NoSuchTableException;
 import tajo.catalog.function.AggFunction;
 import tajo.catalog.function.GeneralFunction;
-import tajo.catalog.proto.CatalogProtos.DataType;
 import tajo.catalog.proto.CatalogProtos.FunctionType;
 import tajo.catalog.proto.CatalogProtos.IndexMethod;
 import tajo.catalog.proto.CatalogProtos.StoreType;
+import tajo.common.TajoDataTypes.DataType;
+import tajo.common.TajoDataTypes.Type;
+import tajo.common.exception.NotImplementedException;
 import tajo.datum.DatumFactory;
 import tajo.engine.eval.*;
 import tajo.engine.parser.QueryBlock.*;
@@ -230,18 +232,53 @@ public final class QueryAnalyzer {
 
   private Schema parseCreateTableDef(final CommonTree ast) {
     Schema tableDef = new Schema();
-    DataType type;
+    Type type;
     for (int i = 0; i < ast.getChildCount(); i++) {
-      switch(ast.getChild(i).getChild(1).getType()) {
-        case NQLParser.BOOL: type = DataType.BOOLEAN; break;
-        case NQLParser.BYTE: type = DataType.BYTE; break;
-        case NQLParser.INT: type = DataType.INT; break;
-        case NQLParser.LONG: type = DataType.LONG; break;
-        case NQLParser.FLOAT: type = DataType.FLOAT; break;
-        case NQLParser.DOUBLE: type = DataType.DOUBLE; break;
-        case NQLParser.TEXT: type = DataType.STRING; break;
-        case NQLParser.BYTES: type = DataType.BYTES; break;
-        case NQLParser.IPv4: type = DataType.IPv4; break;
+      Tree child = ast.getChild(i).getChild(1);
+      switch(child.getType()) {
+        case NQLParser.BOOLEAN: type = Type.BOOLEAN; break;
+        case NQLParser.BIT: type = Type.BIT; break;
+
+        case NQLParser.INT1:
+        case NQLParser.INT2:
+          type = Type.INT2;
+          break;
+
+        case NQLParser.INT4: type = Type.INT4; break;
+        case NQLParser.INT8: type = Type.INT8; break;
+        case NQLParser.FLOAT4: type = Type.FLOAT4; break;
+        case NQLParser.FLOAT:
+          if (child.getChildCount() > 0) {
+            int length = Integer.valueOf(child.getChild(0).getText());
+            if (length < 1 || length > 53) {
+              throw new InvalidQueryException("ERROR: floating point precision " + length + " is out of range");
+            }
+            if (length > 25) {
+              type = Type.FLOAT8;
+            } else {
+              type = Type.FLOAT4;
+            }
+          } else { // no given length
+            type = Type.FLOAT8;
+          }
+          break;
+        case NQLParser.FLOAT8:
+          type = Type.FLOAT8; break;
+        case NQLParser.TEXT: type = Type.TEXT; break;
+        case NQLParser.BLOB: type = Type.BLOB; break;
+        case NQLParser.INET4: type = Type.INET4;
+          break;
+
+        case NQLParser.CHAR:
+        case NQLParser.NCHAR:
+        case NQLParser.NUMERIC:
+        case NQLParser.VARCHAR:
+        case NQLParser.NVARCHAR:
+        case NQLParser.BINARY:
+        case NQLParser.VARBINARY:
+          throw new NotImplementedException("ERROR: " + child.toString() +
+              " type is not supported yet");
+
         default: throw new InvalidQueryException(ast.toStringTree());
       }
 
@@ -782,7 +819,7 @@ public final class QueryAnalyzer {
 
     if (evalNode instanceof ConstEval) {
       ConstEval fetchFirst = (ConstEval) evalNode;
-      LimitClause limitClause = new LimitClause(fetchFirst.getValue().asLong());
+      LimitClause limitClause = new LimitClause(fetchFirst.getValue().asInt8());
       return limitClause;
     }
 
@@ -970,15 +1007,15 @@ public final class QueryAnalyzer {
 
       // constants
       case NQLParser.DIGIT:
-        return new ConstEval(DatumFactory.createInt(
+        return new ConstEval(DatumFactory.createInt4(
             Integer.valueOf(ast.getText())));
 
-      case NQLParser.REAL:
-        return new ConstEval(DatumFactory.createDouble(
-            Double.valueOf(ast.getText())));
+      case NQLParser.REAL_NUMBER:
+        return new ConstEval(DatumFactory.createFloat4(
+            Float.valueOf(ast.getText())));
 
       case NQLParser.STRING:
-        return new ConstEval(DatumFactory.createString(ast.getText()));
+        return new ConstEval(DatumFactory.createText(ast.getText()));
 
       // unary expression
       case NQLParser.NOT:
@@ -1025,8 +1062,8 @@ public final class QueryAnalyzer {
           paramTypes[i] = givenArgs[i].getValueType()[0];
         }
         if (!catalog.containFunction(signature, paramTypes)) {
-          throw new UndefinedFunctionException(TCatUtil.
-              getCanonicalName(signature, paramTypes));
+          throw new UndefinedFunctionException(
+              CatalogUtil.getCanonicalName(signature, paramTypes));
         }
         FunctionDesc funcDesc = catalog.getFunction(signature, paramTypes);
 
@@ -1050,7 +1087,8 @@ public final class QueryAnalyzer {
         // Getting the first argument
         EvalNode colRef = createEvalTree(context, tree, ast.getChild(0));
 
-        FunctionDesc countVals = catalog.getFunction("count", DataType.ANY);
+        FunctionDesc countVals = catalog.getFunction("count",
+            CatalogUtil.newDataTypeWithoutLen(Type.ANY));
         tree.setAggregation();
         try {
           return new AggFuncCallEval(countVals, (AggFunction) countVals.newInstance(),
@@ -1151,13 +1189,13 @@ public final class QueryAnalyzer {
   public EvalNode parseDigitByTypeInfer(final PlanningContext context,
                                         final QueryBlock block, final Tree tree,
                                         DataType type) {
-    switch (type) {
-      case SHORT:
-        return new ConstEval(DatumFactory.createShort(tree.getText()));
-      case INT:
-        return new ConstEval(DatumFactory.createInt(tree.getText()));
-      case LONG:
-        return new ConstEval(DatumFactory.createLong(tree.getText()));
+    switch (type.getType()) {
+      case INT2:
+        return new ConstEval(DatumFactory.createInt2(tree.getText()));
+      case INT4:
+        return new ConstEval(DatumFactory.createInt4(tree.getText()));
+      case INT8 :
+        return new ConstEval(DatumFactory.createInt8(tree.getText()));
       default: return createEvalTree(context, block, tree);
     }
   }
@@ -1165,11 +1203,11 @@ public final class QueryAnalyzer {
   private EvalNode parseRealByTypeInfer(final PlanningContext context,
                                         final QueryBlock block, final Tree tree,
                                         DataType type) {
-    switch (type) {
-      case FLOAT:
-        return new ConstEval(DatumFactory.createFloat(tree.getText()));
-      case DOUBLE:
-        return new ConstEval(DatumFactory.createDouble(tree.getText()));
+    switch (type.getType()) {
+      case FLOAT4:
+        return new ConstEval(DatumFactory.createFloat4(tree.getText()));
+      case FLOAT8:
+        return new ConstEval(DatumFactory.createFloat8(tree.getText()));
       default: return createEvalTree(context, block, tree);
     }
   }
@@ -1178,11 +1216,11 @@ public final class QueryAnalyzer {
                                           final QueryBlock block,
                                           final Tree tree,
                                           DataType type) {
-    switch (type) {
+    switch (type.getType()) {
       case CHAR:
         return new ConstEval(DatumFactory.createChar(tree.getText().charAt(0)));
-      case STRING:
-        return new ConstEval(DatumFactory.createString(tree.getText()));
+      case TEXT:
+        return new ConstEval(DatumFactory.createText(tree.getText()));
       default: return createEvalTree(context, block, tree);
     }
   }
@@ -1214,7 +1252,7 @@ public final class QueryAnalyzer {
               exprs[fieldId].getValueType()[0]);
           break;
 
-        case NQLParser.REAL:
+        case NQLParser.REAL_NUMBER:
           exprs[constId] = parseRealByTypeInfer(context, block, constAst,
               exprs[fieldId].getValueType()[0]);
           break;

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-backend/src/main/java/tajo/engine/parser/QueryBlock.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/parser/QueryBlock.java b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/parser/QueryBlock.java
index 86b37d6..e2851a0 100644
--- a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/parser/QueryBlock.java
+++ b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/parser/QueryBlock.java
@@ -27,6 +27,7 @@ import tajo.catalog.Schema;
 import tajo.catalog.SortSpec;
 import tajo.catalog.TableDesc;
 import tajo.catalog.proto.CatalogProtos;
+import tajo.common.TajoDataTypes.Type;
 import tajo.engine.eval.EvalNode;
 import tajo.engine.json.GsonCreator;
 import tajo.engine.planner.JoinType;
@@ -399,7 +400,7 @@ public class QueryBlock extends ParseTree {
       this.eval = eval;
       if (eval.getType() == EvalNode.Type.AGG_FUNCTION &&
           eval.getValueType().length > 1) { // hack for partial result
-        this.column = new Column(eval.getName(), CatalogProtos.DataType.ARRAY);
+        this.column = new Column(eval.getName(), Type.ARRAY);
       } else {
         this.column = new Column(eval.getName(), eval.getValueType()[0]);
       }

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-backend/src/main/java/tajo/engine/planner/LogicalPlanner.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/planner/LogicalPlanner.java b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/planner/LogicalPlanner.java
index 19f22e2..91e5cdc 100644
--- a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/planner/LogicalPlanner.java
+++ b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/planner/LogicalPlanner.java
@@ -27,7 +27,7 @@ import tajo.catalog.CatalogService;
 import tajo.catalog.Column;
 import tajo.catalog.Schema;
 import tajo.catalog.proto.CatalogProtos;
-import tajo.catalog.proto.CatalogProtos.DataType;
+import tajo.common.TajoDataTypes.DataType;
 import tajo.engine.eval.*;
 import tajo.engine.parser.*;
 import tajo.engine.parser.QueryBlock.*;

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-backend/src/main/java/tajo/engine/planner/PlannerUtil.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/planner/PlannerUtil.java b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/planner/PlannerUtil.java
index aeb44b0..edf2e4b 100644
--- a/tajo-core/tajo-core-backend/src/main/java/tajo/engine/planner/PlannerUtil.java
+++ b/tajo-core/tajo-core-backend/src/main/java/tajo/engine/planner/PlannerUtil.java
@@ -23,10 +23,12 @@ import com.google.common.collect.Lists;
 import com.google.common.collect.Sets;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import tajo.catalog.CatalogUtil;
 import tajo.catalog.Column;
 import tajo.catalog.Schema;
 import tajo.catalog.SortSpec;
-import tajo.catalog.proto.CatalogProtos;
+import tajo.common.TajoDataTypes.DataType;
+import tajo.common.TajoDataTypes.Type;
 import tajo.engine.eval.*;
 import tajo.engine.parser.QueryBlock;
 import tajo.engine.planner.logical.*;
@@ -196,7 +198,7 @@ public class PlannerUtil {
             }
             if (func.getValueType().length > 1) { // hack for partial result
               secondFunc.setArgs(new EvalNode[] {new FieldEval(
-                  new Column("column_"+firstTargetId, CatalogProtos.DataType.ARRAY))});
+                  new Column("column_"+firstTargetId, Type.ARRAY))});
             } else {
               secondFunc.setArgs(new EvalNode [] {new FieldEval(
                   new Column("column_"+firstTargetId, newTarget.getEvalTree().getValueType()[0]))});
@@ -586,9 +588,9 @@ public class PlannerUtil {
   public static Schema targetToSchema(QueryBlock.Target[] targets) {
     Schema schema = new Schema();
     for(QueryBlock.Target t : targets) {
-      CatalogProtos.DataType type;
+      DataType type;
       if (t.getEvalTree().getValueType().length > 1) {
-        type = CatalogProtos.DataType.ARRAY;
+        type = CatalogUtil.newDataTypeWithoutLen(Type.ARRAY);
       } else {
         type = t.getEvalTree().getValueType()[0];
       }


[04/11] TAJO-57: Recognize Parser and Catalog Standard SQL data types. (hyunsik)

Posted by hy...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-backend/src/test/java/tajo/worker/TestRangeRetrieverHandler.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/tajo/worker/TestRangeRetrieverHandler.java b/tajo-core/tajo-core-backend/src/test/java/tajo/worker/TestRangeRetrieverHandler.java
index f32f756..1caa8de 100644
--- a/tajo-core/tajo-core-backend/src/test/java/tajo/worker/TestRangeRetrieverHandler.java
+++ b/tajo-core/tajo-core-backend/src/test/java/tajo/worker/TestRangeRetrieverHandler.java
@@ -30,8 +30,8 @@ import tajo.QueryIdFactory;
 import tajo.TajoTestingCluster;
 import tajo.TaskAttemptContext;
 import tajo.catalog.*;
-import tajo.catalog.proto.CatalogProtos.DataType;
 import tajo.catalog.proto.CatalogProtos.StoreType;
+import tajo.common.TajoDataTypes.Type;
 import tajo.conf.TajoConf;
 import tajo.datum.Datum;
 import tajo.datum.DatumFactory;
@@ -84,8 +84,8 @@ public class TestRangeRetrieverHandler {
     planner = new LogicalPlanner(catalog);
 
     schema = new Schema();
-    schema.addColumn("empId", DataType.INT);
-    schema.addColumn("age", DataType.INT);
+    schema.addColumn("empId", Type.INT4);
+    schema.addColumn("age", Type.INT4);
   }
 
   @After
@@ -103,7 +103,7 @@ public class TestRangeRetrieverHandler {
     Tuple firstTuple = null;
     Tuple lastTuple;
 
-    TableMeta employeeMeta = TCatUtil.newTableMeta(schema, StoreType.CSV);
+    TableMeta employeeMeta = CatalogUtil.newTableMeta(schema, StoreType.CSV);
 
     Path tableDir = StorageUtil.concatPath(testDir, "testGet", "table.csv");
     fs.mkdirs(tableDir.getParent());
@@ -114,8 +114,8 @@ public class TestRangeRetrieverHandler {
     for (int i = 0; i < TEST_TUPLE; i++) {
       tuple.put(
           new Datum[] {
-              DatumFactory.createInt(i),
-              DatumFactory.createInt(i+5)
+              DatumFactory.createInt4(i),
+              DatumFactory.createInt4(i + 5)
           });
       appender.addTuple(tuple);
 
@@ -173,12 +173,12 @@ public class TestRangeRetrieverHandler {
 
     Tuple keytuple = new VTuple(2);
     for(int i = 1 ; i < TEST_TUPLE ; i ++) {
-      keytuple.put(0, DatumFactory.createInt(i));
-      keytuple.put(1, DatumFactory.createInt(i + 5));
+      keytuple.put(0, DatumFactory.createInt4(i));
+      keytuple.put(1, DatumFactory.createInt4(i + 5));
       long offsets = reader.find(keytuple);
       scanner.seek(offsets);
       tuple = scanner.next();
-      assertTrue("[seek check " + (i) + " ]" , i == tuple.get(0).asInt());
+      assertTrue("[seek check " + (i) + " ]" , i == tuple.get(0).asInt4());
       //assertTrue("[seek check " + (i) + " ]" , ("name_" + i).equals(tuple.get(1).asChars()));
     }
 
@@ -216,7 +216,7 @@ public class TestRangeRetrieverHandler {
     Tuple firstTuple = null;
     Tuple lastTuple;
 
-    TableMeta meta = TCatUtil.newTableMeta(schema, StoreType.CSV);
+    TableMeta meta = CatalogUtil.newTableMeta(schema, StoreType.CSV);
     Path tablePath = StorageUtil.concatPath(testDir, "testGetFromDescendingOrder", "table.csv");
     fs.mkdirs(tablePath.getParent());
     Appender appender = sm.getAppender(conf, meta, tablePath);
@@ -225,8 +225,8 @@ public class TestRangeRetrieverHandler {
     for (int i = (TEST_TUPLE - 1); i >= 0 ; i--) {
       tuple.put(
           new Datum[] {
-              DatumFactory.createInt(i),
-              DatumFactory.createInt(i+5)
+              DatumFactory.createInt4(i),
+              DatumFactory.createInt4(i + 5)
           });
       appender.addTuple(tuple);
 
@@ -283,12 +283,12 @@ public class TestRangeRetrieverHandler {
 
     Tuple keytuple = new VTuple(2);
     for(int i = (TEST_TUPLE - 1) ; i >= 0; i --) {
-      keytuple.put(0, DatumFactory.createInt(i));
-      keytuple.put(1, DatumFactory.createInt(i + 5));
+      keytuple.put(0, DatumFactory.createInt4(i));
+      keytuple.put(1, DatumFactory.createInt4(i + 5));
       long offsets = reader.find(keytuple);
       scanner.seek(offsets);
       tuple = scanner.next();
-      assertTrue("[seek check " + (i) + " ]" , i == tuple.get(0).asInt());
+      assertTrue("[seek check " + (i) + " ]" , i == tuple.get(0).asInt4());
     }
 
     TupleRange totalRange = new TupleRange(keySchema, lastTuple, firstTuple);

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-storage/src/main/java/tajo/storage/CSVFile.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/CSVFile.java b/tajo-core/tajo-core-storage/src/main/java/tajo/storage/CSVFile.java
index 1311f7f..e0b3119 100644
--- a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/CSVFile.java
+++ b/tajo-core/tajo-core-storage/src/main/java/tajo/storage/CSVFile.java
@@ -30,10 +30,7 @@ import tajo.catalog.Column;
 import tajo.catalog.Schema;
 import tajo.catalog.TableMeta;
 import tajo.catalog.statistics.TableStat;
-import tajo.datum.ArrayDatum;
-import tajo.datum.Datum;
-import tajo.datum.DatumFactory;
-import tajo.datum.DatumType;
+import tajo.datum.*;
 import tajo.storage.exception.AlreadyExistsStorageException;
 import tajo.storage.json.GsonCreator;
 
@@ -94,49 +91,50 @@ public class CSVFile {
         if (enabledStats) {
           stats.analyzeField(i, datum);
         }
-        if (datum.type() == DatumType.NULL) {
+        if (datum instanceof NullDatum) {
         } else {
           col = schema.getColumn(i);
-          switch (col.getDataType()) {
+          switch (col.getDataType().getType()) {
           case BOOLEAN:
             sb.append(tuple.getBoolean(i));
             break;
-          case BYTE:
+          case BIT:
             sb.append(new String(Base64.encodeBase64(tuple.getByte(i)
                 .asByteArray(), false)));
             break;
-          case BYTES:
+          case BLOB:
             sb.append(new String(Base64.encodeBase64(tuple.getBytes(i)
                 .asByteArray(), false)));
             break;
           case CHAR:
             sb.append(tuple.getChar(i));
             break;
-          case STRING:
-            sb.append(tuple.getString(i));
+//          case STRING:
+//            sb.append(tuple.getString(i));
+//            break;
+          case TEXT:
+            TextDatum td = tuple.getText(i);
+            sb.append(td.toString());
             break;
-          case STRING2:
-            sb.append(tuple.getString2(i));
-            break;
-          case SHORT:
+          case INT2:
             sb.append(tuple.getShort(i));
             break;
-          case INT:
+          case INT4:
             sb.append(tuple.getInt(i));
             break;
-          case LONG:
+          case INT8:
             sb.append(tuple.getLong(i));
             break;
-          case FLOAT:
+          case FLOAT4:
             sb.append(tuple.getFloat(i));
             break;
-          case DOUBLE:
+          case FLOAT8:
             sb.append(tuple.getDouble(i));
             break;
-          case IPv4:
+          case INET4:
             sb.append(tuple.getIPv4(i));
             break;
-          case IPv6:
+          case INET6:
             sb.append(tuple.getIPv6(i));
           case ARRAY:
             /*
@@ -356,44 +354,44 @@ public class CSVFile {
             if (cell.equals("")) {
               tuple.put(i, DatumFactory.createNullDatum());
             } else {
-              switch (field.getDataType()) {
+              switch (field.getDataType().getType()) {
               case BOOLEAN:
                 tuple.put(i, DatumFactory.createBool(cell));
                 break;
-              case BYTE:
+              case BIT:
                 tuple.put(i,
-                    DatumFactory.createByte(Base64.decodeBase64(cell)[0]));
+                    DatumFactory.createBit(Base64.decodeBase64(cell)[0]));
                 break;
               case CHAR:
                 tuple.put(i, DatumFactory.createChar(cell.charAt(0)));
                 break;
-              case BYTES:
+              case BLOB:
                 tuple.put(i,
-                    DatumFactory.createBytes(Base64.decodeBase64(cell)));
-                break;
-              case SHORT:
-                tuple.put(i, DatumFactory.createShort(cell));
+                    DatumFactory.createBlob(Base64.decodeBase64(cell)));
                 break;
-              case INT:
-                tuple.put(i, DatumFactory.createInt(cell));
+              case INT2:
+                tuple.put(i, DatumFactory.createInt2(cell));
                 break;
-              case LONG:
-                tuple.put(i, DatumFactory.createLong(cell));
+              case INT4:
+                tuple.put(i, DatumFactory.createInt4(cell));
                 break;
-              case FLOAT:
-                tuple.put(i, DatumFactory.createFloat(cell));
+              case INT8:
+                tuple.put(i, DatumFactory.createInt8(cell));
                 break;
-              case DOUBLE:
-                tuple.put(i, DatumFactory.createDouble(cell));
+              case FLOAT4:
+                tuple.put(i, DatumFactory.createFloat4(cell));
                 break;
-              case STRING:
-                tuple.put(i, DatumFactory.createString(cell));
+              case FLOAT8:
+                tuple.put(i, DatumFactory.createFloat8(cell));
                 break;
-              case STRING2:
-                tuple.put(i, DatumFactory.createString2(cell));
+//              case STRING:
+//                tuple.put(i, DatumFactory.createText(cell));
+//                break;
+              case TEXT:
+                tuple.put(i, DatumFactory.createText(cell));
                 break;
-              case IPv4:
-                tuple.put(i, DatumFactory.createIPv4(cell));
+              case INET4:
+                tuple.put(i, DatumFactory.createInet4(cell));
                 break;
               case ARRAY:
                 Datum data = GsonCreator.getInstance().fromJson(cell,

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-storage/src/main/java/tajo/storage/Fragment.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/Fragment.java b/tajo-core/tajo-core-storage/src/main/java/tajo/storage/Fragment.java
index 1122b7b..18a9311 100644
--- a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/Fragment.java
+++ b/tajo-core/tajo-core-storage/src/main/java/tajo/storage/Fragment.java
@@ -59,7 +59,7 @@ public class Fragment implements TableDesc, Comparable<Fragment>, SchemaObject {
       long length, String [] dataLocations) {
     this();
     TableMeta newMeta = new TableMetaImpl(meta.getProto());
-    SchemaProto newSchemaProto = TCatUtil.getQualfiedSchema(fragmentId, meta
+    SchemaProto newSchemaProto = CatalogUtil.getQualfiedSchema(fragmentId, meta
         .getSchema().getProto());
     newMeta.setSchema(new Schema(newSchemaProto));
     this.set(fragmentId, path, newMeta, start, length);

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-storage/src/main/java/tajo/storage/FrameTuple.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/FrameTuple.java b/tajo-core/tajo-core-storage/src/main/java/tajo/storage/FrameTuple.java
index 946a799..916399e 100644
--- a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/FrameTuple.java
+++ b/tajo-core/tajo-core-storage/src/main/java/tajo/storage/FrameTuple.java
@@ -72,7 +72,7 @@ public class FrameTuple implements Tuple {
 
   @Override
   public boolean isNull(int fieldid) {
-    return get(fieldid).type() == DatumType.NULL;
+    return get(fieldid) instanceof NullDatum;
   }
 
   @Override
@@ -123,13 +123,13 @@ public class FrameTuple implements Tuple {
   }
 
   @Override
-  public BoolDatum getBoolean(int fieldId) {
-    return (BoolDatum) get(fieldId);
+  public BooleanDatum getBoolean(int fieldId) {
+    return (BooleanDatum) get(fieldId);
   }
 
   @Override
-  public ByteDatum getByte(int fieldId) {
-    return (ByteDatum) get(fieldId);
+  public BitDatum getByte(int fieldId) {
+    return (BitDatum) get(fieldId);
   }
 
   @Override
@@ -138,38 +138,38 @@ public class FrameTuple implements Tuple {
   }
 
   @Override
-  public BytesDatum getBytes(int fieldId) {
-    return (BytesDatum) get(fieldId);
+  public BlobDatum getBytes(int fieldId) {
+    return (BlobDatum) get(fieldId);
   }
 
   @Override
-  public ShortDatum getShort(int fieldId) {
-    return (ShortDatum) get(fieldId);
+  public Int2Datum getShort(int fieldId) {
+    return (Int2Datum) get(fieldId);
   }
 
   @Override
-  public IntDatum getInt(int fieldId) {
-    return (IntDatum) get(fieldId);
+  public Int4Datum getInt(int fieldId) {
+    return (Int4Datum) get(fieldId);
   }
 
   @Override
-  public LongDatum getLong(int fieldId) {
-    return (LongDatum) get(fieldId);
+  public Int8Datum getLong(int fieldId) {
+    return (Int8Datum) get(fieldId);
   }
 
   @Override
-  public FloatDatum getFloat(int fieldId) {
-    return (FloatDatum) get(fieldId);
+  public Float4Datum getFloat(int fieldId) {
+    return (Float4Datum) get(fieldId);
   }
 
   @Override
-  public DoubleDatum getDouble(int fieldId) {
-    return (DoubleDatum) get(fieldId);
+  public Float8Datum getDouble(int fieldId) {
+    return (Float8Datum) get(fieldId);
   }
 
   @Override
-  public IPv4Datum getIPv4(int fieldId) {
-    return (IPv4Datum) get(fieldId);
+  public Inet4Datum getIPv4(int fieldId) {
+    return (Inet4Datum) get(fieldId);
   }
 
   @Override
@@ -188,13 +188,13 @@ public class FrameTuple implements Tuple {
   }
 
   @Override
-  public StringDatum getString(int fieldId) {
-    return (StringDatum) get(fieldId);
+  public TextDatum getString(int fieldId) {
+    return (TextDatum) get(fieldId);
   }
 
   @Override
-  public StringDatum2 getString2(int fieldId) {
-    return (StringDatum2) get(fieldId);
+  public TextDatum getText(int fieldId) {
+    return (TextDatum) get(fieldId);
   }
 
   public String toString() {

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-storage/src/main/java/tajo/storage/RawFile.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/RawFile.java b/tajo-core/tajo-core-storage/src/main/java/tajo/storage/RawFile.java
index 74994b5..cdd65d6 100644
--- a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/RawFile.java
+++ b/tajo-core/tajo-core-storage/src/main/java/tajo/storage/RawFile.java
@@ -21,8 +21,8 @@ package tajo.storage;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import tajo.catalog.TableMeta;
-import tajo.catalog.proto.CatalogProtos.DataType;
 import tajo.catalog.statistics.TableStat;
+import tajo.common.TajoDataTypes.DataType;
 import tajo.datum.ArrayDatum;
 import tajo.datum.Datum;
 import tajo.datum.DatumFactory;
@@ -39,7 +39,7 @@ import java.nio.channels.FileChannel;
 public class RawFile {
   public static class RawFileScanner extends FileScanner implements SeekableScanner {
     private FileChannel channel;
-    private DataType [] columnTypes;
+    private DataType[] columnTypes;
     private Path path;
 
     private ByteBuffer buffer;
@@ -143,69 +143,69 @@ public class RawFile {
           continue;
         }
 
-        switch (columnTypes[i]) {
+        switch (columnTypes[i].getType()) {
           case BOOLEAN :
             tuple.put(i, DatumFactory.createBool(buffer.get()));
             break;
 
-          case BYTE :
-            tuple.put(i, DatumFactory.createByte(buffer.get()));
+          case BIT :
+            tuple.put(i, DatumFactory.createBit(buffer.get()));
             break;
 
           case CHAR :
             tuple.put(i, DatumFactory.createChar(buffer.getChar()));
             break;
 
-          case SHORT :
-            tuple.put(i, DatumFactory.createShort(buffer.getShort()));
+          case INT2 :
+            tuple.put(i, DatumFactory.createInt2(buffer.getShort()));
             break;
 
-          case INT :
-            tuple.put(i, DatumFactory.createInt(buffer.getInt()));
+          case INT4 :
+            tuple.put(i, DatumFactory.createInt4(buffer.getInt()));
             break;
 
-          case LONG :
-            tuple.put(i, DatumFactory.createLong(buffer.getLong()));
+          case INT8 :
+            tuple.put(i, DatumFactory.createInt8(buffer.getLong()));
             break;
 
-          case FLOAT :
-            tuple.put(i, DatumFactory.createFloat(buffer.getFloat()));
+          case FLOAT4 :
+            tuple.put(i, DatumFactory.createFloat4(buffer.getFloat()));
             break;
 
-          case DOUBLE :
-            tuple.put(i, DatumFactory.createDouble(buffer.getDouble()));
+          case FLOAT8 :
+            tuple.put(i, DatumFactory.createFloat8(buffer.getDouble()));
             break;
 
-          case STRING :
-            // TODO - shoud use CharsetEncoder / CharsetDecoder
-            int strSize = buffer.getInt();
-            byte [] strBytes = new byte[strSize];
-            buffer.get(strBytes);
-            tuple.put(i, DatumFactory.createString(new String(strBytes)));
-            break;
+//          case TEXT :
+//            // TODO - shoud use CharsetEncoder / CharsetDecoder
+//            int strSize = buffer.getInt();
+//            byte [] strBytes = new byte[strSize];
+//            buffer.get(strBytes);
+//            tuple.put(i, DatumFactory.createText(new String(strBytes)));
+//            break;
 
-          case STRING2 :
+          case TEXT :
             // TODO - shoud use CharsetEncoder / CharsetDecoder
             int strSize2 = buffer.getInt();
             byte [] strBytes2 = new byte[strSize2];
             buffer.get(strBytes2);
-            tuple.put(i, DatumFactory.createString2(new String(strBytes2)));
+            tuple.put(i, DatumFactory.createText(new String(strBytes2)));
             break;
 
-          case BYTES :
+          case BLOB :
             int byteSize = buffer.getInt();
             byte [] rawBytes = new byte[byteSize];
             buffer.get(rawBytes);
-            tuple.put(i, DatumFactory.createBytes(rawBytes));
+            tuple.put(i, DatumFactory.createBlob(rawBytes));
             break;
 
-          case IPv4 :
+          case INET4 :
             byte [] ipv4Bytes = new byte[4];
             buffer.get(ipv4Bytes);
-            tuple.put(i, DatumFactory.createIPv4(ipv4Bytes));
+            tuple.put(i, DatumFactory.createInet4(ipv4Bytes));
             break;
 
-          case ARRAY:
+          case ARRAY :
             int arrayByteSize = buffer.getInt();
             byte [] arrayBytes = new byte[arrayByteSize];
             buffer.get(arrayBytes);
@@ -352,9 +352,9 @@ public class RawFile {
           recordOffset = 0;
         }
 
-        switch(columnTypes[i]) {
-          case BOOLEAN :
-          case BYTE :
+        switch(columnTypes[i].getType()) {
+          case BOOLEAN:
+          case BIT:
             buffer.put(t.get(i).asByte());
             break;
 
@@ -362,36 +362,36 @@ public class RawFile {
             buffer.putChar(t.get(i).asChar());
             break;
 
-          case SHORT :
-            buffer.putShort(t.get(i).asShort());
+          case INT2 :
+            buffer.putShort(t.get(i).asInt2());
             break;
 
-          case INT :
-            buffer.putInt(t.get(i).asInt());
+          case INT4 :
+            buffer.putInt(t.get(i).asInt4());
             break;
 
-          case LONG :
-            buffer.putLong(t.get(i).asLong());
+          case INT8 :
+            buffer.putLong(t.get(i).asInt8());
             break;
 
-          case FLOAT :
-            buffer.putFloat(t.get(i).asFloat());
+          case FLOAT4 :
+            buffer.putFloat(t.get(i).asFloat4());
             break;
 
-          case DOUBLE:
-            buffer.putDouble(t.get(i).asDouble());
+          case FLOAT8 :
+            buffer.putDouble(t.get(i).asFloat8());
             break;
 
-          case STRING:
-            byte [] strBytes = t.get(i).asByteArray();
-            if (flushBufferAndReplace(recordOffset, strBytes.length + 4)) {
-              recordOffset = 0;
-            }
-            buffer.putInt(strBytes.length);
-            buffer.put(strBytes);
-            break;
+//          case TEXT :
+//            byte [] strBytes = t.get(i).asByteArray();
+//            if (flushBufferAndReplace(recordOffset, strBytes.length + 4)) {
+//              recordOffset = 0;
+//            }
+//            buffer.putInt(strBytes.length);
+//            buffer.put(strBytes);
+//            break;
 
-          case STRING2:
+          case TEXT:
             byte [] strBytes2 = t.get(i).asByteArray();
             if (flushBufferAndReplace(recordOffset, strBytes2.length + 4)) {
               recordOffset = 0;
@@ -400,7 +400,7 @@ public class RawFile {
             buffer.put(strBytes2);
             break;
 
-          case BYTES:
+          case BLOB :
             byte [] rawBytes = t.get(i).asByteArray();
             if (flushBufferAndReplace(recordOffset, rawBytes.length + 4)) {
               recordOffset = 0;
@@ -409,11 +409,11 @@ public class RawFile {
             buffer.put(rawBytes);
             break;
 
-          case IPv4:
+          case INET4 :
             buffer.put(t.get(i).asByteArray());
             break;
 
-          case ARRAY:
+          case ARRAY :
             ArrayDatum array = (ArrayDatum) t.get(i);
             String json = array.toJSON();
             byte [] jsonBytes = json.getBytes();

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-storage/src/main/java/tajo/storage/RowFile.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/RowFile.java b/tajo-core/tajo-core-storage/src/main/java/tajo/storage/RowFile.java
index 119fcf4..91f6715 100644
--- a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/RowFile.java
+++ b/tajo-core/tajo-core-storage/src/main/java/tajo/storage/RowFile.java
@@ -196,75 +196,75 @@ public class RowFile {
       for (i = 0; i < schema.getColumnNum(); i++) {
         if (!nullFlags.get(i)) {
           col = schema.getColumn(i);
-          switch (col.getDataType()) {
-            case BOOLEAN:
+          switch (col.getDataType().getType()) {
+            case BOOLEAN :
               datum = DatumFactory.createBool(buffer.get());
               tuple.put(i, datum);
               break;
 
-            case BYTE:
-              datum = DatumFactory.createByte(buffer.get());
+            case BIT:
+              datum = DatumFactory.createBit(buffer.get());
               tuple.put(i, datum );
               break;
 
-            case CHAR:
+            case CHAR :
               datum = DatumFactory.createChar(buffer.getChar());
               tuple.put(i, datum);
               break;
 
-            case SHORT:
-              datum = DatumFactory.createShort(buffer.getShort());
+            case INT2 :
+              datum = DatumFactory.createInt2(buffer.getShort());
               tuple.put(i, datum );
               break;
 
-            case INT:
-              datum = DatumFactory.createInt(buffer.getInt());
+            case INT4 :
+              datum = DatumFactory.createInt4(buffer.getInt());
               tuple.put(i, datum );
               break;
 
-            case LONG:
-              datum = DatumFactory.createLong(buffer.getLong());
+            case INT8 :
+              datum = DatumFactory.createInt8(buffer.getLong());
               tuple.put(i, datum );
               break;
 
-            case FLOAT:
-              datum = DatumFactory.createFloat(buffer.getFloat());
+            case FLOAT4 :
+              datum = DatumFactory.createFloat4(buffer.getFloat());
               tuple.put(i, datum);
               break;
 
-            case DOUBLE:
-              datum = DatumFactory.createDouble(buffer.getDouble());
+            case FLOAT8 :
+              datum = DatumFactory.createFloat8(buffer.getDouble());
               tuple.put(i, datum);
               break;
 
-            case STRING:
-              short len = buffer.getShort();
-              byte[] buf = new byte[len];
-              buffer.get(buf, 0, len);
-              datum = DatumFactory.createString(buf);
-              tuple.put(i, datum);
-              break;
+//            case TEXT :
+//              short len = buffer.getShort();
+//              byte[] buf = new byte[len];
+//              buffer.get(buf, 0, len);
+//              datum = DatumFactory.createText(buf);
+//              tuple.put(i, datum);
+//              break;
 
-            case STRING2:
+            case TEXT:
               short bytelen = buffer.getShort();
               byte[] strbytes = new byte[bytelen];
               buffer.get(strbytes, 0, bytelen);
-              datum = DatumFactory.createString2(strbytes);
+              datum = DatumFactory.createText(strbytes);
               tuple.put(i, datum);
               break;
 
-            case BYTES:
+            case BLOB:
               short bytesLen = buffer.getShort();
               byte [] bytesBuf = new byte[bytesLen];
               buffer.get(bytesBuf);
-              datum = DatumFactory.createBytes(bytesBuf);
+              datum = DatumFactory.createBlob(bytesBuf);
               tuple.put(i, datum);
               break;
 
-            case IPv4:
+            case INET4 :
               byte[] ipv4 = new byte[4];
               buffer.get(ipv4, 0, 4);
-              datum = DatumFactory.createIPv4(ipv4);
+              datum = DatumFactory.createInet4(ipv4);
               tuple.put(i, datum);
               break;
 
@@ -391,12 +391,12 @@ public class RowFile {
           nullFlags.set(i);
         } else {
           col = schema.getColumn(i);
-          switch (col.getDataType()) {
+          switch (col.getDataType().getType()) {
             case BOOLEAN:
 //              out.writeBoolean(t.getByte(i).asBool());
               buffer.put(t.getBoolean(i).asByte());
               break;
-            case BYTE:
+            case BIT:
 //              out.writeByte(t.getByte(i).asByte());
               buffer.put(t.getByte(i).asByte());
               break;
@@ -404,55 +404,55 @@ public class RowFile {
 //              out.writeChar(t.getChar(i).asChar());
               buffer.putChar(t.getChar(i).asChar());
               break;
-            case STRING:
-              byte[] buf = t.getString(i).asByteArray();
-              if (buf.length > 256) {
-                buf = new byte[256];
-                byte[] str = t.getString(i).asByteArray();
-                System.arraycopy(str, 0, buf, 0, 256);
-              }
-//              out.writeShort(buf.length);
-//              out.write(buf, 0, buf.length);
-              buffer.putShort((short)buf.length);
-              buffer.put(buf, 0, buf.length);
-              break;
-            case STRING2:
-              byte[] strbytes = t.getString2(i).asByteArray();
+//            case TEXT:
+//              byte[] buf = t.getString(i).asByteArray();
+//              if (buf.length > 256) {
+//                buf = new byte[256];
+//                byte[] str = t.getString(i).asByteArray();
+//                System.arraycopy(str, 0, buf, 0, 256);
+//              }
+////              out.writeShort(buf.length);
+////              out.write(buf, 0, buf.length);
+//              buffer.putShort((short)buf.length);
+//              buffer.put(buf, 0, buf.length);
+//              break;
+            case TEXT:
+              byte [] strbytes = t.getText(i).asByteArray();
               buffer.putShort((short)strbytes.length);
               buffer.put(strbytes, 0, strbytes.length);
               break;
-            case SHORT:
+            case INT2:
 //              out.writeShort(t.getShort(i).asShort());
-              buffer.putShort(t.getShort(i).asShort());
+              buffer.putShort(t.getShort(i).asInt2());
               break;
-            case INT:
+            case INT4:
 //              out.writeInt(t.getInt(i).asInt());
-              buffer.putInt(t.getInt(i).asInt());
+              buffer.putInt(t.getInt(i).asInt4());
               break;
-            case LONG:
+            case INT8:
 //              out.writeLong(t.getLong(i).asLong());
-              buffer.putLong(t.getLong(i).asLong());
+              buffer.putLong(t.getLong(i).asInt8());
               break;
-            case FLOAT:
+            case FLOAT4:
 //              out.writeFloat(t.getFloat(i).asFloat());
-              buffer.putFloat(t.getFloat(i).asFloat());
+              buffer.putFloat(t.getFloat(i).asFloat4());
               break;
-            case DOUBLE:
+            case FLOAT8:
 //              out.writeDouble(t.getDouble(i).asDouble());
-              buffer.putDouble(t.getDouble(i).asDouble());
+              buffer.putDouble(t.getDouble(i).asFloat8());
               break;
-            case BYTES:
+            case BLOB:
               byte [] bytes = t.getBytes(i).asByteArray();
 //              out.writeInt(bytes.length);
 //              out.write(bytes);
               buffer.putShort((short)bytes.length);
               buffer.put(bytes);
               break;
-            case IPv4:
+            case INET4:
 //              out.write(t.getIPv4Bytes(i));
               buffer.put(t.getIPv4Bytes(i));
               break;
-            case IPv6:
+            case INET6:
 //              out.write(t.getIPv6Bytes(i));
               buffer.put(t.getIPv6Bytes(i));
             case ARRAY: {

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-storage/src/main/java/tajo/storage/RowStoreUtil.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/RowStoreUtil.java b/tajo-core/tajo-core-storage/src/main/java/tajo/storage/RowStoreUtil.java
index 88766a7..20071f1 100644
--- a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/RowStoreUtil.java
+++ b/tajo-core/tajo-core-storage/src/main/java/tajo/storage/RowStoreUtil.java
@@ -54,80 +54,92 @@ public class RowStoreUtil {
       for (int i =0; i < schema.getColumnNum(); i++) {
         col = schema.getColumn(i);
 
-        switch (col.getDataType()) {
-          case BYTE:
+        switch (col.getDataType().getType()) {
+          case BOOLEAN: tuple.put(i, DatumFactory.createBool(bb.get())); break;
+          case BIT:
             byte b = bb.get();
             if(b == 0) {
               tuple.put(i, DatumFactory.createNullDatum());
-            }else {
-              tuple.put(i, DatumFactory.createByte(b));
-            }break;
+            } else {
+              tuple.put(i, DatumFactory.createBit(b));
+            }
+            break;
+
           case CHAR:
             byte c = bb.get();
             if(c == 0) {
               tuple.put(i, DatumFactory.createNullDatum());
-            }else {
+            } else {
               tuple.put(i, DatumFactory.createChar(c));
-            }break;
-          case BOOLEAN: tuple.put(i, DatumFactory.createBool(bb.get())); break;
-          case SHORT:
+            }
+            break;
+
+          case INT2:
             short s = bb.getShort();
             if(s < Short.MIN_VALUE + 1) {
               tuple.put(i, DatumFactory.createNullDatum());
             }else {
-              tuple.put(i, DatumFactory.createShort(s));
-            }break;
-          case INT:
+              tuple.put(i, DatumFactory.createInt2(s));
+            }
+            break;
+
+          case INT4:
             int i_ = bb.getInt();
             if ( i_ < Integer.MIN_VALUE + 1) {
               tuple.put(i, DatumFactory.createNullDatum());
             }else {
-            tuple.put(i, DatumFactory.createInt(i_));
+            tuple.put(i, DatumFactory.createInt4(i_));
             }break;
-          case LONG:
+          case INT8:
             long l = bb.getLong();
             if ( l < Long.MIN_VALUE + 1) {
               tuple.put(i, DatumFactory.createNullDatum());
             }else {
-              tuple.put(i, DatumFactory.createLong(l));
-            }break;
-          case FLOAT:
+              tuple.put(i, DatumFactory.createInt8(l));
+            }
+            break;
+
+          case FLOAT4:
             float f = bb.getFloat();
             if (Float.isNaN(f)) {
               tuple.put(i, DatumFactory.createNullDatum());
             }else {
-              tuple.put(i, DatumFactory.createFloat(f));
-            }break;
-          case DOUBLE:
+              tuple.put(i, DatumFactory.createFloat4(f));
+            }
+            break;
+
+          case FLOAT8:
             double d = bb.getDouble();
             if(Double.isNaN(d)) {
               tuple.put(i, DatumFactory.createNullDatum());
             }else {
-              tuple.put(i, DatumFactory.createDouble(d));
+              tuple.put(i, DatumFactory.createFloat8(d));
             }break;
-          case STRING:
+          case TEXT:
             byte [] _string = new byte[bb.getInt()];
             bb.get(_string);
             String str = new String(_string);
             if(str.compareTo("NULL") == 0) {
               tuple.put(i, DatumFactory.createNullDatum());
             }else {
-            tuple.put(i, DatumFactory.createString(str));
-            }break;
-          case BYTES:
+            tuple.put(i, DatumFactory.createText(str));
+            }
+            break;
+
+          case BLOB:
             byte [] _bytes = new byte[bb.getInt()];
             bb.get(_bytes);
             if(Bytes.compareTo(bytes, Bytes.toBytes("NULL")) == 0) {
               tuple.put(i, DatumFactory.createNullDatum());
             }else {
-            tuple.put(i, DatumFactory.createBytes(_bytes));
+            tuple.put(i, DatumFactory.createBlob(_bytes));
             }break;
-          case IPv4:
+          case INET4:
             byte [] _ipv4 = new byte[4];
             bb.get(_ipv4);
-            tuple.put(i, DatumFactory.createIPv4(_ipv4));
+            tuple.put(i, DatumFactory.createInet4(_ipv4));
             break;
-          case IPv6:
+          case INET6:
             // TODO - to be implemented
         }
       }
@@ -143,30 +155,30 @@ public class RowStoreUtil {
       Column col;
       for (int i = 0; i < schema.getColumnNum(); i++) {
         col = schema.getColumn(i);
-        switch (col.getDataType()) {
-          case BYTE: bb.put(tuple.get(i).asByte()); break;
-          case CHAR: bb.put(tuple.get(i).asByte()); break;
+        switch (col.getDataType().getType()) {
           case BOOLEAN: bb.put(tuple.get(i).asByte()); break;
-          case SHORT: bb.putShort(tuple.get(i).asShort()); break;
-          case INT: bb.putInt(tuple.get(i).asInt()); break;
-          case LONG: bb.putLong(tuple.get(i).asLong()); break;
-          case FLOAT: bb.putFloat(tuple.get(i).asFloat()); break;
-          case DOUBLE: bb.putDouble(tuple.get(i).asDouble()); break;
-          case STRING:
+          case BIT: bb.put(tuple.get(i).asByte()); break;
+          case CHAR: bb.put(tuple.get(i).asByte()); break;
+          case INT2: bb.putShort(tuple.get(i).asInt2()); break;
+          case INT4: bb.putInt(tuple.get(i).asInt4()); break;
+          case INT8: bb.putLong(tuple.get(i).asInt8()); break;
+          case FLOAT4: bb.putFloat(tuple.get(i).asFloat4()); break;
+          case FLOAT8: bb.putDouble(tuple.get(i).asFloat8()); break;
+          case TEXT:
             byte [] _string = tuple.get(i).asByteArray();
             bb.putInt(_string.length);
             bb.put(_string);
             break;
-          case BYTES:
+          case BLOB:
             byte [] bytes = tuple.get(i).asByteArray();
             bb.putInt(bytes.length);
             bb.put(bytes);
             break;
-          case IPv4:
+          case INET4:
             byte [] ipBytes = tuple.getIPv4Bytes(i);
             bb.put(ipBytes);
             break;
-          case IPv6: bb.put(tuple.getIPv6Bytes(i)); break;
+          case INET6: bb.put(tuple.getIPv6Bytes(i)); break;
           default:
         }
       }

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-storage/src/main/java/tajo/storage/StorageUtil.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/StorageUtil.java b/tajo-core/tajo-core-storage/src/main/java/tajo/storage/StorageUtil.java
index 293148e..0fb07ab 100644
--- a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/StorageUtil.java
+++ b/tajo-core/tajo-core-storage/src/main/java/tajo/storage/StorageUtil.java
@@ -40,18 +40,19 @@ public class StorageUtil {
   }
 
   public static int getColByteSize(Column col) {
-    switch(col.getDataType()) {
+    switch(col.getDataType().getType()) {
     case BOOLEAN: return 1;
     case CHAR: return 1;
-    case BYTE: return 1;
-    case SHORT: return 2;
-    case INT: return 4;
-    case LONG: return 8;
-    case FLOAT: return 4;
-    case DOUBLE: return 8;
-    case IPv4: return 4;
-    case IPv6: return 32;
-    case STRING: return 256;
+    case BIT: return 1;
+    case INT2: return 2;
+    case INT4: return 4;
+    case INT8: return 8;
+    case FLOAT4: return 4;
+    case FLOAT8: return 8;
+    case INET4: return 4;
+    case INET6: return 32;
+    case TEXT: return 256;
+    case BLOB: return 256;
     default: return 0;
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-storage/src/main/java/tajo/storage/TableStatistics.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/TableStatistics.java b/tajo-core/tajo-core-storage/src/main/java/tajo/storage/TableStatistics.java
index 5d93af1..e51e7de 100644
--- a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/TableStatistics.java
+++ b/tajo-core/tajo-core-storage/src/main/java/tajo/storage/TableStatistics.java
@@ -19,11 +19,13 @@
 package tajo.storage;
 
 import tajo.catalog.Schema;
-import tajo.catalog.proto.CatalogProtos;
 import tajo.catalog.statistics.ColumnStat;
 import tajo.catalog.statistics.TableStat;
+import tajo.common.TajoDataTypes;
+import tajo.common.TajoDataTypes.DataType;
+import tajo.common.TajoDataTypes.Type;
 import tajo.datum.Datum;
-import tajo.datum.DatumType;
+import tajo.datum.NullDatum;
 
 /**
  * This class is not thread-safe.
@@ -51,10 +53,10 @@ public class TableStatistics {
     numNulls = new long[schema.getColumnNum()];
     comparable = new boolean[schema.getColumnNum()];
 
-    CatalogProtos.DataType type;
+    DataType type;
     for (int i = 0; i < schema.getColumnNum(); i++) {
       type = schema.getColumn(i).getDataType();
-      if (type == CatalogProtos.DataType.ARRAY) {
+      if (type.getType() == Type.ARRAY) {
         comparable[i] = false;
       } else {
         comparable[i] = true;
@@ -83,12 +85,12 @@ public class TableStatistics {
   }
 
   public void analyzeField(int idx, Datum datum) {
-    if (datum.type() == DatumType.NULL) {
+    if (datum instanceof NullDatum) {
       numNulls[idx]++;
       return;
     }
 
-    if (datum.type() != DatumType.ARRAY) {
+    if (datum.type() != TajoDataTypes.Type.ARRAY) {
       if (comparable[idx]) {
         if (!maxValues.contains(idx) ||
             maxValues.get(idx).compareTo(datum) < 0) {

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-storage/src/main/java/tajo/storage/Tuple.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/Tuple.java b/tajo-core/tajo-core-storage/src/main/java/tajo/storage/Tuple.java
index e96c524..4544f41 100644
--- a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/Tuple.java
+++ b/tajo-core/tajo-core-storage/src/main/java/tajo/storage/Tuple.java
@@ -46,25 +46,25 @@ public interface Tuple {
 	
 	public long getOffset();
 
-	public BoolDatum getBoolean(int fieldId);
+	public BooleanDatum getBoolean(int fieldId);
 	
-	public ByteDatum getByte(int fieldId);
+	public BitDatum getByte(int fieldId);
 
   public CharDatum getChar(int fieldId);
 	
-	public BytesDatum getBytes(int fieldId);
+	public BlobDatum getBytes(int fieldId);
 	
-	public ShortDatum getShort(int fieldId);
+	public Int2Datum getShort(int fieldId);
 	
-	public IntDatum getInt(int fieldId);
+	public Int4Datum getInt(int fieldId);
 	
-	public LongDatum getLong(int fieldId);
+	public Int8Datum getLong(int fieldId);
 	
-	public FloatDatum getFloat(int fieldId);
+	public Float4Datum getFloat(int fieldId);
 	
-	public DoubleDatum getDouble(int fieldId);
+	public Float8Datum getDouble(int fieldId);
 	
-	public IPv4Datum getIPv4(int fieldId);
+	public Inet4Datum getIPv4(int fieldId);
 	
 	public byte [] getIPv4Bytes(int fieldId);
 	
@@ -72,7 +72,7 @@ public interface Tuple {
 	
 	public byte [] getIPv6Bytes(int fieldId);
 	
-	public StringDatum getString(int fieldId);
+	public TextDatum getString(int fieldId);
 
-  public StringDatum2 getString2(int fieldId);
+  public TextDatum getText(int fieldId);
 }

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-storage/src/main/java/tajo/storage/TupleComparator.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/TupleComparator.java b/tajo-core/tajo-core-storage/src/main/java/tajo/storage/TupleComparator.java
index bbd0cc6..0b5c235 100644
--- a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/TupleComparator.java
+++ b/tajo-core/tajo-core-storage/src/main/java/tajo/storage/TupleComparator.java
@@ -24,7 +24,7 @@ import tajo.catalog.SortSpec;
 import tajo.catalog.proto.CatalogProtos.SortSpecProto;
 import tajo.common.ProtoObject;
 import tajo.datum.Datum;
-import tajo.datum.DatumType;
+import tajo.datum.NullDatum;
 import tajo.storage.index.IndexProtos.TupleComparatorProto;
 
 import java.util.Comparator;
@@ -82,11 +82,11 @@ public class TupleComparator implements Comparator<Tuple>, ProtoObject<TupleComp
       left = tuple1.get(sortKeyIds[i]);
       right = tuple2.get(sortKeyIds[i]);
 
-      if (left.type() == DatumType.NULL || right.type() == DatumType.NULL) {
+      if (left instanceof NullDatum || right instanceof NullDatum) {
         if (!left.equals(right)) {
-          if (left.type() == DatumType.NULL) {
+          if (left instanceof NullDatum) {
             compVal = 1;
-          } else if (right.type() == DatumType.NULL) {
+          } else if (right instanceof NullDatum) {
             compVal = -1;
           }
           if (nullFirsts[i]) {

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-storage/src/main/java/tajo/storage/VTuple.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/VTuple.java b/tajo-core/tajo-core-storage/src/main/java/tajo/storage/VTuple.java
index 89f1230..74894bc 100644
--- a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/VTuple.java
+++ b/tajo-core/tajo-core-storage/src/main/java/tajo/storage/VTuple.java
@@ -55,7 +55,7 @@ public class VTuple implements Tuple {
 
   @Override
   public boolean isNull(int fieldid) {
-    return values[fieldid].type() == DatumType.NULL;
+    return values[fieldid] instanceof NullDatum;
   }
 
   @Override
@@ -106,44 +106,44 @@ public class VTuple implements Tuple {
 	}
 	
 	@Override
-	public BoolDatum getBoolean(int fieldId) {
-		return (BoolDatum) values[fieldId];
+	public BooleanDatum getBoolean(int fieldId) {
+		return (BooleanDatum) values[fieldId];
 	}
 
-	public ByteDatum getByte(int fieldId) {
-		return (ByteDatum) values[fieldId];
+	public BitDatum getByte(int fieldId) {
+		return (BitDatum) values[fieldId];
 	}
 
   public CharDatum getChar(int fieldId) {
     return (CharDatum) values[fieldId];
   }
 
-	public BytesDatum getBytes(int fieldId) {
-		return (BytesDatum) values[fieldId];
+	public BlobDatum getBytes(int fieldId) {
+		return (BlobDatum) values[fieldId];
 	}
 
-	public ShortDatum getShort(int fieldId) {
-		return (ShortDatum) values[fieldId];
+	public Int2Datum getShort(int fieldId) {
+		return (Int2Datum) values[fieldId];
 	}
 
-	public IntDatum getInt(int fieldId) {
-		return (IntDatum) values[fieldId];			
+	public Int4Datum getInt(int fieldId) {
+		return (Int4Datum) values[fieldId];
 	}
 
-	public LongDatum getLong(int fieldId) {
-		return (LongDatum) values[fieldId];
+	public Int8Datum getLong(int fieldId) {
+		return (Int8Datum) values[fieldId];
 	}
 
-	public FloatDatum getFloat(int fieldId) {
-		return (FloatDatum) values[fieldId];
+	public Float4Datum getFloat(int fieldId) {
+		return (Float4Datum) values[fieldId];
 	}
 
-	public DoubleDatum getDouble(int fieldId) {
-		return (DoubleDatum) values[fieldId];
+	public Float8Datum getDouble(int fieldId) {
+		return (Float8Datum) values[fieldId];
 	}
 
-	public IPv4Datum getIPv4(int fieldId) {
-		return (IPv4Datum) values[fieldId];
+	public Inet4Datum getIPv4(int fieldId) {
+		return (Inet4Datum) values[fieldId];
 	}
 
 	public byte[] getIPv4Bytes(int fieldId) {
@@ -158,13 +158,13 @@ public class VTuple implements Tuple {
 	  throw new InvalidCastException("IPv6 is unsupported yet");
 	}
 
-	public StringDatum getString(int fieldId) {
-		return (StringDatum) values[fieldId];
+	public TextDatum getString(int fieldId) {
+		return (TextDatum) values[fieldId];
 	}
 
   @Override
-  public StringDatum2 getString2(int fieldId) {
-    return (StringDatum2) values[fieldId];
+  public TextDatum getText(int fieldId) {
+    return (TextDatum) values[fieldId];
   }
 
   public String toString() {

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/BasicBlock.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/BasicBlock.java b/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/BasicBlock.java
deleted file mode 100644
index 56b376a..0000000
--- a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/BasicBlock.java
+++ /dev/null
@@ -1,140 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package tajo.storage.hcfile;
-
-import com.google.common.collect.Lists;
-import tajo.datum.Datum;
-
-import java.io.IOException;
-import java.util.List;
-
-public class BasicBlock extends UpdatableSeekableBlock {
-  private final int MAX_BLOCK_SIZE = 65535; // 64KB
-  private BlockMeta meta;
-  private List<Datum> datums = Lists.newArrayList();
-  private int pos;
-  private int next;
-  int size;
-
-  public BasicBlock() {
-    size = 0;
-  }
-
-  public BasicBlock(BlockMeta meta) {
-    this.setMeta(meta);
-  }
-
-  public void setMeta(BlockMeta meta) {
-    this.meta = meta;
-  }
-
-  @Override
-  public BlockMeta getMeta() throws IOException {
-    return meta;
-  }
-
-  @Override
-  public Datum[] asArray() throws IOException {
-    return datums.toArray(new Datum[datums.size()]);
-  }
-
-  @Override
-  public Datum next() throws IOException {
-    if (next >= datums.size()) {
-      return null;
-    }
-    pos = next;
-    Datum datum = datums.get(next++);
-    return datum;
-  }
-
-  @Override
-  public int getPos() throws IOException {
-    return pos;
-  }
-
-  @Override
-  public int getSize() throws IOException {
-    return this.size;
-  }
-
-  @Override
-  public int getDataNum() throws IOException {
-    return datums.size();
-  }
-
-  @Override
-  public void first() throws IOException {
-    pos = 0;
-    next = 0;
-  }
-
-  @Override
-  public void last() throws IOException {
-    pos = datums.size() - 1;
-    next = datums.size() - 1;
-  }
-
-  @Override
-  public void pos(long pos) throws IOException {
-    this.pos = (int)pos;
-    this.next = this.pos;
-  }
-
-  @Override
-  public void appendValue(Datum data) throws IOException {
-    this.size += ColumnStoreUtil.getWrittenSize(data);
-    datums.add(data);
-  }
-
-  public boolean isAppendable(Datum datum) throws IOException {
-    return Integer.SIZE/8 // block size
-        + this.size
-        + ColumnStoreUtil.getWrittenSize(datum)
-        < MAX_BLOCK_SIZE;
-  }
-
-  @Override
-  public void setValues(Datum[] data) throws IOException {
-    if (datums == null) {
-      datums = Lists.newArrayList();
-    } else {
-      this.clear();
-    }
-    for (Datum d : data) {
-      this.appendValue(d);
-    }
-  }
-
-  @Override
-  public Datum removeValue() throws IOException {
-    this.size -= ColumnStoreUtil.getWrittenSize(datums.get(pos));
-    return datums.remove(pos);
-  }
-
-  @Override
-  public void clear() throws IOException {
-    datums.clear();
-    pos = 0;
-    next = 0;
-    meta.setRecordNum(0);
-    meta.setStartRid(0);
-    size = 0;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/Block.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/Block.java b/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/Block.java
deleted file mode 100644
index a4180e0..0000000
--- a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/Block.java
+++ /dev/null
@@ -1,71 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package tajo.storage.hcfile;
-
-import tajo.datum.Datum;
-
-import java.io.IOException;
-
-/**
- * Read/write unit
- */
-public interface Block {
-
-  void setMeta(BlockMeta meta);
-
-  /**
-   * Get the metadata of the block
-   *
-   * @return the metadata
-   */
-  BlockMeta getMeta() throws IOException;
-
-  /**
-   * Return the whole values as an array.
-   *
-   * @return the array of values in the block
-   */
-  Datum[] asArray() throws IOException;
-
-  /**
-   * Return a datum of the current row pointed by the cursor,
-   * and advance the cursor
-   *
-   * @return retrieve null if the scanner has no more datum.
-   * Otherwise it returns one datum.
-   */
-  Datum next() throws IOException;
-
-  /**
-   * Get the position of the cursor.
-   *
-   * @return the position of the value which is currently pointed by the cursor
-   */
-  int getPos() throws IOException;
-
-  /**
-   * Return the number of bytes of data in the block.
-   *
-   * @return the number of bytes
-   * @throws IOException
-   */
-  int getSize() throws IOException;
-
-  int getDataNum() throws IOException;
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/BlockMeta.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/BlockMeta.java b/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/BlockMeta.java
deleted file mode 100644
index fa7c718..0000000
--- a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/BlockMeta.java
+++ /dev/null
@@ -1,50 +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 tajo.storage.hcfile;
-
-import tajo.catalog.proto.CatalogProtos.DataType;
-
-public interface BlockMeta {
-
-  /**
-   * Return the type of this column
-   *
-   * @return the type of this column
-   */
-  public DataType getType();
-
-  /**
-   * Return the number of records in this file
-   *
-   * @return the number of records in this file
-   */
-  public int getRecordNum();
-
-  public BlockMeta setRecordNum(int recordNum);
-
-  public BlockMeta setStartRid(long rid);
-
-  public long getStartRid();
-
-  public boolean isSorted();
-
-  public boolean isContiguous();
-
-  public boolean isCompressed();
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/BlockReader.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/BlockReader.java b/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/BlockReader.java
deleted file mode 100644
index 762c7dc..0000000
--- a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/BlockReader.java
+++ /dev/null
@@ -1,57 +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 tajo.storage.hcfile;
-
-import tajo.catalog.proto.CatalogProtos.DataType;
-import tajo.datum.Datum;
-import tajo.storage.exception.UnknownDataTypeException;
-import tajo.storage.hcfile.reader.Reader;
-import tajo.storage.hcfile.reader.TypeReader;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-
-/**
- * BlockReader reads data from the disk
- * and fills a block.
- */
-public class BlockReader implements Reader {
-  private TypeReader reader;
-
-  public BlockReader(DataType type)
-      throws IOException, UnknownDataTypeException {
-    reader = TypeReader.get(type);
-  }
-
-  @Override
-  public UpdatableBlock read(BlockMeta meta, ByteBuffer buffer) throws IOException {
-    UpdatableBlock block = new BasicBlock(meta);
-    Datum datum;
-    while ((datum=reader.read(buffer)) != null) {
-      block.appendValue(datum);
-    }
-    block.getMeta().setRecordNum(block.getDataNum());
-    return block;
-  }
-
-  @Override
-  public void close() throws IOException {
-    reader.close();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/BlockWriter.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/BlockWriter.java b/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/BlockWriter.java
deleted file mode 100644
index a4e9691..0000000
--- a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/BlockWriter.java
+++ /dev/null
@@ -1,57 +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 tajo.storage.hcfile;
-
-import org.apache.hadoop.fs.FSDataOutputStream;
-import tajo.catalog.proto.CatalogProtos.DataType;
-import tajo.datum.Datum;
-import tajo.storage.exception.UnknownDataTypeException;
-import tajo.storage.hcfile.writer.TypeWriter;
-import tajo.storage.hcfile.writer.Writer;
-
-import java.io.IOException;
-
-/**
- * BlockWriter writes a block to the disk.
- */
-public class BlockWriter implements Writer {
-  private TypeWriter writer;
-
-  public BlockWriter(FSDataOutputStream out, DataType type)
-      throws IOException, UnknownDataTypeException {
-    writer = TypeWriter.get(out, type);
-  }
-
-  public void write(Block block) throws IOException {
-    writer.getOutputStream().writeInt(block.getSize());
-    for (Datum datum : block.asArray()) {
-      writer.write(datum);
-    }
-  }
-
-  @Override
-  public void close() throws IOException {
-    writer.close();
-  }
-
-  @Override
-  public long getPos() throws IOException {
-    return writer.getPos();
-  }
-}

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

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

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/ColumnFileScanner.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/ColumnFileScanner.java b/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/ColumnFileScanner.java
deleted file mode 100644
index 5aedab1..0000000
--- a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/ColumnFileScanner.java
+++ /dev/null
@@ -1,33 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package tajo.storage.hcfile;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.Path;
-
-public abstract class ColumnFileScanner implements ColumnScanner {
-  protected final Configuration conf;
-  protected final Path path;
-
-  public ColumnFileScanner(Configuration conf, Path path) {
-    this.conf = conf;
-    this.path = path;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/ColumnMeta.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/ColumnMeta.java b/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/ColumnMeta.java
deleted file mode 100644
index 8cdc07c..0000000
--- a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/ColumnMeta.java
+++ /dev/null
@@ -1,48 +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 tajo.storage.hcfile;
-
-import tajo.catalog.proto.CatalogProtos.CompressType;
-import tajo.catalog.proto.CatalogProtos.DataType;
-import tajo.catalog.proto.CatalogProtos.StoreType;
-
-public interface ColumnMeta {
-
-  StoreType getStoreType();
-
-  /**
-   * Return the type of this column
-   *
-   * @return the type of this column
-   */
-  DataType getDataType();
-
-  CompressType getCompressType();
-
-  /**
-   * Indicating this column is compressed or not
-   *
-   * @return return true if this column is compressed, otherwise return false
-   */
-  boolean isCompressed();
-
-  boolean isSorted();
-
-  boolean isContiguous();
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/ColumnReader.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/ColumnReader.java b/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/ColumnReader.java
deleted file mode 100644
index fbe8757..0000000
--- a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/ColumnReader.java
+++ /dev/null
@@ -1,42 +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 tajo.storage.hcfile;
-
-import tajo.datum.Datum;
-
-import java.io.Closeable;
-import java.io.IOException;
-
-public interface ColumnReader extends Closeable, Seekable {
-
-  @Override
-  void first() throws IOException;
-
-  @Override
-  void last() throws IOException;
-
-  Datum get() throws IOException;
-
-  void pos(long rid) throws IOException;
-
-  long getPos() throws IOException;
-
-  @Override
-  void close() throws IOException;
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/ColumnScanner.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/ColumnScanner.java b/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/ColumnScanner.java
deleted file mode 100644
index 1c70422..0000000
--- a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/ColumnScanner.java
+++ /dev/null
@@ -1,107 +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 tajo.storage.hcfile;
-
-import tajo.datum.Datum;
-
-import java.io.Closeable;
-import java.io.IOException;
-
-/**
- * Scanner interface for columnar files
- */
-public interface ColumnScanner extends Closeable, Seekable {
-
-  ColumnMeta getMeta() throws IOException;
-
-  /**
-   * Move the cursor to the first row.
-   * After this is called, get() retrieves the first datum.
-   *
-   * @throws IOException if internal I/O error occurs during reset method
-   */
-  @Override
-  void first() throws IOException;
-
-  /**
-   * Move the cursor to the last row.
-   * After this is called, get() retrieves the last datum.
-   *
-   * @throws IOException if internal I/O error occurs during reset method
-   */
-  @Override
-  void last() throws IOException;
-
-  /**
-   * Return a datum of the current row pointed by the cursor,
-   * and advance the cursor
-   *
-   * @return retrieve null if the scanner has no more datum.
-   * Otherwise it returns one datum.
-   *
-   * @throws IOException if internal I/O error occurs during get method
-   */
-  Datum get() throws IOException;
-
-  /**
-   * Move the cursor to the given position
-   *
-   * @param rid indicates the desired position
-   * @throws IOException if the desired position is larger than the max row id
-   */
-  @Override
-  void pos(long rid) throws IOException;
-
-  /**
-   * Retrieves the get block
-   *
-   * @return retrieve null if the scanner has no more block.
-   * Otherwise it returns one block.
-   *
-   * @throws IOException if internal I/O error occurs during reset method
-   */
-  Block getBlock() throws IOException;
-
-  /**
-   * Returns an array of datums in the get block.
-   *
-   * @return retrieve null if the scanner has no more block.
-   * Otherwise it returns an array of datums in the get block.
-   *
-   * @throws IOException if internal I/O error occurs during get method
-   */
-  Datum[] getBlockAsArray() throws IOException;
-
-
-  /**
-   * Return the id of the current row.
-   *
-   * @return the id of the current row
-   * @throws IOException if internal I/O error occurs during get method
-   */
-  long getPos() throws IOException;
-
-  /**
-   * Close scanner
-   *
-   * @throws IOException if internal I/O error occurs during close method
-   */
-  @Override
-  void close() throws IOException;
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/ColumnStoreUtil.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/ColumnStoreUtil.java b/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/ColumnStoreUtil.java
deleted file mode 100644
index d40c49e..0000000
--- a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/ColumnStoreUtil.java
+++ /dev/null
@@ -1,36 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package tajo.storage.hcfile;
-
-import tajo.datum.Datum;
-
-public class ColumnStoreUtil {
-
-  public static int getWrittenSize(Datum datum) {
-    switch (datum.type()) {
-      case BYTES:
-      case STRING:
-      case STRING2:
-      case IPv4:
-        return Integer.SIZE/8 + datum.size();
-      default:
-        return datum.size();
-    }
-  }
-}

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

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/CompressedBlockReader.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/CompressedBlockReader.java b/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/CompressedBlockReader.java
deleted file mode 100644
index c128ce6..0000000
--- a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/CompressedBlockReader.java
+++ /dev/null
@@ -1,53 +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 tajo.storage.hcfile;
-
-import tajo.catalog.proto.CatalogProtos.CompressType;
-import tajo.catalog.proto.CatalogProtos.DataType;
-import tajo.storage.exception.UnknownCodecException;
-import tajo.storage.hcfile.compress.Codec;
-import tajo.storage.hcfile.reader.Reader;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-
-public class CompressedBlockReader implements Reader {
-  private Codec codec;
-  private DataType dataType;
-
-  public CompressedBlockReader(DataType dataType, CompressType compType)
-      throws UnknownCodecException, IOException {
-    codec = Codec.get(compType);
-    this.dataType = dataType;
-  }
-
-  @Override
-  public UpdatableBlock read(BlockMeta meta, ByteBuffer buffer) throws IOException {
-    CompressedBlock compressedBlock = null;
-
-    // TODO
-
-    return compressedBlock;
-  }
-
-  @Override
-  public void close() throws IOException {
-
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/CompressedBlockWriter.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/CompressedBlockWriter.java b/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/CompressedBlockWriter.java
deleted file mode 100644
index 22a8c91..0000000
--- a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/CompressedBlockWriter.java
+++ /dev/null
@@ -1,62 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package tajo.storage.hcfile;
-
-import org.apache.hadoop.fs.FSDataOutputStream;
-import tajo.catalog.proto.CatalogProtos.CompressType;
-import tajo.catalog.proto.CatalogProtos.DataType;
-import tajo.storage.exception.UnknownCodecException;
-import tajo.storage.hcfile.compress.Codec;
-import tajo.storage.hcfile.writer.Writer;
-
-import java.io.IOException;
-
-public class CompressedBlockWriter implements Writer {
-  private Codec codec;
-  private DataType dataType;
-  private FSDataOutputStream out;
-
-  public CompressedBlockWriter(FSDataOutputStream out,
-                               DataType dataType, CompressType compType)
-      throws UnknownCodecException, IOException {
-    codec = Codec.get(compType);
-    this.dataType = dataType;
-    this.out = out;
-  }
-
-  @Override
-  public void write(Block block) throws IOException {
-    byte[] in = null;
-    byte[] buffer;
-
-    // TODO
-    buffer = codec.compress(in);
-    out.write(buffer);
-  }
-
-  @Override
-  public long getPos() throws IOException {
-    return out.getPos();
-  }
-
-  @Override
-  public void close() throws IOException {
-
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/c1c6f83e/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/HBlockMetaImpl.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/HBlockMetaImpl.java b/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/HBlockMetaImpl.java
deleted file mode 100644
index 584d2ce..0000000
--- a/tajo-core/tajo-core-storage/src/main/java/tajo/storage/hcfile/HBlockMetaImpl.java
+++ /dev/null
@@ -1,94 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package tajo.storage.hcfile;
-
-import tajo.catalog.proto.CatalogProtos.DataType;
-
-/**
- * Meta information for blocks.
- * BlockMeta is not stored and it is only maintained in the memory
- */
-public class HBlockMetaImpl implements BlockMeta {
-  private DataType dataType;
-  private int recordNum;
-  private long startRid;
-  private boolean sorted;
-  private boolean contiguous;
-  private boolean compressed;
-
-  public HBlockMetaImpl() {
-
-  }
-
-  public HBlockMetaImpl(DataType dataType, int recordNum, long startRid,
-                        boolean sorted, boolean contiguous, boolean compressed) {
-    this.set(dataType, recordNum, startRid, sorted, contiguous, compressed);
-  }
-
-  public void set(DataType dataType, int recordNum, long startRid,
-                  boolean sorted, boolean contiguous, boolean compressed) {
-    this.dataType = dataType;
-    this.recordNum = recordNum;
-    this.startRid = startRid;
-    this.sorted = sorted;
-    this.contiguous = contiguous;
-    this.compressed = compressed;
-  }
-
-  @Override
-  public BlockMeta setStartRid(long startRid) {
-    this.startRid = startRid;
-    return this;
-  }
-
-  @Override
-  public BlockMeta setRecordNum(int recordNum) {
-    this.recordNum = recordNum;
-    return this;
-  }
-
-  @Override
-  public DataType getType() {
-    return this.dataType;
-  }
-
-  @Override
-  public int getRecordNum() {
-    return this.recordNum;
-  }
-
-  public long getStartRid() {
-    return this.startRid;
-  }
-
-  @Override
-  public boolean isSorted() {
-    return this.sorted;
-  }
-
-  @Override
-  public boolean isContiguous() {
-    return this.contiguous;
-  }
-
-  @Override
-  public boolean isCompressed() {
-    return this.compressed;
-  }
-}