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

[09/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-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();
         }
       }