You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by jc...@apache.org on 2017/08/24 21:38:36 UTC

[4/4] hive git commit: HIVE-16614: Support "set local time zone" statement (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)

HIVE-16614: Support "set local time zone" statement (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)


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

Branch: refs/heads/master
Commit: 34eebff194e81180202d198200e84058c4910d95
Parents: 5522a2d
Author: Jesus Camacho Rodriguez <jc...@apache.org>
Authored: Wed Aug 23 12:08:04 2017 -0700
Committer: Jesus Camacho Rodriguez <jc...@apache.org>
Committed: Thu Aug 24 14:27:18 2017 -0700

----------------------------------------------------------------------
 .../hadoop/hive/common/type/TimestampTZ.java    | 122 +---
 .../hive/common/type/TimestampTZUtil.java       | 150 +++++
 .../org/apache/hadoop/hive/conf/HiveConf.java   |  16 +
 .../hive/common/type/TestTimestampTZ.java       |  63 ++-
 .../org/apache/hive/jdbc/HiveBaseResultSet.java |   6 +-
 .../java/org/apache/hive/jdbc/JdbcColumn.java   |   8 +-
 .../hadoop/hive/ql/exec/FunctionRegistry.java   |   4 +-
 .../hadoop/hive/ql/exec/GroupByOperator.java    |   2 +-
 .../hive/ql/exec/SerializationUtilities.java    |   7 +-
 .../optimizer/ConstantPropagateProcFactory.java |   6 +-
 .../hive/ql/optimizer/calcite/HiveType.java     |  48 ++
 .../calcite/translator/ExprNodeConverter.java   |  10 +
 .../calcite/translator/RexNodeConverter.java    |   3 +-
 .../translator/SqlFunctionConverter.java        |   3 +
 .../calcite/translator/TypeConverter.java       |  17 +-
 .../hive/ql/parse/DDLSemanticAnalyzer.java      |  20 +-
 .../org/apache/hadoop/hive/ql/parse/HiveLexer.g |   2 +-
 .../apache/hadoop/hive/ql/parse/HiveParser.g    |  10 +-
 .../hive/ql/parse/TypeCheckProcFactory.java     |  21 +-
 .../hadoop/hive/ql/processors/SetProcessor.java |  11 +
 .../apache/hadoop/hive/ql/stats/StatsUtils.java |  12 +-
 .../apache/hadoop/hive/ql/udf/UDFToBoolean.java |   2 +-
 .../apache/hadoop/hive/ql/udf/UDFToString.java  |   4 +-
 .../hadoop/hive/ql/udf/generic/GenericUDF.java  |   6 +-
 .../hive/ql/udf/generic/GenericUDFDate.java     |   4 +-
 .../generic/GenericUDFToTimestampLocalTZ.java   | 113 ++++
 .../ql/udf/generic/GenericUDFToTimestampTZ.java |  89 ---
 .../test/queries/clientpositive/localtimezone.q |  95 ++++
 .../test/queries/clientpositive/timestamptz.q   |  16 +-
 .../test/queries/clientpositive/timestamptz_1.q |   4 +-
 .../test/queries/clientpositive/timestamptz_2.q |   3 +-
 .../clientpositive/annotate_stats_select.q.out  |   4 +-
 .../clientpositive/constantfolding.q.out        |   4 +-
 .../results/clientpositive/localtimezone.q.out  | 558 +++++++++++++++++++
 .../results/clientpositive/timestamptz.q.out    |  48 +-
 .../results/clientpositive/timestamptz_1.q.out  |  26 +-
 .../results/clientpositive/timestamptz_2.q.out  |  34 +-
 serde/if/serde.thrift                           |   4 +-
 .../src/gen/thrift/gen-cpp/serde_constants.cpp  |   4 +-
 serde/src/gen/thrift/gen-cpp/serde_constants.h  |   2 +-
 .../hadoop/hive/serde/serdeConstants.java       |   4 +-
 .../org/apache/hadoop/hive/serde/Types.php      |   8 +-
 .../org_apache_hadoop_hive_serde/constants.py   |   4 +-
 serde/src/gen/thrift/gen-rb/serde_constants.rb  |   4 +-
 .../apache/hadoop/hive/serde2/SerDeUtils.java   |   6 +-
 .../binarysortable/BinarySortableSerDe.java     |  25 +-
 .../serde2/io/TimestampLocalTZWritable.java     | 449 +++++++++++++++
 .../hive/serde2/io/TimestampTZWritable.java     | 427 --------------
 .../hadoop/hive/serde2/lazy/LazyFactory.java    |   6 +-
 .../hive/serde2/lazy/LazySerDeParameters.java   |  19 +-
 .../hive/serde2/lazy/LazyTimestampLocalTZ.java  | 102 ++++
 .../hive/serde2/lazy/LazyTimestampTZ.java       |  91 ---
 .../hadoop/hive/serde2/lazy/LazyUtils.java      |   6 +-
 .../LazyPrimitiveObjectInspectorFactory.java    |   8 +-
 .../LazyTimestampLocalTZObjectInspector.java    |  52 ++
 .../LazyTimestampTZObjectInspector.java         |  43 --
 .../serde2/lazybinary/LazyBinaryFactory.java    |   6 +-
 .../hive/serde2/lazybinary/LazyBinarySerDe.java |   8 +-
 .../lazybinary/LazyBinaryTimestampLocalTZ.java  |  43 ++
 .../lazybinary/LazyBinaryTimestampTZ.java       |  36 --
 .../hive/serde2/lazybinary/LazyBinaryUtils.java |   6 +-
 .../ObjectInspectorConverters.java              |  10 +-
 .../objectinspector/ObjectInspectorUtils.java   |  22 +-
 .../PrimitiveObjectInspector.java               |   2 +-
 .../JavaTimestampLocalTZObjectInspector.java    | 103 ++++
 .../JavaTimestampTZObjectInspector.java         |  76 ---
 .../PrimitiveObjectInspectorConverter.java      |  23 +-
 .../PrimitiveObjectInspectorFactory.java        |  32 +-
 .../PrimitiveObjectInspectorUtils.java          | 121 ++--
 ...SettableTimestampLocalTZObjectInspector.java |  34 ++
 .../SettableTimestampTZObjectInspector.java     |  34 --
 .../TimestampLocalTZObjectInspector.java        |  29 +
 .../primitive/TimestampTZObjectInspector.java   |  29 -
 ...ConstantTimestampLocalTZObjectInspector.java |  40 ++
 ...tableConstantTimestampTZObjectInspector.java |  36 --
 ...WritableTimestampLocalTZObjectInspector.java | 107 ++++
 .../WritableTimestampTZObjectInspector.java     |  79 ---
 .../apache/hadoop/hive/serde2/thrift/Type.java  |  10 +-
 .../typeinfo/TimestampLocalTZTypeInfo.java      | 104 ++++
 .../hive/serde2/typeinfo/TypeInfoFactory.java   |  21 +-
 .../hive/serde2/typeinfo/TypeInfoUtils.java     |  24 +-
 .../hive/serde2/io/TestTimestampTZWritable.java |  20 +-
 service-rpc/if/TCLIService.thrift               |   8 +-
 .../thrift/gen-cpp/TCLIService_constants.cpp    |   2 +-
 .../gen/thrift/gen-cpp/TCLIService_types.cpp    |   4 +-
 .../src/gen/thrift/gen-cpp/TCLIService_types.h  |   2 +-
 .../rpc/thrift/TCLIServiceConstants.java        |   4 +-
 .../apache/hive/service/rpc/thrift/TTypeId.java |   4 +-
 service-rpc/src/gen/thrift/gen-php/Types.php    |   6 +-
 .../gen/thrift/gen-py/TCLIService/constants.py  |   2 +-
 .../src/gen/thrift/gen-py/TCLIService/ttypes.py |   6 +-
 .../thrift/gen-rb/t_c_l_i_service_constants.rb  |   2 +-
 .../gen/thrift/gen-rb/t_c_l_i_service_types.rb  |   6 +-
 .../apache/hive/service/cli/ColumnValue.java    |   2 +-
 .../apache/hive/service/cli/TypeDescriptor.java |   2 +-
 95 files changed, 2550 insertions(+), 1370 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/34eebff1/common/src/java/org/apache/hadoop/hive/common/type/TimestampTZ.java
----------------------------------------------------------------------
diff --git a/common/src/java/org/apache/hadoop/hive/common/type/TimestampTZ.java b/common/src/java/org/apache/hadoop/hive/common/type/TimestampTZ.java
index ed83871..c8e1a0f 100644
--- a/common/src/java/org/apache/hadoop/hive/common/type/TimestampTZ.java
+++ b/common/src/java/org/apache/hadoop/hive/common/type/TimestampTZ.java
@@ -17,66 +17,21 @@
  */
 package org.apache.hadoop.hive.common.type;
 
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.sql.Timestamp;
-import java.text.DateFormat;
-import java.text.SimpleDateFormat;
-import java.time.DateTimeException;
 import java.time.Instant;
-import java.time.LocalDate;
-import java.time.LocalTime;
 import java.time.ZoneId;
 import java.time.ZoneOffset;
 import java.time.ZonedDateTime;
-import java.time.format.DateTimeFormatter;
-import java.time.format.DateTimeFormatterBuilder;
-import java.time.format.DateTimeParseException;
-import java.time.format.TextStyle;
-import java.time.temporal.ChronoField;
-import java.time.temporal.TemporalAccessor;
-import java.util.regex.Matcher;
-import java.util.regex.Pattern;
 
 /**
  * This is the internal type for Timestamp with time zone.
- * A wrapper of ZonedDateTime which automatically convert the Zone to UTC.
  * The full qualified input format of Timestamp with time zone is
  * "yyyy-MM-dd HH:mm:ss[.SSS...] zoneid/zoneoffset", where the time and zone parts are optional.
  * If time part is absent, a default '00:00:00.0' will be used.
  * If zone part is absent, the system time zone will be used.
- * All timestamp with time zone will be converted and stored as UTC retaining the instant.
- * E.g. "2017-04-14 18:00:00 Asia/Shanghai" will be converted to
- * "2017-04-14 10:00:00.0 Z".
  */
 public class TimestampTZ implements Comparable<TimestampTZ> {
 
-  private static final DateTimeFormatter formatter;
-  private static final ZoneId UTC = ZoneOffset.UTC;
-  private static final ZonedDateTime EPOCH = ZonedDateTime.ofInstant(Instant.EPOCH, UTC);
-  private static final LocalTime DEFAULT_LOCAL_TIME = LocalTime.of(0, 0);
-  private static final Pattern SINGLE_DIGIT_PATTERN = Pattern.compile("[\\+-]\\d:\\d\\d");
-  private static final Logger LOG = LoggerFactory.getLogger(TimestampTZ.class);
-
-  private static final ThreadLocal<DateFormat> CONVERT_FORMATTER =
-      ThreadLocal.withInitial(() -> new SimpleDateFormat("yyyy-MM-dd HH:mm:ss"));
-
-  static {
-    DateTimeFormatterBuilder builder = new DateTimeFormatterBuilder();
-    // Date part
-    builder.append(DateTimeFormatter.ofPattern("yyyy-MM-dd"));
-    // Time part
-    builder.optionalStart().appendLiteral(" ").append(DateTimeFormatter.ofPattern("HH:mm:ss")).
-        optionalStart().appendFraction(ChronoField.NANO_OF_SECOND, 1, 9, true).
-        optionalEnd().optionalEnd();
-
-    // Zone part
-    builder.optionalStart().appendLiteral(" ").optionalEnd();
-    builder.optionalStart().appendZoneText(TextStyle.NARROW).optionalEnd();
-
-    formatter = builder.toFormatter();
-  }
+  private static final ZonedDateTime EPOCH = ZonedDateTime.ofInstant(Instant.EPOCH, ZoneOffset.UTC);
 
   private ZonedDateTime zonedDateTime;
 
@@ -88,13 +43,18 @@ public class TimestampTZ implements Comparable<TimestampTZ> {
     setZonedDateTime(zonedDateTime);
   }
 
-  public TimestampTZ(long seconds, int nanos) {
-    set(seconds, nanos);
+  public TimestampTZ(long seconds, int nanos, ZoneId timeZone) {
+    set(seconds, nanos, timeZone);
   }
 
-  public void set(long seconds, int nanos) {
+  /**
+   * Obtains an instance of Instant using seconds from the epoch of 1970-01-01T00:00:00Z and
+   * nanosecond fraction of second. Then, it creates a zoned date-time with the same instant
+   * as that specified but in the given time-zone.
+   */
+  public void set(long seconds, int nanos, ZoneId timeZone) {
     Instant instant = Instant.ofEpochSecond(seconds, nanos);
-    setZonedDateTime(ZonedDateTime.ofInstant(instant, UTC));
+    setZonedDateTime(ZonedDateTime.ofInstant(instant, timeZone));
   }
 
   public ZonedDateTime getZonedDateTime() {
@@ -102,12 +62,12 @@ public class TimestampTZ implements Comparable<TimestampTZ> {
   }
 
   public void setZonedDateTime(ZonedDateTime zonedDateTime) {
-    this.zonedDateTime = zonedDateTime != null ? zonedDateTime.withZoneSameInstant(UTC) : EPOCH;
+    this.zonedDateTime = zonedDateTime != null ? zonedDateTime : EPOCH;
   }
 
   @Override
   public String toString() {
-    return zonedDateTime.format(formatter);
+    return zonedDateTime.format(TimestampTZUtil.FORMATTER);
   }
 
   @Override
@@ -136,62 +96,4 @@ public class TimestampTZ implements Comparable<TimestampTZ> {
     return zonedDateTime.toInstant().getNano();
   }
 
-  public static TimestampTZ parse(String s) {
-    // need to handle offset with single digital hour, see JDK-8066806
-    s = handleSingleDigitHourOffset(s);
-    ZonedDateTime zonedDateTime;
-    try {
-      zonedDateTime = ZonedDateTime.parse(s, formatter);
-    } catch (DateTimeParseException e) {
-      // try to be more tolerant
-      // if the input is invalid instead of incomplete, we'll hit exception here again
-      TemporalAccessor accessor = formatter.parse(s);
-      // LocalDate must be present
-      LocalDate localDate = LocalDate.from(accessor);
-      LocalTime localTime;
-      ZoneId zoneId;
-      try {
-        localTime = LocalTime.from(accessor);
-      } catch (DateTimeException e1) {
-        localTime = DEFAULT_LOCAL_TIME;
-      }
-      try {
-        zoneId = ZoneId.from(accessor);
-      } catch (DateTimeException e2) {
-        // TODO: in future this may come from user specified zone (via set time zone command)
-        zoneId = ZoneId.systemDefault();
-      }
-      zonedDateTime = ZonedDateTime.of(localDate, localTime, zoneId);
-    }
-
-    return new TimestampTZ(zonedDateTime);
-  }
-
-  private static String handleSingleDigitHourOffset(String s) {
-    Matcher matcher = SINGLE_DIGIT_PATTERN.matcher(s);
-    if (matcher.find()) {
-      int index = matcher.start() + 1;
-      s = s.substring(0, index) + "0" + s.substring(index, s.length());
-    }
-    return s;
-  }
-
-  public static TimestampTZ parseOrNull(String s) {
-    try {
-      return parse(s);
-    } catch (DateTimeParseException e) {
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Invalid string " + s + " for TIMESTAMP WITH TIME ZONE", e);
-      }
-      return null;
-    }
-  }
-
-  // Converts Date to TimestampTZ. The conversion is done text-wise since
-  // Date/Timestamp should be treated as description of date/time.
-  public static TimestampTZ convert(java.util.Date date) {
-    String s = date instanceof Timestamp ? date.toString() : CONVERT_FORMATTER.get().format(date);
-    // TODO: in future this may come from user specified zone (via set time zone command)
-    return parse(s + " " + ZoneId.systemDefault().getId());
-  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/34eebff1/common/src/java/org/apache/hadoop/hive/common/type/TimestampTZUtil.java
----------------------------------------------------------------------
diff --git a/common/src/java/org/apache/hadoop/hive/common/type/TimestampTZUtil.java b/common/src/java/org/apache/hadoop/hive/common/type/TimestampTZUtil.java
new file mode 100644
index 0000000..c49aefd
--- /dev/null
+++ b/common/src/java/org/apache/hadoop/hive/common/type/TimestampTZUtil.java
@@ -0,0 +1,150 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.hadoop.hive.common.type;
+
+import java.sql.Timestamp;
+import java.text.DateFormat;
+import java.text.SimpleDateFormat;
+import java.time.DateTimeException;
+import java.time.LocalDate;
+import java.time.LocalTime;
+import java.time.ZoneId;
+import java.time.ZonedDateTime;
+import java.time.format.DateTimeFormatter;
+import java.time.format.DateTimeFormatterBuilder;
+import java.time.format.DateTimeParseException;
+import java.time.format.TextStyle;
+import java.time.temporal.ChronoField;
+import java.time.temporal.TemporalAccessor;
+import java.util.Date;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class TimestampTZUtil {
+
+  private static final Logger LOG = LoggerFactory.getLogger(TimestampTZ.class);
+
+  private static final LocalTime DEFAULT_LOCAL_TIME = LocalTime.of(0, 0);
+  private static final Pattern SINGLE_DIGIT_PATTERN = Pattern.compile("[\\+-]\\d:\\d\\d");
+
+  private static final ThreadLocal<DateFormat> CONVERT_FORMATTER =
+      ThreadLocal.withInitial(() -> new SimpleDateFormat("yyyy-MM-dd HH:mm:ss"));
+
+  static final DateTimeFormatter FORMATTER;
+  static {
+    DateTimeFormatterBuilder builder = new DateTimeFormatterBuilder();
+    // Date part
+    builder.append(DateTimeFormatter.ofPattern("yyyy-MM-dd"));
+    // Time part
+    builder.optionalStart().appendLiteral(" ").append(DateTimeFormatter.ofPattern("HH:mm:ss")).
+        optionalStart().appendFraction(ChronoField.NANO_OF_SECOND, 1, 9, true).
+        optionalEnd().optionalEnd();
+
+    // Zone part
+    builder.optionalStart().appendLiteral(" ").optionalEnd();
+    builder.optionalStart().appendZoneText(TextStyle.NARROW).optionalEnd();
+
+    FORMATTER = builder.toFormatter();
+  }
+
+  public static TimestampTZ parse(String s) {
+    return parse(s, null);
+  }
+
+  public static TimestampTZ parse(String s, ZoneId defaultTimeZone) {
+    // need to handle offset with single digital hour, see JDK-8066806
+    s = handleSingleDigitHourOffset(s);
+    ZonedDateTime zonedDateTime;
+    try {
+      zonedDateTime = ZonedDateTime.parse(s, FORMATTER);
+    } catch (DateTimeParseException e) {
+      // try to be more tolerant
+      // if the input is invalid instead of incomplete, we'll hit exception here again
+      TemporalAccessor accessor = FORMATTER.parse(s);
+      // LocalDate must be present
+      LocalDate localDate = LocalDate.from(accessor);
+      LocalTime localTime;
+      ZoneId zoneId;
+      try {
+        localTime = LocalTime.from(accessor);
+      } catch (DateTimeException e1) {
+        localTime = DEFAULT_LOCAL_TIME;
+      }
+      try {
+        zoneId = ZoneId.from(accessor);
+      } catch (DateTimeException e2) {
+        if (defaultTimeZone == null) {
+          throw new DateTimeException("Time Zone not available");
+        }
+        zoneId = defaultTimeZone;
+      }
+      zonedDateTime = ZonedDateTime.of(localDate, localTime, zoneId);
+    }
+
+    if (defaultTimeZone == null) {
+      return new TimestampTZ(zonedDateTime);
+    }
+    return new TimestampTZ(zonedDateTime.withZoneSameInstant(defaultTimeZone));
+  }
+
+  private static String handleSingleDigitHourOffset(String s) {
+    Matcher matcher = SINGLE_DIGIT_PATTERN.matcher(s);
+    if (matcher.find()) {
+      int index = matcher.start() + 1;
+      s = s.substring(0, index) + "0" + s.substring(index, s.length());
+    }
+    return s;
+  }
+
+
+  public static TimestampTZ parseOrNull(String s, ZoneId defaultTimeZone) {
+    try {
+      return parse(s, defaultTimeZone);
+    } catch (DateTimeParseException e) {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Invalid string " + s + " for TIMESTAMP WITH TIME ZONE", e);
+      }
+      return null;
+    }
+  }
+
+  // Converts Date to TimestampTZ. The conversion is done text-wise since
+  // Date/Timestamp should be treated as description of date/time.
+  public static TimestampTZ convert(Date date, ZoneId defaultTimeZone) {
+    String s = date instanceof Timestamp ? date.toString() : CONVERT_FORMATTER.get().format(date);
+    return parse(s, defaultTimeZone);
+  }
+
+  public static ZoneId parseTimeZone(String timeZoneStr) {
+    if (timeZoneStr == null || timeZoneStr.trim().isEmpty() ||
+        timeZoneStr.trim().toLowerCase().equals("local")) {
+      // default
+      return ZoneId.systemDefault();
+    }
+    try {
+      return ZoneId.of(timeZoneStr);
+    } catch (DateTimeException e1) {
+      // default
+      throw new RuntimeException("Invalid time zone displacement value");
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/34eebff1/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
----------------------------------------------------------------------
diff --git a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
index 2fb004c..5a2a436 100644
--- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
+++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
@@ -25,6 +25,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.common.FileUtils;
 import org.apache.hadoop.hive.common.classification.InterfaceAudience;
 import org.apache.hadoop.hive.common.classification.InterfaceAudience.LimitedPrivate;
+import org.apache.hadoop.hive.common.type.TimestampTZUtil;
 import org.apache.hadoop.hive.conf.Validator.PatternSet;
 import org.apache.hadoop.hive.conf.Validator.RangeValidator;
 import org.apache.hadoop.hive.conf.Validator.RatioValidator;
@@ -52,6 +53,7 @@ import java.net.URI;
 import java.net.URL;
 import java.net.URLDecoder;
 import java.net.URLEncoder;
+import java.time.ZoneId;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.HashMap;
@@ -976,6 +978,12 @@ public class HiveConf extends Configuration {
     // whether session is running in silent mode or not
     HIVESESSIONSILENT("hive.session.silent", false, ""),
 
+    HIVE_LOCAL_TIME_ZONE("hive.local.time.zone", "LOCAL",
+        "Sets the time-zone for displaying and interpreting time stamps. If this property value is set to\n" +
+        "LOCAL, it is not specified, or it is not a correct time-zone, the system default time-zone will be\n " +
+        "used instead. Time-zone IDs can be specified as region-based zone IDs (based on IANA time-zone data),\n" +
+        "abbreviated zone IDs, or offset IDs."),
+
     HIVE_SESSION_HISTORY_ENABLED("hive.session.history.enabled", false,
         "Whether to log Hive query, query plan, runtime statistics etc."),
 
@@ -4289,6 +4297,14 @@ public class HiveConf extends Configuration {
   }
 
   /**
+   * Obtains the local time-zone ID.
+   */
+  public ZoneId getLocalTimeZone() {
+    String timeZoneStr = getVar(ConfVars.HIVE_LOCAL_TIME_ZONE);
+    return TimestampTZUtil.parseTimeZone(timeZoneStr);
+  }
+
+  /**
    * @param paramList  list of parameter strings
    * @return list of parameter strings with "." replaced by "\."
    */

http://git-wip-us.apache.org/repos/asf/hive/blob/34eebff1/common/src/test/org/apache/hadoop/hive/common/type/TestTimestampTZ.java
----------------------------------------------------------------------
diff --git a/common/src/test/org/apache/hadoop/hive/common/type/TestTimestampTZ.java b/common/src/test/org/apache/hadoop/hive/common/type/TestTimestampTZ.java
index 0cef77a..cb4b570 100644
--- a/common/src/test/org/apache/hadoop/hive/common/type/TestTimestampTZ.java
+++ b/common/src/test/org/apache/hadoop/hive/common/type/TestTimestampTZ.java
@@ -22,6 +22,7 @@ import org.junit.Assert;
 import org.junit.Test;
 
 import java.sql.Timestamp;
+import java.time.ZoneId;
 import java.time.format.DateTimeParseException;
 import java.util.TimeZone;
 
@@ -29,8 +30,8 @@ public class TestTimestampTZ {
   @Test
   public void testConvertToUTC() {
     String s = "2017-04-14 18:00:00 Asia/Shanghai";
-    TimestampTZ timestampTZ = TimestampTZ.parse(s);
-    Assert.assertEquals("2017-04-14 10:00:00.0 Z", timestampTZ.toString());
+    TimestampTZ timestampTZ = TimestampTZUtil.parse(s, ZoneId.of("UTC"));
+    Assert.assertEquals("2017-04-14 10:00:00.0 UTC", timestampTZ.toString());
   }
 
   @Test
@@ -39,10 +40,10 @@ public class TestTimestampTZ {
     String s2 = "2017-04-14 10:00:00.00 GMT";
     String s3 = "2017-04-14 18:00:00 UTC+08:00";
     String s4 = "2017-04-14 18:00:00 Europe/London";
-    TimestampTZ tstz1 = TimestampTZ.parse(s1);
-    TimestampTZ tstz2 = TimestampTZ.parse(s2);
-    TimestampTZ tstz3 = TimestampTZ.parse(s3);
-    TimestampTZ tstz4 = TimestampTZ.parse(s4);
+    TimestampTZ tstz1 = TimestampTZUtil.parse(s1);
+    TimestampTZ tstz2 = TimestampTZUtil.parse(s2);
+    TimestampTZ tstz3 = TimestampTZUtil.parse(s3);
+    TimestampTZ tstz4 = TimestampTZUtil.parse(s4);
 
     Assert.assertEquals(tstz1, tstz2);
     Assert.assertEquals(tstz1, tstz3);
@@ -55,47 +56,47 @@ public class TestTimestampTZ {
   public void testDST() {
     String s1 = "2005-04-03 02:01:00 America/Los_Angeles";
     String s2 = "2005-04-03 03:01:00 America/Los_Angeles";
-    Assert.assertEquals(TimestampTZ.parse(s1), TimestampTZ.parse(s2));
+    Assert.assertEquals(TimestampTZUtil.parse(s1), TimestampTZUtil.parse(s2));
   }
 
   @Test
   public void testFromToInstant() {
     String s1 = "2017-04-14 18:00:00 UTC";
-    TimestampTZ tstz = TimestampTZ.parse(s1);
+    TimestampTZ tstz = TimestampTZUtil.parse(s1);
     long seconds = tstz.getEpochSecond();
     int nanos = tstz.getNanos();
-    Assert.assertEquals(tstz, new TimestampTZ(seconds, nanos));
+    Assert.assertEquals(tstz, new TimestampTZ(seconds, nanos, ZoneId.of("UTC")));
 
     nanos += 123000000;
-    Assert.assertEquals("2017-04-14 18:00:00.123 Z", new TimestampTZ(seconds, nanos).toString());
+    Assert.assertEquals("2017-04-14 18:00:00.123 UTC", new TimestampTZ(seconds, nanos, ZoneId.of("UTC")).toString());
 
     seconds -= 3;
-    Assert.assertEquals("2017-04-14 17:59:57.123 Z", new TimestampTZ(seconds, nanos).toString());
+    Assert.assertEquals("2017-04-14 17:59:57.123 UTC", new TimestampTZ(seconds, nanos, ZoneId.of("UTC")).toString());
   }
 
   @Test
   public void testVariations() {
     // Omitting zone or time part is allowed
-    TimestampTZ.parse("2017-01-01 13:33:00");
-    TimestampTZ.parse("2017-11-08 Europe/London");
-    TimestampTZ.parse("2017-05-20");
-    TimestampTZ.parse("2017-11-08GMT");
-    TimestampTZ.parse("2017-10-11 GMT+8:00");
-    TimestampTZ.parse("2017-05-08 07:45:00-3:00");
+    TimestampTZUtil.parse("2017-01-01 13:33:00", ZoneId.of("UTC"));
+    TimestampTZUtil.parse("2017-11-08 Europe/London");
+    TimestampTZUtil.parse("2017-05-20", ZoneId.of("UTC"));
+    TimestampTZUtil.parse("2017-11-08GMT");
+    TimestampTZUtil.parse("2017-10-11 GMT+8:00");
+    TimestampTZUtil.parse("2017-05-08 07:45:00-3:00");
   }
 
   @Test
   public void testInvalidStrings() {
     // invalid zone
     try {
-      TimestampTZ.parse("2017-01-01 13:33:00 foo");
+      TimestampTZUtil.parse("2017-01-01 13:33:00 foo");
       Assert.fail("Invalid timezone ID should cause exception");
     } catch (DateTimeParseException e) {
       // expected
     }
     // invalid time part
     try {
-      TimestampTZ.parse("2017-01-01 13:33:61");
+      TimestampTZUtil.parse("2017-01-01 13:33:61");
       Assert.fail("Invalid time should cause exception");
     } catch (DateTimeParseException e) {
       // expected
@@ -108,10 +109,26 @@ public class TestTimestampTZ {
     try {
       // Use system zone when converting from timestamp to timestamptz
       String s = "2017-06-12 23:12:56.34";
-      TimeZone.setDefault(TimeZone.getTimeZone("Europe/London"));
-      TimestampTZ tstz1 = TimestampTZ.convert(Timestamp.valueOf(s));
-      TimeZone.setDefault(TimeZone.getTimeZone("America/Los_Angeles"));
-      TimestampTZ tstz2 = TimestampTZ.convert(Timestamp.valueOf(s));
+      TimestampTZ tstz1 = TimestampTZUtil.convert(
+          Timestamp.valueOf(s),
+          TimeZone.getTimeZone("Europe/London").toZoneId());
+      TimestampTZ tstz2 = TimestampTZUtil.convert(
+          Timestamp.valueOf(s),
+          TimeZone.getTimeZone("America/Los_Angeles").toZoneId());
+      Assert.assertTrue(tstz1.compareTo(tstz2) < 0);
+    } finally {
+      TimeZone.setDefault(defaultZone);
+    }
+  }
+
+  @Test
+  public void testConvertFromTimestamp2() {
+    TimeZone defaultZone = TimeZone.getDefault();
+    try {
+      // Use system zone when converting from timestamp to timestamptz
+      String s = "2017-06-12 23:12:56.34";
+      TimestampTZ tstz1 = TimestampTZUtil.parse(s + " " + TimeZone.getTimeZone("Europe/London").getID());
+      TimestampTZ tstz2 = TimestampTZUtil.parse(s + " " + TimeZone.getTimeZone("America/Los_Angeles").getID());
       Assert.assertTrue(tstz1.compareTo(tstz2) < 0);
     } finally {
       TimeZone.setDefault(defaultZone);

http://git-wip-us.apache.org/repos/asf/hive/blob/34eebff1/jdbc/src/java/org/apache/hive/jdbc/HiveBaseResultSet.java
----------------------------------------------------------------------
diff --git a/jdbc/src/java/org/apache/hive/jdbc/HiveBaseResultSet.java b/jdbc/src/java/org/apache/hive/jdbc/HiveBaseResultSet.java
index 6742423..e833862 100644
--- a/jdbc/src/java/org/apache/hive/jdbc/HiveBaseResultSet.java
+++ b/jdbc/src/java/org/apache/hive/jdbc/HiveBaseResultSet.java
@@ -47,7 +47,7 @@ import java.util.Map;
 
 import org.apache.hadoop.hive.common.type.HiveIntervalDayTime;
 import org.apache.hadoop.hive.common.type.HiveIntervalYearMonth;
-import org.apache.hadoop.hive.common.type.TimestampTZ;
+import org.apache.hadoop.hive.common.type.TimestampTZUtil;
 import org.apache.hadoop.hive.serde2.thrift.Type;
 import org.apache.hive.service.cli.TableSchema;
 
@@ -443,8 +443,8 @@ public abstract class HiveBaseResultSet implements ResultSet {
         return value;
       case TIMESTAMP_TYPE:
         return Timestamp.valueOf((String) value);
-      case TIMESTAMPTZ_TYPE:
-        return TimestampTZ.parse((String) value);
+      case TIMESTAMPLOCALTZ_TYPE:
+        return TimestampTZUtil.parse((String) value);
       case DECIMAL_TYPE:
         return new BigDecimal((String)value);
       case DATE_TYPE:

http://git-wip-us.apache.org/repos/asf/hive/blob/34eebff1/jdbc/src/java/org/apache/hive/jdbc/JdbcColumn.java
----------------------------------------------------------------------
diff --git a/jdbc/src/java/org/apache/hive/jdbc/JdbcColumn.java b/jdbc/src/java/org/apache/hive/jdbc/JdbcColumn.java
index bf42f0d..8d9604e 100644
--- a/jdbc/src/java/org/apache/hive/jdbc/JdbcColumn.java
+++ b/jdbc/src/java/org/apache/hive/jdbc/JdbcColumn.java
@@ -145,8 +145,8 @@ public class JdbcColumn {
       return Type.DATE_TYPE;
     } else if ("timestamp".equalsIgnoreCase(type)) {
       return Type.TIMESTAMP_TYPE;
-    } else if (serdeConstants.TIMESTAMPTZ_TYPE_NAME.equalsIgnoreCase(type)) {
-      return Type.TIMESTAMPTZ_TYPE;
+    } else if (serdeConstants.TIMESTAMPLOCALTZ_TYPE_NAME.equalsIgnoreCase(type)) {
+      return Type.TIMESTAMPLOCALTZ_TYPE;
     } else if ("interval_year_month".equalsIgnoreCase(type)) {
       return Type.INTERVAL_YEAR_MONTH_TYPE;
     } else if ("interval_day_time".equalsIgnoreCase(type)) {
@@ -200,8 +200,8 @@ public class JdbcColumn {
       return serdeConstants.BIGINT_TYPE_NAME;
     } else if ("timestamp".equalsIgnoreCase(type)) {
       return serdeConstants.TIMESTAMP_TYPE_NAME;
-    } else if (serdeConstants.TIMESTAMPTZ_TYPE_NAME.equalsIgnoreCase(type)) {
-      return serdeConstants.TIMESTAMPTZ_TYPE_NAME;
+    } else if (serdeConstants.TIMESTAMPLOCALTZ_TYPE_NAME.equalsIgnoreCase(type)) {
+      return serdeConstants.TIMESTAMPLOCALTZ_TYPE_NAME;
     } else if ("date".equalsIgnoreCase(type)) {
       return serdeConstants.DATE_TYPE_NAME;
     } else if ("interval_year_month".equalsIgnoreCase(type)) {

http://git-wip-us.apache.org/repos/asf/hive/blob/34eebff1/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java
index 9795f3e..1d1d2a3 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java
@@ -402,7 +402,7 @@ public final class FunctionRegistry {
 
     system.registerGenericUDF(serdeConstants.DATE_TYPE_NAME, GenericUDFToDate.class);
     system.registerGenericUDF(serdeConstants.TIMESTAMP_TYPE_NAME, GenericUDFTimestamp.class);
-    system.registerGenericUDF(serdeConstants.TIMESTAMPTZ_TYPE_NAME, GenericUDFToTimestampTZ.class);
+    system.registerGenericUDF(serdeConstants.TIMESTAMPLOCALTZ_TYPE_NAME, GenericUDFToTimestampLocalTZ.class);
     system.registerGenericUDF(serdeConstants.INTERVAL_YEAR_MONTH_TYPE_NAME, GenericUDFToIntervalYearMonth.class);
     system.registerGenericUDF(serdeConstants.INTERVAL_DAY_TIME_TYPE_NAME, GenericUDFToIntervalDayTime.class);
     system.registerGenericUDF(serdeConstants.BINARY_TYPE_NAME, GenericUDFToBinary.class);
@@ -1538,7 +1538,7 @@ public final class FunctionRegistry {
         udfClass == GenericUDFToVarchar.class || udfClass == GenericUDFToChar.class ||
         udfClass == GenericUDFTimestamp.class || udfClass == GenericUDFToBinary.class ||
         udfClass == GenericUDFToDate.class || udfClass == GenericUDFToDecimal.class ||
-        udfClass == GenericUDFToTimestampTZ.class;
+        udfClass == GenericUDFToTimestampLocalTZ.class;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hive/blob/34eebff1/ql/src/java/org/apache/hadoop/hive/ql/exec/GroupByOperator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/GroupByOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/GroupByOperator.java
index 9c5e7e2..d3dfd21 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/GroupByOperator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/GroupByOperator.java
@@ -472,7 +472,7 @@ public class GroupByOperator extends Operator<GroupByDesc> {
       keyPositionsSize.add(new Integer(pos));
       return javaObjectOverHead;
     case TIMESTAMP:
-    case TIMESTAMPTZ:
+    case TIMESTAMPLOCALTZ:
       return javaObjectOverHead + javaSizePrimitiveType;
     default:
       return javaSizeUnknownType;

http://git-wip-us.apache.org/repos/asf/hive/blob/34eebff1/ql/src/java/org/apache/hadoop/hive/ql/exec/SerializationUtilities.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/SerializationUtilities.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/SerializationUtilities.java
index 8902f6c..8d523ca 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/SerializationUtilities.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/SerializationUtilities.java
@@ -29,6 +29,7 @@ import java.lang.reflect.Array;
 import java.lang.reflect.Field;
 import java.net.URI;
 import java.sql.Timestamp;
+import java.time.ZoneId;
 import java.util.Arrays;
 import java.util.HashMap;
 import java.util.List;
@@ -46,6 +47,8 @@ import org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat;
 import org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat;
 import org.apache.hadoop.hive.ql.io.RCFileInputFormat;
 import org.apache.hadoop.hive.ql.log.PerfLogger;
+import org.apache.hadoop.hive.ql.metadata.Hive;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.plan.AbstractOperatorDesc;
 import org.apache.hadoop.hive.ql.plan.BaseWork;
 import org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc;
@@ -316,13 +319,15 @@ public class SerializationUtilities {
     public void write(Kryo kryo, Output output, TimestampTZ object) {
       output.writeLong(object.getEpochSecond());
       output.writeInt(object.getNanos());
+      output.writeString(object.getZonedDateTime().getZone().getId());
     }
 
     @Override
     public TimestampTZ read(Kryo kryo, Input input, Class<TimestampTZ> type) {
       long seconds = input.readLong();
       int nanos = input.readInt();
-      return new TimestampTZ(seconds, nanos);
+      String zoneId = input.readString();
+      return new TimestampTZ(seconds, nanos, ZoneId.of(zoneId));
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/34eebff1/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConstantPropagateProcFactory.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConstantPropagateProcFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConstantPropagateProcFactory.java
index 517ce31..b0a2da8 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConstantPropagateProcFactory.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConstantPropagateProcFactory.java
@@ -961,9 +961,9 @@ public final class ConstantPropagateProcFactory {
         TypeInfo typeInfo = poi.getTypeInfo();
         o = poi.getPrimitiveJavaObject(o);
         if (typeInfo.getTypeName().contains(serdeConstants.DECIMAL_TYPE_NAME)
-            || typeInfo.getTypeName()
-                .contains(serdeConstants.VARCHAR_TYPE_NAME)
-            || typeInfo.getTypeName().contains(serdeConstants.CHAR_TYPE_NAME)) {
+            || typeInfo.getTypeName().contains(serdeConstants.VARCHAR_TYPE_NAME)
+            || typeInfo.getTypeName().contains(serdeConstants.CHAR_TYPE_NAME)
+            || typeInfo.getTypeName().contains(serdeConstants.TIMESTAMPLOCALTZ_TYPE_NAME)) {
           return new ExprNodeConstantDesc(typeInfo, o);
         }
       } else if (udf instanceof GenericUDFStruct

http://git-wip-us.apache.org/repos/asf/hive/blob/34eebff1/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveType.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveType.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveType.java
new file mode 100644
index 0000000..31a628b
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveType.java
@@ -0,0 +1,48 @@
+/**
+ * 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.hadoop.hive.ql.optimizer.calcite;
+
+import org.apache.calcite.sql.type.AbstractSqlType;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * Hive-specific type.
+ *
+ * TODO: Created to represent timestamp with time-zone type.
+ * It can be removed once the type exists in Calcite.
+ */
+public class HiveType extends AbstractSqlType {
+  private final Class clazz;
+
+  public HiveType(Class clazz) {
+    super(SqlTypeName.NULL, true, null);
+    this.clazz = clazz;
+    computeDigest();
+  }
+
+  protected void generateTypeString(StringBuilder sb, boolean withDetail) {
+    sb.append("HiveType(");
+    sb.append(clazz);
+    sb.append(")");
+  }
+
+  public Class getTypeClass() {
+    return clazz;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/34eebff1/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/ExprNodeConverter.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/ExprNodeConverter.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/ExprNodeConverter.java
index f974cc9..3dcceab 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/ExprNodeConverter.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/ExprNodeConverter.java
@@ -48,8 +48,10 @@ import org.apache.calcite.util.TimestampString;
 import org.apache.hadoop.hive.common.type.HiveDecimal;
 import org.apache.hadoop.hive.common.type.HiveIntervalDayTime;
 import org.apache.hadoop.hive.common.type.HiveIntervalYearMonth;
+import org.apache.hadoop.hive.common.type.TimestampTZ;
 import org.apache.hadoop.hive.ql.exec.UDFArgumentException;
 import org.apache.hadoop.hive.ql.optimizer.ConstantPropagateProcFactory;
+import org.apache.hadoop.hive.ql.optimizer.calcite.HiveType;
 import org.apache.hadoop.hive.ql.optimizer.calcite.translator.ASTConverter.RexVisitor;
 import org.apache.hadoop.hive.ql.optimizer.calcite.translator.ASTConverter.Schema;
 import org.apache.hadoop.hive.ql.parse.ASTNode;
@@ -265,8 +267,12 @@ public class ExprNodeConverter extends RexVisitorImpl<ExprNodeDesc> {
       case INTERVAL_MINUTE_SECOND:
       case INTERVAL_SECOND:
         return new ExprNodeConstantDesc(TypeInfoFactory.intervalDayTimeTypeInfo, null);
+      case NULL:
       case OTHER:
       default:
+        if (lType instanceof HiveType && ((HiveType) lType).getTypeClass() == TimestampTZ.class) {
+          return new ExprNodeConstantDesc(TypeInfoFactory.timestampLocalTZTypeInfo, null);
+        }
         return new ExprNodeConstantDesc(TypeInfoFactory.voidTypeInfo, null);
       }
     } else {
@@ -334,8 +340,12 @@ public class ExprNodeConverter extends RexVisitorImpl<ExprNodeDesc> {
         return new ExprNodeConstantDesc(TypeInfoFactory.intervalDayTimeTypeInfo,
                 new HiveIntervalDayTime(secsBd));
       }
+      case NULL:
       case OTHER:
       default:
+        if (lType instanceof HiveType && ((HiveType) lType).getTypeClass() == TimestampTZ.class) {
+          return new ExprNodeConstantDesc(TypeInfoFactory.timestampLocalTZTypeInfo, literal.getValue3());
+        }
         return new ExprNodeConstantDesc(TypeInfoFactory.voidTypeInfo, literal.getValue3());
       }
     }

http://git-wip-us.apache.org/repos/asf/hive/blob/34eebff1/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/RexNodeConverter.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/RexNodeConverter.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/RexNodeConverter.java
index 7665f56..abbffb0 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/RexNodeConverter.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/RexNodeConverter.java
@@ -87,6 +87,7 @@ import org.apache.hadoop.hive.ql.udf.generic.GenericUDFToBinary;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDFToChar;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDFToDate;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDFToDecimal;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDFToTimestampLocalTZ;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDFToUnixTimeStamp;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDFToVarchar;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDFUnixTimeStamp;
@@ -382,7 +383,7 @@ public class RexNodeConverter {
           || (udf instanceof GenericUDFToDecimal) || (udf instanceof GenericUDFToDate)
           // Calcite can not specify the scale for timestamp. As a result, all
           // the millisecond part will be lost
-          || (udf instanceof GenericUDFTimestamp)
+          || (udf instanceof GenericUDFTimestamp) || (udf instanceof GenericUDFToTimestampLocalTZ)
           || (udf instanceof GenericUDFToBinary) || castExprUsingUDFBridge(udf)) {
         castExpr = cluster.getRexBuilder().makeAbstractCast(
             TypeConverter.convert(func.getTypeInfo(), cluster.getTypeFactory()),

http://git-wip-us.apache.org/repos/asf/hive/blob/34eebff1/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/SqlFunctionConverter.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/SqlFunctionConverter.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/SqlFunctionConverter.java
index a9198a1..31a088b 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/SqlFunctionConverter.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/SqlFunctionConverter.java
@@ -181,6 +181,9 @@ public class SqlFunctionConverter {
         castUDF = FunctionRegistry.getFunctionInfo("double");
       } else if (castType.equals(TypeInfoFactory.timestampTypeInfo)) {
         castUDF = FunctionRegistry.getFunctionInfo("timestamp");
+      } else if (castType.equals(TypeInfoFactory.timestampLocalTZTypeInfo)) {
+        castUDF = handleCastForParameterizedType(castType,
+            FunctionRegistry.getFunctionInfo(serdeConstants.TIMESTAMPLOCALTZ_TYPE_NAME));
       } else if (castType.equals(TypeInfoFactory.dateTypeInfo)) {
         castUDF = FunctionRegistry.getFunctionInfo("date");
       } else if (castType instanceof DecimalTypeInfo) {

http://git-wip-us.apache.org/repos/asf/hive/blob/34eebff1/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/TypeConverter.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/TypeConverter.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/TypeConverter.java
index 2df7588..34886f3 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/TypeConverter.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/TypeConverter.java
@@ -36,10 +36,12 @@ import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.calcite.util.ConversionUtil;
 import org.apache.hadoop.hive.common.type.HiveChar;
 import org.apache.hadoop.hive.common.type.HiveVarchar;
+import org.apache.hadoop.hive.common.type.TimestampTZ;
 import org.apache.hadoop.hive.ql.exec.ColumnInfo;
 import org.apache.hadoop.hive.ql.exec.RowSchema;
 import org.apache.hadoop.hive.ql.optimizer.calcite.CalciteSemanticException;
 import org.apache.hadoop.hive.ql.optimizer.calcite.CalciteSemanticException.UnsupportedFeature;
+import org.apache.hadoop.hive.ql.optimizer.calcite.HiveType;
 import org.apache.hadoop.hive.ql.optimizer.calcite.translator.SqlFunctionConverter.HiveToken;
 import org.apache.hadoop.hive.ql.parse.HiveParser;
 import org.apache.hadoop.hive.ql.parse.RowResolver;
@@ -200,8 +202,8 @@ public class TypeConverter {
     case TIMESTAMP:
       convertedType = dtFactory.createSqlType(SqlTypeName.TIMESTAMP);
       break;
-    case TIMESTAMPTZ:
-      convertedType = dtFactory.createSqlType(SqlTypeName.OTHER);
+    case TIMESTAMPLOCALTZ:
+      convertedType = new HiveType(TimestampTZ.class);
       break;
     case INTERVAL_YEAR_MONTH:
       convertedType = dtFactory.createSqlIntervalType(
@@ -359,8 +361,13 @@ public class TypeConverter {
         return TypeInfoFactory.getPrimitiveTypeInfo(serdeConstants.STRING_TYPE_NAME);
       else
         return TypeInfoFactory.getCharTypeInfo(charLength);
+    case NULL:
     case OTHER:
     default:
+      if (rType instanceof HiveType && ((HiveType) rType).getTypeClass() == TimestampTZ.class) {
+        // TODO: This block should be removed when we upgrade Calcite to use local time-zone
+        return TypeInfoFactory.timestampLocalTZTypeInfo;
+      }
       return TypeInfoFactory.voidTypeInfo;
     }
 
@@ -389,6 +396,12 @@ public class TypeConverter {
           .getPrecision()), String.valueOf(calciteType.getScale()));
     }
       break;
+    case NULL:
+      if (calciteType instanceof HiveType && ((HiveType) calciteType).getTypeClass() == TimestampTZ.class) {
+        ht = new HiveToken(HiveParser.TOK_TIMESTAMPLOCALTZ, "TOK_TIMESTAMPLOCALTZ");
+        break;
+      }
+      // fall-through
     default:
       ht = calciteToHiveTypeNameMap.get(calciteType.getSqlTypeName().getName());
     }

http://git-wip-us.apache.org/repos/asf/hive/blob/34eebff1/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java
index a0e5171..a054abb 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java
@@ -144,6 +144,7 @@ import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorConverters.C
 import org.apache.hadoop.hive.serde2.typeinfo.CharTypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.DecimalTypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.TimestampLocalTZTypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils;
 import org.apache.hadoop.hive.serde2.typeinfo.VarcharTypeInfo;
@@ -198,7 +199,7 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
     TokenToTypeName.put(HiveParser.TOK_DATE, serdeConstants.DATE_TYPE_NAME);
     TokenToTypeName.put(HiveParser.TOK_DATETIME, serdeConstants.DATETIME_TYPE_NAME);
     TokenToTypeName.put(HiveParser.TOK_TIMESTAMP, serdeConstants.TIMESTAMP_TYPE_NAME);
-    TokenToTypeName.put(HiveParser.TOK_TIMESTAMPTZ, serdeConstants.TIMESTAMPTZ_TYPE_NAME);
+    TokenToTypeName.put(HiveParser.TOK_TIMESTAMPLOCALTZ, serdeConstants.TIMESTAMPLOCALTZ_TYPE_NAME);
     TokenToTypeName.put(HiveParser.TOK_INTERVAL_YEAR_MONTH, serdeConstants.INTERVAL_YEAR_MONTH_TYPE_NAME);
     TokenToTypeName.put(HiveParser.TOK_INTERVAL_DAY_TIME, serdeConstants.INTERVAL_DAY_TIME_TYPE_NAME);
     TokenToTypeName.put(HiveParser.TOK_DECIMAL, serdeConstants.DECIMAL_TYPE_NAME);
@@ -222,10 +223,21 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
       VarcharTypeInfo varcharTypeInfo = ParseUtils.getVarcharTypeInfo(node);
       typeName = varcharTypeInfo.getQualifiedName();
       break;
+    case HiveParser.TOK_TIMESTAMPLOCALTZ:
+      HiveConf conf;
+      try {
+        conf = Hive.get().getConf();
+      } catch (HiveException e) {
+        throw new SemanticException(e);
+      }
+      TimestampLocalTZTypeInfo timestampLocalTZTypeInfo = TypeInfoFactory.getTimestampTZTypeInfo(
+          conf.getLocalTimeZone());
+      typeName = timestampLocalTZTypeInfo.getQualifiedName();
+      break;
     case HiveParser.TOK_DECIMAL:
-        DecimalTypeInfo decTypeInfo = ParseUtils.getDecimalTypeTypeInfo(node);
-        typeName = decTypeInfo.getQualifiedName();
-        break;
+      DecimalTypeInfo decTypeInfo = ParseUtils.getDecimalTypeTypeInfo(node);
+      typeName = decTypeInfo.getQualifiedName();
+      break;
     default:
       typeName = TokenToTypeName.get(token);
     }

http://git-wip-us.apache.org/repos/asf/hive/blob/34eebff1/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveLexer.g
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveLexer.g b/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveLexer.g
index da52cd5..b5792ac 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveLexer.g
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveLexer.g
@@ -130,7 +130,7 @@ KW_PRECISION: 'PRECISION';
 KW_DATE: 'DATE';
 KW_DATETIME: 'DATETIME';
 KW_TIMESTAMP: 'TIMESTAMP';
-KW_TIMESTAMPTZ: 'TIMESTAMPTZ';
+KW_TIMESTAMPLOCALTZ: 'TIMESTAMPLOCALTZ';
 KW_TIME: 'TIME';
 KW_ZONE: 'ZONE';
 KW_INTERVAL: 'INTERVAL';

http://git-wip-us.apache.org/repos/asf/hive/blob/34eebff1/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g b/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g
index 1386e65..429e0d9 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g
@@ -126,7 +126,7 @@ TOK_DATELITERAL;
 TOK_DATETIME;
 TOK_TIMESTAMP;
 TOK_TIMESTAMPLITERAL;
-TOK_TIMESTAMPTZ;
+TOK_TIMESTAMPLOCALTZ;
 TOK_INTERVAL_YEAR_MONTH;
 TOK_INTERVAL_YEAR_MONTH_LITERAL;
 TOK_INTERVAL_DAY_TIME;
@@ -495,7 +495,7 @@ import org.apache.hadoop.hive.conf.HiveConf;
     xlateMap.put("KW_DATE", "DATE");
     xlateMap.put("KW_DATETIME", "DATETIME");
     xlateMap.put("KW_TIMESTAMP", "TIMESTAMP");
-    xlateMap.put("KW_TIMESTAMPTZ", "TIMESTAMPTZ");
+    xlateMap.put("KW_TIMESTAMPLOCALTZ", "TIMESTAMPLOCALTZ");
     xlateMap.put("KW_TIME", "TIME");
     xlateMap.put("KW_ZONE", "ZONE");
     xlateMap.put("KW_STRING", "STRING");
@@ -2463,8 +2463,10 @@ primitiveType
     | KW_DATE          ->    TOK_DATE
     | KW_DATETIME      ->    TOK_DATETIME
     | KW_TIMESTAMP     ->    TOK_TIMESTAMP
-    | KW_TIMESTAMPTZ   ->    TOK_TIMESTAMPTZ
-    | KW_TIMESTAMP KW_WITH KW_TIME KW_ZONE -> TOK_TIMESTAMPTZ
+    | KW_TIMESTAMPLOCALTZ   ->    TOK_TIMESTAMPLOCALTZ
+    //| KW_TIMESTAMPTZ   ->    TOK_TIMESTAMPTZ
+    | KW_TIMESTAMP KW_WITH KW_LOCAL KW_TIME KW_ZONE -> TOK_TIMESTAMPLOCALTZ
+    //| KW_TIMESTAMP KW_WITH KW_TIME KW_ZONE -> TOK_TIMESTAMPTZ
     // Uncomment to allow intervals as table column types
     //| KW_INTERVAL KW_YEAR KW_TO KW_MONTH -> TOK_INTERVAL_YEAR_MONTH
     //| KW_INTERVAL KW_DAY KW_TO KW_SECOND -> TOK_INTERVAL_DAY_TIME

http://git-wip-us.apache.org/repos/asf/hive/blob/34eebff1/ql/src/java/org/apache/hadoop/hive/ql/parse/TypeCheckProcFactory.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/TypeCheckProcFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/TypeCheckProcFactory.java
index 632b9c6..391ee08 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/TypeCheckProcFactory.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/TypeCheckProcFactory.java
@@ -37,6 +37,7 @@ import org.apache.hadoop.hive.common.type.HiveChar;
 import org.apache.hadoop.hive.common.type.HiveDecimal;
 import org.apache.hadoop.hive.common.type.HiveIntervalDayTime;
 import org.apache.hadoop.hive.common.type.HiveIntervalYearMonth;
+import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.ql.ErrorMsg;
 import org.apache.hadoop.hive.ql.exec.ColumnInfo;
 import org.apache.hadoop.hive.ql.exec.FunctionInfo;
@@ -52,6 +53,8 @@ import org.apache.hadoop.hive.ql.lib.NodeProcessor;
 import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
 import org.apache.hadoop.hive.ql.lib.Rule;
 import org.apache.hadoop.hive.ql.lib.RuleRegExp;
+import org.apache.hadoop.hive.ql.metadata.Hive;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.lib.ExpressionWalker;
 import org.apache.hadoop.hive.ql.optimizer.ConstantPropagateProcFactory;
 import org.apache.hadoop.hive.ql.optimizer.calcite.CalciteSubquerySemanticException;
@@ -85,6 +88,7 @@ import org.apache.hadoop.hive.serde2.typeinfo.ListTypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.MapTypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.StructTypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.TimestampLocalTZTypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils;
@@ -747,8 +751,8 @@ public class TypeCheckProcFactory {
           serdeConstants.DATE_TYPE_NAME);
       conversionFunctionTextHashMap.put(HiveParser.TOK_TIMESTAMP,
           serdeConstants.TIMESTAMP_TYPE_NAME);
-      conversionFunctionTextHashMap.put(HiveParser.TOK_TIMESTAMPTZ,
-          serdeConstants.TIMESTAMPTZ_TYPE_NAME);
+      conversionFunctionTextHashMap.put(HiveParser.TOK_TIMESTAMPLOCALTZ,
+          serdeConstants.TIMESTAMPLOCALTZ_TYPE_NAME);
       conversionFunctionTextHashMap.put(HiveParser.TOK_INTERVAL_YEAR_MONTH,
           serdeConstants.INTERVAL_YEAR_MONTH_TYPE_NAME);
       conversionFunctionTextHashMap.put(HiveParser.TOK_INTERVAL_DAY_TIME,
@@ -1001,6 +1005,19 @@ public class TypeCheckProcFactory {
                 ((SettableUDF)genericUDF).setTypeInfo(varcharTypeInfo);
               }
               break;
+            case HiveParser.TOK_TIMESTAMPLOCALTZ:
+              TimestampLocalTZTypeInfo timestampLocalTZTypeInfo = new TimestampLocalTZTypeInfo();
+              HiveConf conf;
+              try {
+                conf = Hive.get().getConf();
+              } catch (HiveException e) {
+                throw new SemanticException(e);
+              }
+              timestampLocalTZTypeInfo.setTimeZone(conf.getLocalTimeZone());
+              if (genericUDF != null) {
+                ((SettableUDF)genericUDF).setTypeInfo(timestampLocalTZTypeInfo);
+              }
+              break;
             case HiveParser.TOK_DECIMAL:
               DecimalTypeInfo decTypeInfo = ParseUtils.getDecimalTypeTypeInfo(funcNameNode);
               if (genericUDF != null) {

http://git-wip-us.apache.org/repos/asf/hive/blob/34eebff1/ql/src/java/org/apache/hadoop/hive/ql/processors/SetProcessor.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/processors/SetProcessor.java b/ql/src/java/org/apache/hadoop/hive/ql/processors/SetProcessor.java
index 1458211..5a2c8cf 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/processors/SetProcessor.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/processors/SetProcessor.java
@@ -29,6 +29,8 @@ import java.util.Properties;
 import java.util.Set;
 import java.util.SortedMap;
 import java.util.TreeMap;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.conf.HiveConf;
@@ -69,6 +71,9 @@ public class SetProcessor implements CommandProcessor {
 
   private static final String[] PASSWORD_STRINGS = new String[] {"password", "paswd", "pswd"};
 
+  private static final Pattern TIME_ZONE_PATTERN =
+      Pattern.compile("^time(\\s)+zone\\s", Pattern.CASE_INSENSITIVE);
+
   public static boolean getBoolean(String value) {
     if (value.equals("on") || value.equals("true")) {
       return true;
@@ -383,6 +388,12 @@ public class SetProcessor implements CommandProcessor {
       return createProcessorSuccessResponse();
     }
 
+    // Special handling for time-zone
+    Matcher matcher = TIME_ZONE_PATTERN.matcher(nwcmd);
+    if (matcher.find()) {
+      nwcmd = HiveConf.ConfVars.HIVE_LOCAL_TIME_ZONE.varname + "=" + nwcmd.substring(matcher.end());
+    }
+
     String[] part = new String[2];
     int eqIndex = nwcmd.indexOf('=');
 

http://git-wip-us.apache.org/repos/asf/hive/blob/34eebff1/ql/src/java/org/apache/hadoop/hive/ql/stats/StatsUtils.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/stats/StatsUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/stats/StatsUtils.java
index eb02a91..487a823 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/stats/StatsUtils.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/stats/StatsUtils.java
@@ -99,7 +99,7 @@ import org.apache.hadoop.hive.serde2.objectinspector.primitive.WritableLongObjec
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.WritableShortObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.WritableStringObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.WritableTimestampObjectInspector;
-import org.apache.hadoop.hive.serde2.objectinspector.primitive.WritableTimestampTZObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.WritableTimestampLocalTZObjectInspector;
 import org.apache.hadoop.hive.serde2.typeinfo.CharTypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
@@ -834,7 +834,7 @@ public class StatsUtils {
       cs.setAvgColLen(csd.getBinaryStats().getAvgColLen());
       cs.setNumNulls(csd.getBinaryStats().getNumNulls());
     } else if (colTypeLowerCase.equals(serdeConstants.TIMESTAMP_TYPE_NAME) ||
-        colTypeLowerCase.equals(serdeConstants.TIMESTAMPTZ_TYPE_NAME)) {
+        colTypeLowerCase.equals(serdeConstants.TIMESTAMPLOCALTZ_TYPE_NAME)) {
       cs.setAvgColLen(JavaDataModel.get().lengthOfTimestamp());
     } else if (colTypeLowerCase.startsWith(serdeConstants.DECIMAL_TYPE_NAME)) {
       cs.setAvgColLen(JavaDataModel.get().lengthOfDecimal());
@@ -1144,7 +1144,7 @@ public class StatsUtils {
         || colTypeLowerCase.equals("long")) {
       return JavaDataModel.get().primitive2();
     } else if (colTypeLowerCase.equals(serdeConstants.TIMESTAMP_TYPE_NAME) ||
-        colTypeLowerCase.equals(serdeConstants.TIMESTAMPTZ_TYPE_NAME)) {
+        colTypeLowerCase.equals(serdeConstants.TIMESTAMPLOCALTZ_TYPE_NAME)) {
       return JavaDataModel.get().lengthOfTimestamp();
     } else if (colTypeLowerCase.equals(serdeConstants.DATE_TYPE_NAME)) {
       return JavaDataModel.get().lengthOfDate();
@@ -1182,7 +1182,7 @@ public class StatsUtils {
     } else if (colTypeLowerCase.equals(serdeConstants.BOOLEAN_TYPE_NAME)) {
       return JavaDataModel.get().lengthForBooleanArrayOfSize(length);
     } else if (colTypeLowerCase.equals(serdeConstants.TIMESTAMP_TYPE_NAME) ||
-        colTypeLowerCase.equals(serdeConstants.TIMESTAMPTZ_TYPE_NAME)) {
+        colTypeLowerCase.equals(serdeConstants.TIMESTAMPLOCALTZ_TYPE_NAME)) {
       return JavaDataModel.get().lengthForTimestampArrayOfSize(length);
     } else if (colTypeLowerCase.equals(serdeConstants.DATE_TYPE_NAME)) {
       return JavaDataModel.get().lengthForDateArrayOfSize(length);
@@ -1268,7 +1268,7 @@ public class StatsUtils {
     } else if (oi instanceof WritableShortObjectInspector) {
       return JavaDataModel.get().primitive1();
     } else if (oi instanceof WritableTimestampObjectInspector ||
-        oi instanceof WritableTimestampTZObjectInspector) {
+        oi instanceof WritableTimestampLocalTZObjectInspector) {
       return JavaDataModel.get().lengthOfTimestamp();
     }
 
@@ -1648,7 +1648,7 @@ public class StatsUtils {
           int acl = (int) Math.round(cs.getAvgColLen());
           sizeOf = JavaDataModel.get().lengthForByteArrayOfSize(acl);
         } else if (colTypeLowerCase.equals(serdeConstants.TIMESTAMP_TYPE_NAME) ||
-            colTypeLowerCase.equals(serdeConstants.TIMESTAMPTZ_TYPE_NAME)) {
+            colTypeLowerCase.equals(serdeConstants.TIMESTAMPLOCALTZ_TYPE_NAME)) {
           sizeOf = JavaDataModel.get().lengthOfTimestamp();
         } else if (colTypeLowerCase.startsWith(serdeConstants.DECIMAL_TYPE_NAME)) {
           sizeOf = JavaDataModel.get().lengthOfDecimal();

http://git-wip-us.apache.org/repos/asf/hive/blob/34eebff1/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFToBoolean.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFToBoolean.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFToBoolean.java
index 1605877..d291e36 100755
--- a/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFToBoolean.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFToBoolean.java
@@ -33,7 +33,7 @@ import org.apache.hadoop.hive.serde2.io.DateWritable;
 import org.apache.hadoop.hive.serde2.io.DoubleWritable;
 import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
 import org.apache.hadoop.hive.serde2.io.ShortWritable;
-import org.apache.hadoop.hive.serde2.io.TimestampTZWritable;
+import org.apache.hadoop.hive.serde2.io.TimestampLocalTZWritable;
 import org.apache.hadoop.hive.serde2.io.TimestampWritable;
 import org.apache.hadoop.io.BooleanWritable;
 import org.apache.hadoop.io.FloatWritable;

http://git-wip-us.apache.org/repos/asf/hive/blob/34eebff1/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFToString.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFToString.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFToString.java
index c10552a..05da2b4 100755
--- a/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFToString.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFToString.java
@@ -25,7 +25,7 @@ import org.apache.hadoop.hive.serde2.io.DateWritable;
 import org.apache.hadoop.hive.serde2.io.DoubleWritable;
 import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
 import org.apache.hadoop.hive.serde2.io.ShortWritable;
-import org.apache.hadoop.hive.serde2.io.TimestampTZWritable;
+import org.apache.hadoop.hive.serde2.io.TimestampLocalTZWritable;
 import org.apache.hadoop.hive.serde2.io.TimestampWritable;
 import org.apache.hadoop.hive.serde2.lazy.LazyInteger;
 import org.apache.hadoop.hive.serde2.lazy.LazyLong;
@@ -153,7 +153,7 @@ public class UDFToString extends UDF {
     }
   }
 
-  public Text evaluate(TimestampTZWritable i) {
+  public Text evaluate(TimestampLocalTZWritable i) {
     if (i == null) {
       return null;
     } else {

http://git-wip-us.apache.org/repos/asf/hive/blob/34eebff1/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDF.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDF.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDF.java
index 0898de6..ef8dcf0 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDF.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDF.java
@@ -412,7 +412,7 @@ public abstract class GenericUDF implements Closeable {
     case TIMESTAMP:
     case DATE:
     case VOID:
-    case TIMESTAMPTZ:
+    case TIMESTAMPLOCALTZ:
       outOi = PrimitiveObjectInspectorFactory.writableDateObjectInspector;
       break;
     default:
@@ -435,7 +435,7 @@ public abstract class GenericUDF implements Closeable {
     case CHAR:
     case TIMESTAMP:
     case DATE:
-    case TIMESTAMPTZ:
+    case TIMESTAMPLOCALTZ:
       break;
     default:
       throw new UDFArgumentTypeException(i, getFuncName()
@@ -510,7 +510,7 @@ public abstract class GenericUDF implements Closeable {
       break;
     case TIMESTAMP:
     case DATE:
-    case TIMESTAMPTZ:
+    case TIMESTAMPLOCALTZ:
       Object writableValue = converters[i].convert(obj);
       date = ((DateWritable) writableValue).get();
       break;

http://git-wip-us.apache.org/repos/asf/hive/blob/34eebff1/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFDate.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFDate.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFDate.java
index 4247afd..3885abc 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFDate.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFDate.java
@@ -87,7 +87,7 @@ public class GenericUDFDate extends GenericUDF {
       timestampConverter = new TimestampConverter(argumentOI,
         PrimitiveObjectInspectorFactory.writableTimestampObjectInspector);
       break;
-    case TIMESTAMPTZ:
+    case TIMESTAMPLOCALTZ:
     case DATE:
       dateWritableConverter = ObjectInspectorConverters.getConverter(argumentOI,
           PrimitiveObjectInspectorFactory.writableDateObjectInspector);
@@ -121,7 +121,7 @@ public class GenericUDFDate extends GenericUDF {
           .getTimestamp();
       output.set(DateWritable.millisToDays(ts.getTime()));
       break;
-    case TIMESTAMPTZ:
+    case TIMESTAMPLOCALTZ:
     case DATE:
       DateWritable dw = (DateWritable) dateWritableConverter.convert(arguments[0].get());
       output.set(dw);

http://git-wip-us.apache.org/repos/asf/hive/blob/34eebff1/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFToTimestampLocalTZ.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFToTimestampLocalTZ.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFToTimestampLocalTZ.java
new file mode 100644
index 0000000..754497d
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFToTimestampLocalTZ.java
@@ -0,0 +1,113 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.hadoop.hive.ql.udf.generic;
+
+import org.apache.hadoop.hive.ql.exec.Description;
+import org.apache.hadoop.hive.ql.exec.UDFArgumentException;
+import org.apache.hadoop.hive.ql.exec.UDFArgumentLengthException;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.udf.SettableUDF;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorConverter;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.SettableTimestampLocalTZObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorConverter.TimestampLocalTZConverter;
+import org.apache.hadoop.hive.serde2.typeinfo.TimestampLocalTZTypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
+
+/**
+ * Convert from string to TIMESTAMP WITH LOCAL TIME ZONE.
+ */
+@Description(name = "timestamp with local time zone",
+    value = "CAST(STRING as TIMESTAMP WITH LOCAL TIME ZONE) - returns the" +
+        "timestamp with local time zone represented by string.",
+    extended = "The string should be of format 'yyyy-MM-dd HH:mm:ss[.SSS...] ZoneId/ZoneOffset'. " +
+        "Examples of ZoneId and ZoneOffset are Asia/Shanghai and GMT+08:00. " +
+        "The time and zone parts are optional. If time is absent, '00:00:00.0' will be used. " +
+        "If zone is absent, the system time zone will be used.")
+public class GenericUDFToTimestampLocalTZ extends GenericUDF implements SettableUDF {
+
+  private transient PrimitiveObjectInspector argumentOI;
+  private transient PrimitiveObjectInspectorConverter.TimestampLocalTZConverter converter;
+
+  private TimestampLocalTZTypeInfo typeInfo;
+
+  @Override
+  public ObjectInspector initialize(ObjectInspector[] arguments) throws UDFArgumentException {
+    if (arguments.length < 1) {
+      throw new UDFArgumentLengthException(
+          "The function CAST as TIMESTAMP WITH LOCAL TIME ZONE requires at least one argument, got "
+              + arguments.length);
+    }
+    try {
+      argumentOI = (PrimitiveObjectInspector) arguments[0];
+      switch (argumentOI.getPrimitiveCategory()) {
+      case CHAR:
+      case VARCHAR:
+      case STRING:
+      case DATE:
+      case TIMESTAMP:
+      case TIMESTAMPLOCALTZ:
+        break;
+      default:
+        throw new UDFArgumentException("CAST as TIMESTAMP WITH LOCAL TIME ZONE only allows" +
+            "string/date/timestamp/timestamp with time zone types");
+      }
+    } catch (ClassCastException e) {
+      throw new UDFArgumentException(
+          "The function CAST as TIMESTAMP WITH LOCAL TIME ZONE takes only primitive types");
+    }
+    SettableTimestampLocalTZObjectInspector outputOI = (SettableTimestampLocalTZObjectInspector)
+          PrimitiveObjectInspectorFactory.getPrimitiveWritableObjectInspector(typeInfo);
+    converter = new TimestampLocalTZConverter(argumentOI, outputOI);
+    return outputOI;
+  }
+
+  @Override
+  public Object evaluate(DeferredObject[] arguments) throws HiveException {
+    Object o0 = arguments[0].get();
+    if (o0 == null) {
+      return null;
+    }
+    return converter.convert(o0);
+  }
+
+  @Override
+  public String getDisplayString(String[] children) {
+    assert (children.length == 1);
+    StringBuilder sb = new StringBuilder();
+    sb.append("CAST( ");
+    sb.append(children[0]);
+    sb.append(" AS ");
+    sb.append(typeInfo.getTypeName());
+    sb.append(")");
+    return sb.toString();
+  }
+
+  @Override
+  public TypeInfo getTypeInfo() {
+    return typeInfo;
+  }
+
+  @Override
+  public void setTypeInfo(TypeInfo typeInfo) throws UDFArgumentException {
+    this.typeInfo = (TimestampLocalTZTypeInfo) typeInfo;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/34eebff1/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFToTimestampTZ.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFToTimestampTZ.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFToTimestampTZ.java
deleted file mode 100644
index e96012b..0000000
--- a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFToTimestampTZ.java
+++ /dev/null
@@ -1,89 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * 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.hadoop.hive.ql.udf.generic;
-
-import org.apache.hadoop.hive.ql.exec.Description;
-import org.apache.hadoop.hive.ql.exec.UDFArgumentException;
-import org.apache.hadoop.hive.ql.exec.UDFArgumentLengthException;
-import org.apache.hadoop.hive.ql.metadata.HiveException;
-import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
-import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector;
-import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorConverter;
-import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory;
-
-/**
- * Convert from string to TIMESTAMP WITH TIME ZONE.
- */
-@Description(name = "timestamp with time zone",
-    value = "CAST(STRING as TIMESTAMP WITH TIME ZONE) - returns the" +
-        "timestamp with time zone represented by string.",
-    extended = "The string should be of format 'yyyy-MM-dd HH:mm:ss[.SSS...] ZoneId/ZoneOffset'. " +
-        "Examples of ZoneId and ZoneOffset are Asia/Shanghai and GMT+08:00. " +
-        "The time and zone parts are optional. If time is absent, '00:00:00.0' will be used. " +
-        "If zone is absent, the system time zone will be used.")
-public class GenericUDFToTimestampTZ extends GenericUDF {
-
-  private transient PrimitiveObjectInspector argumentOI;
-  private transient PrimitiveObjectInspectorConverter.TimestampTZConverter converter;
-
-
-  @Override
-  public ObjectInspector initialize(ObjectInspector[] arguments) throws UDFArgumentException {
-    if (arguments.length < 1) {
-      throw new UDFArgumentLengthException(
-          "The function CAST as TIMESTAMP WITH TIME ZONE requires at least one argument, got "
-              + arguments.length);
-    }
-    try {
-      argumentOI = (PrimitiveObjectInspector) arguments[0];
-      switch (argumentOI.getPrimitiveCategory()) {
-      case CHAR:
-      case VARCHAR:
-      case STRING:
-      case DATE:
-      case TIMESTAMP:
-      case TIMESTAMPTZ:
-        break;
-      default:
-        throw new UDFArgumentException("CAST as TIMESTAMP WITH TIME ZONE only allows" +
-            "string/date/timestamp/timestamp with time zone types");
-      }
-    } catch (ClassCastException e) {
-      throw new UDFArgumentException(
-          "The function CAST as TIMESTAMP WITH TIME ZONE takes only primitive types");
-    }
-    converter = new PrimitiveObjectInspectorConverter.TimestampTZConverter(argumentOI,
-        PrimitiveObjectInspectorFactory.writableTimestampTZObjectInspector);
-    return PrimitiveObjectInspectorFactory.writableTimestampTZObjectInspector;
-  }
-
-  @Override
-  public Object evaluate(DeferredObject[] arguments) throws HiveException {
-    Object o0 = arguments[0].get();
-    if (o0 == null) {
-      return null;
-    }
-    return converter.convert(o0);
-  }
-
-  @Override
-  public String getDisplayString(String[] children) {
-    assert (children.length == 1);
-    return "CAST(" + children[0] + " AS TIMESTAMP WITH TIME ZONE)";
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/34eebff1/ql/src/test/queries/clientpositive/localtimezone.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/localtimezone.q b/ql/src/test/queries/clientpositive/localtimezone.q
new file mode 100644
index 0000000..27b036b
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/localtimezone.q
@@ -0,0 +1,95 @@
+drop table `date_test`;
+drop table `timestamp_test`;
+drop table `timestamptz_test`;
+
+create table `date_test` (`mydate1` date);
+
+insert into `date_test` VALUES
+  ('2011-01-01 01:01:01.123'),
+  ('2011-01-01 01:01:01.123 Europe/Rome'),
+  ('2011-01-01 01:01:01.123 GMT-05:00'),
+  ('2011-01-01 01:01:01.12345678912'),
+  ('2011-01-01 01:01:01.12345678912 Europe/Rome'),
+  ('2011-01-01 01:01:01.12345678912 GMT-05:00'),
+  ('2011-01-01 01:01:01.12345678912 xyz');
+
+create table `timestamp_test` (`mydate1` timestamp);
+
+insert into `timestamp_test` VALUES
+  ('2011-01-01 01:01:01.123'),
+  ('2011-01-01 01:01:01.123 Europe/Rome'),
+  ('2011-01-01 01:01:01.123 GMT-05:00'),
+  ('2011-01-01 01:01:01.12345678912'),
+  ('2011-01-01 01:01:01.12345678912 Europe/Rome'),
+  ('2011-01-01 01:01:01.12345678912 GMT-05:00'),
+  ('2011-01-01 01:01:01.12345678912 xyz');
+
+create table `timestamptz_test` (`mydate1` timestamp with local time zone);
+
+insert into `timestamptz_test` VALUES
+  ('2011-01-01 01:01:01.123'),
+  ('2011-01-01 01:01:01.123 Europe/Rome'),
+  ('2011-01-01 01:01:01.123 GMT-05:00'),
+  ('2011-01-01 01:01:01.12345678912'),
+  ('2011-01-01 01:01:01.12345678912 Europe/Rome'),
+  ('2011-01-01 01:01:01.12345678912 GMT-05:00'),
+  ('2011-01-01 01:01:01.12345678912 xyz');
+
+select * from `date_test`;
+select * from `timestamp_test`;
+select * from `timestamptz_test`;
+
+set time zone Europe/Rome;
+
+select * from `date_test`;
+select * from `timestamp_test`;
+select * from `timestamptz_test`;
+
+set hive.local.time.zone=America/Los_Angeles;
+
+select * from `date_test`;
+select * from `timestamp_test`;
+select * from `timestamptz_test`;
+
+set time  zone GMT-07:00;
+
+select * from `date_test`;
+select * from `timestamp_test`;
+select * from `timestamptz_test`;
+
+select extract(year from `mydate1`) from `timestamptz_test`;
+select extract(quarter from `mydate1`) from `timestamptz_test`;
+select extract(month from `mydate1`) from `timestamptz_test`;
+select extract(day from `mydate1`) from `timestamptz_test`;
+select extract(hour from `mydate1`) from `timestamptz_test`;
+select extract(minute from `mydate1`) from `timestamptz_test`;
+select extract(second from `mydate1`) from `timestamptz_test`;
+
+select cast(`mydate1` as date) from `timestamptz_test`;
+select cast(`mydate1` as timestamp with local time zone) from `date_test`;
+select cast(`mydate1` as timestamp) from `timestamptz_test`;
+select cast(`mydate1` as timestamp with local time zone) from `timestamp_test`;
+
+select `mydate1` from `timestamptz_test` group by `mydate1`;
+select a.`mydate1` as c1, b.`mydate1` as c2
+from `timestamptz_test` a join `timestamptz_test` b
+on a.`mydate1` = b.`mydate1`;
+
+create table `timestamptz_test2` (`mydate1` timestamp with local time zone, `item` string, `price` double);
+insert into `timestamptz_test2` VALUES
+  ('2011-01-01 01:01:01.123', 'laptop 1', 9.2),
+  ('2011-01-01 01:01:01.123', 'mouse 1', 3.1),
+  ('2011-01-01 01:01:01.123 Europe/Rome', 'keyboard 1', 4.2),
+  ('2011-01-01 01:01:01.123 GMT-05:00', 'keyboard 2', 3.9),
+  ('2011-01-01 01:01:01.123 GMT-05:00', 'keyboard 3', 3.99),
+  ('2011-01-01 01:01:01.12345678912', 'mouse 2', 4.594),
+  ('2011-01-01 01:01:01.12345678912 Europe/Rome', 'laptop 2', 10),
+  ('2011-01-01 01:01:01.12345678912 Europe/Rome', 'hdmi', 1.25),
+  ('2011-01-01 01:01:01.12345678912 GMT-05:00', 'pin', null),
+  ('2011-01-01 01:01:01.12345678912 xyz', 'cable', 0.0);
+select `item`, `price`,
+rank() over (partition by `mydate1` order by `price`) as r
+from `timestamptz_test2`;
+select `item`, `price`,
+rank() over (partition by cast(`mydate1` as date) order by `price`) as r
+from `timestamptz_test2`;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/34eebff1/ql/src/test/queries/clientpositive/timestamptz.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/timestamptz.q b/ql/src/test/queries/clientpositive/timestamptz.q
index 176fefd..be76157 100644
--- a/ql/src/test/queries/clientpositive/timestamptz.q
+++ b/ql/src/test/queries/clientpositive/timestamptz.q
@@ -1,11 +1,11 @@
-explain select cast('2005-01-03 02:01:00 GMT' as timestamp with time zone);
-select cast('2005-01-03 02:01:00 GMT' as timestamp with time zone);
+explain select cast('2005-01-03 02:01:00 GMT' as timestamp with local time zone);
+select cast('2005-01-03 02:01:00 GMT' as timestamp with local time zone);
 
-explain select cast('2016-01-03 12:26:34.0123 America/Los_Angeles' as timestamptz);
-select cast('2016-01-03 12:26:34.0123 America/Los_Angeles' as timestamptz);
+explain select cast('2016-01-03 12:26:34.0123 America/Los_Angeles' as timestamplocaltz);
+select cast('2016-01-03 12:26:34.0123 America/Los_Angeles' as timestamplocaltz);
 
-explain select cast('2016-01-03Europe/London' as timestamptz);
-select cast('2016-01-03Europe/London' as timestamptz);
+explain select cast('2016-01-03Europe/London' as timestamplocaltz);
+select cast('2016-01-03Europe/London' as timestamplocaltz);
 
-explain select cast('2016-01-03 13:34:56.38 +1:00' as timestamptz);
-select cast('2016-01-03 13:34:56.38 +1:00' as timestamptz);
+explain select cast('2016-01-03 13:34:56.38 +1:00' as timestamplocaltz);
+select cast('2016-01-03 13:34:56.38 +1:00' as timestamplocaltz);

http://git-wip-us.apache.org/repos/asf/hive/blob/34eebff1/ql/src/test/queries/clientpositive/timestamptz_1.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/timestamptz_1.q b/ql/src/test/queries/clientpositive/timestamptz_1.q
index c11aea2..63270a3 100644
--- a/ql/src/test/queries/clientpositive/timestamptz_1.q
+++ b/ql/src/test/queries/clientpositive/timestamptz_1.q
@@ -2,9 +2,9 @@ set hive.fetch.task.conversion=more;
 
 drop table tstz1;
 
-create table tstz1(t timestamp with time zone);
+create table tstz1(t timestamp with local time zone);
 
-insert overwrite table tstz1 select cast('2016-01-03 12:26:34 America/Los_Angeles' as timestamp with time zone);
+insert overwrite table tstz1 select cast('2016-01-03 12:26:34 America/Los_Angeles' as timestamp with local time zone);
 select cast(t as string) from tstz1;
 select cast(t as date) from tstz1;
 select cast(t as timestamp) from tstz1;

http://git-wip-us.apache.org/repos/asf/hive/blob/34eebff1/ql/src/test/queries/clientpositive/timestamptz_2.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/timestamptz_2.q b/ql/src/test/queries/clientpositive/timestamptz_2.q
index a335f52..da5e8d9 100644
--- a/ql/src/test/queries/clientpositive/timestamptz_2.q
+++ b/ql/src/test/queries/clientpositive/timestamptz_2.q
@@ -1,8 +1,9 @@
 set hive.fetch.task.conversion=more;
+set time zone UTC;
 
 drop table tstz2;
 
-create table tstz2(t timestamp with time zone);
+create table tstz2(t timestamp with local time zone);
 
 insert into table tstz2 values
   ('2005-04-03 03:01:00.04067 GMT-07:00'),('2005-01-03 02:01:00 GMT'),('2005-01-03 06:01:00 GMT+04:00'),

http://git-wip-us.apache.org/repos/asf/hive/blob/34eebff1/ql/src/test/results/clientpositive/annotate_stats_select.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/annotate_stats_select.q.out b/ql/src/test/results/clientpositive/annotate_stats_select.q.out
index 67d134b..e3f08ea 100644
--- a/ql/src/test/results/clientpositive/annotate_stats_select.q.out
+++ b/ql/src/test/results/clientpositive/annotate_stats_select.q.out
@@ -470,9 +470,9 @@ STAGE PLANS:
           alias: alltypes_orc
           Statistics: Num rows: 2 Data size: 1686 Basic stats: COMPLETE Column stats: COMPLETE
           Select Operator
-            expressions: null (type: date)
+            expressions: 1970-12-31 (type: date)
             outputColumnNames: _col0
-            Statistics: Num rows: 2 Data size: 56 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 2 Data size: 112 Basic stats: COMPLETE Column stats: COMPLETE
             ListSink
 
 PREHOOK: query: explain select cast("58.174" as DECIMAL) from alltypes_orc

http://git-wip-us.apache.org/repos/asf/hive/blob/34eebff1/ql/src/test/results/clientpositive/constantfolding.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/constantfolding.q.out b/ql/src/test/results/clientpositive/constantfolding.q.out
index 10e185f..f9a9d24 100644
--- a/ql/src/test/results/clientpositive/constantfolding.q.out
+++ b/ql/src/test/results/clientpositive/constantfolding.q.out
@@ -205,9 +205,9 @@ STAGE PLANS:
           alias: src
           Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
           Select Operator
-            expressions: null (type: date)
+            expressions: 1970-12-31 (type: date)
             outputColumnNames: _col0
-            Statistics: Num rows: 500 Data size: 56 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 500 Data size: 28000 Basic stats: COMPLETE Column stats: COMPLETE
             ListSink
 
 PREHOOK: query: CREATE TABLE dest1(c1 STRING) STORED AS TEXTFILE