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/12/10 03:03:25 UTC

git commit: TAJO-62: Implement Timestamp Datum type. (hyunsik)

Updated Branches:
  refs/heads/master ad6ff6ede -> ea5aa8f47


TAJO-62: Implement Timestamp Datum type. (hyunsik)


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

Branch: refs/heads/master
Commit: ea5aa8f471c6a0cd8d97bb00eaaceabcdc50b666
Parents: ad6ff6e
Author: Hyunsik Choi <hy...@apache.org>
Authored: Mon Dec 9 21:35:27 2013 +0900
Committer: Hyunsik Choi <hy...@apache.org>
Committed: Mon Dec 9 21:39:17 2013 +0900

----------------------------------------------------------------------
 CHANGES.txt                                     |   2 +
 tajo-common/pom.xml                             |   6 +
 .../org/apache/tajo/datum/DatumFactory.java     |   9 ++
 .../org/apache/tajo/datum/TimestampDatum.java   | 140 +++++++++++++++++++
 .../org/apache/tajo/json/CommonGsonHelper.java  |   2 +
 .../apache/tajo/json/TimestampDatumAdapter.java |  41 ++++++
 .../apache/tajo/datum/TestTimestampDatum.java   | 100 +++++++++++++
 .../function/datetime/ToCharTimestamp.java      |  66 +++++++++
 .../engine/function/datetime/ToTimestamp.java   |  44 ++++++
 .../java/org/apache/tajo/master/TajoMaster.java |  16 +++
 .../engine/function/TestDateTimeFunctions.java  |  41 ++++++
 .../engine/query/TestCreateTableStatement.java  |  57 ++++++++
 .../test/queries/create_table_various_types.sql |  48 +++++++
 .../storage/BinarySerializeDeserialize.java     |   3 +-
 .../java/org/apache/tajo/storage/RawFile.java   |   8 ++
 .../tajo/storage/TextSerializeDeserialize.java  |   5 +
 tajo-project/pom.xml                            |  12 +-
 17 files changed, 592 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/ea5aa8f4/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 86facbb..4b55a7c 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -4,6 +4,8 @@ Release 0.8.0 - unreleased
 
   NEW FEATURES
 
+    TAJO-62: Implement Timestamp Datum type. (hyunsik)
+
     TAJO-307: Implement chr(int) function. (DaeMyung Kang via hyunsik)
 
     TAJO-365: Implement degrees/radians function. (DaeMyung Kang via hyunsik)

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/ea5aa8f4/tajo-common/pom.xml
----------------------------------------------------------------------
diff --git a/tajo-common/pom.xml b/tajo-common/pom.xml
index cc969ab..f83b30e 100644
--- a/tajo-common/pom.xml
+++ b/tajo-common/pom.xml
@@ -161,6 +161,12 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xs
       <type>jar</type>
       <scope>compile</scope>
     </dependency>
+    <dependency>
+      <groupId>joda-time</groupId>
+      <artifactId>joda-time</artifactId>
+      <type>jar</type>
+      <scope>compile</scope>
+    </dependency>
   </dependencies>
 
   <profiles>

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/ea5aa8f4/tajo-common/src/main/java/org/apache/tajo/datum/DatumFactory.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/datum/DatumFactory.java b/tajo-common/src/main/java/org/apache/tajo/datum/DatumFactory.java
index 2ce02d4..6fd9c35 100644
--- a/tajo-common/src/main/java/org/apache/tajo/datum/DatumFactory.java
+++ b/tajo-common/src/main/java/org/apache/tajo/datum/DatumFactory.java
@@ -21,6 +21,7 @@ package org.apache.tajo.datum;
 import org.apache.tajo.common.TajoDataTypes.DataType;
 import org.apache.tajo.common.TajoDataTypes.Type;
 import org.apache.tajo.util.Bytes;
+import org.joda.time.DateTime;
 
 public class DatumFactory {
 
@@ -168,6 +169,14 @@ public class DatumFactory {
     return new TextDatum(val);
   }
 
+  public static TimestampDatum createTimeStamp(long instance) {
+    return new TimestampDatum(instance);
+  }
+
+  public static TimestampDatum createTimeStamp(String val) {
+    return new TimestampDatum(DateTime.parse(val));
+  }
+
   public static TextDatum createText(byte[] val) {
     return new TextDatum(val);
   }

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/ea5aa8f4/tajo-common/src/main/java/org/apache/tajo/datum/TimestampDatum.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/datum/TimestampDatum.java b/tajo-common/src/main/java/org/apache/tajo/datum/TimestampDatum.java
new file mode 100644
index 0000000..c9bbd92
--- /dev/null
+++ b/tajo-common/src/main/java/org/apache/tajo/datum/TimestampDatum.java
@@ -0,0 +1,140 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.datum;
+
+import org.apache.tajo.common.TajoDataTypes;
+import org.apache.tajo.datum.exception.InvalidCastException;
+import org.apache.tajo.datum.exception.InvalidOperationException;
+import org.apache.tajo.util.Bytes;
+import org.joda.time.DateTime;
+
+public class TimestampDatum extends Datum {
+  public static final int SIZE = 8;
+  /** ISO 8601/SQL standard format - ex) 1997-12-17 07:37:16-08 */
+  public static final String DEFAULT_FORMAT = "yyyy-MM-dd HH:mm:ss";
+  private DateTime dateTime;
+
+  public TimestampDatum(long instant) {
+    super(TajoDataTypes.Type.TIMESTAMP);
+    dateTime = new DateTime(instant);
+  }
+
+  public TimestampDatum(DateTime dateTime) {
+    super(TajoDataTypes.Type.TIMESTAMP);
+    this.dateTime = dateTime;
+  }
+
+  public TimestampDatum(byte [] bytes) {
+    super(TajoDataTypes.Type.TIMESTAMP);
+    this.dateTime = new DateTime(Bytes.toLong(bytes));
+  }
+
+  public int getYear() {
+    return dateTime.getYear();
+  }
+
+  public int getMonthOfYear() {
+    return dateTime.getMonthOfYear();
+  }
+
+  public int getDayOfWeek() {
+    return dateTime.getDayOfWeek();
+  }
+
+  public int getDayOfMonth() {
+    return dateTime.getDayOfMonth();
+  }
+
+  public int getHourOfDay() {
+    return dateTime.getHourOfDay();
+  }
+
+  public int getSecondOfDay() {
+    return dateTime.getSecondOfDay();
+  }
+
+  public int getSecondOfMinute() {
+    return dateTime.getSecondOfMinute();
+  }
+
+  public int getMillisOfSecond() {
+    return dateTime.getMillisOfSecond();
+  }
+
+  public String toString() {
+    return asChars();
+  }
+
+  @Override
+  public int asInt4() {
+    throw new InvalidCastException();
+  }
+
+  @Override
+  public long asInt8() {
+    return dateTime.getMillis();
+  }
+
+  @Override
+  public float asFloat4() {
+    throw new InvalidCastException();
+  }
+
+  @Override
+  public double asFloat8() {
+    throw new InvalidCastException();
+  }
+
+  @Override
+  public String asChars() {
+    return dateTime.toString(DEFAULT_FORMAT);
+  }
+
+  public String toChars(String format) {
+    return dateTime.toString(format);
+  }
+
+  @Override
+  public int size() {
+    return 8;
+  }
+
+  @Override
+  public byte [] asByteArray() {
+    return Bytes.toBytes(dateTime.getMillis());
+  }
+
+  @Override
+  public int compareTo(Datum datum) {
+    if (datum.type() == TajoDataTypes.Type.TIMESTAMP) {
+      return dateTime.compareTo(((TimestampDatum)datum).dateTime);
+    } else {
+      throw new InvalidOperationException();
+    }
+  }
+
+  public boolean equals(Object obj) {
+    if (obj instanceof TimestampDatum) {
+      TimestampDatum another = (TimestampDatum) obj;
+      return dateTime.isEqual(another.dateTime);
+    } else {
+      throw new InvalidOperationException();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/ea5aa8f4/tajo-common/src/main/java/org/apache/tajo/json/CommonGsonHelper.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/json/CommonGsonHelper.java b/tajo-common/src/main/java/org/apache/tajo/json/CommonGsonHelper.java
index 3badbc0..b6d6779 100644
--- a/tajo-common/src/main/java/org/apache/tajo/json/CommonGsonHelper.java
+++ b/tajo-common/src/main/java/org/apache/tajo/json/CommonGsonHelper.java
@@ -21,6 +21,7 @@ package org.apache.tajo.json;
 import com.google.gson.Gson;
 import com.google.gson.GsonBuilder;
 import org.apache.tajo.datum.Datum;
+import org.apache.tajo.datum.TimestampDatum;
 import org.apache.tajo.util.TUtil;
 
 import java.lang.reflect.Type;
@@ -35,6 +36,7 @@ public class CommonGsonHelper {
 	
 	private static Map<Type, GsonSerDerAdapter> registerAdapters() {
     Map<Type, GsonSerDerAdapter> adapters = TUtil.newHashMap();
+    adapters.put(TimestampDatum.class, new TimestampDatumAdapter());
     adapters.put(Datum.class, new DatumAdapter());
 
     return adapters;

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/ea5aa8f4/tajo-common/src/main/java/org/apache/tajo/json/TimestampDatumAdapter.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/json/TimestampDatumAdapter.java b/tajo-common/src/main/java/org/apache/tajo/json/TimestampDatumAdapter.java
new file mode 100644
index 0000000..f963eb2
--- /dev/null
+++ b/tajo-common/src/main/java/org/apache/tajo/json/TimestampDatumAdapter.java
@@ -0,0 +1,41 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.json;
+
+import com.google.gson.*;
+import org.apache.tajo.datum.TimestampDatum;
+
+import java.lang.reflect.Type;
+
+public class TimestampDatumAdapter implements GsonSerDerAdapter<TimestampDatum> {
+
+	@Override
+	public TimestampDatum deserialize(JsonElement json, Type typeOfT,
+                                    JsonDeserializationContext context) throws JsonParseException {
+		JsonObject jsonObject = json.getAsJsonObject();
+		return new TimestampDatum(jsonObject.get("val").getAsLong());
+	}
+
+	@Override
+	public JsonElement serialize(TimestampDatum src, Type typeOfSrc, JsonSerializationContext context) {
+		JsonObject jsonObj = new JsonObject();
+		jsonObj.addProperty("value", src.asInt8());
+		return jsonObj;
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/ea5aa8f4/tajo-common/src/test/java/org/apache/tajo/datum/TestTimestampDatum.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/test/java/org/apache/tajo/datum/TestTimestampDatum.java b/tajo-common/src/test/java/org/apache/tajo/datum/TestTimestampDatum.java
new file mode 100644
index 0000000..7a62d06
--- /dev/null
+++ b/tajo-common/src/test/java/org/apache/tajo/datum/TestTimestampDatum.java
@@ -0,0 +1,100 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.datum;
+
+import org.apache.tajo.common.TajoDataTypes.Type;
+import org.apache.tajo.datum.exception.InvalidCastException;
+import org.junit.Test;
+
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+
+public class TestTimestampDatum {
+	
+	@Test
+	public final void testType() {
+    long instance = 1386577582;
+		Datum d = DatumFactory.createTimeStamp(instance);
+    assertEquals(Type.TIMESTAMP, d.type());
+	}
+	
+	@Test(expected = InvalidCastException.class)
+	public final void testAsInt4() {
+    long instance = 1386577582;
+    Datum d = DatumFactory.createTimeStamp(instance);
+    d.asInt4();
+	}
+
+	@Test
+	public final void testAsInt8() {
+    long instance = 1386577582;
+    Datum d = DatumFactory.createTimeStamp(instance);
+    assertEquals(instance, d.asInt8());
+	}
+
+  @Test(expected = InvalidCastException.class)
+	public final void testAsFloat4() {
+    long instance = 1386577582;
+    Datum d = DatumFactory.createTimeStamp(instance);
+    d.asFloat4();
+	}
+
+  @Test(expected = InvalidCastException.class)
+	public final void testAsFloat8() {
+    long instance = 1386577582;
+    Datum d = DatumFactory.createTimeStamp(instance);
+    d.asFloat8();
+	}
+
+	@Test
+	public final void testAsText() {
+    long instance = 1386577582;
+    Datum d = DatumFactory.createTimeStamp(instance);
+    System.out.println(d.asChars());
+	}
+
+  @Test
+  public final void testAsByteArray() {
+    long instance = 1386577582;
+    TimestampDatum d = DatumFactory.createTimeStamp(instance);
+    TimestampDatum copy = new TimestampDatum(d.asByteArray());
+    assertEquals(d.asInt8(), copy.asInt8());
+  }
+	
+	@Test
+  public final void testSize() {
+    long instance = 1386577582;
+    Datum d = DatumFactory.createTimeStamp(instance);
+    assertEquals(TimestampDatum.SIZE, d.size());
+  }
+
+  @Test
+  public final void testAsTextBytes() {
+    long instance = 1386577582;
+    Datum d = DatumFactory.createTimeStamp(instance);
+    assertArrayEquals(d.toString().getBytes(), d.asTextBytes());
+  }
+
+  @Test
+  public final void testToJson() {
+    long instance = 1386577582;
+    Datum d = DatumFactory.createTimeStamp(instance);
+    System.out.println(d.toJson());
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/ea5aa8f4/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/datetime/ToCharTimestamp.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/datetime/ToCharTimestamp.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/datetime/ToCharTimestamp.java
new file mode 100644
index 0000000..42f406e
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/datetime/ToCharTimestamp.java
@@ -0,0 +1,66 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.engine.function.datetime;
+
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.datum.Datum;
+import org.apache.tajo.datum.DatumFactory;
+import org.apache.tajo.datum.NullDatum;
+import org.apache.tajo.engine.eval.FunctionEval;
+import org.apache.tajo.engine.function.GeneralFunction;
+import org.apache.tajo.storage.Tuple;
+import org.joda.time.format.DateTimeFormat;
+import org.joda.time.format.DateTimeFormatter;
+
+import static org.apache.tajo.common.TajoDataTypes.Type.INT8;
+import static org.apache.tajo.common.TajoDataTypes.Type.TEXT;
+
+public class ToCharTimestamp extends GeneralFunction {
+  private boolean constantFormat;
+  private DateTimeFormatter formatter;
+
+  public ToCharTimestamp() {
+    super(new Column[] {
+        new Column("timestamp", INT8),
+        new Column("format", TEXT)
+    });
+  }
+
+  @Override
+  public void init(FunctionEval.ParamType[] paramTypes) {
+    if (paramTypes[1] == FunctionEval.ParamType.CONSTANT) {
+      constantFormat = true;
+    }
+  }
+
+
+  @Override
+  public Datum eval(Tuple params) {
+    Datum valueDatum = params.get(0);
+    Datum pattern = params.get(1);
+    if(valueDatum instanceof NullDatum || params instanceof NullDatum) {
+      return NullDatum.get();
+    }
+
+    if (formatter == null || !constantFormat) {
+      formatter = DateTimeFormat.forPattern(pattern.asChars());
+    }
+    return DatumFactory.createText(formatter.print(valueDatum.asInt8()));
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/ea5aa8f4/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/datetime/ToTimestamp.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/datetime/ToTimestamp.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/datetime/ToTimestamp.java
new file mode 100644
index 0000000..f333d20
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/datetime/ToTimestamp.java
@@ -0,0 +1,44 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.engine.function.datetime;
+
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.datum.Datum;
+import org.apache.tajo.datum.DatumFactory;
+import org.apache.tajo.datum.NullDatum;
+import org.apache.tajo.engine.function.GeneralFunction;
+import org.apache.tajo.storage.Tuple;
+
+import static org.apache.tajo.common.TajoDataTypes.Type.INT8;
+
+public class ToTimestamp extends GeneralFunction {
+  public ToTimestamp() {
+    super(new Column[] {new Column("timestamp", INT8)});
+  }
+
+
+  @Override
+  public Datum eval(Tuple params) {
+    Datum value = params.get(0);
+    if (value instanceof NullDatum) {
+      return NullDatum.get();
+    }
+    return DatumFactory.createTimeStamp(value.asInt8());
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/ea5aa8f4/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoMaster.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoMaster.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoMaster.java
index 40696b9..cc8c6a6 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoMaster.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoMaster.java
@@ -46,6 +46,8 @@ import org.apache.tajo.conf.TajoConf.ConfVars;
 import org.apache.tajo.engine.function.Country;
 import org.apache.tajo.engine.function.InCountry;
 import org.apache.tajo.engine.function.builtin.*;
+import org.apache.tajo.engine.function.datetime.ToCharTimestamp;
+import org.apache.tajo.engine.function.datetime.ToTimestamp;
 import org.apache.tajo.engine.function.math.*;
 import org.apache.tajo.engine.function.string.*;
 import org.apache.tajo.master.querymaster.QueryJobManager;
@@ -707,6 +709,20 @@ public class TajoMaster extends CompositeService {
             CatalogUtil.newSimpleDataType(Type.FLOAT8),
             CatalogUtil.newSimpleDataTypeArray(Type.FLOAT8)));
 
+    // Date Time
+    sqlFuncs.add(
+        new FunctionDesc("to_timestamp", ToTimestamp.class, FunctionType.GENERAL,
+            CatalogUtil.newSimpleDataType(Type.TIMESTAMP),
+            CatalogUtil.newSimpleDataTypeArray(Type.INT8)));
+    sqlFuncs.add(
+        new FunctionDesc("to_timestamp", ToTimestamp.class, FunctionType.GENERAL,
+            CatalogUtil.newSimpleDataType(Type.TIMESTAMP),
+            CatalogUtil.newSimpleDataTypeArray(Type.INT4)));
+    sqlFuncs.add(
+        new FunctionDesc("to_char", ToCharTimestamp.class, FunctionType.GENERAL,
+            CatalogUtil.newSimpleDataType(Type.TEXT),
+            CatalogUtil.newSimpleDataTypeArray(Type.TIMESTAMP, Type.TEXT)));
+
     return sqlFuncs;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/ea5aa8f4/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/function/TestDateTimeFunctions.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/function/TestDateTimeFunctions.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/function/TestDateTimeFunctions.java
new file mode 100644
index 0000000..eb22a96
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/function/TestDateTimeFunctions.java
@@ -0,0 +1,41 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.engine.function;
+
+
+import org.apache.tajo.engine.eval.ExprTestBase;
+import org.junit.Test;
+
+import java.io.IOException;
+
+public class TestDateTimeFunctions extends ExprTestBase {
+
+  @Test
+  public void testToTimestamp() throws IOException {
+    testSimpleEval("select to_timestamp(cast (1386577582 as int8));", new String[]{"1970-01-17 10:09:37"});
+    testSimpleEval("select to_timestamp(cast ('1386577582' as int8));", new String[]{"1970-01-17 10:09:37"});
+    testSimpleEval("select to_timestamp(cast (1386577582 as int8)) < to_timestamp(cast (1386577583 as int8));",
+        new String[]{"t"});
+  }
+
+  @Test
+  public void testToChar() throws IOException {
+    testSimpleEval("select to_char(to_timestamp(1386577582), 'yyyy-MM');", new String[]{"1970-01"});
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/ea5aa8f4/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/query/TestCreateTableStatement.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/query/TestCreateTableStatement.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/query/TestCreateTableStatement.java
new file mode 100644
index 0000000..416f613
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/query/TestCreateTableStatement.java
@@ -0,0 +1,57 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.engine.query;
+
+import org.apache.tajo.IntegrationTest;
+import org.apache.tajo.TajoTestingCluster;
+import org.apache.tajo.TpchTestBase;
+import org.apache.tajo.catalog.CatalogService;
+import org.apache.tajo.util.FileUtil;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import java.io.File;
+import java.io.IOException;
+import java.sql.ResultSet;
+
+import static org.junit.Assert.assertTrue;
+
+@Category(IntegrationTest.class)
+public class TestCreateTableStatement {
+  private static TpchTestBase tpch;
+  public TestCreateTableStatement() throws IOException {
+    super();
+  }
+
+  @BeforeClass
+  public static void setUp() throws Exception {
+    tpch = TpchTestBase.getInstance();
+  }
+
+  @Test
+  public final void testVariousTypes() throws Exception {
+    String tableName ="various_types";
+    ResultSet res = tpch.execute(FileUtil.readTextFile(new File("src/test/queries/create_table_various_types.sql")));
+    res.close();
+    TajoTestingCluster cluster = tpch.getTestingCluster();
+    CatalogService catalog = cluster.getMaster().getCatalog();
+    assertTrue(catalog.existsTable(tableName));
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/ea5aa8f4/tajo-core/tajo-core-backend/src/test/queries/create_table_various_types.sql
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/queries/create_table_various_types.sql b/tajo-core/tajo-core-backend/src/test/queries/create_table_various_types.sql
new file mode 100644
index 0000000..92a4990
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/test/queries/create_table_various_types.sql
@@ -0,0 +1,48 @@
+create table various_types (
+  col0 bit,
+  col1 BIT(10),
+  col2 bit varying,
+  col3 bit VARYING(10),
+  col4 tinyint,
+  col5 smallInt,
+  col6 integer,
+  col7 biginT,
+  col8 real,
+  col9 float,
+  col10 float(53),
+  col11 double,
+  col12 doublE precision,
+  col13 numeric,
+  col14 numeric(10),
+  col15 numeric(10,2),
+  col16 decimal,
+  col17 decimal(10),
+  col18 decimal(10,2),
+  col19 char,
+  col20 character,
+  col21 chaR(10),
+  col22 character(10),
+  col23 varchar,
+  col24 character varying,
+  col25 varchar(255),
+  col26 character varying (255),
+  col27 nchar,
+  col28 nchar(255),
+  col29 national character,
+  col30 national character(255),
+  col31 nvarchar,
+  col32 nvarchar(255),
+  col33 natIonal character varying,
+  col34 national character varying (255),
+  col35 date,
+  col36 time,
+  col37 timetz,
+  col38 time With time zone,
+  col39 timesTamptz,
+  col40 timestamp with time zone,
+  col41 binary,
+  col42 binary(10),
+  col43 varbinary(10),
+  col44 binary Varying(10),
+  col45 blOb
+);
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/ea5aa8f4/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/BinarySerializeDeserialize.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/BinarySerializeDeserialize.java b/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/BinarySerializeDeserialize.java
index 4c3b6ff..27de655 100644
--- a/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/BinarySerializeDeserialize.java
+++ b/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/BinarySerializeDeserialize.java
@@ -87,8 +87,9 @@ public class BinarySerializeDeserialize implements SerializeDeserialize {
         out.write(bytes, 0, length);
         break;
       case NULL_TYPE:
-      default:
         break;
+      default:
+        throw new IOException("Does not support type");
     }
     return length;
   }

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/ea5aa8f4/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/RawFile.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/RawFile.java b/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/RawFile.java
index 88a4005..e3ddd09 100644
--- a/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/RawFile.java
+++ b/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/RawFile.java
@@ -203,6 +203,10 @@ public class RawFile {
             tuple.put(i, DatumFactory.createText(new String(strBytes2)));
             break;
 
+          case TIMESTAMP:
+            tuple.put(i, DatumFactory.createTimeStamp(buffer.getLong()));
+            break;
+
           case BLOB : {
             //byte [] rawBytes = getColumnBytes();
             int byteSize = buffer.getInt();
@@ -447,6 +451,10 @@ public class RawFile {
             buffer.put(strBytes2);
             break;
 
+          case TIMESTAMP:
+            buffer.putLong(t.get(i).asInt8());
+            break;
+
           case BLOB : {
             byte [] rawBytes = t.get(i).asByteArray();
             if (flushBufferAndReplace(recordOffset, rawBytes.length + 4)) {

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/ea5aa8f4/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/TextSerializeDeserialize.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/TextSerializeDeserialize.java b/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/TextSerializeDeserialize.java
index 67fb8b8..4280783 100644
--- a/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/TextSerializeDeserialize.java
+++ b/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/TextSerializeDeserialize.java
@@ -76,6 +76,7 @@ public class TextSerializeDeserialize implements SerializeDeserialize {
       case FLOAT4:
       case FLOAT8:
       case INET4:
+      case TIMESTAMP:
         bytes = datum.asTextBytes();
         length = bytes.length;
         out.write(bytes);
@@ -143,6 +144,10 @@ public class TextSerializeDeserialize implements SerializeDeserialize {
             : DatumFactory.createText(chars);
         break;
       }
+      case TIMESTAMP:
+        datum = isNull(bytes, offset, length, nullCharacters) ? NullDatum.get()
+            : DatumFactory.createTimeStamp(new String(bytes, offset, length));
+        break;
       case PROTOBUF: {
         if (isNull(bytes, offset, length, nullCharacters)) {
           datum = NullDatum.get();

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/ea5aa8f4/tajo-project/pom.xml
----------------------------------------------------------------------
diff --git a/tajo-project/pom.xml b/tajo-project/pom.xml
index 55aa52d..089f80c 100644
--- a/tajo-project/pom.xml
+++ b/tajo-project/pom.xml
@@ -788,18 +788,16 @@
         <version>1.9.5-rc1</version>
         <scope>test</scope>
       </dependency>
-      <!--
-      <dependency>
-        <groupId>org.jboss.netty</groupId>
-        <artifactId>netty</artifactId>
-        <version>3.2.4.Final</version>
-      </dependency>
-      -->
       <dependency>
         <groupId>io.netty</groupId>
         <artifactId>netty</artifactId>
         <version>3.6.6.Final</version>
       </dependency>
+      <dependency>
+        <groupId>joda-time</groupId>
+        <artifactId>joda-time</artifactId>
+        <version>2.3</version>
+      </dependency>
     </dependencies>
   </dependencyManagement>
   <profiles>