You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@tajo.apache.org by jh...@apache.org on 2016/05/18 01:23:37 UTC

tajo git commit: TAJO-2150: Fix invalid implementation of date time in jdbc.

Repository: tajo
Updated Branches:
  refs/heads/master 1e540bbb2 -> 3de377461


TAJO-2150: Fix invalid implementation of date time in jdbc.

Closes #1025


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

Branch: refs/heads/master
Commit: 3de37746113d6534963396616485d5cc5ff50501
Parents: 1e540bb
Author: Jinho Kim <jh...@apache.org>
Authored: Wed May 18 10:22:09 2016 +0900
Committer: Jinho Kim <jh...@apache.org>
Committed: Wed May 18 10:22:09 2016 +0900

----------------------------------------------------------------------
 CHANGES                                         |   2 +
 .../org/apache/tajo/jdbc/TajoResultSetBase.java | 170 +++++++++++++++++--
 .../java/org/apache/tajo/datum/DateDatum.java   |   2 +-
 .../tajo/util/datetime/DateTimeConstants.java   |  10 ++
 .../apache/tajo/util/datetime/DateTimeUtil.java | 117 ++++++++++++-
 .../org/apache/tajo/jdbc/TestResultSet.java     |  76 ++++-----
 .../apache/tajo/storage/jdbc/JdbcScanner.java   |   5 +-
 7 files changed, 318 insertions(+), 64 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tajo/blob/3de37746/CHANGES
----------------------------------------------------------------------
diff --git a/CHANGES b/CHANGES
index aaf6c84..8f6c0fe 100644
--- a/CHANGES
+++ b/CHANGES
@@ -149,6 +149,8 @@ Release 0.12.0 - unreleased
 
   BUG FIXES
 
+    TAJO-2150: Fix invalid implementation of date time in jdbc. (jinho)
+
     TAJO-2151: Fix broken CI. (jihoon)
 
     TAJO-2158: The concat_ws function can't support a tab separator.

http://git-wip-us.apache.org/repos/asf/tajo/blob/3de37746/tajo-client/src/main/java/org/apache/tajo/jdbc/TajoResultSetBase.java
----------------------------------------------------------------------
diff --git a/tajo-client/src/main/java/org/apache/tajo/jdbc/TajoResultSetBase.java b/tajo-client/src/main/java/org/apache/tajo/jdbc/TajoResultSetBase.java
index 9198a1e..ce72856 100644
--- a/tajo-client/src/main/java/org/apache/tajo/jdbc/TajoResultSetBase.java
+++ b/tajo-client/src/main/java/org/apache/tajo/jdbc/TajoResultSetBase.java
@@ -22,7 +22,9 @@ import org.apache.tajo.QueryId;
 import org.apache.tajo.SessionVars;
 import org.apache.tajo.catalog.Schema;
 import org.apache.tajo.common.TajoDataTypes;
+import org.apache.tajo.datum.DatumFactory;
 import org.apache.tajo.datum.TimestampDatum;
+import org.apache.tajo.error.Errors;
 import org.apache.tajo.storage.Tuple;
 import org.apache.tajo.util.datetime.DateTimeUtil;
 import org.apache.tajo.util.datetime.TimeMeta;
@@ -277,26 +279,71 @@ public abstract class TajoResultSetBase implements ResultSet {
 
   @Override
   public Date getDate(int fieldId) throws SQLException {
-    return getDate(cur, null, fieldId - 1);
+    return getDate(cur, fieldId - 1);
   }
 
   @Override
   public Date getDate(String name) throws SQLException {
-    return getDate(cur, null, findColumn(name));
+    return getDate(cur, findColumn(name));
   }
 
   @Override
   public Date getDate(int fieldId, Calendar x) throws SQLException {
-    return getDate(cur, x.getTimeZone(), fieldId - 1);
+    return getDate(cur, x, fieldId - 1);
   }
 
   @Override
   public Date getDate(String name, Calendar x) throws SQLException {
-    return getDate(cur, x.getTimeZone(), findColumn(name));
+    return getDate(cur, x, findColumn(name));
   }
 
-  private Date getDate(Tuple tuple, TimeZone tz, int index) throws SQLException {
-    return handleNull(tuple, index) ? null : toDate(tuple.asDatum(index).asTimeMeta(), tz);
+  private Date getDate(Tuple tuple, int index) throws SQLException {
+    if (handleNull(tuple, index)) return null;
+
+    switch (tuple.type(index)) {
+    case DATE:
+      return toDate(tuple.asDatum(index).asTimeMeta(), null);
+    case TIMESTAMP:
+      return toDate(tuple.asDatum(index).asTimeMeta(), timezone);
+    case TEXT:
+      return toDate(DatumFactory.createDate(tuple.asDatum(index)).asTimeMeta(), null);
+    default:
+      throw new TajoSQLException(Errors.ResultCode.INVALID_VALUE_FOR_CAST, tuple.type(index).name(), "date");
+    }
+  }
+
+  /**
+   * This method uses the given calendar to construct an appropriate millisecond
+   * value for the time if the underlying database does not store timezone information.
+   */
+  private Date getDate(Tuple tuple, Calendar cal, int index) throws SQLException {
+    if (handleNull(tuple, index)) return null;
+
+    TimeZone tz = cal == null ? TimeZone.getDefault() : cal.getTimeZone();
+    long javaTime;
+
+    switch (tuple.type(index)) {
+    case DATE: {
+      TimeMeta tm = tuple.asDatum(index).asTimeMeta();
+      javaTime = DateTimeUtil.convertTimeZone(toDate(tm, null).getTime(), TimeZone.getDefault(), tz);
+      break;
+    }
+    case TIMESTAMP: {
+      Date date = DateTimeUtil.convertToDate(getTimestamp(tuple, index), timezone);
+      javaTime = DateTimeUtil.convertTimeZone(date.getTime(), timezone, tz);
+      break;
+    }
+    case TEXT: {
+      TimeMeta  tm = DatumFactory.createDate(tuple.asDatum(index)).asTimeMeta();
+      javaTime = DateTimeUtil.convertTimeZone(toDate(tm, null).getTime(), TimeZone.getDefault(), tz);
+      break;
+    }
+    default:
+      throw new TajoSQLException(Errors.ResultCode.INVALID_VALUE_FOR_CAST, tuple.type(index).name(), "date");
+    }
+
+    // remove time part
+    return DateTimeUtil.convertToDate(javaTime, TimeZone.getDefault());
   }
 
   private Date toDate(TimeMeta tm, TimeZone tz) {
@@ -305,26 +352,71 @@ public abstract class TajoResultSetBase implements ResultSet {
 
   @Override
   public Time getTime(int fieldId) throws SQLException {
-    return getTime(cur, null, fieldId - 1);
+    return getTime(cur, fieldId - 1);
   }
 
   @Override
   public Time getTime(String name) throws SQLException {
-    return getTime(cur, null, findColumn(name));
+    return getTime(cur, findColumn(name));
   }
 
   @Override
   public Time getTime(int fieldId, Calendar x) throws SQLException {
-    return getTime(cur, x.getTimeZone(), fieldId - 1);
+    return getTime(cur, x, fieldId - 1);
   }
 
   @Override
   public Time getTime(String name, Calendar x) throws SQLException {
-    return getTime(cur, x.getTimeZone(), findColumn(name));
+    return getTime(cur, x, findColumn(name));
+  }
+
+  private Time getTime(Tuple tuple, int index) throws SQLException {
+    if (handleNull(tuple, index)) return null;
+
+    switch (tuple.type(index)) {
+    case DATE:
+    case TIME:
+      return toTime(tuple.asDatum(index).asTimeMeta(), null);
+    case TIMESTAMP:
+      return toTime(tuple.asDatum(index).asTimeMeta(), timezone);
+    case TEXT:
+      return toTime(DatumFactory.createTime(tuple.asDatum(index)).asTimeMeta(), null);
+    default:
+      throw new TajoSQLException(Errors.ResultCode.INVALID_VALUE_FOR_CAST, tuple.type(index).name(), "time");
+    }
   }
 
-  private Time getTime(Tuple tuple, TimeZone tz, int index) throws SQLException {
-    return handleNull(tuple, index) ? null : toTime(tuple.asDatum(index).asTimeMeta(), tz);
+  /**
+   * This method uses the given calendar to construct an appropriate millisecond
+   * value for the time if the underlying database does not store timezone information.
+   */
+  private Time getTime(Tuple tuple, Calendar cal, int index) throws SQLException {
+    if (handleNull(tuple, index)) return null;
+
+    TimeZone tz = cal == null ? TimeZone.getDefault() : cal.getTimeZone();
+    long javaTime;
+
+    switch (tuple.type(index)) {
+    case DATE:
+    case TIME: {
+      TimeMeta tm = tuple.asDatum(index).asTimeMeta();
+      javaTime = DateTimeUtil.convertTimeZone(toTime(tm, null).getTime(), TimeZone.getDefault(), tz);
+      break;
+    }
+    case TIMESTAMP: {
+      return DateTimeUtil.convertToTime(getTimestamp(tuple, index), timezone);
+    }
+    case TEXT: {
+      TimeMeta tm = DatumFactory.createTime(tuple.asDatum(index)).asTimeMeta();
+      javaTime = DateTimeUtil.convertTimeZone(toTime(tm, null).getTime(), TimeZone.getDefault(), tz);
+      break;
+    }
+    default:
+      throw new TajoSQLException(Errors.ResultCode.INVALID_VALUE_FOR_CAST, tuple.type(index).name(), "time");
+    }
+
+    // remove date part
+    return DateTimeUtil.convertToTime(javaTime, TimeZone.getDefault());
   }
 
   private Time toTime(TimeMeta tm, TimeZone tz) {
@@ -333,26 +425,68 @@ public abstract class TajoResultSetBase implements ResultSet {
 
   @Override
   public Timestamp getTimestamp(int fieldId) throws SQLException {
-    return getTimestamp(cur, null, fieldId - 1);
+    return getTimestamp(cur, fieldId - 1);
   }
 
   @Override
   public Timestamp getTimestamp(String name) throws SQLException {
-    return getTimestamp(cur, null, findColumn(name));
+    return getTimestamp(cur, findColumn(name));
   }
 
   @Override
   public Timestamp getTimestamp(int fieldId, Calendar x) throws SQLException {
-    return getTimestamp(cur, x.getTimeZone(), fieldId - 1);
+    return getTimestamp(cur, x, fieldId - 1);
   }
 
   @Override
   public Timestamp getTimestamp(String name, Calendar x) throws SQLException {
-    return getTimestamp(cur, x.getTimeZone(), findColumn(name));
+    return getTimestamp(cur, x, findColumn(name));
+  }
+
+  private Timestamp getTimestamp(Tuple tuple, int index) throws SQLException {
+    if (handleNull(tuple, index)) return null;
+
+    TimeMeta tm;
+    switch (tuple.type(index)) {
+    case DATE:
+    case TIME:
+      tm = tuple.asDatum(index).asTimeMeta();
+      DateTimeUtil.toUTCTimezone(tm, timezone);
+      return toTimestamp(tm, timezone);
+    case TIMESTAMP:
+      return toTimestamp(tuple.asDatum(index).asTimeMeta(), timezone);
+    case TEXT:
+      tm = DatumFactory.createTimestamp(tuple.asDatum(index), timezone).asTimeMeta();
+      return toTimestamp(tm, timezone);
+    default:
+      throw new TajoSQLException(Errors.ResultCode.INVALID_VALUE_FOR_CAST, tuple.type(index).name(), "timestamp");
+    }
   }
 
-  private Timestamp getTimestamp(Tuple tuple, TimeZone tz, int index) throws SQLException {
-    return handleNull(tuple, index) ? null : toTimestamp(tuple.asDatum(index).asTimeMeta(), tz);
+  /**
+   * This method uses the given calendar to construct an appropriate millisecond
+   * value for the time if the underlying database does not store timezone information.
+   */
+  private Timestamp getTimestamp(Tuple tuple, Calendar cal, int index) throws SQLException {
+    if (handleNull(tuple, index)) return null;
+
+    TimeMeta tm;
+    TimeZone tz = cal == null ? TimeZone.getDefault() : cal.getTimeZone();
+
+    switch (tuple.type(index)) {
+    case DATE:
+    case TIME:
+      tm = tuple.asDatum(index).asTimeMeta();
+      DateTimeUtil.toUTCTimezone(tm, tz);
+      return toTimestamp(tm, null);
+    case TIMESTAMP:
+      return toTimestamp(tuple.asDatum(index).asTimeMeta(), timezone);
+    case TEXT:
+      tm = DatumFactory.createTimestamp(tuple.asDatum(index), tz).asTimeMeta();
+      return toTimestamp(tm, null);
+    default:
+      throw new TajoSQLException(Errors.ResultCode.INVALID_VALUE_FOR_CAST, tuple.type(index).name(), "timestamp");
+    }
   }
 
   private Timestamp toTimestamp(TimeMeta tm, TimeZone tz) {

http://git-wip-us.apache.org/repos/asf/tajo/blob/3de37746/tajo-common/src/main/java/org/apache/tajo/datum/DateDatum.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/datum/DateDatum.java b/tajo-common/src/main/java/org/apache/tajo/datum/DateDatum.java
index acc5408..fffdca8 100644
--- a/tajo-common/src/main/java/org/apache/tajo/datum/DateDatum.java
+++ b/tajo-common/src/main/java/org/apache/tajo/datum/DateDatum.java
@@ -33,7 +33,7 @@ import org.apache.tajo.util.datetime.TimeMeta;
 public class DateDatum extends Datum {
   public static final int SIZE = 4;
 
-  // Dates are stored in UTC.
+  // Dates are stored by local time.
   private int jdate;
 
   public DateDatum(int value) {

http://git-wip-us.apache.org/repos/asf/tajo/blob/3de37746/tajo-common/src/main/java/org/apache/tajo/util/datetime/DateTimeConstants.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/util/datetime/DateTimeConstants.java b/tajo-common/src/main/java/org/apache/tajo/util/datetime/DateTimeConstants.java
index 002ae0b..96e2c1f 100644
--- a/tajo-common/src/main/java/org/apache/tajo/util/datetime/DateTimeConstants.java
+++ b/tajo-common/src/main/java/org/apache/tajo/util/datetime/DateTimeConstants.java
@@ -76,6 +76,16 @@ public class DateTimeConstants {
 
   /** == DateTimeUtil.toJulianDate(JULIAN_MAXYEAR, 1, 1) */
   public static final int JULIAN_MAX = 2147483494;
+
+  // This is an implementation copied from PGStatement in pgsql jdbc
+  // We can't use Long.MAX_VALUE or Long.MIN_VALUE for java.sql.date
+  // because this would break the 'normalization contract' of the
+  // java.sql.Date API.
+  // The follow values are the nearest MAX/MIN values with hour,
+  // minute, second, millisecond set to 0 - this is used for
+  // -infinity / infinity representation in Java
+  public static final long DATE_POSITIVE_INFINITY = 9223372036825200000l;
+  public static final long DATE_NEGATIVE_INFINITY = -9223372036832400000l;
   
   /** the first ISO day of week */
   public static final int MONDAY = 1;

http://git-wip-us.apache.org/repos/asf/tajo/blob/3de37746/tajo-common/src/main/java/org/apache/tajo/util/datetime/DateTimeUtil.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/util/datetime/DateTimeUtil.java b/tajo-common/src/main/java/org/apache/tajo/util/datetime/DateTimeUtil.java
index 8f43c7e..a827c6b 100644
--- a/tajo-common/src/main/java/org/apache/tajo/util/datetime/DateTimeUtil.java
+++ b/tajo-common/src/main/java/org/apache/tajo/util/datetime/DateTimeUtil.java
@@ -30,6 +30,8 @@ import javax.annotation.Nullable;
 import java.sql.Date;
 import java.sql.Time;
 import java.sql.Timestamp;
+import java.util.Calendar;
+import java.util.GregorianCalendar;
 import java.util.TimeZone;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
@@ -44,6 +46,13 @@ public class DateTimeUtil {
   /** maximum possible number of fields in a date * string */
   private static int MAXDATEFIELDS = 25;
 
+  private static final TimeZone defaultTz = TimeZone.getDefault();
+
+  /**
+   * Number of milliseconds in one day.
+   */
+  public static final int ONEDAY = 24 * 3600 * 1000;
+
   public static boolean isJulianCalendar(int year, int month, int day) {
     return year <= 1752 && month <= 9 && day < 14;
   }
@@ -362,13 +371,18 @@ public class DateTimeUtil {
     long javaTime = DateTimeUtil.julianTimeToJavaTime(DateTimeUtil.toJulianTimestamp(tm));
 
     if (tz != null) {
-      int offset = tz.getOffset(javaTime) - TimeZone.getDefault().getOffset(javaTime);
+      int offset = tz.getOffset(javaTime) - defaultTz.getOffset(javaTime);
       return new Timestamp(javaTime + offset);
     } else {
       return new Timestamp(javaTime);
     }
   }
 
+  public static long convertTimeZone(long javaTime, TimeZone from, TimeZone to) {
+    int offset = from.getOffset(javaTime) - to.getOffset(javaTime);
+    return javaTime + offset;
+  }
+
   public static Time toJavaTime(TimeMeta tm, @Nullable TimeZone tz) {
     if (tz != null) {
       DateTimeUtil.toUserTimezone(tm, tz);
@@ -384,6 +398,107 @@ public class DateTimeUtil {
   }
 
   /**
+   * Extracts the date part from a timestamp.
+   *
+   * @param timestamp The timestamp from which to extract the date.
+   * @param tz The time zone of the date.
+   * @return The extracted date.
+   */
+  public static Date convertToDate(Timestamp timestamp, TimeZone tz) {
+    return convertToDate(timestamp.getTime(), tz);
+  }
+
+  private static boolean isSimpleTimeZone(String id) {
+    return id.startsWith("GMT") || id.startsWith("UTC");
+  }
+
+  /**
+   * Extracts the date part from a timestamp.
+   *
+   * @param millis The java time
+   * @param tz The time zone of the date.
+   * @return The extracted date.
+   */
+  public static Date convertToDate(long millis, TimeZone tz) {
+    if (tz == null) {
+      tz = defaultTz;
+    }
+    if (isSimpleTimeZone(tz.getID())) {
+      // Truncate to 00:00 of the day.
+      // Suppose the input date is 7 Jan 15:40 GMT+02:00 (that is 13:40 UTC)
+      // We want it to become 7 Jan 00:00 GMT+02:00
+      // 1) Make sure millis becomes 15:40 in UTC, so add offset
+      int offset = tz.getRawOffset();
+      millis += offset;
+      // 2) Truncate hours, minutes, etc. Day is always 86400 seconds, no matter what leap seconds
+      // are
+      millis = millis / ONEDAY * ONEDAY;
+      // 2) Now millis is 7 Jan 00:00 UTC, however we need that in GMT+02:00, so subtract some
+      // offset
+      millis -= offset;
+      // Now we have brand-new 7 Jan 00:00 GMT+02:00
+      return new Date(millis);
+    }
+    Calendar cal = new GregorianCalendar();
+    cal.setTimeZone(tz);
+    cal.setTimeInMillis(millis);
+    cal.set(Calendar.HOUR_OF_DAY, 0);
+    cal.set(Calendar.MINUTE, 0);
+    cal.set(Calendar.SECOND, 0);
+    cal.set(Calendar.MILLISECOND, 0);
+    return new Date(cal.getTimeInMillis());
+  }
+
+  /**
+   * Extracts the time part from a timestamp.
+   *
+   * @param timestamp The timestamp from which to extract the time.
+   * @param tz The time zone of the time.
+   * @return The extracted time.
+   */
+  public static Time convertToTime(Timestamp timestamp, TimeZone tz) {
+    return convertToTime(timestamp.getTime(), tz);
+  }
+
+  /**
+   * Extracts the time part from a timestamp.
+   *
+   * @param millis The java time
+   * @param tz The time zone of the time.
+   * @return The extracted time.
+   */
+  public static Time convertToTime(long millis, TimeZone tz) {
+    if (tz == null) {
+      tz = defaultTz;
+    }
+    if (isSimpleTimeZone(tz.getID())) {
+      // Leave just time part of the day.
+      // Suppose the input date is 2015 7 Jan 15:40 GMT+02:00 (that is 13:40 UTC)
+      // We want it to become 1970 1 Jan 15:40 GMT+02:00
+      // 1) Make sure millis becomes 15:40 in UTC, so add offset
+      int offset = tz.getRawOffset();
+      millis += offset;
+      // 2) Truncate year, month, day. Day is always 86400 seconds, no matter what leap seconds are
+      millis = millis % ONEDAY;
+      // 2) Now millis is 1970 1 Jan 15:40 UTC, however we need that in GMT+02:00, so subtract some
+      // offset
+      millis -= offset;
+      // Now we have brand-new 1970 1 Jan 15:40 GMT+02:00
+      return new Time(millis);
+    }
+
+    Calendar cal = new GregorianCalendar();
+    cal.setTimeZone(tz);
+    cal.setTimeInMillis(millis);
+    cal.set(Calendar.ERA, GregorianCalendar.AD);
+    cal.set(Calendar.YEAR, 1970);
+    cal.set(Calendar.MONTH, 0);
+    cal.set(Calendar.DAY_OF_MONTH, 1);
+
+    return new Time(cal.getTimeInMillis());
+  }
+
+  /**
    * Calculate julian timestamp.
    * @param years
    * @param months

http://git-wip-us.apache.org/repos/asf/tajo/blob/3de37746/tajo-jdbc/src/test/java/org/apache/tajo/jdbc/TestResultSet.java
----------------------------------------------------------------------
diff --git a/tajo-jdbc/src/test/java/org/apache/tajo/jdbc/TestResultSet.java b/tajo-jdbc/src/test/java/org/apache/tajo/jdbc/TestResultSet.java
index 17fd759..3ed3787 100644
--- a/tajo-jdbc/src/test/java/org/apache/tajo/jdbc/TestResultSet.java
+++ b/tajo-jdbc/src/test/java/org/apache/tajo/jdbc/TestResultSet.java
@@ -212,61 +212,51 @@ public class TestResultSet {
 
       assertTrue(res.next());
 
-      Date date = res.getDate(1);
-      assertNotNull(date);
-      assertEquals(Date.valueOf("2014-01-01"), date);
-
-      date = res.getDate("col1");
-      assertNotNull(date);
-      assertEquals(Date.valueOf("2014-01-01"), date);
-
-      Time time = res.getTime(2);
-      assertNotNull(time);
-      assertEquals(Time.valueOf("01:00:00"), time);
-
-      time = res.getTime("col2");
-      assertNotNull(time);
-      assertEquals(Time.valueOf("01:00:00"), time);
-
-      Timestamp timestamp = res.getTimestamp(3);
-      assertNotNull(timestamp);
-      assertEquals(Timestamp.valueOf("2014-01-01 01:00:00"), timestamp);
+      assertEquals(Date.valueOf("2014-01-01"), res.getDate(1));
+      assertEquals(Timestamp.valueOf("2014-01-01 00:00:00"), res.getTimestamp(1));
+      assertEquals(Time.valueOf("00:00:00"), res.getTime(1));
+      assertEquals(res.getDate(1), res.getDate("col1"));
+
+      try {
+        // Does not support
+        fail(res.getDate(2).toString());
+        fail(res.getTimestamp(2).toString());
+      } catch (TajoSQLException e) {
+      }
+      assertEquals(Time.valueOf("01:00:00"), res.getTime(2));
+      assertEquals(res.getTime(2), res.getTime("col2"));
 
-      timestamp = res.getTimestamp("col3");
-      assertNotNull(timestamp);
-      assertEquals(Timestamp.valueOf("2014-01-01 01:00:00"), timestamp);
+      assertEquals(Date.valueOf("2014-01-01"), res.getDate(3));
+      assertEquals(Time.valueOf("01:00:00"), res.getTime(3));
+      assertEquals(Timestamp.valueOf("2014-01-01 01:00:00.0"), res.getTimestamp(3));
+      assertEquals(res.getTimestamp(3), res.getTimestamp("col3"));
 
       // assert with timezone
-
       //Current timezone + 1 hour
       TimeZone tz = TimeZone.getDefault();
       tz.setRawOffset(tz.getRawOffset() + (int) TimeUnit.HOURS.toMillis(1));
 
       Calendar cal = Calendar.getInstance(tz);
       assertEquals(tz.getRawOffset(), cal.getTimeZone().getRawOffset());
-      date = res.getDate(1, cal);
-      assertNotNull(date);
-      assertEquals("2014-01-01", date.toString());
-
-      date = res.getDate("col1", cal);
-      assertNotNull(date);
-      assertEquals("2014-01-01", date.toString());
 
-      time = res.getTime(2);
-      assertNotNull(time);
-      assertEquals("01:00:00", time.toString());
+      assertEquals(Date.valueOf("2013-12-31"), res.getDate(1, cal));
+      assertEquals(Time.valueOf("23:00:00"), res.getTime(1, cal));
+      assertEquals(Timestamp.valueOf("2013-12-31 23:00:00.0"), res.getTimestamp(1, cal));
+      assertEquals(res.getDate(1, cal), res.getDate("col1", cal));
 
-      time = res.getTime("col2");
-      assertNotNull(time);
-      assertEquals("01:00:00", time.toString());
-
-      timestamp = res.getTimestamp(3, cal);
-      assertNotNull(timestamp);
-      assertEquals("2014-01-01 02:00:00.0", timestamp.toString());
+      try {
+        // Does not support
+        fail(res.getDate(2, cal).toString());
+        fail(res.getTimestamp(2, cal).toString());
+      } catch (TajoSQLException e) {
+      }
+      assertEquals(Time.valueOf("00:00:00"), res.getTime(2, cal));
+      assertEquals(res.getTime(2, cal), res.getTime("col2", cal));
 
-      timestamp = res.getTimestamp("col3", cal);
-      assertNotNull(timestamp);
-      assertEquals("2014-01-01 02:00:00.0", timestamp.toString());
+      assertEquals(Date.valueOf("2013-12-31"), res.getDate(3, cal));
+      assertEquals(Time.valueOf("01:00:00"), res.getTime(3, cal));
+      assertEquals(Timestamp.valueOf("2014-01-01 01:00:00.0"), res.getTimestamp(3, cal));
+      assertEquals(res.getTimestamp(3, cal), res.getTimestamp("col3", cal));
     } finally {
       if (res != null) {
         res.close();

http://git-wip-us.apache.org/repos/asf/tajo/blob/3de37746/tajo-storage/tajo-storage-jdbc/src/main/java/org/apache/tajo/storage/jdbc/JdbcScanner.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-jdbc/src/main/java/org/apache/tajo/storage/jdbc/JdbcScanner.java b/tajo-storage/tajo-storage-jdbc/src/main/java/org/apache/tajo/storage/jdbc/JdbcScanner.java
index d7e1c3b..a58b861 100644
--- a/tajo-storage/tajo-storage-jdbc/src/main/java/org/apache/tajo/storage/jdbc/JdbcScanner.java
+++ b/tajo-storage/tajo-storage-jdbc/src/main/java/org/apache/tajo/storage/jdbc/JdbcScanner.java
@@ -36,6 +36,7 @@ import org.apache.tajo.plan.logical.LogicalNode;
 import org.apache.tajo.storage.Scanner;
 import org.apache.tajo.storage.Tuple;
 import org.apache.tajo.storage.VTuple;
+import org.apache.tajo.util.datetime.DateTimeUtil;
 
 import java.io.Closeable;
 import java.io.IOException;
@@ -231,7 +232,9 @@ public abstract class JdbcScanner implements Scanner {
           tuple.put(column_idx, DatumFactory.createDate(1900 + date.getYear(), 1 + date.getMonth(), date.getDate()));
           break;
         case TIME:
-          tuple.put(column_idx, new TimeDatum(resultSet.getTime(resultIdx).getTime() * 1000));
+          final Time time = resultSet.getTime(resultIdx);
+          tuple.put(column_idx, new TimeDatum(
+              DateTimeUtil.toTime(time.getHours(), time.getMinutes(), time.getSeconds(), 0)));
           break;
         case TIMESTAMP:
           tuple.put(column_idx,