You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@tajo.apache.org by ji...@apache.org on 2013/12/28 07:36:28 UTC

[31/50] [abbrv] git commit: TAJO-437: Timestamp literal support. (hyunsik)

TAJO-437: Timestamp literal support. (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/4319ded3
Tree: http://git-wip-us.apache.org/repos/asf/incubator-tajo/tree/4319ded3
Diff: http://git-wip-us.apache.org/repos/asf/incubator-tajo/diff/4319ded3

Branch: refs/heads/DAG-execplan
Commit: 4319ded3aa974dfca35fd2630b97b8f2601507c4
Parents: e8c5c27
Author: Hyunsik Choi <hy...@apache.org>
Authored: Mon Dec 23 14:30:00 2013 +0900
Committer: Hyunsik Choi <hy...@apache.org>
Committed: Mon Dec 23 14:30:00 2013 +0900

----------------------------------------------------------------------
 CHANGES.txt                                     |  2 +
 .../java/org/apache/tajo/algebra/DateValue.java | 63 ++++++++++++++++
 .../org/apache/tajo/algebra/NullLiteral.java    | 31 ++++++++
 .../java/org/apache/tajo/algebra/NullValue.java | 31 --------
 .../java/org/apache/tajo/algebra/OpType.java    | 35 ++++-----
 .../java/org/apache/tajo/algebra/TimeValue.java | 78 ++++++++++++++++++++
 .../apache/tajo/algebra/TimestampLiteral.java   | 57 ++++++++++++++
 .../org/apache/tajo/datum/TimestampDatum.java   |  4 +
 .../apache/tajo/json/ClassNameDeserializer.java | 44 -----------
 .../org/apache/tajo/json/CommonGsonHelper.java  |  6 --
 .../org/apache/tajo/json/DateDatumAdapter.java  | 41 ----------
 .../java/org/apache/tajo/json/DatumAdapter.java | 31 ++++++--
 .../java/org/apache/tajo/json/GsonObject.java   |  2 +-
 .../org/apache/tajo/json/PathDeserializer.java  | 40 ----------
 .../org/apache/tajo/json/PathSerializer.java    |  3 +-
 .../org/apache/tajo/json/TimeDatumAdapter.java  | 41 ----------
 .../apache/tajo/json/TimestampDatumAdapter.java | 41 ----------
 .../org/apache/tajo/engine/parser/SQLLexer.g4   |  6 +-
 .../org/apache/tajo/engine/parser/SQLParser.g4  |  9 +++
 .../tajo/engine/parser/HiveConverter.java       |  4 +-
 .../apache/tajo/engine/parser/SQLAnalyzer.java  | 49 +++++++++++-
 .../tajo/engine/planner/AlgebraVisitor.java     |  9 ++-
 .../tajo/engine/planner/BaseAlgebraVisitor.java | 45 ++++++++---
 .../tajo/engine/planner/LogicalPlanner.java     | 61 ++++++++++++++-
 .../apache/tajo/engine/eval/ExprTestBase.java   |  6 +-
 .../tajo/engine/eval/TestSQLDateTimeTypes.java  | 32 ++++++++
 .../tajo/engine/eval/TestSQLExpression.java     |  4 +-
 .../src/test/queries/complex_union_1.sql        |  4 +-
 28 files changed, 477 insertions(+), 302 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/4319ded3/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 24f574a..2ef8bf3 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -4,6 +4,8 @@ Release 0.8.0 - unreleased
 
   NEW FEATURES
 
+    TAJO-437: Timestamp literal support. (hyunsik)
+
     TAJO-382: Implement encode/decode functions. (Seungun Choe via jihoon)
 
     TAJO-436: Implement ceiling(FLOAT8) function. (DaeMyung Kang via hyunsik)

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/4319ded3/tajo-algebra/src/main/java/org/apache/tajo/algebra/DateValue.java
----------------------------------------------------------------------
diff --git a/tajo-algebra/src/main/java/org/apache/tajo/algebra/DateValue.java b/tajo-algebra/src/main/java/org/apache/tajo/algebra/DateValue.java
new file mode 100644
index 0000000..ad9d575
--- /dev/null
+++ b/tajo-algebra/src/main/java/org/apache/tajo/algebra/DateValue.java
@@ -0,0 +1,63 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.algebra;
+
+import com.google.common.base.Objects;
+
+public class DateValue {
+  private String years;
+  private String months;
+  private String days;
+
+  public DateValue(String years, String months, String days) {
+
+    this.years = years;
+    this.months = months;
+    this.days = days;
+  }
+
+  public String getYears() {
+    return years;
+  }
+
+  public String getMonths() {
+    return months;
+  }
+
+  public String getDays() {
+    return days;
+  }
+
+  public String toString() {
+    return String.format("%s-%s-%s", years, months, days);
+  }
+
+  public int hashCode() {
+    return Objects.hashCode(years, months, days);
+  }
+
+  @Override
+  public boolean equals(Object object) {
+    if (object instanceof DateValue) {
+      DateValue another = (DateValue) object;
+      return years.equals(another.years) && months.equals(another.months) && days.equals(another.days);
+    }
+    return false;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/4319ded3/tajo-algebra/src/main/java/org/apache/tajo/algebra/NullLiteral.java
----------------------------------------------------------------------
diff --git a/tajo-algebra/src/main/java/org/apache/tajo/algebra/NullLiteral.java b/tajo-algebra/src/main/java/org/apache/tajo/algebra/NullLiteral.java
new file mode 100644
index 0000000..e4e7749
--- /dev/null
+++ b/tajo-algebra/src/main/java/org/apache/tajo/algebra/NullLiteral.java
@@ -0,0 +1,31 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.algebra;
+
+public class NullLiteral extends Expr {
+
+  public NullLiteral() {
+    super(OpType.NullLiteral);
+  }
+
+  @Override
+  boolean equalsTo(Expr expr) {
+    return expr instanceof NullLiteral;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/4319ded3/tajo-algebra/src/main/java/org/apache/tajo/algebra/NullValue.java
----------------------------------------------------------------------
diff --git a/tajo-algebra/src/main/java/org/apache/tajo/algebra/NullValue.java b/tajo-algebra/src/main/java/org/apache/tajo/algebra/NullValue.java
deleted file mode 100644
index 9be1a2f..0000000
--- a/tajo-algebra/src/main/java/org/apache/tajo/algebra/NullValue.java
+++ /dev/null
@@ -1,31 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tajo.algebra;
-
-public class NullValue extends Expr {
-
-  public NullValue() {
-    super(OpType.Null);
-  }
-
-  @Override
-  boolean equalsTo(Expr expr) {
-    return expr instanceof NullValue;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/4319ded3/tajo-algebra/src/main/java/org/apache/tajo/algebra/OpType.java
----------------------------------------------------------------------
diff --git a/tajo-algebra/src/main/java/org/apache/tajo/algebra/OpType.java b/tajo-algebra/src/main/java/org/apache/tajo/algebra/OpType.java
index cddba5d..a58f103 100644
--- a/tajo-algebra/src/main/java/org/apache/tajo/algebra/OpType.java
+++ b/tajo-algebra/src/main/java/org/apache/tajo/algebra/OpType.java
@@ -25,20 +25,21 @@ import java.lang.reflect.Type;
 public enum OpType {
 
   // relational operators
-  Projection(Projection.class), // 0
-  Limit(Limit.class), // 1
-  Sort(Sort.class), // 2
-  Having(Having.class), // 3
-  Aggregation(Aggregation.class), // 4
-  Join(Join.class), // 5
-  Filter(Selection.class), // 6
-  Union(SetOperation.class), // 7
-  Except(SetOperation.class), // 8
-  Intersect(SetOperation.class), // 9
-  SimpleTableSubQuery(SimpleTableSubQuery.class), // 10
-  TablePrimaryTableSubQuery(TablePrimarySubQuery.class), // 11
-  RelationList(RelationList.class), // 12
-  Relation(Relation.class), // 13
+  Projection(Projection.class),
+  Limit(Limit.class),
+  Sort(Sort.class),
+  Having(Having.class),
+  Aggregation(Aggregation.class),
+  Join(Join.class),
+  Filter(Selection.class),
+  Union(SetOperation.class),
+  Except(SetOperation.class),
+  Intersect(SetOperation.class),
+  SimpleTableSubQuery(SimpleTableSubQuery.class),
+  TablePrimaryTableSubQuery(TablePrimarySubQuery.class),
+  RelationList(RelationList.class),
+  Relation(Relation.class),
+  ScalarSubQuery(ScalarSubQuery.class),
 
   // Data definition language
   CreateTable(CreateTable.class),
@@ -92,11 +93,11 @@ public enum OpType {
   GeneralSetFunction(GeneralSetFunctionExpr.class),
 
   // Literal
+  DataType(DataTypeExpr.class),
   Cast(CastExpr.class),
-  ScalarSubQuery(ScalarSubQuery.class),
   Literal(LiteralValue.class),
-  Null(NullValue.class),
-  DataType(DataTypeExpr.class);
+  NullLiteral(NullLiteral.class),
+  TimestampLiteral(TimestampLiteral.class);
 
   private Class baseClass;
 

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/4319ded3/tajo-algebra/src/main/java/org/apache/tajo/algebra/TimeValue.java
----------------------------------------------------------------------
diff --git a/tajo-algebra/src/main/java/org/apache/tajo/algebra/TimeValue.java b/tajo-algebra/src/main/java/org/apache/tajo/algebra/TimeValue.java
new file mode 100644
index 0000000..916c3d2
--- /dev/null
+++ b/tajo-algebra/src/main/java/org/apache/tajo/algebra/TimeValue.java
@@ -0,0 +1,78 @@
+/**
+ * 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.algebra;
+
+import com.google.common.base.Objects;
+
+public class TimeValue {
+  private String hours;
+  private String minutes;
+  private String seconds;
+  private String secondsFraction; // optional
+
+  public TimeValue(String hours, String minutes, String seconds) {
+    this.hours = hours;
+    this.minutes = minutes;
+    this.seconds = seconds;
+  }
+
+  public String getHours() {
+    return hours;
+  }
+
+  public String getMinutes() {
+    return minutes;
+  }
+
+  public String getSeconds() {
+    return seconds;
+  }
+
+  public boolean hasSecondsFraction() {
+    return secondsFraction != null;
+  }
+
+  public void setSecondsFraction(String secondsFraction) {
+    this.secondsFraction = secondsFraction;
+  }
+
+  public String getSecondsFraction() {
+    return secondsFraction;
+  }
+
+  public boolean equals(Object object) {
+    if (object instanceof TimeValue) {
+      TimeValue another = (TimeValue) object;
+      return hours.equals(another.hours) && minutes.equals(another.minutes) && seconds.equals(another.seconds);
+    }
+    return false;
+  }
+
+  public String toString() {
+    if (hasSecondsFraction()) {
+      return String.format("%s:%s:%s.%s", hours, minutes, seconds, secondsFraction);
+    } else {
+      return String.format("%s:%s:%s", hours, minutes, seconds);
+    }
+  }
+
+  public int hashCode() {
+    return Objects.hashCode(hours, minutes, seconds);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/4319ded3/tajo-algebra/src/main/java/org/apache/tajo/algebra/TimestampLiteral.java
----------------------------------------------------------------------
diff --git a/tajo-algebra/src/main/java/org/apache/tajo/algebra/TimestampLiteral.java b/tajo-algebra/src/main/java/org/apache/tajo/algebra/TimestampLiteral.java
new file mode 100644
index 0000000..cb6f5a1
--- /dev/null
+++ b/tajo-algebra/src/main/java/org/apache/tajo/algebra/TimestampLiteral.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.algebra;
+
+import com.google.common.base.Objects;
+
+public class TimestampLiteral extends Expr {
+  private DateValue date;
+  private TimeValue time;
+
+  public TimestampLiteral(DateValue date, TimeValue time) {
+    super(OpType.TimestampLiteral);
+    this.date = date;
+    this.time = time;
+  }
+
+  public DateValue getDate() {
+    return date;
+  }
+
+  public TimeValue getTime() {
+    return time;
+  }
+
+  public String toString() {
+    return date + " " + time;
+  }
+
+  public int hashCode() {
+    return Objects.hashCode(date, time);
+  }
+
+  @Override
+  boolean equalsTo(Expr expr) {
+    if (expr instanceof TimestampLiteral) {
+      TimestampLiteral another = (TimestampLiteral) expr;
+      return date.equals(another.date) && time.equals(another.time);
+    }
+    return false;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/4319ded3/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
index 83e10b4..4c53caa 100644
--- a/tajo-common/src/main/java/org/apache/tajo/datum/TimestampDatum.java
+++ b/tajo-common/src/main/java/org/apache/tajo/datum/TimestampDatum.java
@@ -53,6 +53,10 @@ public class TimestampDatum extends Datum {
     this.dateTime = DateTime.parse(datetime, DEFAULT_FORMATTER);
   }
 
+  public DateTime getDateTime() {
+    return dateTime;
+  }
+
   public int getYear() {
     return dateTime.getYear();
   }

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

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/4319ded3/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 947c5a3..3badbc0 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
@@ -20,10 +20,7 @@ package org.apache.tajo.json;
 
 import com.google.gson.Gson;
 import com.google.gson.GsonBuilder;
-import org.apache.tajo.datum.DateDatum;
-import org.apache.tajo.datum.TimeDatum;
 import org.apache.tajo.datum.Datum;
-import org.apache.tajo.datum.TimestampDatum;
 import org.apache.tajo.util.TUtil;
 
 import java.lang.reflect.Type;
@@ -39,9 +36,6 @@ public class CommonGsonHelper {
 	private static Map<Type, GsonSerDerAdapter> registerAdapters() {
     Map<Type, GsonSerDerAdapter> adapters = TUtil.newHashMap();
     adapters.put(Datum.class, new DatumAdapter());
-    adapters.put(TimestampDatum.class, new TimestampDatumAdapter());
-    adapters.put(DateDatum.class, new DateDatumAdapter());
-    adapters.put(TimeDatum.class, new TimeDatumAdapter());
 
     return adapters;
 	}

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

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/4319ded3/tajo-common/src/main/java/org/apache/tajo/json/DatumAdapter.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/json/DatumAdapter.java b/tajo-common/src/main/java/org/apache/tajo/json/DatumAdapter.java
index fe20a35..278caec 100644
--- a/tajo-common/src/main/java/org/apache/tajo/json/DatumAdapter.java
+++ b/tajo-common/src/main/java/org/apache/tajo/json/DatumAdapter.java
@@ -20,8 +20,7 @@ package org.apache.tajo.json;
 
 import com.google.gson.*;
 import org.apache.tajo.common.TajoDataTypes;
-import org.apache.tajo.datum.Datum;
-import org.apache.tajo.datum.DatumFactory;
+import org.apache.tajo.datum.*;
 
 import java.lang.reflect.Type;
 
@@ -32,16 +31,36 @@ public class DatumAdapter implements GsonSerDerAdapter<Datum> {
 			JsonDeserializationContext context) throws JsonParseException {
 		JsonObject jsonObject = json.getAsJsonObject();
 		String typeName = jsonObject.get("type").getAsString();
-		return context.deserialize(jsonObject.get("body"),
-        DatumFactory.getDatumClass(TajoDataTypes.Type.valueOf(typeName)));
+    TajoDataTypes.Type type = TajoDataTypes.Type.valueOf(jsonObject.get("type").getAsString());
+    switch (type) {
+    case DATE:
+      return new DateDatum(jsonObject.get("value").getAsInt());
+    case TIME:
+      return new TimeDatum(jsonObject.get("value").getAsLong());
+    case TIMESTAMP:
+      return new TimestampDatum(jsonObject.get("value").getAsLong());
+    default:
+      return context.deserialize(jsonObject.get("body"),
+          DatumFactory.getDatumClass(TajoDataTypes.Type.valueOf(typeName)));
+    }
 	}
 
 	@Override
 	public JsonElement serialize(Datum src, Type typeOfSrc, JsonSerializationContext context) {
 		JsonObject jsonObj = new JsonObject();
 		jsonObj.addProperty("type", src.type().name());
-		JsonElement jsonElem = context.serialize(src);
-		jsonObj.add("body", jsonElem);
+    switch (src.type()) {
+    case DATE:
+      jsonObj.addProperty("value", src.asInt4());
+      break;
+    case TIME:
+    case TIMESTAMP:
+      jsonObj.addProperty("value", src.asInt8());
+      break;
+    default:
+      jsonObj.add("body", context.serialize(src));
+    }
+
 		return jsonObj;
 	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/4319ded3/tajo-common/src/main/java/org/apache/tajo/json/GsonObject.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/json/GsonObject.java b/tajo-common/src/main/java/org/apache/tajo/json/GsonObject.java
index b7e69c1..3c49a21 100644
--- a/tajo-common/src/main/java/org/apache/tajo/json/GsonObject.java
+++ b/tajo-common/src/main/java/org/apache/tajo/json/GsonObject.java
@@ -19,5 +19,5 @@
 package org.apache.tajo.json;
 
 public interface GsonObject {
-  String toJson();
+  public String toJson();
 }

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

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/4319ded3/tajo-common/src/main/java/org/apache/tajo/json/PathSerializer.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/json/PathSerializer.java b/tajo-common/src/main/java/org/apache/tajo/json/PathSerializer.java
index bcd1bcf..c286ae3 100644
--- a/tajo-common/src/main/java/org/apache/tajo/json/PathSerializer.java
+++ b/tajo-common/src/main/java/org/apache/tajo/json/PathSerializer.java
@@ -35,8 +35,7 @@ public class PathSerializer implements GsonSerDerAdapter<Path> {
 	}
 
   @Override
-  public Path deserialize(JsonElement arg0, Type arg1,
-                          JsonDeserializationContext context) throws JsonParseException {
+  public Path deserialize(JsonElement arg0, Type arg1, JsonDeserializationContext context) throws JsonParseException {
     return new Path(arg0.getAsJsonPrimitive().getAsString());
   }
 }

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

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/4319ded3/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
deleted file mode 100644
index 6b362a4..0000000
--- a/tajo-common/src/main/java/org/apache/tajo/json/TimestampDatumAdapter.java
+++ /dev/null
@@ -1,41 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package 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("val", src.asInt8());
-		return jsonObj;
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/4319ded3/tajo-core/tajo-core-backend/src/main/antlr4/org/apache/tajo/engine/parser/SQLLexer.g4
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/antlr4/org/apache/tajo/engine/parser/SQLLexer.g4 b/tajo-core/tajo-core-backend/src/main/antlr4/org/apache/tajo/engine/parser/SQLLexer.g4
index 3bc9ff4..184304a 100644
--- a/tajo-core/tajo-core-backend/src/main/antlr4/org/apache/tajo/engine/parser/SQLLexer.g4
+++ b/tajo-core/tajo-core-backend/src/main/antlr4/org/apache/tajo/engine/parser/SQLLexer.g4
@@ -391,6 +391,8 @@ MODULAR : '%';
 DOT : '.';
 UNDERLINE : '_';
 VERTICAL_BAR : '|';
+QUOTE : '\'';
+DOUBLE_QUOTE : '"';
 
 NUMBER : Digit+;
 
@@ -422,6 +424,7 @@ Identifier
   | Regular_Identifier
   ;
 
+fragment
 Regular_Identifier
   : ('a'..'z'|'A'..'Z'|'_') ('a'..'z'|'A'..'Z'|Digit|'_')*
   ;
@@ -439,8 +442,7 @@ fragment
 Extended_Control_Characters         :   '\u0080' .. '\u009F';
 
 Character_String_Literal
-  : '\'' ( ESC_SEQ | ~('\\'|'\'') )* '\''
-  | '"' ( ESC_SEQ | ~('\\'|'"') )* '"'
+  : QUOTE ( ESC_SEQ | ~('\\'|'\'') )* QUOTE
   ;
 
 fragment

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/4319ded3/tajo-core/tajo-core-backend/src/main/antlr4/org/apache/tajo/engine/parser/SQLParser.g4
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/antlr4/org/apache/tajo/engine/parser/SQLParser.g4 b/tajo-core/tajo-core-backend/src/main/antlr4/org/apache/tajo/engine/parser/SQLParser.g4
index 26d2fd9..1a7f903 100644
--- a/tajo-core/tajo-core-backend/src/main/antlr4/org/apache/tajo/engine/parser/SQLParser.g4
+++ b/tajo-core/tajo-core-backend/src/main/antlr4/org/apache/tajo/engine/parser/SQLParser.g4
@@ -182,9 +182,18 @@ unsigned_literal
 
 general_literal
   : Character_String_Literal
+  | datetime_literal
   | boolean_literal
   ;
 
+datetime_literal
+  : timestamp_literal
+  ;
+
+timestamp_literal
+  : TIMESTAMP timestamp_string=Character_String_Literal
+  ;
+
 boolean_literal
   : TRUE | FALSE | UNKNOWN
   ;

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/4319ded3/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/parser/HiveConverter.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/parser/HiveConverter.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/parser/HiveConverter.java
index 6de1bff..308faf0 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/parser/HiveConverter.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/parser/HiveConverter.java
@@ -1048,7 +1048,7 @@ public class HiveConverter extends HiveParserBaseVisitor<Expr> {
 
   @Override
   public Expr visitNullCondition(HiveParser.NullConditionContext ctx) {
-    return new NullValue();
+    return new NullLiteral();
   }
 
   @Override
@@ -1069,7 +1069,7 @@ public class HiveConverter extends HiveParserBaseVisitor<Expr> {
     Expr current = null;
 
     if (ctx.KW_NULL() != null) {
-      current = new NullValue();
+      current = new NullLiteral();
     }
     if (ctx.constant() != null) {
       current = visitConstant(ctx.constant());

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/4319ded3/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/parser/SQLAnalyzer.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/parser/SQLAnalyzer.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/parser/SQLAnalyzer.java
index d9e7135..a2777c5 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/parser/SQLAnalyzer.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/parser/SQLAnalyzer.java
@@ -37,6 +37,9 @@ import java.util.Map;
 
 import static org.apache.tajo.algebra.Aggregation.GroupElement;
 import static org.apache.tajo.algebra.CreateTable.*;
+
+import org.apache.tajo.algebra.DateValue;
+import org.apache.tajo.algebra.TimeValue;
 import static org.apache.tajo.common.TajoDataTypes.Type;
 import static org.apache.tajo.engine.parser.SQLParser.*;
 
@@ -441,7 +444,7 @@ public class SQLAnalyzer extends SQLParserBaseVisitor<Expr> {
 
   private Expr buildCaseResult(ResultContext result) {
     if (result.NULL() != null) {
-      return new NullValue();
+      return new NullLiteral();
     } else {
       return visitValue_expression(result.value_expression());
     }
@@ -464,7 +467,7 @@ public class SQLAnalyzer extends SQLParserBaseVisitor<Expr> {
 
   @Override public Expr visitCommon_value_expression(SQLParser.Common_value_expressionContext ctx) {
     if (checkIfExist(ctx.NULL())) {
-      return new NullValue();
+      return new NullLiteral();
     } else {
       return visitChildren(ctx);
     }
@@ -826,6 +829,8 @@ public class SQLAnalyzer extends SQLParserBaseVisitor<Expr> {
     return target;
   }
 
+
+
   @Override
   public Expr visitCharacter_string_type(SQLParser.Character_string_typeContext ctx) {
     return new LiteralValue(stripQuote(ctx.getText()), LiteralType.String);
@@ -1249,8 +1254,48 @@ public class SQLAnalyzer extends SQLParserBaseVisitor<Expr> {
   public Expr visitGeneral_literal(SQLParser.General_literalContext ctx) {
     if (checkIfExist(ctx.Character_String_Literal())) {
       return new LiteralValue(stripQuote(ctx.Character_String_Literal().getText()), LiteralType.String);
+    } else if (checkIfExist(ctx.datetime_literal())) {
+      return visitDatetime_literal(ctx.datetime_literal());
     } else {
       return new BooleanLiteral(checkIfExist(ctx.boolean_literal().TRUE()));
     }
   }
+
+  @Override
+  public Expr visitDatetime_literal(@NotNull SQLParser.Datetime_literalContext ctx) {
+    return visitTimestamp_literal(ctx.timestamp_literal());
+  }
+
+  @Override
+  public Expr visitTimestamp_literal(SQLParser.Timestamp_literalContext ctx) {
+    String timestampStr = stripQuote(ctx.timestamp_string.getText());
+    String [] parts = timestampStr.split(" ");
+    String datePart = parts[0];
+    String timePart = parts[1];
+    return new TimestampLiteral(parseDate(datePart), parseTime(timePart));
+  }
+
+  private DateValue parseDate(String datePart) {
+    // e.g., 1980-04-01
+    String [] parts = datePart.split("-");
+    return new DateValue(parts[0], parts[1], parts[2]);
+  }
+
+  private TimeValue parseTime(String timePart) {
+    // e.g., 12:01:50.399
+    String [] parts = timePart.split(":");
+
+    TimeValue time;
+    boolean hasFractionOfSeconds = parts[2].indexOf('.') > 0;
+    if (hasFractionOfSeconds) {
+      String [] secondsParts = parts[2].split(".");
+      time = new TimeValue(parts[0], parts[1], secondsParts[0]);
+      if (secondsParts.length == 2) {
+        time.setSecondsFraction(secondsParts[1]);
+      }
+    } else {
+      time = new TimeValue(parts[0], parts[1], parts[2]);
+    }
+    return time;
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/4319ded3/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/AlgebraVisitor.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/AlgebraVisitor.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/AlgebraVisitor.java
index 0bc957d..385158e 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/AlgebraVisitor.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/AlgebraVisitor.java
@@ -38,6 +38,7 @@ public interface AlgebraVisitor<CONTEXT, RESULT> {
   RESULT visitTableSubQuery(CONTEXT ctx, Stack<OpType> stack, TablePrimarySubQuery expr) throws PlanningException;
   RESULT visitRelationList(CONTEXT ctx, Stack<OpType> stack, RelationList expr) throws PlanningException;
   RESULT visitRelation(CONTEXT ctx, Stack<OpType> stack, Relation expr) throws PlanningException;
+  RESULT visitScalarSubQuery(CONTEXT ctx, Stack<OpType> stack, ScalarSubQuery expr) throws PlanningException;
 
   // Data definition language
   RESULT visitCreateTable(CONTEXT ctx, Stack<OpType> stack, CreateTable expr) throws PlanningException;
@@ -93,8 +94,10 @@ public interface AlgebraVisitor<CONTEXT, RESULT> {
 
   // Literal
   RESULT visitCastExpr(CONTEXT ctx, Stack<OpType> stack, CastExpr expr) throws PlanningException;
-  RESULT visitScalarSubQuery(CONTEXT ctx, Stack<OpType> stack, ScalarSubQuery expr) throws PlanningException;
-  RESULT visitLiteral(CONTEXT ctx, Stack<OpType> stack, LiteralValue expr) throws PlanningException;
-  RESULT visitNullValue(CONTEXT ctx, Stack<OpType> stack, NullValue expr) throws PlanningException;
+
   RESULT visitDataType(CONTEXT ctx, Stack<OpType> stack, DataTypeExpr expr) throws PlanningException;
+  RESULT visitLiteral(CONTEXT ctx, Stack<OpType> stack, LiteralValue expr) throws PlanningException;
+  RESULT visitNullLiteral(CONTEXT ctx, Stack<OpType> stack, NullLiteral expr) throws PlanningException;
+  RESULT visitTimestampLiteral(CONTEXT ctx, Stack<OpType> stack, TimestampLiteral expr) throws PlanningException;
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/4319ded3/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/BaseAlgebraVisitor.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/BaseAlgebraVisitor.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/BaseAlgebraVisitor.java
index 2195e9d..f2cf1f8 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/BaseAlgebraVisitor.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/BaseAlgebraVisitor.java
@@ -216,8 +216,8 @@ public class BaseAlgebraVisitor<CONTEXT, RESULT> implements AlgebraVisitor<CONTE
     case Literal:
       current = visitLiteral(ctx, stack, (LiteralValue) expr);
       break;
-    case Null:
-      current = visitNullValue(ctx, stack, (NullValue) expr);
+    case NullLiteral:
+      current = visitNullLiteral(ctx, stack, (NullLiteral) expr);
       break;
     case DataType:
       current = visitDataType(ctx, stack, (DataTypeExpr) expr);
@@ -332,6 +332,14 @@ public class BaseAlgebraVisitor<CONTEXT, RESULT> implements AlgebraVisitor<CONTE
     return null;
   }
 
+  @Override
+  public RESULT visitScalarSubQuery(CONTEXT ctx, Stack<OpType> stack, ScalarSubQuery expr) throws PlanningException {
+    stack.push(OpType.ScalarSubQuery);
+    RESULT result = visit(ctx, stack, expr.getSubQuery());
+    stack.pop();
+    return result;
+  }
+
   ///////////////////////////////////////////////////////////////////////////////////////////////////////////
   // Data Definition Language Section
   ///////////////////////////////////////////////////////////////////////////////////////////////////////////
@@ -512,6 +520,10 @@ public class BaseAlgebraVisitor<CONTEXT, RESULT> implements AlgebraVisitor<CONTE
     return visitDefaultBinaryExpr(ctx, stack, expr);
   }
 
+  ///////////////////////////////////////////////////////////////////////////////////////////////////////////
+  // Arithmetic Operators
+  ///////////////////////////////////////////////////////////////////////////////////////////////////////////
+
   @Override
   public RESULT visitPlus(CONTEXT ctx, Stack<OpType> stack, BinaryOperator expr) throws PlanningException {
     return visitDefaultBinaryExpr(ctx, stack, expr);
@@ -537,6 +549,10 @@ public class BaseAlgebraVisitor<CONTEXT, RESULT> implements AlgebraVisitor<CONTE
     return visitDefaultBinaryExpr(ctx, stack, expr);
   }
 
+  ///////////////////////////////////////////////////////////////////////////////////////////////////////////
+  // Other Expressions
+  ///////////////////////////////////////////////////////////////////////////////////////////////////////////
+
   @Override
   public RESULT visitSign(CONTEXT ctx, Stack<OpType> stack, SignedExpr expr) throws PlanningException {
     return visitDefaultUnaryExpr(ctx, stack, expr);
@@ -567,6 +583,10 @@ public class BaseAlgebraVisitor<CONTEXT, RESULT> implements AlgebraVisitor<CONTE
     return result;
   }
 
+  ///////////////////////////////////////////////////////////////////////////////////////////////////////////
+  // General Set Section
+  ///////////////////////////////////////////////////////////////////////////////////////////////////////////
+
   @Override
   public RESULT visitCountRowsFunction(CONTEXT ctx, Stack<OpType> stack, CountRowsFunctionExpr expr)
       throws PlanningException {
@@ -585,18 +605,19 @@ public class BaseAlgebraVisitor<CONTEXT, RESULT> implements AlgebraVisitor<CONTE
     return result;
   }
 
+  ///////////////////////////////////////////////////////////////////////////////////////////////////////////
+  // Literal Section
+  ///////////////////////////////////////////////////////////////////////////////////////////////////////////
+
   @Override
-  public RESULT visitCastExpr(CONTEXT ctx, Stack<OpType> stack, CastExpr expr) throws PlanningException {
-    stack.push(OpType.Cast);
-    RESULT result = visit(ctx, stack, expr.getOperand());
-    stack.pop();
-    return result;
+  public RESULT visitDataType(CONTEXT ctx, Stack<OpType> stack, DataTypeExpr expr) throws PlanningException {
+    return null;
   }
 
   @Override
-  public RESULT visitScalarSubQuery(CONTEXT ctx, Stack<OpType> stack, ScalarSubQuery expr) throws PlanningException {
-    stack.push(OpType.ScalarSubQuery);
-    RESULT result = visit(ctx, stack, expr.getSubQuery());
+  public RESULT visitCastExpr(CONTEXT ctx, Stack<OpType> stack, CastExpr expr) throws PlanningException {
+    stack.push(OpType.Cast);
+    RESULT result = visit(ctx, stack, expr.getOperand());
     stack.pop();
     return result;
   }
@@ -607,12 +628,12 @@ public class BaseAlgebraVisitor<CONTEXT, RESULT> implements AlgebraVisitor<CONTE
   }
 
   @Override
-  public RESULT visitNullValue(CONTEXT ctx, Stack<OpType> stack, NullValue expr) throws PlanningException {
+  public RESULT visitNullLiteral(CONTEXT ctx, Stack<OpType> stack, NullLiteral expr) throws PlanningException {
     return null;
   }
 
   @Override
-  public RESULT visitDataType(CONTEXT ctx, Stack<OpType> stack, DataTypeExpr expr) throws PlanningException {
+  public RESULT visitTimestampLiteral(CONTEXT ctx, Stack<OpType> stack, TimestampLiteral expr) throws PlanningException {
     return null;
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/4319ded3/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/LogicalPlanner.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/LogicalPlanner.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/LogicalPlanner.java
index 605b9df..6581c66 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/LogicalPlanner.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/LogicalPlanner.java
@@ -38,6 +38,7 @@ import org.apache.tajo.common.TajoDataTypes.DataType;
 import org.apache.tajo.datum.Datum;
 import org.apache.tajo.datum.DatumFactory;
 import org.apache.tajo.datum.NullDatum;
+import org.apache.tajo.datum.TimestampDatum;
 import org.apache.tajo.engine.eval.*;
 import org.apache.tajo.engine.exception.InvalidQueryException;
 import org.apache.tajo.engine.exception.UndefinedFunctionException;
@@ -49,6 +50,7 @@ import org.apache.tajo.engine.planner.logical.*;
 import org.apache.tajo.engine.utils.SchemaUtil;
 import org.apache.tajo.exception.InternalException;
 import org.apache.tajo.util.TUtil;
+import org.joda.time.DateTime;
 
 import java.util.Collection;
 import java.util.List;
@@ -57,6 +59,8 @@ import java.util.Stack;
 import static org.apache.tajo.algebra.Aggregation.GroupType;
 import static org.apache.tajo.algebra.CreateTable.ColumnPartition;
 import static org.apache.tajo.algebra.CreateTable.PartitionType;
+
+import org.apache.tajo.algebra.DateValue;
 import static org.apache.tajo.catalog.proto.CatalogProtos.FunctionType;
 import static org.apache.tajo.engine.planner.LogicalPlan.BlockType;
 
@@ -1040,12 +1044,11 @@ public class LogicalPlanner extends BaseAlgebraVisitor<LogicalPlanner.PlanContex
     Expression SECTION
    ===============================================================================================*/
 
-  public EvalNode createEvalTree(LogicalPlan plan, QueryBlock block, final Expr expr)
-      throws PlanningException {
+  public EvalNode createEvalTree(LogicalPlan plan, QueryBlock block, final Expr expr)throws PlanningException {
 
     switch(expr.getType()) {
       // constants
-      case Null:
+      case NullLiteral:
         return new ConstEval(NullDatum.get());
 
       case Literal:
@@ -1065,6 +1068,56 @@ public class LogicalPlanner extends BaseAlgebraVisitor<LogicalPlanner.PlanContex
             throw new RuntimeException("Unsupported type: " + literal.getValueType());
         }
 
+      case TimestampLiteral: {
+        TimestampLiteral timestampLiteral = (TimestampLiteral) expr;
+        DateValue dateValue = timestampLiteral.getDate();
+        TimeValue timeValue = timestampLiteral.getTime();
+        int years;
+        int months;
+        int days;
+        int hours;
+        int minutes;
+        int seconds;
+
+        years = Integer.valueOf(dateValue.getYears());
+        if (!(1 <= years && years <= 9999)) {
+          throw new PlanningException(String.format("Years (%d) must be between 1 and 9999 integer value", years));
+        }
+        months = Integer.valueOf(dateValue.getMonths());
+        if (!(1 <= months && months <= 12)) {
+          throw new PlanningException(String.format("Months (%d) must be between 1 and 12 integer value", months));
+        }
+        days = Integer.valueOf(dateValue.getDays());
+        if (!(1<= days && days <= 31)) {
+          throw new PlanningException(String.format("Days (%d) must be between 1 and 31 integer value", days));
+        }
+
+        hours = Integer.valueOf(timeValue.getHours());
+        if (!(0 <= hours && hours <= 23)) {
+          throw new PlanningException(String.format("Hours (%d) must be between 0 and 24 integer value", hours));
+        }
+        minutes = Integer.valueOf(timeValue.getMinutes());
+        if (!(0 <= minutes && minutes <= 59)) {
+          throw new PlanningException(String.format("Minutes (%d) must be between 0 and 59 integer value", minutes));
+        }
+        seconds = Integer.valueOf(timeValue.getSeconds());
+        if (!(0 <= seconds && seconds <= 59)) {
+          throw new PlanningException(String.format("Seconds (%d) must be between 0 and 59 integer value", seconds));
+        }
+
+        DateTime dateTime;
+        if (timeValue.hasSecondsFraction()) {
+          int secondsFraction = Integer.valueOf(timeValue.getSecondsFraction());
+          if (!(0 <= secondsFraction && secondsFraction <= 999))
+          throw new PlanningException(String.format("Seconds (%d) must be between 0 and 999 integer value", seconds));
+          dateTime = new DateTime(years, months, days, hours, minutes, seconds, secondsFraction);
+        } else {
+          dateTime = new DateTime(years, months, days, hours, minutes, seconds);
+        }
+
+        return new ConstEval(new TimestampDatum(dateTime));
+      }
+
       case Sign:
         SignedExpr signedExpr = (SignedExpr) expr;
         EvalNode numericExpr = createEvalTree(plan, block, signedExpr.getChild());
@@ -1205,7 +1258,7 @@ public class LogicalPlanner extends BaseAlgebraVisitor<LogicalPlanner.PlanContex
         Expr[] params = function.getParams();
         if (params == null) {
             params = new Expr[1];
-            params[0] = new NullValue();
+            params[0] = new NullLiteral();
         }
 
         EvalNode[] givenArgs = new EvalNode[params.length];

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/4319ded3/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/eval/ExprTestBase.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/eval/ExprTestBase.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/eval/ExprTestBase.java
index ca1259b..3c150ec 100644
--- a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/eval/ExprTestBase.java
+++ b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/eval/ExprTestBase.java
@@ -42,7 +42,7 @@ import org.junit.BeforeClass;
 import java.io.IOException;
 
 import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.assertFalse;
 
 public class ExprTestBase {
   private static TajoTestingCluster util;
@@ -69,7 +69,7 @@ public class ExprTestBase {
   }
 
   private static void assertJsonSerDer(EvalNode expr) {
-    String json = expr.toJson();
+    String json = CoreGsonHelper.toJson(expr, EvalNode.class);
     EvalNode fromJson = CoreGsonHelper.fromJson(json, EvalNode.class);
     assertEquals(expr, fromJson);
   }
@@ -122,7 +122,7 @@ public class ExprTestBase {
     try {
       targets = getRawTargets(query);
     } catch (PlanningException e) {
-      assertTrue("Wrong query statement: " + query, false);
+      assertFalse(e.getMessage(), true);
     }
 
     EvalContext [] evalContexts = new EvalContext[targets.length];

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/4319ded3/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/eval/TestSQLDateTimeTypes.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/eval/TestSQLDateTimeTypes.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/eval/TestSQLDateTimeTypes.java
new file mode 100644
index 0000000..fb1699f
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/eval/TestSQLDateTimeTypes.java
@@ -0,0 +1,32 @@
+/**
+ * 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.eval;
+
+import org.junit.Test;
+
+import java.io.IOException;
+
+public class TestSQLDateTimeTypes extends ExprTestBase {
+
+  @Test
+  public void testToTimestamp() throws IOException {
+    testSimpleEval("select to_char(TIMESTAMP '1970-01-17 10:09:37', 'yyyy-MM-dd HH:mm:ss');",
+        new String[]{"1970-01-17 10:09:37"});
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/4319ded3/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/eval/TestSQLExpression.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/eval/TestSQLExpression.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/eval/TestSQLExpression.java
index 2d76fec..9af33ad 100644
--- a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/eval/TestSQLExpression.java
+++ b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/eval/TestSQLExpression.java
@@ -62,8 +62,8 @@ public class TestSQLExpression extends ExprTestBase {
 
   @Test
   public void testCastWithNestedFunction() throws IOException {
-    testSimpleEval("select to_timestamp(CAST(split_part('1386577650.123', '.', 1) as INT8));",
-        new String[] {"1970-01-17 10:09:37"});
+    testSimpleEval("select to_char(to_timestamp(CAST(split_part('1386577650.123', '.', 1) as INT8)), " +
+        "'yyyy-MM-dd HH:mm:ss');", new String[] {"1970-01-17 10:09:37"});
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/4319ded3/tajo-core/tajo-core-backend/src/test/queries/complex_union_1.sql
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/queries/complex_union_1.sql b/tajo-core/tajo-core-backend/src/test/queries/complex_union_1.sql
index 66ea3ff..9020ad7 100644
--- a/tajo-core/tajo-core-backend/src/test/queries/complex_union_1.sql
+++ b/tajo-core/tajo-core-backend/src/test/queries/complex_union_1.sql
@@ -9,7 +9,7 @@ FROM
   SELECT
     l_orderkey,
     l_partkey,
-    "abc" as query
+    'abc' as query
   FROM
     lineitem
   WHERE
@@ -20,7 +20,7 @@ FROM
   SELECT
     l_orderkey,
     l_partkey,
-    "bbc" as query
+    'bbc' as query
   FROM
     lineitem
   WHERE