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:42 UTC

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

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);
   }