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 2014/12/09 02:56:25 UTC

[04/10] tajo git commit: TAJO-1234: Rearrange timezone in date/time types.

TAJO-1234: Rearrange timezone in date/time types.

Closes #290


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

Branch: refs/heads/index_support
Commit: facd1ddcc85a088ab8ba4e42dce2c99f6fe7e5de
Parents: 95cf4b9
Author: Hyunsik Choi <hy...@apache.org>
Authored: Mon Dec 8 16:43:53 2014 +0900
Committer: Hyunsik Choi <hy...@apache.org>
Committed: Mon Dec 8 16:43:53 2014 +0900

----------------------------------------------------------------------
 CHANGES                                         |   2 +
 .../org/apache/tajo/client/QueryClient.java     |   2 +
 .../org/apache/tajo/client/QueryClientImpl.java |   8 +-
 .../apache/tajo/client/SessionConnection.java   |  33 ++++
 .../org/apache/tajo/client/TajoClientUtil.java  |   3 +-
 .../org/apache/tajo/jdbc/FetchResultSet.java    |   1 +
 .../apache/tajo/jdbc/TajoMemoryResultSet.java   |   5 +-
 .../org/apache/tajo/jdbc/TajoResultSet.java     |   2 +
 .../org/apache/tajo/jdbc/TajoResultSetBase.java |  38 +++-
 .../java/org/apache/tajo/OverridableConf.java   |   7 +-
 .../main/java/org/apache/tajo/SessionVars.java  |   4 +-
 .../java/org/apache/tajo/TajoConstants.java     |   2 +
 .../java/org/apache/tajo/conf/TajoConf.java     |  25 +--
 .../main/java/org/apache/tajo/datum/Datum.java  |   2 +-
 .../org/apache/tajo/datum/DatumFactory.java     |  57 ++++--
 .../tajo/json/TimeZoneGsonSerdeAdapter.java     |  40 ++++
 .../apache/tajo/storage/StorageConstants.java   |  22 ++-
 .../apache/tajo/util/datetime/DateTimeUtil.java |  23 ++-
 .../apache/tajo/datum/TestTimestampDatum.java   |   7 +-
 .../tajo/engine/function/builtin/AvgDouble.java |   3 -
 .../engine/function/datetime/CurrentDate.java   |  20 +-
 .../function/datetime/DatePartFromTime.java     |  20 +-
 .../datetime/DatePartFromTimestamp.java         |  18 +-
 .../function/datetime/ToCharTimestamp.java      |  18 +-
 .../function/datetime/ToTimestampText.java      |  15 +-
 .../tajo/engine/function/string/BTrim.java      |   4 +-
 .../tajo/engine/function/string/LTrim.java      |   4 +-
 .../tajo/engine/function/string/Lpad.java       |   4 +-
 .../tajo/engine/function/string/RTrim.java      |   4 +-
 .../engine/function/string/RegexpReplace.java   |   4 +-
 .../tajo/engine/function/string/Rpad.java       |   4 +-
 .../apache/tajo/engine/json/CoreGsonHelper.java |   2 +
 .../apache/tajo/engine/query/QueryContext.java  |   4 +-
 .../java/org/apache/tajo/QueryTestCaseBase.java |   8 +
 .../org/apache/tajo/TajoTestingCluster.java     |   3 +
 .../apache/tajo/engine/eval/ExprTestBase.java   |  59 +++---
 .../tajo/engine/eval/TestIntervalType.java      |  46 ++---
 .../tajo/engine/eval/TestSQLDateTimeTypes.java  |  18 +-
 .../tajo/engine/eval/TestSQLExpression.java     |  67 ++++---
 .../function/TestConditionalExpressions.java    |  16 +-
 .../engine/function/TestDateTimeFunctions.java  | 186 ++++++++++++-------
 .../tajo/engine/query/TestSelectQuery.java      |  50 ++++-
 .../apache/tajo/engine/query/TestSortQuery.java |  23 +--
 .../org/apache/tajo/jdbc/TestResultSet.java     |  13 +-
 .../java/org/apache/tajo/jdbc/TestTajoJdbc.java |   6 -
 .../TestSelectQuery/timezoned/table1.tbl        |   3 +
 .../TestSelectQuery/datetime_table_ddl.sql      |   4 +
 .../datetime_table_timezoned_ddl.sql            |   4 +
 .../TestSelectQuery/testTimezonedTable1.sql     |   1 +
 .../TestSelectQuery/testTimezonedTable2.sql     |   1 +
 .../TestSelectQuery/testTimezonedTable3.sql     |   1 +
 .../TestSelectQuery/testTimezonedTable1.result  |   5 +
 .../TestSelectQuery/testTimezonedTable2.result  |   5 +
 .../TestSelectQuery/testTimezonedTable3.result  |   5 +
 tajo-docs/src/main/sphinx/table_management.rst  |   5 +-
 .../src/main/sphinx/table_management/csv.rst    |   4 +-
 .../sphinx/table_management/table_overview.rst  |  98 ++++++++++
 .../apache/tajo/jdbc/TajoMetaDataResultSet.java |   2 +
 .../org/apache/tajo/plan/ExprAnnotator.java     |  96 ++++++----
 .../org/apache/tajo/plan/LogicalPlanner.java    |   8 +
 .../org/apache/tajo/plan/expr/CastEval.java     |  28 ++-
 .../tajo/plan/expr/GeneralFunctionEval.java     |   8 +-
 .../tajo/plan/function/GeneralFunction.java     |   4 +-
 .../plan/serder/EvalTreeProtoDeserializer.java  |   7 +-
 .../plan/serder/EvalTreeProtoSerializer.java    |   3 +
 .../apache/tajo/plan/serder/PlanGsonHelper.java |   2 +
 .../org/apache/tajo/plan/util/PlannerUtil.java  |   4 +
 tajo-plan/src/main/proto/Plan.proto             |   1 +
 .../storage/TextSerializerDeserializer.java     |   7 +-
 .../tajo/storage/text/CSVLineDeserializer.java  |   2 +-
 .../tajo/storage/text/CSVLineSerializer.java    |   2 +-
 .../text/TextFieldSerializerDeserializer.java   |  46 ++++-
 72 files changed, 902 insertions(+), 356 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tajo/blob/facd1ddc/CHANGES
----------------------------------------------------------------------
diff --git a/CHANGES b/CHANGES
index a59e107..acc72b9 100644
--- a/CHANGES
+++ b/CHANGES
@@ -85,6 +85,8 @@ Release 0.9.1 - unreleased
 
   BUG FIXES
 
+    TAJO-1234: Rearrange timezone in date/time types. (hyunsik)
+
     TAJO-1231: Implicit table properties in session are not stored in 
     table property. (hyunsik)
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/facd1ddc/tajo-client/src/main/java/org/apache/tajo/client/QueryClient.java
----------------------------------------------------------------------
diff --git a/tajo-client/src/main/java/org/apache/tajo/client/QueryClient.java b/tajo-client/src/main/java/org/apache/tajo/client/QueryClient.java
index 9b24663..32ef97d 100644
--- a/tajo-client/src/main/java/org/apache/tajo/client/QueryClient.java
+++ b/tajo-client/src/main/java/org/apache/tajo/client/QueryClient.java
@@ -42,6 +42,8 @@ public interface QueryClient extends Closeable {
 
   public SessionIdProto getSessionId();
 
+  public Map<String, String> getClientSideSessionVars();
+
   public String getBaseDatabase();
 
   @Override

http://git-wip-us.apache.org/repos/asf/tajo/blob/facd1ddc/tajo-client/src/main/java/org/apache/tajo/client/QueryClientImpl.java
----------------------------------------------------------------------
diff --git a/tajo-client/src/main/java/org/apache/tajo/client/QueryClientImpl.java b/tajo-client/src/main/java/org/apache/tajo/client/QueryClientImpl.java
index 5b78959..1cee515 100644
--- a/tajo-client/src/main/java/org/apache/tajo/client/QueryClientImpl.java
+++ b/tajo-client/src/main/java/org/apache/tajo/client/QueryClientImpl.java
@@ -73,6 +73,11 @@ public class QueryClientImpl implements QueryClient {
   }
 
   @Override
+  public Map<String, String> getClientSideSessionVars() {
+    return connection.getClientSideSessionVars();
+  }
+
+  @Override
   public String getBaseDatabase() {
     return connection.getBaseDatabase();
   }
@@ -455,7 +460,8 @@ public class QueryClientImpl implements QueryClient {
       return new TajoMemoryResultSet(
           new Schema(serializedResultSet.getSchema()),
           serializedResultSet.getSerializedTuplesList(),
-          serializedResultSet.getSerializedTuplesCount());
+          serializedResultSet.getSerializedTuplesCount(),
+          getClientSideSessionVars());
     } catch (Exception e) {
       throw new ServiceException(e.getMessage(), e);
     }

http://git-wip-us.apache.org/repos/asf/tajo/blob/facd1ddc/tajo-client/src/main/java/org/apache/tajo/client/SessionConnection.java
----------------------------------------------------------------------
diff --git a/tajo-client/src/main/java/org/apache/tajo/client/SessionConnection.java b/tajo-client/src/main/java/org/apache/tajo/client/SessionConnection.java
index 42085a2..922984f 100644
--- a/tajo-client/src/main/java/org/apache/tajo/client/SessionConnection.java
+++ b/tajo-client/src/main/java/org/apache/tajo/client/SessionConnection.java
@@ -23,6 +23,7 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.tajo.QueryId;
+import org.apache.tajo.SessionVars;
 import org.apache.tajo.TajoIdProtos;
 import org.apache.tajo.annotation.Nullable;
 import org.apache.tajo.conf.TajoConf;
@@ -39,6 +40,7 @@ import org.jboss.netty.channel.ConnectTimeoutException;
 import java.io.Closeable;
 import java.io.IOException;
 import java.net.InetSocketAddress;
+import java.util.Collections;
 import java.util.List;
 import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
@@ -68,6 +70,9 @@ public class SessionConnection implements Closeable {
 
   private AtomicBoolean closed = new AtomicBoolean(false);
 
+  /** session variable cache */
+  Map<String, String> clientSideSessionVars = new ConcurrentHashMap<String, String>();
+
 
   public SessionConnection(TajoConf conf) throws IOException {
     this(conf, NetUtils.createSocketAddr(conf.getVar(TajoConf.ConfVars.TAJO_MASTER_CLIENT_RPC_ADDRESS)), null);
@@ -105,6 +110,10 @@ public class SessionConnection implements Closeable {
     this.baseDatabase = baseDatabase != null ? baseDatabase : null;
   }
 
+  public Map<String, String> getClientSideSessionVars() {
+    return Collections.unmodifiableMap(clientSideSessionVars);
+  }
+
   public <T> T getStub(QueryId queryId, Class protocolClass, boolean asyncMode) throws NoSuchMethodException,
       ConnectTimeoutException, ClassNotFoundException {
     InetSocketAddress addr = queryMasterMap.get(queryId);
@@ -177,6 +186,17 @@ public class SessionConnection implements Closeable {
       public Boolean call(NettyClientBase client) throws ServiceException {
         checkSessionAndGet(client);
 
+        // keep client-side session variables
+        for (Map.Entry<String, String> entry : variables.entrySet()) {
+          String key = entry.getKey();
+          if (SessionVars.exists(entry.getKey())) {
+            SessionVars configKey = SessionVars.get(key);
+            if (configKey.getMode() == SessionVars.VariableMode.CLI_SIDE_VAR) {
+              clientSideSessionVars.put(key, entry.getValue());
+            }
+          }
+        }
+
         TajoMasterClientProtocolService.BlockingInterface tajoMasterService = client.getStub();
         KeyValueSet keyValueSet = new KeyValueSet();
         keyValueSet.putAll(variables);
@@ -195,6 +215,13 @@ public class SessionConnection implements Closeable {
       public Boolean call(NettyClientBase client) throws ServiceException {
         checkSessionAndGet(client);
 
+        // Remove matched session vars
+        for (String key : variables) {
+          if (clientSideSessionVars.containsKey(key)) {
+            clientSideSessionVars.remove(key);
+          }
+        }
+
         TajoMasterClientProtocolService.BlockingInterface tajoMasterService = client.getStub();
         ClientProtos.UpdateSessionVariableRequest request = ClientProtos.UpdateSessionVariableRequest.newBuilder()
             .setSessionId(sessionId)
@@ -208,6 +235,12 @@ public class SessionConnection implements Closeable {
     return new ServerCallable<String>(connPool, getTajoMasterAddr(), TajoMasterClientProtocol.class, false, true) {
 
       public String call(NettyClientBase client) throws ServiceException {
+
+        // If a desired variable is client side one and exists in the cache, immediately return the variable.
+        if (clientSideSessionVars.containsKey(varname)) {
+          return clientSideSessionVars.get(varname);
+        }
+
         checkSessionAndGet(client);
 
         TajoMasterClientProtocolService.BlockingInterface tajoMasterService = client.getStub();

http://git-wip-us.apache.org/repos/asf/tajo/blob/facd1ddc/tajo-client/src/main/java/org/apache/tajo/client/TajoClientUtil.java
----------------------------------------------------------------------
diff --git a/tajo-client/src/main/java/org/apache/tajo/client/TajoClientUtil.java b/tajo-client/src/main/java/org/apache/tajo/client/TajoClientUtil.java
index 7aed335..bf9d111 100644
--- a/tajo-client/src/main/java/org/apache/tajo/client/TajoClientUtil.java
+++ b/tajo-client/src/main/java/org/apache/tajo/client/TajoClientUtil.java
@@ -88,7 +88,8 @@ public class TajoClientUtil {
       return new TajoMemoryResultSet(
           new Schema(serializedResultSet.getSchema()),
           serializedResultSet.getSerializedTuplesList(),
-          response.getMaxRowNum());
+          response.getMaxRowNum(),
+          client.getClientSideSessionVars());
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/tajo/blob/facd1ddc/tajo-client/src/main/java/org/apache/tajo/jdbc/FetchResultSet.java
----------------------------------------------------------------------
diff --git a/tajo-client/src/main/java/org/apache/tajo/jdbc/FetchResultSet.java b/tajo-client/src/main/java/org/apache/tajo/jdbc/FetchResultSet.java
index 78674b1..18b7c1a 100644
--- a/tajo-client/src/main/java/org/apache/tajo/jdbc/FetchResultSet.java
+++ b/tajo-client/src/main/java/org/apache/tajo/jdbc/FetchResultSet.java
@@ -35,6 +35,7 @@ public class FetchResultSet extends TajoResultSetBase {
   private boolean finished = false;
 
   public FetchResultSet(QueryClient tajoClient, Schema schema, QueryId queryId, int fetchRowNum) {
+    super(tajoClient.getClientSideSessionVars());
     this.tajoClient = tajoClient;
     this.queryId = queryId;
     this.fetchRowNum = fetchRowNum;

http://git-wip-us.apache.org/repos/asf/tajo/blob/facd1ddc/tajo-client/src/main/java/org/apache/tajo/jdbc/TajoMemoryResultSet.java
----------------------------------------------------------------------
diff --git a/tajo-client/src/main/java/org/apache/tajo/jdbc/TajoMemoryResultSet.java b/tajo-client/src/main/java/org/apache/tajo/jdbc/TajoMemoryResultSet.java
index 84fafda..d0898f5 100644
--- a/tajo-client/src/main/java/org/apache/tajo/jdbc/TajoMemoryResultSet.java
+++ b/tajo-client/src/main/java/org/apache/tajo/jdbc/TajoMemoryResultSet.java
@@ -26,6 +26,7 @@ import org.apache.tajo.storage.Tuple;
 import java.io.IOException;
 import java.sql.SQLException;
 import java.util.List;
+import java.util.Map;
 import java.util.concurrent.atomic.AtomicBoolean;
 
 public class TajoMemoryResultSet extends TajoResultSetBase {
@@ -33,7 +34,9 @@ public class TajoMemoryResultSet extends TajoResultSetBase {
   private AtomicBoolean closed = new AtomicBoolean(false);
   private RowStoreUtil.RowStoreDecoder decoder;
 
-  public TajoMemoryResultSet(Schema schema, List<ByteString> serializedTuples, int maxRowNum) {
+  public TajoMemoryResultSet(Schema schema, List<ByteString> serializedTuples, int maxRowNum,
+                             Map<String, String> clientSideSessionVars) {
+    super(clientSideSessionVars);
     this.schema = schema;
     this.totalRow = maxRowNum;
     this.serializedTuples = serializedTuples;

http://git-wip-us.apache.org/repos/asf/tajo/blob/facd1ddc/tajo-client/src/main/java/org/apache/tajo/jdbc/TajoResultSet.java
----------------------------------------------------------------------
diff --git a/tajo-client/src/main/java/org/apache/tajo/jdbc/TajoResultSet.java b/tajo-client/src/main/java/org/apache/tajo/jdbc/TajoResultSet.java
index e99c0ff..e2ac3b1 100644
--- a/tajo-client/src/main/java/org/apache/tajo/jdbc/TajoResultSet.java
+++ b/tajo-client/src/main/java/org/apache/tajo/jdbc/TajoResultSet.java
@@ -56,12 +56,14 @@ public class TajoResultSet extends TajoResultSetBase {
   private AtomicBoolean closed = new AtomicBoolean(false);
 
   public TajoResultSet(QueryClient tajoClient, QueryId queryId) {
+    super(tajoClient.getClientSideSessionVars());
     this.tajoClient = tajoClient;
     this.queryId = queryId;
     init();
   }
 
   public TajoResultSet(QueryClient tajoClient, QueryId queryId, TajoConf conf, TableDesc table) throws IOException {
+    super(tajoClient.getClientSideSessionVars());
     this.tajoClient = tajoClient;
     this.queryId = queryId;
     this.conf = conf;

http://git-wip-us.apache.org/repos/asf/tajo/blob/facd1ddc/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 78d8bde..051ea75 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
@@ -18,14 +18,16 @@
 
 package org.apache.tajo.jdbc;
 
+import org.apache.tajo.SessionVars;
+import org.apache.tajo.TajoConstants;
 import org.apache.tajo.catalog.Schema;
 import org.apache.tajo.common.TajoDataTypes;
-import org.apache.tajo.conf.TajoConf;
 import org.apache.tajo.datum.*;
 import org.apache.tajo.storage.Tuple;
 import org.apache.tajo.util.datetime.DateTimeUtil;
 import org.apache.tajo.util.datetime.TimeMeta;
 
+import javax.annotation.Nullable;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.Reader;
@@ -37,12 +39,30 @@ import java.util.Map;
 import java.util.TimeZone;
 
 public abstract class TajoResultSetBase implements ResultSet {
+  protected final Map<String, String> clientSideSessionVars;
+  protected TimeZone timezone;
+
   protected int curRow;
   protected long totalRow;
   protected boolean wasNull;
   protected Schema schema;
   protected Tuple cur;
 
+  public TajoResultSetBase(@Nullable Map<String, String> clientSideSessionVars) {
+    this.clientSideSessionVars = clientSideSessionVars;
+
+    if (clientSideSessionVars != null) {
+
+      if (clientSideSessionVars.containsKey(SessionVars.TZ.name())) {
+        String timezoneId = clientSideSessionVars.get(SessionVars.TZ.name());
+        this.timezone = TimeZone.getTimeZone(timezoneId);
+      } else {
+        this.timezone = TimeZone.getTimeZone(TajoConstants.DEFAULT_SYSTEM_TIMEZONE);
+      }
+
+    }
+  }
+
   protected void init() {
     cur = null;
     curRow = 0;
@@ -226,13 +246,13 @@ public abstract class TajoResultSetBase implements ResultSet {
       case FLOAT8:  return d.asFloat8();
       case NUMERIC:  return d.asFloat8();
       case DATE: {
-        return getDate((DateDatum)d, TajoConf.getCurrentTimeZone());
+        return getDate((DateDatum)d, timezone);
       }
       case TIME: {
-        return getTime((TimeDatum)d, TajoConf.getCurrentTimeZone());
+        return getTime((TimeDatum)d, timezone);
       }
       case TIMESTAMP: {
-        return getTimestamp((TimestampDatum) d, TajoConf.getCurrentTimeZone());
+        return getTimestamp((TimestampDatum) d, timezone);
       }
       default: return d.asChars();
     }
@@ -289,10 +309,10 @@ public abstract class TajoResultSetBase implements ResultSet {
       case BOOLEAN:
         return String.valueOf(datum.asBool());
       case TIME: {
-        return ((TimeDatum)datum).asChars(TajoConf.getCurrentTimeZone(), false);
+        return ((TimeDatum)datum).asChars(timezone, false);
       }
       case TIMESTAMP: {
-        return ((TimestampDatum)datum).asChars(TajoConf.getCurrentTimeZone(), false);
+        return ((TimestampDatum)datum).asChars(timezone, false);
       }
       default :
         return datum.asChars();
@@ -307,7 +327,7 @@ public abstract class TajoResultSetBase implements ResultSet {
       return null;
     }
 
-    return getDate((DateDatum)datum, TajoConf.getCurrentTimeZone());
+    return getDate((DateDatum)datum, null);
   }
 
   @Override
@@ -347,7 +367,7 @@ public abstract class TajoResultSetBase implements ResultSet {
       return null;
     }
 
-    return getTime((TimeDatum)datum, TajoConf.getCurrentTimeZone());
+    return getTime((TimeDatum)datum, timezone);
 
   }
 
@@ -388,7 +408,7 @@ public abstract class TajoResultSetBase implements ResultSet {
       return null;
     }
 
-    return getTimestamp((TimestampDatum)datum, TajoConf.getCurrentTimeZone());
+    return getTimestamp((TimestampDatum)datum, timezone);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/tajo/blob/facd1ddc/tajo-common/src/main/java/org/apache/tajo/OverridableConf.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/OverridableConf.java b/tajo-common/src/main/java/org/apache/tajo/OverridableConf.java
index 84be00e..b7a5da7 100644
--- a/tajo-common/src/main/java/org/apache/tajo/OverridableConf.java
+++ b/tajo-common/src/main/java/org/apache/tajo/OverridableConf.java
@@ -181,10 +181,13 @@ public class OverridableConf extends KeyValueSet {
   private void assertRegisteredEnum(ConfigKey key) {
     boolean registered = false;
 
-    for (ConfigType c : configTypes) {
-      registered = key.type() == c;
+    if (configTypes != null) {
+      for (ConfigType c : configTypes) {
+        registered = key.type() == c;
+      }
     }
 
+    // default permitted keys
     registered |= key.type() == ConfigType.SESSION || key.type() != ConfigType.SYSTEM;
 
     Preconditions.checkArgument(registered, key.keyname() + " (" + key.type() + ") is not allowed in " +

http://git-wip-us.apache.org/repos/asf/tajo/blob/facd1ddc/tajo-common/src/main/java/org/apache/tajo/SessionVars.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/SessionVars.java b/tajo-common/src/main/java/org/apache/tajo/SessionVars.java
index dd33e12..fa717d1 100644
--- a/tajo-common/src/main/java/org/apache/tajo/SessionVars.java
+++ b/tajo-common/src/main/java/org/apache/tajo/SessionVars.java
@@ -71,8 +71,8 @@ public enum SessionVars implements ConfigKey {
   ON_ERROR_STOP(ConfVars.$CLI_ERROR_STOP, "tsql will exist if an error occurs.", CLI_SIDE_VAR),
 
   // Timezone & Date ----------------------------------------------------------
-  TZ(ConfVars.$TIMEZONE, "Sets timezone", FROM_SHELL_ENV),
-  DATE_ORDER(ConfVars.$DATE_ORDER, "date order (default is YMD)", FROM_SHELL_ENV),
+  TZ(ConfVars.$TIMEZONE, "Sets timezone", CLI_SIDE_VAR),
+  DATE_ORDER(ConfVars.$DATE_ORDER, "date order (default is YMD)", CLI_SIDE_VAR),
 
   // Locales and Character set ------------------------------------------------
   // TODO - they are reserved variables, and we should support them.

http://git-wip-us.apache.org/repos/asf/tajo/blob/facd1ddc/tajo-common/src/main/java/org/apache/tajo/TajoConstants.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/TajoConstants.java b/tajo-common/src/main/java/org/apache/tajo/TajoConstants.java
index 1cc28af..9250edd 100644
--- a/tajo-common/src/main/java/org/apache/tajo/TajoConstants.java
+++ b/tajo-common/src/main/java/org/apache/tajo/TajoConstants.java
@@ -31,6 +31,8 @@ public class TajoConstants {
   public static final String DEFAULT_DATABASE_NAME = "default";
   public static final String DEFAULT_SCHEMA_NAME = "public";
 
+  /** Java 6 only recognize GMT instead of UTC. So, we should keep using GMT. */
+  public static final String DEFAULT_SYSTEM_TIMEZONE = "GMT";
 
   public static final String EMPTY_STRING = "";
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/facd1ddc/tajo-common/src/main/java/org/apache/tajo/conf/TajoConf.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/conf/TajoConf.java b/tajo-common/src/main/java/org/apache/tajo/conf/TajoConf.java
index 312abfb..b00b322 100644
--- a/tajo-common/src/main/java/org/apache/tajo/conf/TajoConf.java
+++ b/tajo-common/src/main/java/org/apache/tajo/conf/TajoConf.java
@@ -44,7 +44,7 @@ import java.util.concurrent.locks.ReentrantReadWriteLock;
 
 public class TajoConf extends Configuration {
 
-  private static TimeZone CURRENT_TIMEZONE;
+  private static TimeZone SYSTEM_TIMEZONE;
   private static int DATE_ORDER = -1;
   private static final ReentrantReadWriteLock confLock = new ReentrantReadWriteLock();
   private static final Lock writeLock = confLock.writeLock();
@@ -83,29 +83,18 @@ public class TajoConf extends Configuration {
   }
 
   private static void confStaticInit() {
-    TimeZone.setDefault(getCurrentTimeZone());
+    TimeZone.setDefault(getSystemTimezone());
     getDateOrder();
   }
 
-  public static TimeZone getCurrentTimeZone() {
+  public static TimeZone getSystemTimezone() {
     writeLock.lock();
     try {
-      if (CURRENT_TIMEZONE == null) {
+      if (SYSTEM_TIMEZONE == null) {
         TajoConf tajoConf = new TajoConf();
-        CURRENT_TIMEZONE = TimeZone.getTimeZone(tajoConf.getVar(ConfVars.$TIMEZONE));
+        SYSTEM_TIMEZONE = TimeZone.getTimeZone(tajoConf.getVar(ConfVars.$TIMEZONE));
       }
-      return CURRENT_TIMEZONE;
-    } finally {
-      writeLock.unlock();
-    }
-  }
-
-  public static TimeZone setCurrentTimeZone(TimeZone timeZone) {
-    writeLock.lock();
-    try {
-      TimeZone oldTimeZone = CURRENT_TIMEZONE;
-      CURRENT_TIMEZONE = timeZone;
-      return oldTimeZone;
+      return SYSTEM_TIMEZONE;
     } finally {
       writeLock.unlock();
     }
@@ -360,7 +349,7 @@ public class TajoConf extends Configuration {
     $CLI_ERROR_STOP("tajo.cli.error.stop", false),
 
     // Timezone & Date ----------------------------------------------------------
-    $TIMEZONE("tajo.timezone", TimeZone.getDefault().getID()),
+    $TIMEZONE("tajo.timezone", TajoConstants.DEFAULT_SYSTEM_TIMEZONE),
     $DATE_ORDER("tajo.date.order", "YMD"),
 
     // FILE FORMAT

http://git-wip-us.apache.org/repos/asf/tajo/blob/facd1ddc/tajo-common/src/main/java/org/apache/tajo/datum/Datum.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/datum/Datum.java b/tajo-common/src/main/java/org/apache/tajo/datum/Datum.java
index 442db71..cfff369 100644
--- a/tajo-common/src/main/java/org/apache/tajo/datum/Datum.java
+++ b/tajo-common/src/main/java/org/apache/tajo/datum/Datum.java
@@ -100,7 +100,7 @@ public abstract class Datum implements Comparable<Datum>, GsonObject {
   }
 
   public byte[] asTextBytes() {
-    return toString().getBytes();
+    return asChars().getBytes();
   }
 
   public boolean isNumeric() {

http://git-wip-us.apache.org/repos/asf/tajo/blob/facd1ddc/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 17cfc7a..11ba791 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
@@ -27,7 +27,9 @@ import org.apache.tajo.util.datetime.DateTimeFormat;
 import org.apache.tajo.util.datetime.DateTimeUtil;
 import org.apache.tajo.util.datetime.TimeMeta;
 
+import javax.annotation.Nullable;
 import java.io.IOException;
+import java.util.TimeZone;
 
 public class DatumFactory {
 
@@ -282,6 +284,12 @@ public class DatumFactory {
     return new TimeDatum(DateTimeUtil.toJulianTime(timeStr));
   }
 
+  public static TimeDatum createTime(String timeStr, TimeZone tz) {
+    TimeMeta tm = DateTimeUtil.decodeDateTime(timeStr);
+    DateTimeUtil.toUTCTimezone(tm, tz);
+    return new TimeDatum(DateTimeUtil.toTime(tm));
+  }
+
   public static TimestampDatum createTimestmpDatumWithJavaMillis(long millis) {
     return new TimestampDatum(DateTimeUtil.javaTimeToJulianTime(millis));
   }
@@ -294,6 +302,12 @@ public class DatumFactory {
     return new TimestampDatum(DateTimeUtil.toJulianTimestamp(datetimeStr));
   }
 
+  public static TimestampDatum createTimestamp(String datetimeStr, TimeZone tz) {
+    TimeMeta tm = DateTimeUtil.decodeDateTime(datetimeStr);
+    DateTimeUtil.toUTCTimezone(tm, tz);
+    return new TimestampDatum(DateTimeUtil.toJulianTimestamp(tm));
+  }
+
   public static IntervalDatum createInterval(String intervalStr) {
     return new IntervalDatum(intervalStr);
   }
@@ -318,13 +332,17 @@ public class DatumFactory {
     }
   }
 
-  public static TimeDatum createTime(Datum datum) {
+  public static TimeDatum createTime(Datum datum, @Nullable TimeZone tz) {
     switch (datum.type()) {
     case INT8:
       return new TimeDatum(datum.asInt8());
+    case CHAR:
+    case VARCHAR:
     case TEXT:
       TimeMeta tm = DateTimeFormat.parseDateTime(datum.asChars(), "HH24:MI:SS.MS");
-      DateTimeUtil.toUTCTimezone(tm);
+      if (tz != null) {
+        DateTimeUtil.toUTCTimezone(tm, tz);
+      }
       return new TimeDatum(DateTimeUtil.toTime(tm));
     case TIME:
       return (TimeDatum) datum;
@@ -333,10 +351,12 @@ public class DatumFactory {
     }
   }
 
-  public static TimestampDatum createTimestamp(Datum datum) {
+  public static TimestampDatum createTimestamp(Datum datum, @Nullable TimeZone tz) {
     switch (datum.type()) {
+      case CHAR:
+      case VARCHAR:
       case TEXT:
-        return parseTimestamp(datum.asChars());
+        return parseTimestamp(datum.asChars(), tz);
       case TIMESTAMP:
         return (TimestampDatum) datum;
       default:
@@ -349,8 +369,8 @@ public class DatumFactory {
     return new TimestampDatum(julianTimestamp);
   }
 
-  public static TimestampDatum parseTimestamp(String str) {
-    return new TimestampDatum(DateTimeUtil.toJulianTimestampWithTZ(str));
+  public static TimestampDatum parseTimestamp(String str, @Nullable TimeZone tz) {
+    return new TimestampDatum(DateTimeUtil.toJulianTimestampWithTZ(str, tz));
   }
 
   public static BlobDatum createBlob(byte[] val) {
@@ -381,7 +401,7 @@ public class DatumFactory {
     return new Inet4Datum(val);
   }
 
-  public static Datum cast(Datum operandDatum, DataType target) {
+  public static Datum cast(Datum operandDatum, DataType target, @Nullable TimeZone tz) {
     switch (target.getType()) {
     case BOOLEAN:
       return DatumFactory.createBool(operandDatum.asBool());
@@ -398,21 +418,24 @@ public class DatumFactory {
       return DatumFactory.createFloat4(operandDatum.asFloat4());
     case FLOAT8:
       return DatumFactory.createFloat8(operandDatum.asFloat8());
+    case VARCHAR:
     case TEXT:
       switch (operandDatum.type()) {
         case TIMESTAMP: {
           TimestampDatum timestampDatum = (TimestampDatum)operandDatum;
-          TimeMeta tm = timestampDatum.toTimeMeta();
-          DateTimeUtil.toUserTimezone(tm);
-          TimestampDatum convertedTimestampDatum = new TimestampDatum(DateTimeUtil.toJulianTimestamp(tm));
-          return DatumFactory.createText(convertedTimestampDatum.asTextBytes());
+          if (tz != null) {
+            return DatumFactory.createText(timestampDatum.asChars(tz, false));
+          } else {
+            return DatumFactory.createText(timestampDatum.asChars());
+          }
         }
         case TIME: {
           TimeDatum timeDatum = (TimeDatum)operandDatum;
-          TimeMeta tm = timeDatum.toTimeMeta();
-          DateTimeUtil.toUserTimezone(tm);
-          TimeDatum convertedTimeDatum = new TimeDatum(DateTimeUtil.toTime(tm));
-          return DatumFactory.createText(convertedTimeDatum.asTextBytes());
+          if (tz != null) {
+            return DatumFactory.createText(timeDatum.asChars(tz, false));
+          } else {
+            return DatumFactory.createText(timeDatum.asChars());
+          }
         }
         default:
           return DatumFactory.createText(operandDatum.asTextBytes());
@@ -420,9 +443,9 @@ public class DatumFactory {
     case DATE:
       return DatumFactory.createDate(operandDatum);
     case TIME:
-      return DatumFactory.createTime(operandDatum);
+      return DatumFactory.createTime(operandDatum, tz);
     case TIMESTAMP:
-      return DatumFactory.createTimestamp(operandDatum);
+      return DatumFactory.createTimestamp(operandDatum, tz);
     case BLOB:
       return DatumFactory.createBlob(operandDatum.asByteArray());
     case INET4:

http://git-wip-us.apache.org/repos/asf/tajo/blob/facd1ddc/tajo-common/src/main/java/org/apache/tajo/json/TimeZoneGsonSerdeAdapter.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/json/TimeZoneGsonSerdeAdapter.java b/tajo-common/src/main/java/org/apache/tajo/json/TimeZoneGsonSerdeAdapter.java
new file mode 100644
index 0000000..38441d3
--- /dev/null
+++ b/tajo-common/src/main/java/org/apache/tajo/json/TimeZoneGsonSerdeAdapter.java
@@ -0,0 +1,40 @@
+/*
+ * 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 java.lang.reflect.Type;
+import java.util.TimeZone;
+
+public class TimeZoneGsonSerdeAdapter implements GsonSerDerAdapter<TimeZone> {
+
+	@Override
+	public JsonElement serialize(TimeZone object, Type arg1, JsonSerializationContext arg2) {
+		return new JsonPrimitive(object.getID());
+	}
+
+  @Override
+  public TimeZone deserialize(JsonElement arg0, Type arg1, JsonDeserializationContext context) throws JsonParseException {
+    return TimeZone.getTimeZone(arg0.getAsJsonPrimitive().getAsString());
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/facd1ddc/tajo-common/src/main/java/org/apache/tajo/storage/StorageConstants.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/storage/StorageConstants.java b/tajo-common/src/main/java/org/apache/tajo/storage/StorageConstants.java
index 459c9c9..d2c6c1c 100644
--- a/tajo-common/src/main/java/org/apache/tajo/storage/StorageConstants.java
+++ b/tajo-common/src/main/java/org/apache/tajo/storage/StorageConstants.java
@@ -18,11 +18,20 @@
 
 package org.apache.tajo.storage;
 
+import org.apache.tajo.TajoConstants;
+
 public class StorageConstants {
-  // table properties
+
+  // Common table properties -------------------------------------------------
+
+  // time zone
+  public static final String TIMEZONE = "timezone";
+
+  // compression
   public static final String COMPRESSION_CODEC = "compression.codec";
   public static final String COMPRESSION_TYPE = "compression.type";
 
+  // Text file properties -------------------------------------------------
   @Deprecated
   public static final String CSVFILE_DELIMITER = "csvfile.delimiter";
   @Deprecated
@@ -47,12 +56,14 @@ public class StorageConstants {
   public static final String TEXT_ERROR_TOLERANCE_MAXNUM = "text.error-tolerance.max-num";
   public static final String DEFAULT_TEXT_ERROR_TOLERANCE_MAXNUM = "0";
 
+  // Sequence file properties -------------------------------------------------
   @Deprecated
   public static final String SEQUENCEFILE_DELIMITER = "sequencefile.delimiter";
   @Deprecated
   public static final String SEQUENCEFILE_NULL = "sequencefile.null";
   public static final String SEQUENCEFILE_SERDE = "sequencefile.serde";
 
+  // RC file properties -------------------------------------------------
   @Deprecated
   public static final String RCFILE_NULL = "rcfile.null";
   public static final String RCFILE_SERDE = "rcfile.serde";
@@ -61,17 +72,22 @@ public class StorageConstants {
   public static final String DEFAULT_BINARY_SERDE = "org.apache.tajo.storage.BinarySerializerDeserializer";
   public static final String DEFAULT_TEXT_SERDE = "org.apache.tajo.storage.TextSerializerDeserializer";
 
+
+  // Parquet file properties -------------------------------------------------
   public static final String PARQUET_DEFAULT_BLOCK_SIZE;
   public static final String PARQUET_DEFAULT_PAGE_SIZE;
   public static final String PARQUET_DEFAULT_COMPRESSION_CODEC_NAME;
   public static final String PARQUET_DEFAULT_IS_DICTIONARY_ENABLED;
   public static final String PARQUET_DEFAULT_IS_VALIDATION_ENABLED;
 
+  public static final int DEFAULT_BLOCK_SIZE = 128 * 1024 * 1024;
+  public static final int DEFAULT_PAGE_SIZE = 1 * 1024 * 1024;
+
+
+  // Avro file properties -------------------------------------------------
   public static final String AVRO_SCHEMA_LITERAL = "avro.schema.literal";
   public static final String AVRO_SCHEMA_URL = "avro.schema.url";
 
-  public static final int DEFAULT_BLOCK_SIZE = 128 * 1024 * 1024;
-  public static final int DEFAULT_PAGE_SIZE = 1 * 1024 * 1024;
   static {
     PARQUET_DEFAULT_BLOCK_SIZE = Integer.toString(DEFAULT_BLOCK_SIZE);
     PARQUET_DEFAULT_PAGE_SIZE = Integer.toString(DEFAULT_PAGE_SIZE);

http://git-wip-us.apache.org/repos/asf/tajo/blob/facd1ddc/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 a9dc4e8..7acb544 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
@@ -25,6 +25,7 @@ import org.apache.tajo.util.datetime.DateTimeConstants.DateStyle;
 import org.apache.tajo.util.datetime.DateTimeConstants.DateToken;
 import org.apache.tajo.util.datetime.DateTimeConstants.TokenField;
 
+import javax.annotation.Nullable;
 import java.util.TimeZone;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
@@ -34,7 +35,6 @@ import java.util.concurrent.atomic.AtomicLong;
  * This Class is originated from j2date in datetime.c of PostgreSQL.
  */
 public class DateTimeUtil {
-
   private static int MAX_FRACTION_LENGTH = 6;
 
   /** maximum possible number of fields in a date * string */
@@ -663,16 +663,20 @@ public class DateTimeUtil {
 
 
   /**
-   * Parse datetime string to julian time.
-   * The result is the local time basis.
+   * Parse datetime string to UTC-based julian time.
+   * The result julian time is adjusted by local timezone.
+   *
    * @param timestampStr
-   * @return
+   * @param tz Local timezone. If it is NULL, UTC will be used by default.
+   * @return UTC-based julian time
    */
-  public static long toJulianTimestampWithTZ(String timestampStr) {
+  public static long toJulianTimestampWithTZ(String timestampStr, @Nullable TimeZone tz) {
     long timestamp = DateTimeUtil.toJulianTimestamp(timestampStr);
     TimeMeta tm = new TimeMeta();
     DateTimeUtil.toJulianTimeMeta(timestamp, tm);
-    DateTimeUtil.toUTCTimezone(tm);
+    if (tz != null) {
+      DateTimeUtil.toUTCTimezone(tm, tz);
+    }
     return DateTimeUtil.toJulianTimestamp(tm);
   }
 
@@ -2060,16 +2064,11 @@ public class DateTimeUtil {
     return date2j(year, mon, mday) - isoweek2j(date2isoyear(year, mon, mday), 1) + 1;
   }
 
-  public static void toUserTimezone(TimeMeta tm) {
-    toUserTimezone(tm, TajoConf.getCurrentTimeZone());
-  }
-
   public static void toUserTimezone(TimeMeta tm, TimeZone timeZone) {
     tm.plusMillis(timeZone.getRawOffset());
   }
 
-  public static void toUTCTimezone(TimeMeta tm) {
-    TimeZone timeZone = TajoConf.getCurrentTimeZone();
+  public static void toUTCTimezone(TimeMeta tm, TimeZone timeZone) {
     tm.plusMillis(0 - timeZone.getRawOffset());
   }
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/facd1ddc/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
index 5886083..277eefc 100644
--- a/tajo-common/src/test/java/org/apache/tajo/datum/TestTimestampDatum.java
+++ b/tajo-common/src/test/java/org/apache/tajo/datum/TestTimestampDatum.java
@@ -19,7 +19,6 @@
 package org.apache.tajo.datum;
 
 import org.apache.tajo.common.TajoDataTypes.Type;
-import org.apache.tajo.conf.TajoConf;
 import org.apache.tajo.exception.InvalidCastException;
 import org.apache.tajo.json.CommonGsonHelper;
 import org.apache.tajo.util.datetime.DateTimeUtil;
@@ -29,8 +28,8 @@ import org.junit.Test;
 import java.util.Calendar;
 import java.util.TimeZone;
 
+import static org.hamcrest.CoreMatchers.is;
 import static org.junit.Assert.*;
-import static org.hamcrest.CoreMatchers.*;
 
 public class TestTimestampDatum {
   private static long javatime;
@@ -40,7 +39,7 @@ public class TestTimestampDatum {
   @BeforeClass
   public static void setUp() {
     javatime = System.currentTimeMillis();
-    calendar = Calendar.getInstance(TajoConf.getCurrentTimeZone());
+    calendar = Calendar.getInstance(TimeZone.getTimeZone("PST"));
     calendar.setTimeInMillis(javatime);
     unixtime = (int) (javatime / 1000);
   }
@@ -151,7 +150,7 @@ public class TestTimestampDatum {
     assertEquals(datum2, datum);
 
     for (int i = 0; i < 100; i++) {
-      TimeZone timeZone = TimeZone.getTimeZone("UTC");
+      TimeZone timeZone = TimeZone.getTimeZone("GMT");
       Calendar cal = Calendar.getInstance(timeZone);
       long jTime = System.currentTimeMillis();
       int uTime = (int)(jTime / 1000);

http://git-wip-us.apache.org/repos/asf/tajo/blob/facd1ddc/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/AvgDouble.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/AvgDouble.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/AvgDouble.java
index 5961076..f337c36 100644
--- a/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/AvgDouble.java
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/AvgDouble.java
@@ -52,9 +52,6 @@ public class AvgDouble extends AggFunction {
     return new AvgContext();
   }
 
-  public void init() {
-  }
-
   @Override
   public void eval(FunctionContext ctx, Tuple params) {
     AvgContext avgCtx = (AvgContext) ctx;

http://git-wip-us.apache.org/repos/asf/tajo/blob/facd1ddc/tajo-core/src/main/java/org/apache/tajo/engine/function/datetime/CurrentDate.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/datetime/CurrentDate.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/datetime/CurrentDate.java
index f43a41f..ccfbd84 100644
--- a/tajo-core/src/main/java/org/apache/tajo/engine/function/datetime/CurrentDate.java
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/datetime/CurrentDate.java
@@ -18,17 +18,24 @@
 
 package org.apache.tajo.engine.function.datetime;
 
+import com.google.gson.annotations.Expose;
+import org.apache.tajo.OverridableConf;
+import org.apache.tajo.SessionVars;
+import org.apache.tajo.TajoConstants;
 import org.apache.tajo.common.TajoDataTypes;
 import org.apache.tajo.datum.DateDatum;
 import org.apache.tajo.datum.Datum;
 import org.apache.tajo.datum.DatumFactory;
-import org.apache.tajo.plan.function.GeneralFunction;
 import org.apache.tajo.engine.function.annotation.Description;
 import org.apache.tajo.engine.function.annotation.ParamTypes;
+import org.apache.tajo.plan.expr.FunctionEval;
+import org.apache.tajo.plan.function.GeneralFunction;
 import org.apache.tajo.storage.Tuple;
 import org.apache.tajo.util.datetime.DateTimeUtil;
 import org.apache.tajo.util.datetime.TimeMeta;
 
+import java.util.TimeZone;
+
 @Description(
     functionName = "current_date",
     description = "Get current date. Result is DATE type.",
@@ -37,19 +44,26 @@ import org.apache.tajo.util.datetime.TimeMeta;
     paramTypes = {@ParamTypes(paramTypes = {})}
 )
 public class CurrentDate extends GeneralFunction {
-  DateDatum datum;
+  @Expose private TimeZone timezone;
+  private DateDatum datum;
 
   public CurrentDate() {
     super(NoArgs);
   }
 
   @Override
+  public void init(OverridableConf context, FunctionEval.ParamType[] types) {
+    String timezoneId = context.get(SessionVars.TZ, TajoConstants.DEFAULT_SYSTEM_TIMEZONE);
+    timezone = TimeZone.getTimeZone(timezoneId);
+  }
+
+  @Override
   public Datum eval(Tuple params) {
     if (datum == null) {
       long julianTimestamp = DateTimeUtil.javaTimeToJulianTime(System.currentTimeMillis());
       TimeMeta tm = new TimeMeta();
       DateTimeUtil.toJulianTimeMeta(julianTimestamp, tm);
-      DateTimeUtil.toUserTimezone(tm);
+      DateTimeUtil.toUserTimezone(tm, timezone);
       datum = DatumFactory.createDate(tm.years, tm.monthOfYear, tm.dayOfMonth);
     }
     return datum;

http://git-wip-us.apache.org/repos/asf/tajo/blob/facd1ddc/tajo-core/src/main/java/org/apache/tajo/engine/function/datetime/DatePartFromTime.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/datetime/DatePartFromTime.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/datetime/DatePartFromTime.java
index 66a54bf..ae09197 100644
--- a/tajo-core/src/main/java/org/apache/tajo/engine/function/datetime/DatePartFromTime.java
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/datetime/DatePartFromTime.java
@@ -18,20 +18,27 @@
 
 package org.apache.tajo.engine.function.datetime;
 
+import com.google.gson.annotations.Expose;
+import org.apache.tajo.OverridableConf;
+import org.apache.tajo.SessionVars;
+import org.apache.tajo.TajoConstants;
 import org.apache.tajo.catalog.Column;
 import org.apache.tajo.common.TajoDataTypes;
 import org.apache.tajo.datum.Datum;
 import org.apache.tajo.datum.DatumFactory;
 import org.apache.tajo.datum.NullDatum;
 import org.apache.tajo.datum.TimeDatum;
-import org.apache.tajo.plan.function.GeneralFunction;
 import org.apache.tajo.engine.function.annotation.Description;
 import org.apache.tajo.engine.function.annotation.ParamTypes;
+import org.apache.tajo.plan.expr.FunctionEval;
+import org.apache.tajo.plan.function.GeneralFunction;
 import org.apache.tajo.storage.Tuple;
 import org.apache.tajo.util.datetime.DateTimeConstants;
 import org.apache.tajo.util.datetime.DateTimeUtil;
 import org.apache.tajo.util.datetime.TimeMeta;
 
+import java.util.TimeZone;
+
 import static org.apache.tajo.common.TajoDataTypes.Type.FLOAT8;
 import static org.apache.tajo.common.TajoDataTypes.Type.TEXT;
 
@@ -44,6 +51,9 @@ import static org.apache.tajo.common.TajoDataTypes.Type.TEXT;
     paramTypes = {@ParamTypes(paramTypes = {TajoDataTypes.Type.TEXT, TajoDataTypes.Type.TIME})}
 )
 public class DatePartFromTime extends GeneralFunction {
+  @Expose  private TimeZone timezone;
+  private DatePartExtractorFromTime extractor = null;
+
   public DatePartFromTime() {
     super(new Column[] {
         new Column("target", FLOAT8),
@@ -51,7 +61,11 @@ public class DatePartFromTime extends GeneralFunction {
     });
   }
 
-  private DatePartExtractorFromTime extractor = null;
+  @Override
+  public void init(OverridableConf context, FunctionEval.ParamType [] types) {
+    String timezoneId = context.get(SessionVars.TZ, TajoConstants.DEFAULT_SYSTEM_TIMEZONE);
+    timezone = TimeZone.getTimeZone(timezoneId);
+  }
 
   @Override
   public Datum eval(Tuple params) {
@@ -93,7 +107,7 @@ public class DatePartFromTime extends GeneralFunction {
     }
 
     TimeMeta tm = time.toTimeMeta();
-    DateTimeUtil.toUserTimezone(tm);
+    DateTimeUtil.toUserTimezone(tm, timezone);
     return extractor.extract(tm);
   }
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/facd1ddc/tajo-core/src/main/java/org/apache/tajo/engine/function/datetime/DatePartFromTimestamp.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/datetime/DatePartFromTimestamp.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/datetime/DatePartFromTimestamp.java
index 843c593..cd195fb 100644
--- a/tajo-core/src/main/java/org/apache/tajo/engine/function/datetime/DatePartFromTimestamp.java
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/datetime/DatePartFromTimestamp.java
@@ -18,9 +18,14 @@
 
 package org.apache.tajo.engine.function.datetime;
 
+import com.google.gson.annotations.Expose;
+import org.apache.tajo.OverridableConf;
+import org.apache.tajo.SessionVars;
+import org.apache.tajo.TajoConstants;
 import org.apache.tajo.catalog.Column;
 import org.apache.tajo.common.TajoDataTypes;
 import org.apache.tajo.datum.*;
+import org.apache.tajo.plan.expr.FunctionEval;
 import org.apache.tajo.plan.function.GeneralFunction;
 import org.apache.tajo.engine.function.annotation.Description;
 import org.apache.tajo.engine.function.annotation.ParamTypes;
@@ -29,6 +34,8 @@ import org.apache.tajo.util.datetime.DateTimeConstants;
 import org.apache.tajo.util.datetime.DateTimeUtil;
 import org.apache.tajo.util.datetime.TimeMeta;
 
+import java.util.TimeZone;
+
 import static org.apache.tajo.common.TajoDataTypes.Type.*;
 
 @Description(
@@ -40,6 +47,9 @@ import static org.apache.tajo.common.TajoDataTypes.Type.*;
     paramTypes = {@ParamTypes(paramTypes = {TajoDataTypes.Type.TEXT, TajoDataTypes.Type.TIMESTAMP})}
 )
 public class DatePartFromTimestamp extends GeneralFunction {
+  @Expose private TimeZone timezone;
+  private DatePartExtractorFromTimestamp extractor = null;
+
   public DatePartFromTimestamp() {
     super(new Column[] {
         new Column("target", FLOAT8),
@@ -47,7 +57,11 @@ public class DatePartFromTimestamp extends GeneralFunction {
     });
   }
 
-  private DatePartExtractorFromTimestamp extractor = null;
+  @Override
+  public void init(OverridableConf context, FunctionEval.ParamType [] types) {
+    String timezoneId = context.get(SessionVars.TZ, TajoConstants.DEFAULT_SYSTEM_TIMEZONE);
+    timezone = TimeZone.getTimeZone(timezoneId);
+  }
 
   @Override
   public Datum eval(Tuple params) {
@@ -115,7 +129,7 @@ public class DatePartFromTimestamp extends GeneralFunction {
     }
 
     TimeMeta tm = timestamp.toTimeMeta();
-    DateTimeUtil.toUserTimezone(tm);
+    DateTimeUtil.toUserTimezone(tm, timezone);
 
     return extractor.extract(tm);
   }

http://git-wip-us.apache.org/repos/asf/tajo/blob/facd1ddc/tajo-core/src/main/java/org/apache/tajo/engine/function/datetime/ToCharTimestamp.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/datetime/ToCharTimestamp.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/datetime/ToCharTimestamp.java
index 02b52e3..9e76677 100644
--- a/tajo-core/src/main/java/org/apache/tajo/engine/function/datetime/ToCharTimestamp.java
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/datetime/ToCharTimestamp.java
@@ -18,6 +18,10 @@
 
 package org.apache.tajo.engine.function.datetime;
 
+import com.google.gson.annotations.Expose;
+import org.apache.tajo.OverridableConf;
+import org.apache.tajo.SessionVars;
+import org.apache.tajo.TajoConstants;
 import org.apache.tajo.catalog.Column;
 import org.apache.tajo.common.TajoDataTypes;
 import org.apache.tajo.datum.Datum;
@@ -33,6 +37,8 @@ import org.apache.tajo.util.datetime.DateTimeFormat;
 import org.apache.tajo.util.datetime.DateTimeUtil;
 import org.apache.tajo.util.datetime.TimeMeta;
 
+import java.util.TimeZone;
+
 import static org.apache.tajo.common.TajoDataTypes.Type.TEXT;
 import static org.apache.tajo.common.TajoDataTypes.Type.TIMESTAMP;
 
@@ -45,6 +51,8 @@ import static org.apache.tajo.common.TajoDataTypes.Type.TIMESTAMP;
   paramTypes = {@ParamTypes(paramTypes = {TajoDataTypes.Type.TIMESTAMP, TajoDataTypes.Type.TEXT})}
 )
 public class ToCharTimestamp extends GeneralFunction {
+  @Expose private TimeZone timezone;
+
   public ToCharTimestamp() {
     super(new Column[] {
         new Column("timestamp", TIMESTAMP),
@@ -53,7 +61,9 @@ public class ToCharTimestamp extends GeneralFunction {
   }
 
   @Override
-  public void init(FunctionEval.ParamType[] paramTypes) {
+  public void init(OverridableConf context, FunctionEval.ParamType[] paramTypes) {
+    String timezoneId = context.get(SessionVars.TZ, TajoConstants.DEFAULT_SYSTEM_TIMEZONE);
+    timezone = TimeZone.getTimeZone(timezoneId);
   }
 
   @Override
@@ -63,11 +73,11 @@ public class ToCharTimestamp extends GeneralFunction {
     }
 
     TimestampDatum valueDatum = (TimestampDatum) params.get(0);
-    TimeMeta tm = valueDatum.toTimeMeta();
-    DateTimeUtil.toUserTimezone(tm);
-
     Datum pattern = params.get(1);
 
+    TimeMeta tm = valueDatum.toTimeMeta();
+    DateTimeUtil.toUserTimezone(tm, timezone);
+
     return DatumFactory.createText(DateTimeFormat.to_char(tm, pattern.asChars()));
   }
 }

http://git-wip-us.apache.org/repos/asf/tajo/blob/facd1ddc/tajo-core/src/main/java/org/apache/tajo/engine/function/datetime/ToTimestampText.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/datetime/ToTimestampText.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/datetime/ToTimestampText.java
index e50bacc..e9beefb 100644
--- a/tajo-core/src/main/java/org/apache/tajo/engine/function/datetime/ToTimestampText.java
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/datetime/ToTimestampText.java
@@ -18,9 +18,13 @@
 
 package org.apache.tajo.engine.function.datetime;
 
+import org.apache.tajo.OverridableConf;
+import org.apache.tajo.SessionVars;
+import org.apache.tajo.TajoConstants;
 import org.apache.tajo.catalog.Column;
 import org.apache.tajo.common.TajoDataTypes;
 import org.apache.tajo.datum.*;
+import org.apache.tajo.plan.expr.FunctionEval;
 import org.apache.tajo.plan.function.GeneralFunction;
 import org.apache.tajo.engine.function.annotation.Description;
 import org.apache.tajo.engine.function.annotation.ParamTypes;
@@ -29,6 +33,8 @@ import org.apache.tajo.util.datetime.DateTimeFormat;
 import org.apache.tajo.util.datetime.DateTimeUtil;
 import org.apache.tajo.util.datetime.TimeMeta;
 
+import java.util.TimeZone;
+
 import static org.apache.tajo.common.TajoDataTypes.Type.TEXT;
 
 @Description(
@@ -41,10 +47,17 @@ import static org.apache.tajo.common.TajoDataTypes.Type.TEXT;
     paramTypes = {@ParamTypes(paramTypes = {TajoDataTypes.Type.TEXT, TajoDataTypes.Type.TEXT})}
 )
 public class ToTimestampText extends GeneralFunction {
+  private TimeZone timezone;
+
   public ToTimestampText() {
     super(new Column[]{new Column("DateTimeText", TEXT), new Column("Pattern", TEXT)});
   }
 
+  public void init(OverridableConf queryContext, FunctionEval.ParamType [] paramTypes) {
+    String timezoneId = queryContext.get(SessionVars.TZ, TajoConstants.DEFAULT_SYSTEM_TIMEZONE);
+    timezone = TimeZone.getTimeZone(timezoneId);
+  }
+
   @Override
   public Datum eval(Tuple params) {
     if(params.isNull(0) || params.isNull(1)) {
@@ -55,7 +68,7 @@ public class ToTimestampText extends GeneralFunction {
     TextDatum patternDatum = (TextDatum) params.get(1);
 
     TimeMeta tm = DateTimeFormat.parseDateTime(dateTimeTextDatum.asChars(), patternDatum.asChars());
-    DateTimeUtil.toUTCTimezone(tm);
+    DateTimeUtil.toUTCTimezone(tm, timezone);
 
     return new TimestampDatum(DateTimeUtil.toJulianTimestamp(tm));
   }

http://git-wip-us.apache.org/repos/asf/tajo/blob/facd1ddc/tajo-core/src/main/java/org/apache/tajo/engine/function/string/BTrim.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/string/BTrim.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/string/BTrim.java
index 3214a5e..9864b0d 100644
--- a/tajo-core/src/main/java/org/apache/tajo/engine/function/string/BTrim.java
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/string/BTrim.java
@@ -20,6 +20,7 @@ package org.apache.tajo.engine.function.string;
 
 import com.google.gson.annotations.Expose;
 import org.apache.commons.lang.StringUtils;
+import org.apache.tajo.OverridableConf;
 import org.apache.tajo.catalog.Column;
 import org.apache.tajo.common.TajoDataTypes;
 import org.apache.tajo.datum.Datum;
@@ -57,7 +58,8 @@ public class BTrim extends GeneralFunction {
     });
   }
 
-  public void init(FunctionEval.ParamType[] paramTypes) {
+  @Override
+  public void init(OverridableConf context, FunctionEval.ParamType[] paramTypes) {
     if (paramTypes.length == 2) {
       hasTrimCharacters = true;
     }

http://git-wip-us.apache.org/repos/asf/tajo/blob/facd1ddc/tajo-core/src/main/java/org/apache/tajo/engine/function/string/LTrim.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/string/LTrim.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/string/LTrim.java
index f345293..5d95c38 100644
--- a/tajo-core/src/main/java/org/apache/tajo/engine/function/string/LTrim.java
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/string/LTrim.java
@@ -20,6 +20,7 @@ package org.apache.tajo.engine.function.string;
 
 import com.google.gson.annotations.Expose;
 import org.apache.commons.lang.StringUtils;
+import org.apache.tajo.OverridableConf;
 import org.apache.tajo.catalog.Column;
 import org.apache.tajo.common.TajoDataTypes;
 import org.apache.tajo.datum.Datum;
@@ -57,7 +58,8 @@ public class LTrim extends GeneralFunction {
     });
   }
 
-  public void init(FunctionEval.ParamType[] paramTypes) {
+  @Override
+  public void init(OverridableConf context, FunctionEval.ParamType[] paramTypes) {
     if (paramTypes.length == 2) {
       hasTrimCharacters = true;
     }

http://git-wip-us.apache.org/repos/asf/tajo/blob/facd1ddc/tajo-core/src/main/java/org/apache/tajo/engine/function/string/Lpad.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/string/Lpad.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/string/Lpad.java
index 58c0951..da11f19 100644
--- a/tajo-core/src/main/java/org/apache/tajo/engine/function/string/Lpad.java
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/string/Lpad.java
@@ -20,6 +20,7 @@ package org.apache.tajo.engine.function.string;
 
 import com.google.gson.annotations.Expose;
 import org.apache.commons.lang.StringUtils;
+import org.apache.tajo.OverridableConf;
 import org.apache.tajo.catalog.Column;
 import org.apache.tajo.common.TajoDataTypes;
 import org.apache.tajo.datum.Datum;
@@ -57,7 +58,8 @@ public class Lpad extends GeneralFunction {
     });
   }
 
-  public void init(FunctionEval.ParamType[] paramTypes) {
+  @Override
+  public void init(OverridableConf context, FunctionEval.ParamType[] paramTypes) {
     if (paramTypes.length == 3) {
       hasFillCharacters = true;
     }

http://git-wip-us.apache.org/repos/asf/tajo/blob/facd1ddc/tajo-core/src/main/java/org/apache/tajo/engine/function/string/RTrim.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/string/RTrim.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/string/RTrim.java
index 8e295fa..03b9c25 100644
--- a/tajo-core/src/main/java/org/apache/tajo/engine/function/string/RTrim.java
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/string/RTrim.java
@@ -20,6 +20,7 @@ package org.apache.tajo.engine.function.string;
 
 import com.google.gson.annotations.Expose;
 import org.apache.commons.lang.StringUtils;
+import org.apache.tajo.OverridableConf;
 import org.apache.tajo.catalog.Column;
 import org.apache.tajo.common.TajoDataTypes;
 import org.apache.tajo.datum.Datum;
@@ -56,7 +57,8 @@ public class RTrim extends GeneralFunction {
     });
   }
 
-  public void init(FunctionEval.ParamType[] paramTypes) {
+  @Override
+  public void init(OverridableConf context, FunctionEval.ParamType[] paramTypes) {
     if (paramTypes.length == 2) {
       hasTrimCharacters = true;
     }

http://git-wip-us.apache.org/repos/asf/tajo/blob/facd1ddc/tajo-core/src/main/java/org/apache/tajo/engine/function/string/RegexpReplace.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/string/RegexpReplace.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/string/RegexpReplace.java
index da06f73..6b888ec 100644
--- a/tajo-core/src/main/java/org/apache/tajo/engine/function/string/RegexpReplace.java
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/string/RegexpReplace.java
@@ -19,6 +19,7 @@
 package org.apache.tajo.engine.function.string;
 
 import com.google.gson.annotations.Expose;
+import org.apache.tajo.OverridableConf;
 import org.apache.tajo.catalog.Column;
 import org.apache.tajo.common.TajoDataTypes;
 import org.apache.tajo.datum.BooleanDatum;
@@ -70,7 +71,8 @@ public class RegexpReplace extends GeneralFunction {
     });
   }
 
-  public void init(ParamType[] paramTypes) {
+  @Override
+  public void init(OverridableConf context, ParamType[] paramTypes) {
     if (paramTypes[0] == ParamType.NULL || paramTypes[1] == ParamType.NULL || paramTypes[2] == ParamType.NULL) {
       isAlwaysNull = true;
     } else if (paramTypes[1] == ParamType.CONSTANT) {

http://git-wip-us.apache.org/repos/asf/tajo/blob/facd1ddc/tajo-core/src/main/java/org/apache/tajo/engine/function/string/Rpad.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/string/Rpad.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/string/Rpad.java
index 9d42848..1509118 100644
--- a/tajo-core/src/main/java/org/apache/tajo/engine/function/string/Rpad.java
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/string/Rpad.java
@@ -20,6 +20,7 @@ package org.apache.tajo.engine.function.string;
 
 import com.google.gson.annotations.Expose;
 import org.apache.commons.lang.StringUtils;
+import org.apache.tajo.OverridableConf;
 import org.apache.tajo.catalog.Column;
 import org.apache.tajo.common.TajoDataTypes;
 import org.apache.tajo.datum.Datum;
@@ -58,7 +59,8 @@ public class Rpad extends GeneralFunction {
     });
   }
 
-  public void init(FunctionEval.ParamType[] paramTypes) {
+  @Override
+  public void init(OverridableConf context, FunctionEval.ParamType[] paramTypes) {
     if (paramTypes.length == 3) {
       hasFillCharacters = true;
     }

http://git-wip-us.apache.org/repos/asf/tajo/blob/facd1ddc/tajo-core/src/main/java/org/apache/tajo/engine/json/CoreGsonHelper.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/json/CoreGsonHelper.java b/tajo-core/src/main/java/org/apache/tajo/engine/json/CoreGsonHelper.java
index 57f2536..5a03bfd 100644
--- a/tajo-core/src/main/java/org/apache/tajo/engine/json/CoreGsonHelper.java
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/json/CoreGsonHelper.java
@@ -38,6 +38,7 @@ import org.apache.tajo.util.TUtil;
 
 import java.lang.reflect.Type;
 import java.util.Map;
+import java.util.TimeZone;
 
 public class CoreGsonHelper {
   private static Gson gson;
@@ -58,6 +59,7 @@ public class CoreGsonHelper {
     adapters.put(AggFunction.class, new FunctionAdapter());
     adapters.put(Datum.class, new DatumAdapter());
     adapters.put(DataType.class, new DataTypeAdapter());
+    adapters.put(TimeZone.class, new TimeZoneGsonSerdeAdapter());
 
     return adapters;
 	}

http://git-wip-us.apache.org/repos/asf/tajo/blob/facd1ddc/tajo-core/src/main/java/org/apache/tajo/engine/query/QueryContext.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/query/QueryContext.java b/tajo-core/src/main/java/org/apache/tajo/engine/query/QueryContext.java
index d8f7f08..47ead40 100644
--- a/tajo-core/src/main/java/org/apache/tajo/engine/query/QueryContext.java
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/query/QueryContext.java
@@ -74,12 +74,12 @@ public class QueryContext extends OverridableConf {
   }
 
   public QueryContext(TajoConf conf, Session session) {
-    super(conf);
+    super(conf, ConfigKey.ConfigType.QUERY, ConfigKey.ConfigType.SESSION);
     putAll(session.getAllVariables());
   }
 
   public QueryContext(TajoConf conf, KeyValueSetProto proto) {
-    super(conf, proto, ConfigKey.ConfigType.QUERY);
+    super(conf, proto, ConfigKey.ConfigType.QUERY, ConfigKey.ConfigType.SESSION);
   }
 
   //-----------------------------------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/tajo/blob/facd1ddc/tajo-core/src/test/java/org/apache/tajo/QueryTestCaseBase.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/QueryTestCaseBase.java b/tajo-core/src/test/java/org/apache/tajo/QueryTestCaseBase.java
index 4e4964e..efcc691 100644
--- a/tajo-core/src/test/java/org/apache/tajo/QueryTestCaseBase.java
+++ b/tajo-core/src/test/java/org/apache/tajo/QueryTestCaseBase.java
@@ -532,6 +532,14 @@ public class QueryTestCaseBase {
    *   replaced by the first and second elements of <code>args</code> respectively</li>. It uses zero-based index.
    * </ul>
    *
+   * Example ddl
+   * <pre>
+   *   CREATE EXTERNAL TABLE ${0} (
+   *     t_timestamp  TIMESTAMP,
+   *     t_date    DATE
+   *   ) USING CSV LOCATION ${table.path}
+   * </pre>
+   *
    * @param ddlFileName A file name, containing a data definition statement.
    * @param dataFileName A file name, containing data rows, which columns have to be separated by vertical bar '|'.
    *                     This file name is used for replacing some format string indicating an external table location.

http://git-wip-us.apache.org/repos/asf/tajo/blob/facd1ddc/tajo-core/src/test/java/org/apache/tajo/TajoTestingCluster.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/TajoTestingCluster.java b/tajo-core/src/test/java/org/apache/tajo/TajoTestingCluster.java
index 603da0c..e5dfb7a 100644
--- a/tajo-core/src/test/java/org/apache/tajo/TajoTestingCluster.java
+++ b/tajo-core/src/test/java/org/apache/tajo/TajoTestingCluster.java
@@ -59,6 +59,7 @@ import java.net.URL;
 import java.sql.ResultSet;
 import java.util.ArrayList;
 import java.util.List;
+import java.util.TimeZone;
 import java.util.UUID;
 
 public class TajoTestingCluster {
@@ -117,6 +118,8 @@ public class TajoTestingCluster {
   }
 
   void initPropertiesAndConfigs() {
+    TimeZone.setDefault(TimeZone.getTimeZone(TajoConstants.DEFAULT_SYSTEM_TIMEZONE));
+
     if (System.getProperty(ConfVars.RESOURCE_MANAGER_CLASS.varname) != null) {
       String testResourceManager = System.getProperty(ConfVars.RESOURCE_MANAGER_CLASS.varname);
       Preconditions.checkState(testResourceManager.equals(TajoWorkerResourceManager.class.getCanonicalName()));

http://git-wip-us.apache.org/repos/asf/tajo/blob/facd1ddc/tajo-core/src/test/java/org/apache/tajo/engine/eval/ExprTestBase.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/eval/ExprTestBase.java b/tajo-core/src/test/java/org/apache/tajo/engine/eval/ExprTestBase.java
index df9ef65..8cd87a8 100644
--- a/tajo-core/src/test/java/org/apache/tajo/engine/eval/ExprTestBase.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/eval/ExprTestBase.java
@@ -58,6 +58,7 @@ import org.junit.BeforeClass;
 
 import java.io.IOException;
 import java.util.List;
+import java.util.TimeZone;
 
 import static org.apache.tajo.TajoConstants.DEFAULT_DATABASE_NAME;
 import static org.apache.tajo.TajoConstants.DEFAULT_TABLESPACE_NAME;
@@ -73,8 +74,8 @@ public class ExprTestBase {
   private static LogicalOptimizer optimizer;
   private static LogicalPlanVerifier annotatedPlanVerifier;
 
-  public static String getUserTimeZoneDisplay() {
-    return DateTimeUtil.getTimeZoneDisplayTime(TajoConf.getCurrentTimeZone());
+  public static String getUserTimeZoneDisplay(TimeZone tz) {
+    return DateTimeUtil.getTimeZoneDisplayTime(tz);
   }
 
   public ExprTestBase() {
@@ -164,7 +165,7 @@ public class ExprTestBase {
       assertJsonSerDer(t.getEvalTree());
     }
     for (Target t : targets) {
-      assertEvalTreeProtoSerDer(t.getEvalTree());
+      assertEvalTreeProtoSerDer(context, t.getEvalTree());
     }
     return targets;
   }
@@ -173,8 +174,19 @@ public class ExprTestBase {
     testEval(null, null, null, query, expected);
   }
 
-  public void testSimpleEval(String query, String [] expected, boolean condition) throws IOException {
-    testEval(null, null, null, null, query, expected, ',', condition);
+  public void testSimpleEval(OverridableConf context, String query, String [] expected) throws IOException {
+    testEval(context, null, null, null, query, expected);
+  }
+
+  public void testSimpleEval(String query, String [] expected, boolean successOrFail)
+      throws IOException {
+
+    testEval(null, null, null, null, query, expected, ',', successOrFail);
+  }
+
+  public void testSimpleEval(OverridableConf context, String query, String [] expected, boolean successOrFail)
+      throws IOException {
+    testEval(context, null, null, null, query, expected, ',', successOrFail);
   }
 
   public void testEval(Schema schema, String tableName, String csvTuple, String query, String [] expected)
@@ -183,10 +195,10 @@ public class ExprTestBase {
         expected, ',', true);
   }
 
-  public void testEval(OverridableConf overideConf, Schema schema, String tableName, String csvTuple, String query,
+  public void testEval(OverridableConf context, Schema schema, String tableName, String csvTuple, String query,
                        String [] expected)
       throws IOException {
-    testEval(overideConf, schema, tableName != null ? CatalogUtil.normalizeIdentifier(tableName) : null, csvTuple,
+    testEval(context, schema, tableName != null ? CatalogUtil.normalizeIdentifier(tableName) : null, csvTuple,
         query, expected, ',', true);
   }
 
@@ -196,16 +208,19 @@ public class ExprTestBase {
         query, expected, delimiter, condition);
   }
 
-  public void testEval(OverridableConf overideConf, Schema schema, String tableName, String csvTuple, String query,
+  public void testEval(OverridableConf context, Schema schema, String tableName, String csvTuple, String query,
                        String [] expected, char delimiter, boolean condition) throws IOException {
-    QueryContext context;
-    if (overideConf == null) {
-      context = LocalTajoTestingUtility.createDummyContext(conf);
+    QueryContext queryContext;
+    if (context == null) {
+      queryContext = LocalTajoTestingUtility.createDummyContext(conf);
     } else {
-      context = LocalTajoTestingUtility.createDummyContext(conf);
-      context.putAll(overideConf);
+      queryContext = LocalTajoTestingUtility.createDummyContext(conf);
+      queryContext.putAll(context);
     }
 
+    String timezoneId = queryContext.get(SessionVars.TZ);
+    TimeZone timeZone = TimeZone.getTimeZone(timezoneId);
+
     LazyTuple lazyTuple;
     VTuple vtuple  = null;
     String qualifiedTableName =
@@ -230,8 +245,8 @@ public class ExprTestBase {
         boolean nullDatum;
         Datum datum = lazyTuple.get(i);
         nullDatum = (datum instanceof TextDatum || datum instanceof CharDatum);
-        nullDatum = nullDatum && datum.asChars().equals("") ||
-            datum.asChars().equals(context.get(SessionVars.NULL_CHAR));
+        nullDatum = nullDatum &&
+            datum.asChars().equals("") || datum.asChars().equals(queryContext.get(SessionVars.NULL_CHAR));
         nullDatum |= datum.isNull();
 
         if (nullDatum) {
@@ -249,10 +264,10 @@ public class ExprTestBase {
     TajoClassLoader classLoader = new TajoClassLoader();
 
     try {
-      targets = getRawTargets(context, query, condition);
+      targets = getRawTargets(queryContext, query, condition);
 
       EvalCodeGenerator codegen = null;
-      if (context.getBool(SessionVars.CODEGEN)) {
+      if (queryContext.getBool(SessionVars.CODEGEN)) {
         codegen = new EvalCodeGenerator(classLoader);
       }
 
@@ -260,7 +275,7 @@ public class ExprTestBase {
       for (int i = 0; i < targets.length; i++) {
         EvalNode eval = targets[i].getEvalTree();
 
-        if (context.getBool(SessionVars.CODEGEN)) {
+        if (queryContext.getBool(SessionVars.CODEGEN)) {
           eval = codegen.compile(inputSchema, eval);
         }
 
@@ -277,9 +292,9 @@ public class ExprTestBase {
         Datum datum = outTuple.get(i);
         String outTupleAsChars;
         if (datum.type() == Type.TIMESTAMP) {
-          outTupleAsChars = ((TimestampDatum) datum).asChars(TajoConf.getCurrentTimeZone(), true);
+          outTupleAsChars = ((TimestampDatum) datum).asChars(timeZone, false);
         } else if (datum.type() == Type.TIME) {
-          outTupleAsChars = ((TimeDatum) datum).asChars(TajoConf.getCurrentTimeZone(), true);
+          outTupleAsChars = ((TimeDatum) datum).asChars(timeZone, false);
         } else {
           outTupleAsChars = datum.asChars();
         }
@@ -302,8 +317,8 @@ public class ExprTestBase {
     }
   }
 
-  public static void assertEvalTreeProtoSerDer(EvalNode evalNode) {
+  public static void assertEvalTreeProtoSerDer(OverridableConf context, EvalNode evalNode) {
     PlanProto.EvalTree converted = EvalTreeProtoSerializer.serialize(evalNode);
-    assertEquals(evalNode, EvalTreeProtoDeserializer.deserialize(converted));
+    assertEquals(evalNode, EvalTreeProtoDeserializer.deserialize(context, converted));
   }
 }

http://git-wip-us.apache.org/repos/asf/tajo/blob/facd1ddc/tajo-core/src/test/java/org/apache/tajo/engine/eval/TestIntervalType.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/eval/TestIntervalType.java b/tajo-core/src/test/java/org/apache/tajo/engine/eval/TestIntervalType.java
index c054fd1..cde370d 100644
--- a/tajo-core/src/test/java/org/apache/tajo/engine/eval/TestIntervalType.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/eval/TestIntervalType.java
@@ -32,28 +32,28 @@ public class TestIntervalType extends ExprTestBase {
     // http://www.postgresql.org/docs/8.2/static/functions-datetime.html
     testSimpleEval("select date '2001-09-28' + 7", new String[]{"2001-10-05"});
     testSimpleEval("select date '2001-09-28' + interval '1 hour'",
-        new String[]{"2001-09-28 01:00:00" + getUserTimeZoneDisplay()});
+        new String[]{"2001-09-28 01:00:00"});
 
     testSimpleEval("select date '2001-09-28' + time '03:00'",
-        new String[]{"2001-09-28 03:00:00" + getUserTimeZoneDisplay()});
+        new String[]{"2001-09-28 03:00:00"});
     testSimpleEval("select time '03:00' + date '2001-09-28'",
-        new String[]{"2001-09-28 03:00:00" + getUserTimeZoneDisplay()});
+        new String[]{"2001-09-28 03:00:00"});
     testSimpleEval("select interval '1 day' + interval '1 hour'", new String[]{"1 day 01:00:00"});
 
     testSimpleEval("select timestamp '2001-09-28 01:00' + interval '23 hours'",
-        new String[]{"2001-09-29 00:00:00" + getUserTimeZoneDisplay()});
+        new String[]{"2001-09-29 00:00:00"});
 
-    testSimpleEval("select time '01:00' + interval '3 hours'", new String[]{"04:00:00" + getUserTimeZoneDisplay()});
+    testSimpleEval("select time '01:00' + interval '3 hours'", new String[]{"04:00:00"});
 
     testSimpleEval("select date '2001-10-01' - date '2001-09-28'", new String[]{"3"});
     testSimpleEval("select date '2001-10-01' - 7", new String[]{"2001-09-24"});
     testSimpleEval("select date '2001-09-28' - interval '1 hour'",
-        new String[]{"2001-09-27 23:00:00" + getUserTimeZoneDisplay()});
+        new String[]{"2001-09-27 23:00:00"});
 
     testSimpleEval("select time '05:00' - time '03:00'", new String[]{"02:00:00"});
-    testSimpleEval("select time '05:00' - interval '2 hours'", new String[]{"03:00:00" + getUserTimeZoneDisplay()});
+    testSimpleEval("select time '05:00' - interval '2 hours'", new String[]{"03:00:00"});
     testSimpleEval("select timestamp '2001-09-28 23:00' - interval '23 hours'",
-        new String[]{"2001-09-28 00:00:00" + getUserTimeZoneDisplay()});
+        new String[]{"2001-09-28 00:00:00"});
 
     testSimpleEval("select interval '1 day' - interval '1 hour'", new String[]{"23:00:00"});
 
@@ -67,32 +67,32 @@ public class TestIntervalType extends ExprTestBase {
   @Test
   public void testCaseByCase() throws Exception {
     testSimpleEval("select date '2001-08-28' + interval '10 day 1 hour'",
-        new String[]{"2001-09-07 01:00:00" + getUserTimeZoneDisplay()});
+        new String[]{"2001-09-07 01:00:00"});
     testSimpleEval("select interval '10 day 01:00:00' + date '2001-08-28'",
-        new String[]{"2001-09-07 01:00:00" + getUserTimeZoneDisplay()});
+        new String[]{"2001-09-07 01:00:00"});
     testSimpleEval("select time '10:20:30' + interval '1 day 01:00:00'",
-        new String[]{"11:20:30" + getUserTimeZoneDisplay()});
+        new String[]{"11:20:30"});
     testSimpleEval("select interval '1 day 01:00:00' + time '10:20:30'",
-        new String[]{"11:20:30" + getUserTimeZoneDisplay()});
+        new String[]{"11:20:30"});
     testSimpleEval("select time '10:20:30' - interval '1 day 01:00:00'",
-        new String[]{"09:20:30" + getUserTimeZoneDisplay()});
+        new String[]{"09:20:30"});
 
     testSimpleEval("select (interval '1 month 20 day' + interval '50 day')", new String[]{"1 month 70 days"});
     testSimpleEval("select date '2013-01-01' + interval '1 month 70 day'",
-        new String[]{"2013-04-12 00:00:00" + getUserTimeZoneDisplay()});
+        new String[]{"2013-04-12 00:00:00"});
     testSimpleEval("select date '2013-01-01' + (interval '1 month 20 day' + interval '50 day')",
-        new String[]{"2013-04-12 00:00:00" + getUserTimeZoneDisplay()});
+        new String[]{"2013-04-12 00:00:00"});
     testSimpleEval("select interval '1 month 70 day' + date '2013-01-01'",
-        new String[]{"2013-04-12 00:00:00" + getUserTimeZoneDisplay()});
+        new String[]{"2013-04-12 00:00:00"});
     testSimpleEval("select date '2013-01-01' - interval '1 month 70 day'",
-        new String[]{"2012-09-22 00:00:00" + getUserTimeZoneDisplay()});
+        new String[]{"2012-09-22 00:00:00"});
 
     testSimpleEval("select timestamp '2001-09-28 23:00' - interval '1 month 2 day 10:20:30'",
-        new String[]{"2001-08-26 12:39:30" + getUserTimeZoneDisplay()});
+        new String[]{"2001-08-26 12:39:30"});
     testSimpleEval("select timestamp '2001-09-28 23:00' + interval '1 month 2 day 10:20:30'",
-        new String[]{"2001-10-31 09:20:30" + getUserTimeZoneDisplay()});
+        new String[]{"2001-10-31 09:20:30"});
     testSimpleEval("select interval '1 month 2 day 10:20:30' + timestamp '2001-09-28 23:00'",
-        new String[]{"2001-10-31 09:20:30" + getUserTimeZoneDisplay()});
+        new String[]{"2001-10-31 09:20:30"});
 
 
     testSimpleEval("select interval '5 month' / 3", new String[]{"1 month 20 days"});
@@ -104,13 +104,13 @@ public class TestIntervalType extends ExprTestBase {
     testSimpleEval("select interval '3 year 5 month 1 hour' / 1.5", new String[]{"2 years 3 months 10 days 00:40:00"});
 
     testSimpleEval("select date '2001-09-28' - time '03:00'",
-        new String[]{"2001-09-27 21:00:00" + getUserTimeZoneDisplay()});
+        new String[]{"2001-09-27 21:00:00"});
 
     testSimpleEval("select date '2014-03-20' + interval '1 day'",
-        new String[]{"2014-03-21 00:00:00" + getUserTimeZoneDisplay()});
+        new String[]{"2014-03-21 00:00:00"});
 
     testSimpleEval("select date '2014-03-20' - interval '1 day'",
-        new String[]{"2014-03-19 00:00:00" + getUserTimeZoneDisplay()});
+        new String[]{"2014-03-19 00:00:00"});
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/tajo/blob/facd1ddc/tajo-core/src/test/java/org/apache/tajo/engine/eval/TestSQLDateTimeTypes.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/eval/TestSQLDateTimeTypes.java b/tajo-core/src/test/java/org/apache/tajo/engine/eval/TestSQLDateTimeTypes.java
index 60f7dcd..fc74339 100644
--- a/tajo-core/src/test/java/org/apache/tajo/engine/eval/TestSQLDateTimeTypes.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/eval/TestSQLDateTimeTypes.java
@@ -26,14 +26,10 @@ public class TestSQLDateTimeTypes extends ExprTestBase {
 
   @Test
   public void testTimestamp() throws IOException {
-    testSimpleEval("select TIMESTAMP '1970-01-17 10:09:37';",
-        new String[]{"1970-01-17 10:09:37" + getUserTimeZoneDisplay()});
-    testSimpleEval("select TIMESTAMP '1970-01-17 10:09:37.5';",
-        new String[]{"1970-01-17 10:09:37.5" + getUserTimeZoneDisplay()});
-    testSimpleEval("select TIMESTAMP '1970-01-17 10:09:37.01';",
-        new String[]{"1970-01-17 10:09:37.01" + getUserTimeZoneDisplay()});
-    testSimpleEval("select TIMESTAMP '1970-01-17 10:09:37.003';",
-        new String[]{"1970-01-17 10:09:37.003" + getUserTimeZoneDisplay()});
+    testSimpleEval("select TIMESTAMP '1970-01-17 10:09:37';", new String[]{"1970-01-17 10:09:37"});
+    testSimpleEval("select TIMESTAMP '1970-01-17 10:09:37.5';", new String[]{"1970-01-17 10:09:37.5"});
+    testSimpleEval("select TIMESTAMP '1970-01-17 10:09:37.01';", new String[]{"1970-01-17 10:09:37.01"});
+    testSimpleEval("select TIMESTAMP '1970-01-17 10:09:37.003';",new String[]{"1970-01-17 10:09:37.003"});
   }
 
   @Test
@@ -44,14 +40,12 @@ public class TestSQLDateTimeTypes extends ExprTestBase {
 
   @Test
   public void testTimeLiteral() throws IOException {
-    testSimpleEval("select TIME '10:09:37';",
-        new String[]{"10:09:37" + getUserTimeZoneDisplay()});
+    testSimpleEval("select TIME '10:09:37';", new String[]{"10:09:37"});
   }
 
   @Test
   public void testDateLiteral() throws IOException {
-    testSimpleEval("select DATE '1970-01-17';",
-        new String[]{"1970-01-17"});
+    testSimpleEval("select DATE '1970-01-17';", new String[]{"1970-01-17"});
   }
 
 }

http://git-wip-us.apache.org/repos/asf/tajo/blob/facd1ddc/tajo-core/src/test/java/org/apache/tajo/engine/eval/TestSQLExpression.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/eval/TestSQLExpression.java b/tajo-core/src/test/java/org/apache/tajo/engine/eval/TestSQLExpression.java
index 6c9892a..4a9e42c 100644
--- a/tajo-core/src/test/java/org/apache/tajo/engine/eval/TestSQLExpression.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/eval/TestSQLExpression.java
@@ -18,6 +18,7 @@
 
 package org.apache.tajo.engine.eval;
 
+import org.apache.tajo.SessionVars;
 import org.apache.tajo.catalog.CatalogUtil;
 import org.apache.tajo.catalog.Schema;
 import org.apache.tajo.catalog.exception.NoSuchFunctionException;
@@ -25,6 +26,7 @@ import org.apache.tajo.common.TajoDataTypes;
 import org.apache.tajo.conf.TajoConf;
 import org.apache.tajo.datum.DatumFactory;
 import org.apache.tajo.datum.TimestampDatum;
+import org.apache.tajo.engine.query.QueryContext;
 import org.apache.tajo.util.datetime.DateTimeUtil;
 import org.junit.Test;
 
@@ -853,41 +855,46 @@ public class TestSQLExpression extends ExprTestBase {
 
   @Test
   public void testCastWithNestedFunction() throws IOException {
-    int unixtime = 1389071574;//(int) (System.currentTimeMillis() / 1000);
+    QueryContext context = new QueryContext(getConf());
+    context.put(SessionVars.TZ, "GMT-6");
+    TimeZone tz = TimeZone.getTimeZone("GMT-6");
+
+    int unixtime = 1389071574; // (int) (System.currentTimeMillis() / 1000);
     TimestampDatum expected = DatumFactory.createTimestmpDatumWithUnixTime(unixtime);
-    testSimpleEval(String.format("select to_timestamp(CAST(split_part('%d.999', '.', 1) as INT8));", unixtime),
-        new String[] {expected.asChars(TajoConf.getCurrentTimeZone(), true)});
+    testSimpleEval(context, String.format("select to_timestamp(CAST(split_part('%d.999', '.', 1) as INT8));", unixtime),
+        new String[] {expected.asChars(tz, false)});
   }
 
   @Test
   public void testCastFromTable() throws IOException {
-    TimeZone originTimeZone = TajoConf.setCurrentTimeZone(TimeZone.getTimeZone("GMT-6"));
-    try {
-      Schema schema = new Schema();
-      schema.addColumn("col1", TEXT);
-      schema.addColumn("col2", TEXT);
-
-      testEval(schema, "table1", "123,234", "select cast(col1 as float) as b, cast(col2 as float) as a from table1",
-          new String[]{"123.0", "234.0"});
-      testEval(schema, "table1", "123,234", "select col1::float, col2::float from table1",
-          new String[]{"123.0", "234.0"});
-
-      TimestampDatum timestamp = DatumFactory.createTimestamp("1980-04-01 01:50:01" +
-          DateTimeUtil.getTimeZoneDisplayTime(TajoConf.getCurrentTimeZone()));
-
-      testEval(schema, "table1", "1980-04-01 01:50:01,234", "select col1::timestamp as t1, col2::float from table1 " +
-              "where t1 = '1980-04-01 01:50:01'::timestamp",
-          new String[]{timestamp.asChars(TajoConf.getCurrentTimeZone(), true), "234.0"}
-      );
-
-      testSimpleEval("select '1980-04-01 01:50:01'::timestamp;",
-          new String[]{timestamp.asChars(TajoConf.getCurrentTimeZone(), true)});
-      testSimpleEval("select '1980-04-01 01:50:01'::timestamp::text", new String[]{"1980-04-01 01:50:01"});
-
-      testSimpleEval("select (cast ('99999'::int8 as text))::int4 + 1", new String[]{"100000"});
-    } finally {
-      TajoConf.setCurrentTimeZone(originTimeZone);
-    }
+    QueryContext queryContext = new QueryContext(getConf());
+    queryContext.put(SessionVars.TZ, "GMT-6");
+    TimeZone tz = TimeZone.getTimeZone("GMT-6");
+
+    Schema schema = new Schema();
+    schema.addColumn("col1", TEXT);
+    schema.addColumn("col2", TEXT);
+
+    testEval(queryContext, schema,
+        "table1",
+        "123,234",
+        "select cast(col1 as float) as b, cast(col2 as float) as a from table1",
+        new String[]{"123.0", "234.0"});
+    testEval(queryContext, schema, "table1", "123,234", "select col1::float, col2::float from table1",
+        new String[]{"123.0", "234.0"});
+
+    TimestampDatum timestamp = DatumFactory.createTimestamp("1980-04-01 01:50:01" +
+        DateTimeUtil.getTimeZoneDisplayTime(tz));
+
+    testEval(queryContext, schema, "table1", "1980-04-01 01:50:01,234",
+        "select col1::timestamp as t1, col2::float from table1 where t1 = '1980-04-01 01:50:01'::timestamp",
+        new String[]{timestamp.asChars(tz, false), "234.0"}
+    );
+
+    testSimpleEval("select '1980-04-01 01:50:01'::timestamp;", new String[]{timestamp.asChars(tz, false)});
+    testSimpleEval("select '1980-04-01 01:50:01'::timestamp::text", new String[]{"1980-04-01 01:50:01"});
+
+    testSimpleEval("select (cast ('99999'::int8 as text))::int4 + 1", new String[]{"100000"});
   }
 
   @Test