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:33 UTC

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

Repository: hive
Updated Branches:
  refs/heads/master 5522a2db1 -> 34eebff19


http://git-wip-us.apache.org/repos/asf/hive/blob/34eebff1/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/TypeInfoUtils.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/TypeInfoUtils.java b/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/TypeInfoUtils.java
index 8be6896..fd7c12a 100644
--- a/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/TypeInfoUtils.java
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/TypeInfoUtils.java
@@ -48,6 +48,8 @@ import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectIn
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorUtils;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorUtils.PrimitiveGrouping;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorUtils.PrimitiveTypeEntry;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * TypeInfoUtils.
@@ -55,6 +57,8 @@ import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectIn
  */
 public final class TypeInfoUtils {
 
+  protected static final Logger LOG = LoggerFactory.getLogger(TypeInfoUtils.class);
+
   public static List<PrimitiveCategory> numericTypeList = new ArrayList<PrimitiveCategory>();
   // The ordering of types here is used to determine which numeric types
   // are common/convertible to one another. Probably better to rely on the
@@ -297,6 +301,23 @@ public final class TypeInfoUtils {
       int end = 1;
       while (end <= typeInfoString.length()) {
         // last character ends a token?
+        // if there are quotes, all the text between the quotes
+        // is considered a single token (this can happen for
+        // timestamp with local time-zone)
+        if (begin > 0 &&
+            typeInfoString.charAt(begin - 1) == '(' &&
+            typeInfoString.charAt(begin) == '\'') {
+          // Ignore starting quote
+          begin++;
+          do {
+            end++;
+          } while (typeInfoString.charAt(end) != '\'');
+        } else if (typeInfoString.charAt(begin) == '\'' &&
+            typeInfoString.charAt(begin + 1) == ')') {
+          // Ignore closing quote
+          begin++;
+          end++;
+        }
         if (end == typeInfoString.length()
             || !isTypeChar(typeInfoString.charAt(end - 1))
             || !isTypeChar(typeInfoString.charAt(end))) {
@@ -443,6 +464,7 @@ public final class TypeInfoUtils {
                 "Type " + typeEntry.typeName+ " only takes one parameter, but " +
                 params.length + " is seen");
           }
+
         case DECIMAL:
           int precision = HiveDecimal.USER_DEFAULT_PRECISION;
           int scale = HiveDecimal.USER_DEFAULT_SCALE;
@@ -462,8 +484,8 @@ public final class TypeInfoUtils {
             throw new IllegalArgumentException("Type decimal only takes two parameter, but " +
                 params.length + " is seen");
           }
-
           return TypeInfoFactory.getDecimalTypeInfo(precision, scale);
+
         default:
           return TypeInfoFactory.getPrimitiveTypeInfo(typeEntry.typeName);
         }

http://git-wip-us.apache.org/repos/asf/hive/blob/34eebff1/serde/src/test/org/apache/hadoop/hive/serde2/io/TestTimestampTZWritable.java
----------------------------------------------------------------------
diff --git a/serde/src/test/org/apache/hadoop/hive/serde2/io/TestTimestampTZWritable.java b/serde/src/test/org/apache/hadoop/hive/serde2/io/TestTimestampTZWritable.java
index 73b81b9..833ceb7 100644
--- a/serde/src/test/org/apache/hadoop/hive/serde2/io/TestTimestampTZWritable.java
+++ b/serde/src/test/org/apache/hadoop/hive/serde2/io/TestTimestampTZWritable.java
@@ -20,11 +20,13 @@ package org.apache.hadoop.hive.serde2.io;
 import com.google.code.tempusfugit.concurrency.RepeatingRule;
 import com.google.code.tempusfugit.concurrency.annotations.Repeating;
 import org.apache.hadoop.hive.common.type.TimestampTZ;
+import org.apache.hadoop.hive.common.type.TimestampTZUtil;
 import org.apache.hadoop.io.WritableComparator;
 import org.junit.Assert;
 import org.junit.Rule;
 import org.junit.Test;
 
+import java.time.ZoneId;
 import java.util.concurrent.ThreadLocalRandom;
 
 public class TestTimestampTZWritable {
@@ -37,7 +39,7 @@ public class TestTimestampTZWritable {
   public void testSeconds() {
     // just 1 VInt
     long seconds = ThreadLocalRandom.current().nextLong(Integer.MAX_VALUE);
-    TimestampTZ tstz = new TimestampTZ(seconds, 0);
+    TimestampTZ tstz = new TimestampTZ(seconds, 0, ZoneId.of("UTC"));
     verifyConversion(tstz);
 
     // 2 VInt
@@ -45,7 +47,7 @@ public class TestTimestampTZWritable {
     if (ThreadLocalRandom.current().nextBoolean()) {
       seconds = -seconds;
     }
-    tstz.set(seconds, 0);
+    tstz.set(seconds, 0, ZoneId.of("UTC"));
     verifyConversion(tstz);
   }
 
@@ -59,7 +61,7 @@ public class TestTimestampTZWritable {
 
     int nanos = ThreadLocalRandom.current().nextInt(999999999) + 1;
 
-    TimestampTZ tstz = new TimestampTZ(seconds, nanos);
+    TimestampTZ tstz = new TimestampTZ(seconds, nanos, ZoneId.of("UTC"));
     verifyConversion(tstz);
   }
 
@@ -69,10 +71,10 @@ public class TestTimestampTZWritable {
     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";
-    TimestampTZWritable writable1 = new TimestampTZWritable(TimestampTZ.parse(s1));
-    TimestampTZWritable writable2 = new TimestampTZWritable(TimestampTZ.parse(s2));
-    TimestampTZWritable writable3 = new TimestampTZWritable(TimestampTZ.parse(s3));
-    TimestampTZWritable writable4 = new TimestampTZWritable(TimestampTZ.parse(s4));
+    TimestampLocalTZWritable writable1 = new TimestampLocalTZWritable(TimestampTZUtil.parse(s1));
+    TimestampLocalTZWritable writable2 = new TimestampLocalTZWritable(TimestampTZUtil.parse(s2));
+    TimestampLocalTZWritable writable3 = new TimestampLocalTZWritable(TimestampTZUtil.parse(s3));
+    TimestampLocalTZWritable writable4 = new TimestampLocalTZWritable(TimestampTZUtil.parse(s4));
 
     Assert.assertEquals(writable1, writable2);
     Assert.assertEquals(writable1, writable3);
@@ -90,9 +92,9 @@ public class TestTimestampTZWritable {
   }
 
   private static void verifyConversion(TimestampTZ srcTstz) {
-    TimestampTZWritable src = new TimestampTZWritable(srcTstz);
+    TimestampLocalTZWritable src = new TimestampLocalTZWritable(srcTstz);
     byte[] bytes = src.getBytes();
-    TimestampTZWritable dest = new TimestampTZWritable(bytes, 0);
+    TimestampLocalTZWritable dest = new TimestampLocalTZWritable(bytes, 0, ZoneId.of("UTC"));
     TimestampTZ destTstz = dest.getTimestampTZ();
     String errMsg = "Src tstz with seconds " + srcTstz.getEpochSecond() + ", nanos " +
         srcTstz.getNanos() + ". Dest tstz with seconds " + destTstz.getEpochSecond() +

http://git-wip-us.apache.org/repos/asf/hive/blob/34eebff1/service-rpc/if/TCLIService.thrift
----------------------------------------------------------------------
diff --git a/service-rpc/if/TCLIService.thrift b/service-rpc/if/TCLIService.thrift
index 1609669..976ca9b 100644
--- a/service-rpc/if/TCLIService.thrift
+++ b/service-rpc/if/TCLIService.thrift
@@ -67,7 +67,7 @@ enum TProtocolVersion {
   // V10 adds support for in place updates via GetOperationStatus
   HIVE_CLI_SERVICE_PROTOCOL_V10
 
-  // V11 adds timestamp with time zone type
+  // V11 adds timestamp with local time zone type
   HIVE_CLI_SERVICE_PROTOCOL_V11
 }
 
@@ -94,7 +94,7 @@ enum TTypeId {
   CHAR_TYPE,
   INTERVAL_YEAR_MONTH_TYPE,
   INTERVAL_DAY_TIME_TYPE,
-  TIMESTAMPTZ_TYPE
+  TIMESTAMPLOCALTZ_TYPE
 }
 
 const set<TTypeId> PRIMITIVE_TYPES = [
@@ -115,7 +115,7 @@ const set<TTypeId> PRIMITIVE_TYPES = [
   TTypeId.CHAR_TYPE,
   TTypeId.INTERVAL_YEAR_MONTH_TYPE,
   TTypeId.INTERVAL_DAY_TIME_TYPE,
-  TTypeId.TIMESTAMPTZ_TYPE
+  TTypeId.TIMESTAMPLOCALTZ_TYPE
 ]
 
 const set<TTypeId> COMPLEX_TYPES = [
@@ -153,7 +153,7 @@ const map<TTypeId,string> TYPE_NAMES = {
   TTypeId.CHAR_TYPE: "CHAR"
   TTypeId.INTERVAL_YEAR_MONTH_TYPE: "INTERVAL_YEAR_MONTH"
   TTypeId.INTERVAL_DAY_TIME_TYPE: "INTERVAL_DAY_TIME"
-  TTypeId.TIMESTAMPTZ_TYPE: "TIMESTAMP WITH TIME ZONE"
+  TTypeId.TIMESTAMPLOCALTZ_TYPE: "TIMESTAMP WITH LOCAL TIME ZONE"
 }
 
 // Thrift does not support recursively defined types or forward declarations,

http://git-wip-us.apache.org/repos/asf/hive/blob/34eebff1/service-rpc/src/gen/thrift/gen-cpp/TCLIService_constants.cpp
----------------------------------------------------------------------
diff --git a/service-rpc/src/gen/thrift/gen-cpp/TCLIService_constants.cpp b/service-rpc/src/gen/thrift/gen-cpp/TCLIService_constants.cpp
index 5ec3426..874a81b 100644
--- a/service-rpc/src/gen/thrift/gen-cpp/TCLIService_constants.cpp
+++ b/service-rpc/src/gen/thrift/gen-cpp/TCLIService_constants.cpp
@@ -60,7 +60,7 @@ TCLIServiceConstants::TCLIServiceConstants() {
   TYPE_NAMES.insert(std::make_pair((TTypeId::type)19, "CHAR"));
   TYPE_NAMES.insert(std::make_pair((TTypeId::type)20, "INTERVAL_YEAR_MONTH"));
   TYPE_NAMES.insert(std::make_pair((TTypeId::type)21, "INTERVAL_DAY_TIME"));
-  TYPE_NAMES.insert(std::make_pair((TTypeId::type)22, "TIMESTAMP WITH TIME ZONE"));
+  TYPE_NAMES.insert(std::make_pair((TTypeId::type)22, "TIMESTAMP WITH LOCAL TIME ZONE"));
 
   CHARACTER_MAXIMUM_LENGTH = "characterMaximumLength";
 

http://git-wip-us.apache.org/repos/asf/hive/blob/34eebff1/service-rpc/src/gen/thrift/gen-cpp/TCLIService_types.cpp
----------------------------------------------------------------------
diff --git a/service-rpc/src/gen/thrift/gen-cpp/TCLIService_types.cpp b/service-rpc/src/gen/thrift/gen-cpp/TCLIService_types.cpp
index 61933e8..ec3f066 100644
--- a/service-rpc/src/gen/thrift/gen-cpp/TCLIService_types.cpp
+++ b/service-rpc/src/gen/thrift/gen-cpp/TCLIService_types.cpp
@@ -64,7 +64,7 @@ int _kTTypeIdValues[] = {
   TTypeId::CHAR_TYPE,
   TTypeId::INTERVAL_YEAR_MONTH_TYPE,
   TTypeId::INTERVAL_DAY_TIME_TYPE,
-  TTypeId::TIMESTAMPTZ_TYPE
+  TTypeId::TIMESTAMPLOCALTZ_TYPE
 };
 const char* _kTTypeIdNames[] = {
   "BOOLEAN_TYPE",
@@ -89,7 +89,7 @@ const char* _kTTypeIdNames[] = {
   "CHAR_TYPE",
   "INTERVAL_YEAR_MONTH_TYPE",
   "INTERVAL_DAY_TIME_TYPE",
-  "TIMESTAMPTZ_TYPE"
+  "TIMESTAMPLOCALTZ_TYPE"
 };
 const std::map<int, const char*> _TTypeId_VALUES_TO_NAMES(::apache::thrift::TEnumIterator(23, _kTTypeIdValues, _kTTypeIdNames), ::apache::thrift::TEnumIterator(-1, NULL, NULL));
 

http://git-wip-us.apache.org/repos/asf/hive/blob/34eebff1/service-rpc/src/gen/thrift/gen-cpp/TCLIService_types.h
----------------------------------------------------------------------
diff --git a/service-rpc/src/gen/thrift/gen-cpp/TCLIService_types.h b/service-rpc/src/gen/thrift/gen-cpp/TCLIService_types.h
index 9483a2a..cb62db0 100644
--- a/service-rpc/src/gen/thrift/gen-cpp/TCLIService_types.h
+++ b/service-rpc/src/gen/thrift/gen-cpp/TCLIService_types.h
@@ -61,7 +61,7 @@ struct TTypeId {
     CHAR_TYPE = 19,
     INTERVAL_YEAR_MONTH_TYPE = 20,
     INTERVAL_DAY_TIME_TYPE = 21,
-    TIMESTAMPTZ_TYPE = 22
+    TIMESTAMPLOCALTZ_TYPE = 22
   };
 };
 

http://git-wip-us.apache.org/repos/asf/hive/blob/34eebff1/service-rpc/src/gen/thrift/gen-javabean/org/apache/hive/service/rpc/thrift/TCLIServiceConstants.java
----------------------------------------------------------------------
diff --git a/service-rpc/src/gen/thrift/gen-javabean/org/apache/hive/service/rpc/thrift/TCLIServiceConstants.java b/service-rpc/src/gen/thrift/gen-javabean/org/apache/hive/service/rpc/thrift/TCLIServiceConstants.java
index 3d15147..955da0c 100644
--- a/service-rpc/src/gen/thrift/gen-javabean/org/apache/hive/service/rpc/thrift/TCLIServiceConstants.java
+++ b/service-rpc/src/gen/thrift/gen-javabean/org/apache/hive/service/rpc/thrift/TCLIServiceConstants.java
@@ -55,7 +55,7 @@ public class TCLIServiceConstants {
     PRIMITIVE_TYPES.add(org.apache.hive.service.rpc.thrift.TTypeId.CHAR_TYPE);
     PRIMITIVE_TYPES.add(org.apache.hive.service.rpc.thrift.TTypeId.INTERVAL_YEAR_MONTH_TYPE);
     PRIMITIVE_TYPES.add(org.apache.hive.service.rpc.thrift.TTypeId.INTERVAL_DAY_TIME_TYPE);
-    PRIMITIVE_TYPES.add(org.apache.hive.service.rpc.thrift.TTypeId.TIMESTAMPTZ_TYPE);
+    PRIMITIVE_TYPES.add(org.apache.hive.service.rpc.thrift.TTypeId.TIMESTAMPLOCALTZ_TYPE);
   }
 
   public static final Set<TTypeId> COMPLEX_TYPES = new HashSet<TTypeId>();
@@ -96,7 +96,7 @@ public class TCLIServiceConstants {
     TYPE_NAMES.put(org.apache.hive.service.rpc.thrift.TTypeId.CHAR_TYPE, "CHAR");
     TYPE_NAMES.put(org.apache.hive.service.rpc.thrift.TTypeId.INTERVAL_YEAR_MONTH_TYPE, "INTERVAL_YEAR_MONTH");
     TYPE_NAMES.put(org.apache.hive.service.rpc.thrift.TTypeId.INTERVAL_DAY_TIME_TYPE, "INTERVAL_DAY_TIME");
-    TYPE_NAMES.put(org.apache.hive.service.rpc.thrift.TTypeId.TIMESTAMPTZ_TYPE, "TIMESTAMP WITH TIME ZONE");
+    TYPE_NAMES.put(org.apache.hive.service.rpc.thrift.TTypeId.TIMESTAMPLOCALTZ_TYPE, "TIMESTAMP WITH LOCAL TIME ZONE");
   }
 
   public static final String CHARACTER_MAXIMUM_LENGTH = "characterMaximumLength";

http://git-wip-us.apache.org/repos/asf/hive/blob/34eebff1/service-rpc/src/gen/thrift/gen-javabean/org/apache/hive/service/rpc/thrift/TTypeId.java
----------------------------------------------------------------------
diff --git a/service-rpc/src/gen/thrift/gen-javabean/org/apache/hive/service/rpc/thrift/TTypeId.java b/service-rpc/src/gen/thrift/gen-javabean/org/apache/hive/service/rpc/thrift/TTypeId.java
index 1b062b7..a698d40 100644
--- a/service-rpc/src/gen/thrift/gen-javabean/org/apache/hive/service/rpc/thrift/TTypeId.java
+++ b/service-rpc/src/gen/thrift/gen-javabean/org/apache/hive/service/rpc/thrift/TTypeId.java
@@ -34,7 +34,7 @@ public enum TTypeId implements org.apache.thrift.TEnum {
   CHAR_TYPE(19),
   INTERVAL_YEAR_MONTH_TYPE(20),
   INTERVAL_DAY_TIME_TYPE(21),
-  TIMESTAMPTZ_TYPE(22);
+  TIMESTAMPLOCALTZ_TYPE(22);
 
   private final int value;
 
@@ -100,7 +100,7 @@ public enum TTypeId implements org.apache.thrift.TEnum {
       case 21:
         return INTERVAL_DAY_TIME_TYPE;
       case 22:
-        return TIMESTAMPTZ_TYPE;
+        return TIMESTAMPLOCALTZ_TYPE;
       default:
         return null;
     }

http://git-wip-us.apache.org/repos/asf/hive/blob/34eebff1/service-rpc/src/gen/thrift/gen-php/Types.php
----------------------------------------------------------------------
diff --git a/service-rpc/src/gen/thrift/gen-php/Types.php b/service-rpc/src/gen/thrift/gen-php/Types.php
index 263b22c..f34a0cf 100644
--- a/service-rpc/src/gen/thrift/gen-php/Types.php
+++ b/service-rpc/src/gen/thrift/gen-php/Types.php
@@ -65,7 +65,7 @@ final class TTypeId {
   const CHAR_TYPE = 19;
   const INTERVAL_YEAR_MONTH_TYPE = 20;
   const INTERVAL_DAY_TIME_TYPE = 21;
-  const TIMESTAMPTZ_TYPE = 22;
+  const TIMESTAMPLOCALTZ_TYPE = 22;
   static public $__names = array(
     0 => 'BOOLEAN_TYPE',
     1 => 'TINYINT_TYPE',
@@ -89,7 +89,7 @@ final class TTypeId {
     19 => 'CHAR_TYPE',
     20 => 'INTERVAL_YEAR_MONTH_TYPE',
     21 => 'INTERVAL_DAY_TIME_TYPE',
-    22 => 'TIMESTAMPTZ_TYPE',
+    22 => 'TIMESTAMPLOCALTZ_TYPE',
   );
 }
 
@@ -10101,7 +10101,7 @@ final class Constant extends \Thrift\Type\TConstant {
             19 => "CHAR",
             20 => "INTERVAL_YEAR_MONTH",
             21 => "INTERVAL_DAY_TIME",
-            22 => "TIMESTAMP WITH TIME ZONE",
+            22 => "TIMESTAMP WITH LOCAL TIME ZONE",
     );
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/34eebff1/service-rpc/src/gen/thrift/gen-py/TCLIService/constants.py
----------------------------------------------------------------------
diff --git a/service-rpc/src/gen/thrift/gen-py/TCLIService/constants.py b/service-rpc/src/gen/thrift/gen-py/TCLIService/constants.py
index edd1edd..0427aa9 100644
--- a/service-rpc/src/gen/thrift/gen-py/TCLIService/constants.py
+++ b/service-rpc/src/gen/thrift/gen-py/TCLIService/constants.py
@@ -62,7 +62,7 @@ TYPE_NAMES = {
     19 : "CHAR",
     20 : "INTERVAL_YEAR_MONTH",
     21 : "INTERVAL_DAY_TIME",
-    22 : "TIMESTAMP WITH TIME ZONE",
+    22 : "TIMESTAMP WITH LOCAL TIME ZONE",
 }
 CHARACTER_MAXIMUM_LENGTH = "characterMaximumLength"
 PRECISION = "precision"

http://git-wip-us.apache.org/repos/asf/hive/blob/34eebff1/service-rpc/src/gen/thrift/gen-py/TCLIService/ttypes.py
----------------------------------------------------------------------
diff --git a/service-rpc/src/gen/thrift/gen-py/TCLIService/ttypes.py b/service-rpc/src/gen/thrift/gen-py/TCLIService/ttypes.py
index 95bd9cc..acc8c3a 100644
--- a/service-rpc/src/gen/thrift/gen-py/TCLIService/ttypes.py
+++ b/service-rpc/src/gen/thrift/gen-py/TCLIService/ttypes.py
@@ -80,7 +80,7 @@ class TTypeId:
   CHAR_TYPE = 19
   INTERVAL_YEAR_MONTH_TYPE = 20
   INTERVAL_DAY_TIME_TYPE = 21
-  TIMESTAMPTZ_TYPE = 22
+  TIMESTAMPLOCALTZ_TYPE = 22
 
   _VALUES_TO_NAMES = {
     0: "BOOLEAN_TYPE",
@@ -105,7 +105,7 @@ class TTypeId:
     19: "CHAR_TYPE",
     20: "INTERVAL_YEAR_MONTH_TYPE",
     21: "INTERVAL_DAY_TIME_TYPE",
-    22: "TIMESTAMPTZ_TYPE",
+    22: "TIMESTAMPLOCALTZ_TYPE",
   }
 
   _NAMES_TO_VALUES = {
@@ -131,7 +131,7 @@ class TTypeId:
     "CHAR_TYPE": 19,
     "INTERVAL_YEAR_MONTH_TYPE": 20,
     "INTERVAL_DAY_TIME_TYPE": 21,
-    "TIMESTAMPTZ_TYPE": 22,
+    "TIMESTAMPLOCALTZ_TYPE": 22,
   }
 
 class TStatusCode:

http://git-wip-us.apache.org/repos/asf/hive/blob/34eebff1/service-rpc/src/gen/thrift/gen-rb/t_c_l_i_service_constants.rb
----------------------------------------------------------------------
diff --git a/service-rpc/src/gen/thrift/gen-rb/t_c_l_i_service_constants.rb b/service-rpc/src/gen/thrift/gen-rb/t_c_l_i_service_constants.rb
index 1f5b604..2772170 100644
--- a/service-rpc/src/gen/thrift/gen-rb/t_c_l_i_service_constants.rb
+++ b/service-rpc/src/gen/thrift/gen-rb/t_c_l_i_service_constants.rb
@@ -63,7 +63,7 @@ TYPE_NAMES = {
     19 => %q"CHAR",
     20 => %q"INTERVAL_YEAR_MONTH",
     21 => %q"INTERVAL_DAY_TIME",
-    22 => %q"TIMESTAMP WITH TIME ZONE",
+    22 => %q"TIMESTAMP WITH LOCAL TIME ZONE",
 }
 
 CHARACTER_MAXIMUM_LENGTH = %q"characterMaximumLength"

http://git-wip-us.apache.org/repos/asf/hive/blob/34eebff1/service-rpc/src/gen/thrift/gen-rb/t_c_l_i_service_types.rb
----------------------------------------------------------------------
diff --git a/service-rpc/src/gen/thrift/gen-rb/t_c_l_i_service_types.rb b/service-rpc/src/gen/thrift/gen-rb/t_c_l_i_service_types.rb
index aa4940a..6695aee 100644
--- a/service-rpc/src/gen/thrift/gen-rb/t_c_l_i_service_types.rb
+++ b/service-rpc/src/gen/thrift/gen-rb/t_c_l_i_service_types.rb
@@ -45,9 +45,9 @@ module TTypeId
   CHAR_TYPE = 19
   INTERVAL_YEAR_MONTH_TYPE = 20
   INTERVAL_DAY_TIME_TYPE = 21
-  TIMESTAMPTZ_TYPE = 22
-  VALUE_MAP = {0 => "BOOLEAN_TYPE", 1 => "TINYINT_TYPE", 2 => "SMALLINT_TYPE", 3 => "INT_TYPE", 4 => "BIGINT_TYPE", 5 => "FLOAT_TYPE", 6 => "DOUBLE_TYPE", 7 => "STRING_TYPE", 8 => "TIMESTAMP_TYPE", 9 => "BINARY_TYPE", 10 => "ARRAY_TYPE", 11 => "MAP_TYPE", 12 => "STRUCT_TYPE", 13 => "UNION_TYPE", 14 => "USER_DEFINED_TYPE", 15 => "DECIMAL_TYPE", 16 => "NULL_TYPE", 17 => "DATE_TYPE", 18 => "VARCHAR_TYPE", 19 => "CHAR_TYPE", 20 => "INTERVAL_YEAR_MONTH_TYPE", 21 => "INTERVAL_DAY_TIME_TYPE", 22 => "TIMESTAMPTZ_TYPE"}
-  VALID_VALUES = Set.new([BOOLEAN_TYPE, TINYINT_TYPE, SMALLINT_TYPE, INT_TYPE, BIGINT_TYPE, FLOAT_TYPE, DOUBLE_TYPE, STRING_TYPE, TIMESTAMP_TYPE, BINARY_TYPE, ARRAY_TYPE, MAP_TYPE, STRUCT_TYPE, UNION_TYPE, USER_DEFINED_TYPE, DECIMAL_TYPE, NULL_TYPE, DATE_TYPE, VARCHAR_TYPE, CHAR_TYPE, INTERVAL_YEAR_MONTH_TYPE, INTERVAL_DAY_TIME_TYPE, TIMESTAMPTZ_TYPE]).freeze
+  TIMESTAMPLOCALTZ_TYPE = 22
+  VALUE_MAP = {0 => "BOOLEAN_TYPE", 1 => "TINYINT_TYPE", 2 => "SMALLINT_TYPE", 3 => "INT_TYPE", 4 => "BIGINT_TYPE", 5 => "FLOAT_TYPE", 6 => "DOUBLE_TYPE", 7 => "STRING_TYPE", 8 => "TIMESTAMP_TYPE", 9 => "BINARY_TYPE", 10 => "ARRAY_TYPE", 11 => "MAP_TYPE", 12 => "STRUCT_TYPE", 13 => "UNION_TYPE", 14 => "USER_DEFINED_TYPE", 15 => "DECIMAL_TYPE", 16 => "NULL_TYPE", 17 => "DATE_TYPE", 18 => "VARCHAR_TYPE", 19 => "CHAR_TYPE", 20 => "INTERVAL_YEAR_MONTH_TYPE", 21 => "INTERVAL_DAY_TIME_TYPE", 22 => "TIMESTAMPLOCALTZ_TYPE"}
+  VALID_VALUES = Set.new([BOOLEAN_TYPE, TINYINT_TYPE, SMALLINT_TYPE, INT_TYPE, BIGINT_TYPE, FLOAT_TYPE, DOUBLE_TYPE, STRING_TYPE, TIMESTAMP_TYPE, BINARY_TYPE, ARRAY_TYPE, MAP_TYPE, STRUCT_TYPE, UNION_TYPE, USER_DEFINED_TYPE, DECIMAL_TYPE, NULL_TYPE, DATE_TYPE, VARCHAR_TYPE, CHAR_TYPE, INTERVAL_YEAR_MONTH_TYPE, INTERVAL_DAY_TIME_TYPE, TIMESTAMPLOCALTZ_TYPE]).freeze
 end
 
 module TStatusCode

http://git-wip-us.apache.org/repos/asf/hive/blob/34eebff1/service/src/java/org/apache/hive/service/cli/ColumnValue.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/cli/ColumnValue.java b/service/src/java/org/apache/hive/service/cli/ColumnValue.java
index 732bc9d..86dac23 100644
--- a/service/src/java/org/apache/hive/service/cli/ColumnValue.java
+++ b/service/src/java/org/apache/hive/service/cli/ColumnValue.java
@@ -201,7 +201,7 @@ public class ColumnValue {
       return dateValue((Date)value);
     case TIMESTAMP_TYPE:
       return timestampValue((Timestamp)value);
-    case TIMESTAMPTZ_TYPE:
+    case TIMESTAMPLOCALTZ_TYPE:
       return timestampTZValue((TimestampTZ) value);
     case INTERVAL_YEAR_MONTH_TYPE:
       return stringValue((HiveIntervalYearMonth) value);

http://git-wip-us.apache.org/repos/asf/hive/blob/34eebff1/service/src/java/org/apache/hive/service/cli/TypeDescriptor.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/cli/TypeDescriptor.java b/service/src/java/org/apache/hive/service/cli/TypeDescriptor.java
index ed903bc..7650898 100644
--- a/service/src/java/org/apache/hive/service/cli/TypeDescriptor.java
+++ b/service/src/java/org/apache/hive/service/cli/TypeDescriptor.java
@@ -116,7 +116,7 @@ public class TypeDescriptor {
       return 10;
     case TIMESTAMP_TYPE:
       return 29;
-    case TIMESTAMPTZ_TYPE:
+    case TIMESTAMPLOCALTZ_TYPE:
       return 31;
     default:
       return null;


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

Posted by jc...@apache.org.
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


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

Posted by jc...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/34eebff1/ql/src/test/results/clientpositive/localtimezone.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/localtimezone.q.out b/ql/src/test/results/clientpositive/localtimezone.q.out
new file mode 100644
index 0000000..3bca0b8
--- /dev/null
+++ b/ql/src/test/results/clientpositive/localtimezone.q.out
@@ -0,0 +1,558 @@
+PREHOOK: query: drop table `date_test`
+PREHOOK: type: DROPTABLE
+POSTHOOK: query: drop table `date_test`
+POSTHOOK: type: DROPTABLE
+PREHOOK: query: drop table `timestamp_test`
+PREHOOK: type: DROPTABLE
+POSTHOOK: query: drop table `timestamp_test`
+POSTHOOK: type: DROPTABLE
+PREHOOK: query: drop table `timestamptz_test`
+PREHOOK: type: DROPTABLE
+POSTHOOK: query: drop table `timestamptz_test`
+POSTHOOK: type: DROPTABLE
+PREHOOK: query: create table `date_test` (`mydate1` date)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@date_test
+POSTHOOK: query: create table `date_test` (`mydate1` date)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@date_test
+PREHOOK: query: 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')
+PREHOOK: type: QUERY
+PREHOOK: Output: default@date_test
+POSTHOOK: query: 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')
+POSTHOOK: type: QUERY
+POSTHOOK: Output: default@date_test
+POSTHOOK: Lineage: date_test.mydate1 EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col1, type:string, comment:), ]
+PREHOOK: query: create table `timestamp_test` (`mydate1` timestamp)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@timestamp_test
+POSTHOOK: query: create table `timestamp_test` (`mydate1` timestamp)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@timestamp_test
+PREHOOK: query: 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')
+PREHOOK: type: QUERY
+PREHOOK: Output: default@timestamp_test
+POSTHOOK: query: 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')
+POSTHOOK: type: QUERY
+POSTHOOK: Output: default@timestamp_test
+POSTHOOK: Lineage: timestamp_test.mydate1 EXPRESSION [(values__tmp__table__2)values__tmp__table__2.FieldSchema(name:tmp_values_col1, type:string, comment:), ]
+PREHOOK: query: create table `timestamptz_test` (`mydate1` timestamp with local time zone)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@timestamptz_test
+POSTHOOK: query: create table `timestamptz_test` (`mydate1` timestamp with local time zone)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@timestamptz_test
+PREHOOK: query: 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')
+PREHOOK: type: QUERY
+PREHOOK: Output: default@timestamptz_test
+POSTHOOK: query: 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')
+POSTHOOK: type: QUERY
+POSTHOOK: Output: default@timestamptz_test
+POSTHOOK: Lineage: timestamptz_test.mydate1 EXPRESSION [(values__tmp__table__3)values__tmp__table__3.FieldSchema(name:tmp_values_col1, type:string, comment:), ]
+PREHOOK: query: select * from `date_test`
+PREHOOK: type: QUERY
+PREHOOK: Input: default@date_test
+#### A masked pattern was here ####
+POSTHOOK: query: select * from `date_test`
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@date_test
+#### A masked pattern was here ####
+2011-01-01
+2010-12-31
+2010-12-31
+2011-01-01
+2010-12-31
+2010-12-31
+NULL
+PREHOOK: query: select * from `timestamp_test`
+PREHOOK: type: QUERY
+PREHOOK: Input: default@timestamp_test
+#### A masked pattern was here ####
+POSTHOOK: query: select * from `timestamp_test`
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@timestamp_test
+#### A masked pattern was here ####
+2011-01-01 01:01:01.123
+2010-12-31 16:01:01.123
+2010-12-31 22:01:01.123
+2011-01-01 01:01:01.123456789
+2010-12-31 16:01:01.123456789
+2010-12-31 22:01:01.123456789
+NULL
+PREHOOK: query: select * from `timestamptz_test`
+PREHOOK: type: QUERY
+PREHOOK: Input: default@timestamptz_test
+#### A masked pattern was here ####
+POSTHOOK: query: select * from `timestamptz_test`
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@timestamptz_test
+#### A masked pattern was here ####
+2011-01-01 01:01:01.123 US/Pacific
+2010-12-31 16:01:01.123 US/Pacific
+2010-12-31 22:01:01.123 US/Pacific
+2011-01-01 01:01:01.123456789 US/Pacific
+2010-12-31 16:01:01.123456789 US/Pacific
+2010-12-31 22:01:01.123456789 US/Pacific
+NULL
+PREHOOK: query: select * from `date_test`
+PREHOOK: type: QUERY
+PREHOOK: Input: default@date_test
+#### A masked pattern was here ####
+POSTHOOK: query: select * from `date_test`
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@date_test
+#### A masked pattern was here ####
+2011-01-01
+2010-12-31
+2010-12-31
+2011-01-01
+2010-12-31
+2010-12-31
+NULL
+PREHOOK: query: select * from `timestamp_test`
+PREHOOK: type: QUERY
+PREHOOK: Input: default@timestamp_test
+#### A masked pattern was here ####
+POSTHOOK: query: select * from `timestamp_test`
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@timestamp_test
+#### A masked pattern was here ####
+2011-01-01 01:01:01.123
+2010-12-31 16:01:01.123
+2010-12-31 22:01:01.123
+2011-01-01 01:01:01.123456789
+2010-12-31 16:01:01.123456789
+2010-12-31 22:01:01.123456789
+NULL
+PREHOOK: query: select * from `timestamptz_test`
+PREHOOK: type: QUERY
+PREHOOK: Input: default@timestamptz_test
+#### A masked pattern was here ####
+POSTHOOK: query: select * from `timestamptz_test`
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@timestamptz_test
+#### A masked pattern was here ####
+2011-01-01 10:01:01.123 Europe/Rome
+2011-01-01 01:01:01.123 Europe/Rome
+2011-01-01 07:01:01.123 Europe/Rome
+2011-01-01 10:01:01.123456789 Europe/Rome
+2011-01-01 01:01:01.123456789 Europe/Rome
+2011-01-01 07:01:01.123456789 Europe/Rome
+NULL
+PREHOOK: query: select * from `date_test`
+PREHOOK: type: QUERY
+PREHOOK: Input: default@date_test
+#### A masked pattern was here ####
+POSTHOOK: query: select * from `date_test`
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@date_test
+#### A masked pattern was here ####
+2011-01-01
+2010-12-31
+2010-12-31
+2011-01-01
+2010-12-31
+2010-12-31
+NULL
+PREHOOK: query: select * from `timestamp_test`
+PREHOOK: type: QUERY
+PREHOOK: Input: default@timestamp_test
+#### A masked pattern was here ####
+POSTHOOK: query: select * from `timestamp_test`
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@timestamp_test
+#### A masked pattern was here ####
+2011-01-01 01:01:01.123
+2010-12-31 16:01:01.123
+2010-12-31 22:01:01.123
+2011-01-01 01:01:01.123456789
+2010-12-31 16:01:01.123456789
+2010-12-31 22:01:01.123456789
+NULL
+PREHOOK: query: select * from `timestamptz_test`
+PREHOOK: type: QUERY
+PREHOOK: Input: default@timestamptz_test
+#### A masked pattern was here ####
+POSTHOOK: query: select * from `timestamptz_test`
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@timestamptz_test
+#### A masked pattern was here ####
+2011-01-01 01:01:01.123 America/Los_Angeles
+2010-12-31 16:01:01.123 America/Los_Angeles
+2010-12-31 22:01:01.123 America/Los_Angeles
+2011-01-01 01:01:01.123456789 America/Los_Angeles
+2010-12-31 16:01:01.123456789 America/Los_Angeles
+2010-12-31 22:01:01.123456789 America/Los_Angeles
+NULL
+PREHOOK: query: select * from `date_test`
+PREHOOK: type: QUERY
+PREHOOK: Input: default@date_test
+#### A masked pattern was here ####
+POSTHOOK: query: select * from `date_test`
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@date_test
+#### A masked pattern was here ####
+2011-01-01
+2010-12-31
+2010-12-31
+2011-01-01
+2010-12-31
+2010-12-31
+NULL
+PREHOOK: query: select * from `timestamp_test`
+PREHOOK: type: QUERY
+PREHOOK: Input: default@timestamp_test
+#### A masked pattern was here ####
+POSTHOOK: query: select * from `timestamp_test`
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@timestamp_test
+#### A masked pattern was here ####
+2011-01-01 01:01:01.123
+2010-12-31 16:01:01.123
+2010-12-31 22:01:01.123
+2011-01-01 01:01:01.123456789
+2010-12-31 16:01:01.123456789
+2010-12-31 22:01:01.123456789
+NULL
+PREHOOK: query: select * from `timestamptz_test`
+PREHOOK: type: QUERY
+PREHOOK: Input: default@timestamptz_test
+#### A masked pattern was here ####
+POSTHOOK: query: select * from `timestamptz_test`
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@timestamptz_test
+#### A masked pattern was here ####
+2011-01-01 02:01:01.123 GMT-07:00
+2010-12-31 17:01:01.123 GMT-07:00
+2010-12-31 23:01:01.123 GMT-07:00
+2011-01-01 02:01:01.123456789 GMT-07:00
+2010-12-31 17:01:01.123456789 GMT-07:00
+2010-12-31 23:01:01.123456789 GMT-07:00
+NULL
+PREHOOK: query: select extract(year from `mydate1`) from `timestamptz_test`
+PREHOOK: type: QUERY
+PREHOOK: Input: default@timestamptz_test
+#### A masked pattern was here ####
+POSTHOOK: query: select extract(year from `mydate1`) from `timestamptz_test`
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@timestamptz_test
+#### A masked pattern was here ####
+2011
+2010
+2010
+2011
+2010
+2010
+NULL
+PREHOOK: query: select extract(quarter from `mydate1`) from `timestamptz_test`
+PREHOOK: type: QUERY
+PREHOOK: Input: default@timestamptz_test
+#### A masked pattern was here ####
+POSTHOOK: query: select extract(quarter from `mydate1`) from `timestamptz_test`
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@timestamptz_test
+#### A masked pattern was here ####
+1
+4
+4
+1
+4
+4
+NULL
+PREHOOK: query: select extract(month from `mydate1`) from `timestamptz_test`
+PREHOOK: type: QUERY
+PREHOOK: Input: default@timestamptz_test
+#### A masked pattern was here ####
+POSTHOOK: query: select extract(month from `mydate1`) from `timestamptz_test`
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@timestamptz_test
+#### A masked pattern was here ####
+1
+12
+12
+1
+12
+12
+NULL
+PREHOOK: query: select extract(day from `mydate1`) from `timestamptz_test`
+PREHOOK: type: QUERY
+PREHOOK: Input: default@timestamptz_test
+#### A masked pattern was here ####
+POSTHOOK: query: select extract(day from `mydate1`) from `timestamptz_test`
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@timestamptz_test
+#### A masked pattern was here ####
+1
+31
+31
+1
+31
+31
+NULL
+PREHOOK: query: select extract(hour from `mydate1`) from `timestamptz_test`
+PREHOOK: type: QUERY
+PREHOOK: Input: default@timestamptz_test
+#### A masked pattern was here ####
+POSTHOOK: query: select extract(hour from `mydate1`) from `timestamptz_test`
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@timestamptz_test
+#### A masked pattern was here ####
+2
+17
+23
+2
+17
+23
+NULL
+PREHOOK: query: select extract(minute from `mydate1`) from `timestamptz_test`
+PREHOOK: type: QUERY
+PREHOOK: Input: default@timestamptz_test
+#### A masked pattern was here ####
+POSTHOOK: query: select extract(minute from `mydate1`) from `timestamptz_test`
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@timestamptz_test
+#### A masked pattern was here ####
+1
+1
+1
+1
+1
+1
+NULL
+PREHOOK: query: select extract(second from `mydate1`) from `timestamptz_test`
+PREHOOK: type: QUERY
+PREHOOK: Input: default@timestamptz_test
+#### A masked pattern was here ####
+POSTHOOK: query: select extract(second from `mydate1`) from `timestamptz_test`
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@timestamptz_test
+#### A masked pattern was here ####
+1
+1
+1
+1
+1
+1
+NULL
+PREHOOK: query: select cast(`mydate1` as date) from `timestamptz_test`
+PREHOOK: type: QUERY
+PREHOOK: Input: default@timestamptz_test
+#### A masked pattern was here ####
+POSTHOOK: query: select cast(`mydate1` as date) from `timestamptz_test`
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@timestamptz_test
+#### A masked pattern was here ####
+2011-01-01
+2010-12-31
+2010-12-31
+2011-01-01
+2010-12-31
+2010-12-31
+NULL
+PREHOOK: query: select cast(`mydate1` as timestamp with local time zone) from `date_test`
+PREHOOK: type: QUERY
+PREHOOK: Input: default@date_test
+#### A masked pattern was here ####
+POSTHOOK: query: select cast(`mydate1` as timestamp with local time zone) from `date_test`
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@date_test
+#### A masked pattern was here ####
+2011-01-01 00:00:00.0 GMT-07:00
+2010-12-31 00:00:00.0 GMT-07:00
+2010-12-31 00:00:00.0 GMT-07:00
+2011-01-01 00:00:00.0 GMT-07:00
+2010-12-31 00:00:00.0 GMT-07:00
+2010-12-31 00:00:00.0 GMT-07:00
+NULL
+PREHOOK: query: select cast(`mydate1` as timestamp) from `timestamptz_test`
+PREHOOK: type: QUERY
+PREHOOK: Input: default@timestamptz_test
+#### A masked pattern was here ####
+POSTHOOK: query: select cast(`mydate1` as timestamp) from `timestamptz_test`
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@timestamptz_test
+#### A masked pattern was here ####
+2011-01-01 02:01:01.123
+2010-12-31 17:01:01.123
+2010-12-31 23:01:01.123
+2011-01-01 02:01:01.123456789
+2010-12-31 17:01:01.123456789
+2010-12-31 23:01:01.123456789
+NULL
+PREHOOK: query: select cast(`mydate1` as timestamp with local time zone) from `timestamp_test`
+PREHOOK: type: QUERY
+PREHOOK: Input: default@timestamp_test
+#### A masked pattern was here ####
+POSTHOOK: query: select cast(`mydate1` as timestamp with local time zone) from `timestamp_test`
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@timestamp_test
+#### A masked pattern was here ####
+2011-01-01 01:01:01.123 GMT-07:00
+2010-12-31 16:01:01.123 GMT-07:00
+2010-12-31 22:01:01.123 GMT-07:00
+2011-01-01 01:01:01.123456789 GMT-07:00
+2010-12-31 16:01:01.123456789 GMT-07:00
+2010-12-31 22:01:01.123456789 GMT-07:00
+NULL
+PREHOOK: query: select `mydate1` from `timestamptz_test` group by `mydate1`
+PREHOOK: type: QUERY
+PREHOOK: Input: default@timestamptz_test
+#### A masked pattern was here ####
+POSTHOOK: query: select `mydate1` from `timestamptz_test` group by `mydate1`
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@timestamptz_test
+#### A masked pattern was here ####
+NULL
+2010-12-31 17:01:01.123 GMT-07:00
+2010-12-31 17:01:01.123456789 GMT-07:00
+2010-12-31 23:01:01.123 GMT-07:00
+2010-12-31 23:01:01.123456789 GMT-07:00
+2011-01-01 02:01:01.123 GMT-07:00
+2011-01-01 02:01:01.123456789 GMT-07:00
+PREHOOK: query: select a.`mydate1` as c1, b.`mydate1` as c2
+from `timestamptz_test` a join `timestamptz_test` b
+on a.`mydate1` = b.`mydate1`
+PREHOOK: type: QUERY
+PREHOOK: Input: default@timestamptz_test
+#### A masked pattern was here ####
+POSTHOOK: query: select a.`mydate1` as c1, b.`mydate1` as c2
+from `timestamptz_test` a join `timestamptz_test` b
+on a.`mydate1` = b.`mydate1`
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@timestamptz_test
+#### A masked pattern was here ####
+2010-12-31 17:01:01.123 GMT-07:00	2010-12-31 17:01:01.123 GMT-07:00
+2010-12-31 17:01:01.123456789 GMT-07:00	2010-12-31 17:01:01.123456789 GMT-07:00
+2010-12-31 23:01:01.123 GMT-07:00	2010-12-31 23:01:01.123 GMT-07:00
+2010-12-31 23:01:01.123456789 GMT-07:00	2010-12-31 23:01:01.123456789 GMT-07:00
+2011-01-01 02:01:01.123 GMT-07:00	2011-01-01 02:01:01.123 GMT-07:00
+2011-01-01 02:01:01.123456789 GMT-07:00	2011-01-01 02:01:01.123456789 GMT-07:00
+PREHOOK: query: create table `timestamptz_test2` (`mydate1` timestamp with local time zone, `item` string, `price` double)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@timestamptz_test2
+POSTHOOK: query: create table `timestamptz_test2` (`mydate1` timestamp with local time zone, `item` string, `price` double)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@timestamptz_test2
+PREHOOK: query: 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)
+PREHOOK: type: QUERY
+PREHOOK: Output: default@timestamptz_test2
+POSTHOOK: query: 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)
+POSTHOOK: type: QUERY
+POSTHOOK: Output: default@timestamptz_test2
+POSTHOOK: Lineage: timestamptz_test2.item SIMPLE [(values__tmp__table__4)values__tmp__table__4.FieldSchema(name:tmp_values_col2, type:string, comment:), ]
+POSTHOOK: Lineage: timestamptz_test2.mydate1 EXPRESSION [(values__tmp__table__4)values__tmp__table__4.FieldSchema(name:tmp_values_col1, type:string, comment:), ]
+POSTHOOK: Lineage: timestamptz_test2.price EXPRESSION [(values__tmp__table__4)values__tmp__table__4.FieldSchema(name:tmp_values_col3, type:string, comment:), ]
+PREHOOK: query: select `item`, `price`,
+rank() over (partition by `mydate1` order by `price`) as r
+from `timestamptz_test2`
+PREHOOK: type: QUERY
+PREHOOK: Input: default@timestamptz_test2
+#### A masked pattern was here ####
+POSTHOOK: query: select `item`, `price`,
+rank() over (partition by `mydate1` order by `price`) as r
+from `timestamptz_test2`
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@timestamptz_test2
+#### A masked pattern was here ####
+cable	0.0	1
+keyboard 1	4.2	1
+hdmi	1.25	1
+laptop 2	10.0	2
+keyboard 2	3.9	1
+keyboard 3	3.99	2
+pin	NULL	1
+mouse 1	3.1	1
+laptop 1	9.2	2
+mouse 2	4.594	1
+PREHOOK: query: select `item`, `price`,
+rank() over (partition by cast(`mydate1` as date) order by `price`) as r
+from `timestamptz_test2`
+PREHOOK: type: QUERY
+PREHOOK: Input: default@timestamptz_test2
+#### A masked pattern was here ####
+POSTHOOK: query: select `item`, `price`,
+rank() over (partition by cast(`mydate1` as date) order by `price`) as r
+from `timestamptz_test2`
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@timestamptz_test2
+#### A masked pattern was here ####
+cable	0.0	1
+pin	NULL	1
+hdmi	1.25	2
+keyboard 2	3.9	3
+keyboard 3	3.99	4
+keyboard 1	4.2	5
+laptop 2	10.0	6
+mouse 1	3.1	1
+mouse 2	4.594	2
+laptop 1	9.2	3

http://git-wip-us.apache.org/repos/asf/hive/blob/34eebff1/ql/src/test/results/clientpositive/timestamptz.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/timestamptz.q.out b/ql/src/test/results/clientpositive/timestamptz.q.out
index 626fe92..31d2e71 100644
--- a/ql/src/test/results/clientpositive/timestamptz.q.out
+++ b/ql/src/test/results/clientpositive/timestamptz.q.out
@@ -1,6 +1,6 @@
-PREHOOK: query: explain select cast('2005-01-03 02:01:00 GMT' as timestamp with time zone)
+PREHOOK: query: explain select cast('2005-01-03 02:01:00 GMT' as timestamp with local time zone)
 PREHOOK: type: QUERY
-POSTHOOK: query: explain select cast('2005-01-03 02:01:00 GMT' as timestamp with time zone)
+POSTHOOK: query: explain select cast('2005-01-03 02:01:00 GMT' as timestamp with local time zone)
 POSTHOOK: type: QUERY
 STAGE DEPENDENCIES:
   Stage-0 is a root stage
@@ -15,23 +15,23 @@ STAGE PLANS:
           Row Limit Per Split: 1
           Statistics: Num rows: 1 Data size: 1 Basic stats: COMPLETE Column stats: COMPLETE
           Select Operator
-            expressions: 2005-01-03 02:01:00.0 Z (type: timestamp with time zone)
+            expressions: 2005-01-02 18:01:00.0 US/Pacific (type: timestamp with local time zone)
             outputColumnNames: _col0
             Statistics: Num rows: 1 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
             ListSink
 
-PREHOOK: query: select cast('2005-01-03 02:01:00 GMT' as timestamp with time zone)
+PREHOOK: query: select cast('2005-01-03 02:01:00 GMT' as timestamp with local time zone)
 PREHOOK: type: QUERY
 PREHOOK: Input: _dummy_database@_dummy_table
 #### A masked pattern was here ####
-POSTHOOK: query: select cast('2005-01-03 02:01:00 GMT' as timestamp with time zone)
+POSTHOOK: query: select cast('2005-01-03 02:01:00 GMT' as timestamp with local time zone)
 POSTHOOK: type: QUERY
 POSTHOOK: Input: _dummy_database@_dummy_table
 #### A masked pattern was here ####
-2005-01-03 02:01:00.0 Z
-PREHOOK: query: explain select cast('2016-01-03 12:26:34.0123 America/Los_Angeles' as timestamptz)
+2005-01-02 18:01:00.0 US/Pacific
+PREHOOK: query: explain select cast('2016-01-03 12:26:34.0123 America/Los_Angeles' as timestamplocaltz)
 PREHOOK: type: QUERY
-POSTHOOK: query: explain select cast('2016-01-03 12:26:34.0123 America/Los_Angeles' as timestamptz)
+POSTHOOK: query: explain select cast('2016-01-03 12:26:34.0123 America/Los_Angeles' as timestamplocaltz)
 POSTHOOK: type: QUERY
 STAGE DEPENDENCIES:
   Stage-0 is a root stage
@@ -46,23 +46,23 @@ STAGE PLANS:
           Row Limit Per Split: 1
           Statistics: Num rows: 1 Data size: 1 Basic stats: COMPLETE Column stats: COMPLETE
           Select Operator
-            expressions: 2016-01-03 20:26:34.0123 Z (type: timestamp with time zone)
+            expressions: 2016-01-03 12:26:34.0123 US/Pacific (type: timestamp with local time zone)
             outputColumnNames: _col0
             Statistics: Num rows: 1 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
             ListSink
 
-PREHOOK: query: select cast('2016-01-03 12:26:34.0123 America/Los_Angeles' as timestamptz)
+PREHOOK: query: select cast('2016-01-03 12:26:34.0123 America/Los_Angeles' as timestamplocaltz)
 PREHOOK: type: QUERY
 PREHOOK: Input: _dummy_database@_dummy_table
 #### A masked pattern was here ####
-POSTHOOK: query: select cast('2016-01-03 12:26:34.0123 America/Los_Angeles' as timestamptz)
+POSTHOOK: query: select cast('2016-01-03 12:26:34.0123 America/Los_Angeles' as timestamplocaltz)
 POSTHOOK: type: QUERY
 POSTHOOK: Input: _dummy_database@_dummy_table
 #### A masked pattern was here ####
-2016-01-03 20:26:34.0123 Z
-PREHOOK: query: explain select cast('2016-01-03Europe/London' as timestamptz)
+2016-01-03 12:26:34.0123 US/Pacific
+PREHOOK: query: explain select cast('2016-01-03Europe/London' as timestamplocaltz)
 PREHOOK: type: QUERY
-POSTHOOK: query: explain select cast('2016-01-03Europe/London' as timestamptz)
+POSTHOOK: query: explain select cast('2016-01-03Europe/London' as timestamplocaltz)
 POSTHOOK: type: QUERY
 STAGE DEPENDENCIES:
   Stage-0 is a root stage
@@ -77,23 +77,23 @@ STAGE PLANS:
           Row Limit Per Split: 1
           Statistics: Num rows: 1 Data size: 1 Basic stats: COMPLETE Column stats: COMPLETE
           Select Operator
-            expressions: 2016-01-03 00:00:00.0 Z (type: timestamp with time zone)
+            expressions: 2016-01-02 16:00:00.0 US/Pacific (type: timestamp with local time zone)
             outputColumnNames: _col0
             Statistics: Num rows: 1 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
             ListSink
 
-PREHOOK: query: select cast('2016-01-03Europe/London' as timestamptz)
+PREHOOK: query: select cast('2016-01-03Europe/London' as timestamplocaltz)
 PREHOOK: type: QUERY
 PREHOOK: Input: _dummy_database@_dummy_table
 #### A masked pattern was here ####
-POSTHOOK: query: select cast('2016-01-03Europe/London' as timestamptz)
+POSTHOOK: query: select cast('2016-01-03Europe/London' as timestamplocaltz)
 POSTHOOK: type: QUERY
 POSTHOOK: Input: _dummy_database@_dummy_table
 #### A masked pattern was here ####
-2016-01-03 00:00:00.0 Z
-PREHOOK: query: explain select cast('2016-01-03 13:34:56.38 +1:00' as timestamptz)
+2016-01-02 16:00:00.0 US/Pacific
+PREHOOK: query: explain select cast('2016-01-03 13:34:56.38 +1:00' as timestamplocaltz)
 PREHOOK: type: QUERY
-POSTHOOK: query: explain select cast('2016-01-03 13:34:56.38 +1:00' as timestamptz)
+POSTHOOK: query: explain select cast('2016-01-03 13:34:56.38 +1:00' as timestamplocaltz)
 POSTHOOK: type: QUERY
 STAGE DEPENDENCIES:
   Stage-0 is a root stage
@@ -108,17 +108,17 @@ STAGE PLANS:
           Row Limit Per Split: 1
           Statistics: Num rows: 1 Data size: 1 Basic stats: COMPLETE Column stats: COMPLETE
           Select Operator
-            expressions: 2016-01-03 12:34:56.38 Z (type: timestamp with time zone)
+            expressions: 2016-01-03 04:34:56.38 US/Pacific (type: timestamp with local time zone)
             outputColumnNames: _col0
             Statistics: Num rows: 1 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
             ListSink
 
-PREHOOK: query: select cast('2016-01-03 13:34:56.38 +1:00' as timestamptz)
+PREHOOK: query: select cast('2016-01-03 13:34:56.38 +1:00' as timestamplocaltz)
 PREHOOK: type: QUERY
 PREHOOK: Input: _dummy_database@_dummy_table
 #### A masked pattern was here ####
-POSTHOOK: query: select cast('2016-01-03 13:34:56.38 +1:00' as timestamptz)
+POSTHOOK: query: select cast('2016-01-03 13:34:56.38 +1:00' as timestamplocaltz)
 POSTHOOK: type: QUERY
 POSTHOOK: Input: _dummy_database@_dummy_table
 #### A masked pattern was here ####
-2016-01-03 12:34:56.38 Z
+2016-01-03 04:34:56.38 US/Pacific

http://git-wip-us.apache.org/repos/asf/hive/blob/34eebff1/ql/src/test/results/clientpositive/timestamptz_1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/timestamptz_1.q.out b/ql/src/test/results/clientpositive/timestamptz_1.q.out
index 75bbfac..4756033 100644
--- a/ql/src/test/results/clientpositive/timestamptz_1.q.out
+++ b/ql/src/test/results/clientpositive/timestamptz_1.q.out
@@ -2,19 +2,19 @@ PREHOOK: query: drop table tstz1
 PREHOOK: type: DROPTABLE
 POSTHOOK: query: drop table tstz1
 POSTHOOK: type: DROPTABLE
-PREHOOK: query: create table tstz1(t timestamp with time zone)
+PREHOOK: query: create table tstz1(t timestamp with local time zone)
 PREHOOK: type: CREATETABLE
 PREHOOK: Output: database:default
 PREHOOK: Output: default@tstz1
-POSTHOOK: query: create table tstz1(t timestamp with time zone)
+POSTHOOK: query: create table tstz1(t timestamp with local time zone)
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@tstz1
-PREHOOK: query: insert overwrite table tstz1 select cast('2016-01-03 12:26:34 America/Los_Angeles' as timestamp with time zone)
+PREHOOK: query: insert overwrite table tstz1 select cast('2016-01-03 12:26:34 America/Los_Angeles' as timestamp with local time zone)
 PREHOOK: type: QUERY
 PREHOOK: Input: _dummy_database@_dummy_table
 PREHOOK: Output: default@tstz1
-POSTHOOK: query: insert overwrite table tstz1 select cast('2016-01-03 12:26:34 America/Los_Angeles' as timestamp with time zone)
+POSTHOOK: query: insert overwrite table tstz1 select cast('2016-01-03 12:26:34 America/Los_Angeles' as timestamp with local time zone)
 POSTHOOK: type: QUERY
 POSTHOOK: Input: _dummy_database@_dummy_table
 POSTHOOK: Output: default@tstz1
@@ -27,7 +27,7 @@ POSTHOOK: query: select cast(t as string) from tstz1
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@tstz1
 #### A masked pattern was here ####
-2016-01-03 20:26:34.0 Z
+2016-01-03 12:26:34.0 US/Pacific
 PREHOOK: query: select cast(t as date) from tstz1
 PREHOOK: type: QUERY
 PREHOOK: Input: default@tstz1
@@ -45,7 +45,7 @@ POSTHOOK: query: select cast(t as timestamp) from tstz1
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@tstz1
 #### A masked pattern was here ####
-2016-01-03 20:26:34
+2016-01-03 12:26:34
 PREHOOK: query: insert overwrite table tstz1 select '2016-01-03 12:26:34.1 GMT'
 PREHOOK: type: QUERY
 PREHOOK: Input: _dummy_database@_dummy_table
@@ -63,7 +63,7 @@ POSTHOOK: query: select cast(t as string) from tstz1
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@tstz1
 #### A masked pattern was here ####
-2016-01-03 12:26:34.1 Z
+2016-01-03 04:26:34.1 US/Pacific
 PREHOOK: query: select cast(t as date) from tstz1
 PREHOOK: type: QUERY
 PREHOOK: Input: default@tstz1
@@ -81,7 +81,7 @@ POSTHOOK: query: select cast(t as timestamp) from tstz1
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@tstz1
 #### A masked pattern was here ####
-2016-01-03 12:26:34.1
+2016-01-03 04:26:34.1
 PREHOOK: query: insert overwrite table tstz1 select '2016-01-03 12:26:34.0123 Europe/London'
 PREHOOK: type: QUERY
 PREHOOK: Input: _dummy_database@_dummy_table
@@ -99,7 +99,7 @@ POSTHOOK: query: select cast(t as string) from tstz1
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@tstz1
 #### A masked pattern was here ####
-2016-01-03 12:26:34.0123 Z
+2016-01-03 04:26:34.0123 US/Pacific
 PREHOOK: query: select cast(t as date) from tstz1
 PREHOOK: type: QUERY
 PREHOOK: Input: default@tstz1
@@ -117,7 +117,7 @@ POSTHOOK: query: select cast(t as timestamp) from tstz1
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@tstz1
 #### A masked pattern was here ####
-2016-01-03 12:26:34.0123
+2016-01-03 04:26:34.0123
 PREHOOK: query: insert overwrite table tstz1 select '2016-01-03 12:26:34.012300 GMT+08:00'
 PREHOOK: type: QUERY
 PREHOOK: Input: _dummy_database@_dummy_table
@@ -135,7 +135,7 @@ POSTHOOK: query: select cast(t as string) from tstz1
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@tstz1
 #### A masked pattern was here ####
-2016-01-03 04:26:34.0123 Z
+2016-01-02 20:26:34.0123 US/Pacific
 PREHOOK: query: select cast(t as date) from tstz1
 PREHOOK: type: QUERY
 PREHOOK: Input: default@tstz1
@@ -144,7 +144,7 @@ POSTHOOK: query: select cast(t as date) from tstz1
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@tstz1
 #### A masked pattern was here ####
-2016-01-03
+2016-01-02
 PREHOOK: query: select cast(t as timestamp) from tstz1
 PREHOOK: type: QUERY
 PREHOOK: Input: default@tstz1
@@ -153,4 +153,4 @@ POSTHOOK: query: select cast(t as timestamp) from tstz1
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@tstz1
 #### A masked pattern was here ####
-2016-01-03 04:26:34.0123
+2016-01-02 20:26:34.0123

http://git-wip-us.apache.org/repos/asf/hive/blob/34eebff1/ql/src/test/results/clientpositive/timestamptz_2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/timestamptz_2.q.out b/ql/src/test/results/clientpositive/timestamptz_2.q.out
index 2666735..8c92f63 100644
--- a/ql/src/test/results/clientpositive/timestamptz_2.q.out
+++ b/ql/src/test/results/clientpositive/timestamptz_2.q.out
@@ -2,11 +2,11 @@ PREHOOK: query: drop table tstz2
 PREHOOK: type: DROPTABLE
 POSTHOOK: query: drop table tstz2
 POSTHOOK: type: DROPTABLE
-PREHOOK: query: create table tstz2(t timestamp with time zone)
+PREHOOK: query: create table tstz2(t timestamp with local time zone)
 PREHOOK: type: CREATETABLE
 PREHOOK: Output: database:default
 PREHOOK: Output: default@tstz2
-POSTHOOK: query: create table tstz2(t timestamp with time zone)
+POSTHOOK: query: create table tstz2(t timestamp with local time zone)
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@tstz2
@@ -29,8 +29,8 @@ POSTHOOK: query: select * from tstz2 where t='2005-01-02 19:01:00 GMT-07:00'
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@tstz2
 #### A masked pattern was here ####
-2005-01-03 02:01:00.0 Z
-2005-01-03 02:01:00.0 Z
+2005-01-03 02:01:00.0 UTC
+2005-01-03 02:01:00.0 UTC
 PREHOOK: query: select * from tstz2 where t>'2013-06-03 02:01:00.30547 GMT+01:00'
 PREHOOK: type: QUERY
 PREHOOK: Input: default@tstz2
@@ -39,7 +39,7 @@ POSTHOOK: query: select * from tstz2 where t>'2013-06-03 02:01:00.30547 GMT+01:0
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@tstz2
 #### A masked pattern was here ####
-2016-01-03 04:26:34.0123 Z
+2016-01-03 04:26:34.0123 UTC
 PREHOOK: query: select min(t),max(t) from tstz2
 PREHOOK: type: QUERY
 PREHOOK: Input: default@tstz2
@@ -48,7 +48,7 @@ POSTHOOK: query: select min(t),max(t) from tstz2
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@tstz2
 #### A masked pattern was here ####
-2005-01-03 02:01:00.0 Z	2016-01-03 04:26:34.0123 Z
+2005-01-03 02:01:00.0 UTC	2016-01-03 04:26:34.0123 UTC
 PREHOOK: query: select t from tstz2 group by t order by t
 PREHOOK: type: QUERY
 PREHOOK: Input: default@tstz2
@@ -57,10 +57,10 @@ POSTHOOK: query: select t from tstz2 group by t order by t
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@tstz2
 #### A masked pattern was here ####
-2005-01-03 02:01:00.0 Z
-2005-04-03 10:01:00.04067 Z
-2013-06-03 01:01:00.30547 Z
-2016-01-03 04:26:34.0123 Z
+2005-01-03 02:01:00.0 UTC
+2005-04-03 10:01:00.04067 UTC
+2013-06-03 01:01:00.30547 UTC
+2016-01-03 04:26:34.0123 UTC
 PREHOOK: query: select * from tstz2 a join tstz2 b on a.t=b.t order by a.t
 PREHOOK: type: QUERY
 PREHOOK: Input: default@tstz2
@@ -69,10 +69,10 @@ POSTHOOK: query: select * from tstz2 a join tstz2 b on a.t=b.t order by a.t
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@tstz2
 #### A masked pattern was here ####
-2005-01-03 02:01:00.0 Z	2005-01-03 02:01:00.0 Z
-2005-01-03 02:01:00.0 Z	2005-01-03 02:01:00.0 Z
-2005-01-03 02:01:00.0 Z	2005-01-03 02:01:00.0 Z
-2005-01-03 02:01:00.0 Z	2005-01-03 02:01:00.0 Z
-2005-04-03 10:01:00.04067 Z	2005-04-03 10:01:00.04067 Z
-2013-06-03 01:01:00.30547 Z	2013-06-03 01:01:00.30547 Z
-2016-01-03 04:26:34.0123 Z	2016-01-03 04:26:34.0123 Z
+2005-01-03 02:01:00.0 UTC	2005-01-03 02:01:00.0 UTC
+2005-01-03 02:01:00.0 UTC	2005-01-03 02:01:00.0 UTC
+2005-01-03 02:01:00.0 UTC	2005-01-03 02:01:00.0 UTC
+2005-01-03 02:01:00.0 UTC	2005-01-03 02:01:00.0 UTC
+2005-04-03 10:01:00.04067 UTC	2005-04-03 10:01:00.04067 UTC
+2013-06-03 01:01:00.30547 UTC	2013-06-03 01:01:00.30547 UTC
+2016-01-03 04:26:34.0123 UTC	2016-01-03 04:26:34.0123 UTC

http://git-wip-us.apache.org/repos/asf/hive/blob/34eebff1/serde/if/serde.thrift
----------------------------------------------------------------------
diff --git a/serde/if/serde.thrift b/serde/if/serde.thrift
index c331242..cbdd604 100644
--- a/serde/if/serde.thrift
+++ b/serde/if/serde.thrift
@@ -64,7 +64,7 @@ const string DECIMAL_TYPE_NAME   = "decimal";
 const string BINARY_TYPE_NAME    = "binary";
 const string INTERVAL_YEAR_MONTH_TYPE_NAME = "interval_year_month";
 const string INTERVAL_DAY_TIME_TYPE_NAME   = "interval_day_time";
-const string TIMESTAMPTZ_TYPE_NAME = "timestamp with time zone";
+const string TIMESTAMPLOCALTZ_TYPE_NAME = "timestamp with local time zone";
 
 const string LIST_TYPE_NAME = "array";
 const string MAP_TYPE_NAME  = "map";
@@ -96,7 +96,7 @@ const set<string> PrimitiveTypes  = [
   INTERVAL_DAY_TIME_TYPE_NAME
   DECIMAL_TYPE_NAME
   BINARY_TYPE_NAME
-  TIMESTAMPTZ_TYPE_NAME
+  TIMESTAMPLOCALTZ_TYPE_NAME
 ],
 
 const set<string> CollectionTypes = [ LIST_TYPE_NAME MAP_TYPE_NAME ],

http://git-wip-us.apache.org/repos/asf/hive/blob/34eebff1/serde/src/gen/thrift/gen-cpp/serde_constants.cpp
----------------------------------------------------------------------
diff --git a/serde/src/gen/thrift/gen-cpp/serde_constants.cpp b/serde/src/gen/thrift/gen-cpp/serde_constants.cpp
index 5674b5e..88cb4ee 100644
--- a/serde/src/gen/thrift/gen-cpp/serde_constants.cpp
+++ b/serde/src/gen/thrift/gen-cpp/serde_constants.cpp
@@ -85,7 +85,7 @@ serdeConstants::serdeConstants() {
 
   INTERVAL_DAY_TIME_TYPE_NAME = "interval_day_time";
 
-  TIMESTAMPTZ_TYPE_NAME = "timestamp with time zone";
+  TIMESTAMPLOCALTZ_TYPE_NAME = "timestamp with local time zone";
 
   LIST_TYPE_NAME = "array";
 
@@ -121,7 +121,7 @@ serdeConstants::serdeConstants() {
   PrimitiveTypes.insert("interval_day_time");
   PrimitiveTypes.insert("decimal");
   PrimitiveTypes.insert("binary");
-  PrimitiveTypes.insert("timestamp with time zone");
+  PrimitiveTypes.insert("timestamp with local time zone");
 
   CollectionTypes.insert("array");
   CollectionTypes.insert("map");

http://git-wip-us.apache.org/repos/asf/hive/blob/34eebff1/serde/src/gen/thrift/gen-cpp/serde_constants.h
----------------------------------------------------------------------
diff --git a/serde/src/gen/thrift/gen-cpp/serde_constants.h b/serde/src/gen/thrift/gen-cpp/serde_constants.h
index e261ed3..12b04b3 100644
--- a/serde/src/gen/thrift/gen-cpp/serde_constants.h
+++ b/serde/src/gen/thrift/gen-cpp/serde_constants.h
@@ -52,7 +52,7 @@ class serdeConstants {
   std::string BINARY_TYPE_NAME;
   std::string INTERVAL_YEAR_MONTH_TYPE_NAME;
   std::string INTERVAL_DAY_TIME_TYPE_NAME;
-  std::string TIMESTAMPTZ_TYPE_NAME;
+  std::string TIMESTAMPLOCALTZ_TYPE_NAME;
   std::string LIST_TYPE_NAME;
   std::string MAP_TYPE_NAME;
   std::string STRUCT_TYPE_NAME;

http://git-wip-us.apache.org/repos/asf/hive/blob/34eebff1/serde/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/serde/serdeConstants.java
----------------------------------------------------------------------
diff --git a/serde/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/serde/serdeConstants.java b/serde/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/serde/serdeConstants.java
index 4da1d3b..62877f9 100644
--- a/serde/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/serde/serdeConstants.java
+++ b/serde/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/serde/serdeConstants.java
@@ -110,7 +110,7 @@ public class serdeConstants {
 
   public static final String INTERVAL_DAY_TIME_TYPE_NAME = "interval_day_time";
 
-  public static final String TIMESTAMPTZ_TYPE_NAME = "timestamp with time zone";
+  public static final String TIMESTAMPLOCALTZ_TYPE_NAME = "timestamp with local time zone";
 
   public static final String LIST_TYPE_NAME = "array";
 
@@ -148,7 +148,7 @@ public class serdeConstants {
     PrimitiveTypes.add("interval_day_time");
     PrimitiveTypes.add("decimal");
     PrimitiveTypes.add("binary");
-    PrimitiveTypes.add("timestamp with time zone");
+    PrimitiveTypes.add("timestamp with local time zone");
   }
 
   public static final Set<String> CollectionTypes = new HashSet<String>();

http://git-wip-us.apache.org/repos/asf/hive/blob/34eebff1/serde/src/gen/thrift/gen-php/org/apache/hadoop/hive/serde/Types.php
----------------------------------------------------------------------
diff --git a/serde/src/gen/thrift/gen-php/org/apache/hadoop/hive/serde/Types.php b/serde/src/gen/thrift/gen-php/org/apache/hadoop/hive/serde/Types.php
index 1a2e8d7..fe32003 100644
--- a/serde/src/gen/thrift/gen-php/org/apache/hadoop/hive/serde/Types.php
+++ b/serde/src/gen/thrift/gen-php/org/apache/hadoop/hive/serde/Types.php
@@ -55,7 +55,7 @@ final class Constant extends \Thrift\Type\TConstant {
   static protected $BINARY_TYPE_NAME;
   static protected $INTERVAL_YEAR_MONTH_TYPE_NAME;
   static protected $INTERVAL_DAY_TIME_TYPE_NAME;
-  static protected $TIMESTAMPTZ_TYPE_NAME;
+  static protected $TIMESTAMPLOCALTZ_TYPE_NAME;
   static protected $LIST_TYPE_NAME;
   static protected $MAP_TYPE_NAME;
   static protected $STRUCT_TYPE_NAME;
@@ -216,8 +216,8 @@ final class Constant extends \Thrift\Type\TConstant {
     return "interval_day_time";
   }
 
-  static protected function init_TIMESTAMPTZ_TYPE_NAME() {
-    return "timestamp with time zone";
+  static protected function init_TIMESTAMPLOCALTZ_TYPE_NAME() {
+    return "timestamp with local time zone";
   }
 
   static protected function init_LIST_TYPE_NAME() {
@@ -272,7 +272,7 @@ final class Constant extends \Thrift\Type\TConstant {
       "interval_day_time" => true,
       "decimal" => true,
       "binary" => true,
-      "timestamp with time zone" => true,
+      "timestamp with local time zone" => true,
     );
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/34eebff1/serde/src/gen/thrift/gen-py/org_apache_hadoop_hive_serde/constants.py
----------------------------------------------------------------------
diff --git a/serde/src/gen/thrift/gen-py/org_apache_hadoop_hive_serde/constants.py b/serde/src/gen/thrift/gen-py/org_apache_hadoop_hive_serde/constants.py
index 79ff525..1e5c919 100644
--- a/serde/src/gen/thrift/gen-py/org_apache_hadoop_hive_serde/constants.py
+++ b/serde/src/gen/thrift/gen-py/org_apache_hadoop_hive_serde/constants.py
@@ -46,7 +46,7 @@ DECIMAL_TYPE_NAME = "decimal"
 BINARY_TYPE_NAME = "binary"
 INTERVAL_YEAR_MONTH_TYPE_NAME = "interval_year_month"
 INTERVAL_DAY_TIME_TYPE_NAME = "interval_day_time"
-TIMESTAMPTZ_TYPE_NAME = "timestamp with time zone"
+TIMESTAMPLOCALTZ_TYPE_NAME = "timestamp with local time zone"
 LIST_TYPE_NAME = "array"
 MAP_TYPE_NAME = "map"
 STRUCT_TYPE_NAME = "struct"
@@ -74,7 +74,7 @@ PrimitiveTypes = set([
   "interval_day_time",
   "decimal",
   "binary",
-  "timestamp with time zone",
+  "timestamp with local time zone",
 ])
 CollectionTypes = set([
   "array",

http://git-wip-us.apache.org/repos/asf/hive/blob/34eebff1/serde/src/gen/thrift/gen-rb/serde_constants.rb
----------------------------------------------------------------------
diff --git a/serde/src/gen/thrift/gen-rb/serde_constants.rb b/serde/src/gen/thrift/gen-rb/serde_constants.rb
index 2810324..d6efcbb 100644
--- a/serde/src/gen/thrift/gen-rb/serde_constants.rb
+++ b/serde/src/gen/thrift/gen-rb/serde_constants.rb
@@ -81,7 +81,7 @@ INTERVAL_YEAR_MONTH_TYPE_NAME = %q"interval_year_month"
 
 INTERVAL_DAY_TIME_TYPE_NAME = %q"interval_day_time"
 
-TIMESTAMPTZ_TYPE_NAME = %q"timestamp with time zone"
+TIMESTAMPLOCALTZ_TYPE_NAME = %q"timestamp with local time zone"
 
 LIST_TYPE_NAME = %q"array"
 
@@ -118,7 +118,7 @@ PrimitiveTypes = Set.new([
   %q"interval_day_time",
   %q"decimal",
   %q"binary",
-  %q"timestamp with time zone",
+  %q"timestamp with local time zone",
 ])
 
 CollectionTypes = Set.new([

http://git-wip-us.apache.org/repos/asf/hive/blob/34eebff1/serde/src/java/org/apache/hadoop/hive/serde2/SerDeUtils.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/SerDeUtils.java b/serde/src/java/org/apache/hadoop/hive/serde2/SerDeUtils.java
index b9d23dd..244ff19 100644
--- a/serde/src/java/org/apache/hadoop/hive/serde2/SerDeUtils.java
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/SerDeUtils.java
@@ -49,7 +49,7 @@ import org.apache.hadoop.hive.serde2.objectinspector.primitive.LongObjectInspect
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.ShortObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.StringObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.TimestampObjectInspector;
-import org.apache.hadoop.hive.serde2.objectinspector.primitive.TimestampTZObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.TimestampLocalTZObjectInspector;
 import org.apache.hadoop.io.BytesWritable;
 import org.apache.hadoop.io.Text;
 import org.slf4j.Logger;
@@ -277,9 +277,9 @@ public final class SerDeUtils {
           sb.append('"');
           break;
         }
-        case TIMESTAMPTZ: {
+        case TIMESTAMPLOCALTZ: {
           sb.append('"');
-          sb.append(((TimestampTZObjectInspector) poi).getPrimitiveWritableObject(o));
+          sb.append(((TimestampLocalTZObjectInspector) poi).getPrimitiveWritableObject(o));
           sb.append('"');
           break;
         }

http://git-wip-us.apache.org/repos/asf/hive/blob/34eebff1/serde/src/java/org/apache/hadoop/hive/serde2/binarysortable/BinarySortableSerDe.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/binarysortable/BinarySortableSerDe.java b/serde/src/java/org/apache/hadoop/hive/serde2/binarysortable/BinarySortableSerDe.java
index f333ae9..6bfe8a9 100644
--- a/serde/src/java/org/apache/hadoop/hive/serde2/binarysortable/BinarySortableSerDe.java
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/binarysortable/BinarySortableSerDe.java
@@ -51,7 +51,7 @@ import org.apache.hadoop.hive.serde2.io.HiveIntervalDayTimeWritable;
 import org.apache.hadoop.hive.serde2.io.HiveIntervalYearMonthWritable;
 import org.apache.hadoop.hive.serde2.io.HiveVarcharWritable;
 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.objectinspector.ListObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.MapObjectInspector;
@@ -77,12 +77,13 @@ import org.apache.hadoop.hive.serde2.objectinspector.primitive.LongObjectInspect
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.ShortObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.StringObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.TimestampObjectInspector;
-import org.apache.hadoop.hive.serde2.objectinspector.primitive.TimestampTZObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.TimestampLocalTZObjectInspector;
 import org.apache.hadoop.hive.serde2.typeinfo.BaseCharTypeInfo;
 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;
@@ -422,14 +423,16 @@ public class BinarySortableSerDe extends AbstractSerDe {
         }
         t.setBinarySortable(bytes, 0);
         return t;
-      case TIMESTAMPTZ:
-        TimestampTZWritable tstz = (reuse == null ? new TimestampTZWritable() :
-            (TimestampTZWritable) reuse);
-        byte[] data = new byte[TimestampTZWritable.BINARY_SORTABLE_LENGTH];
+      case TIMESTAMPLOCALTZ:
+        TimestampLocalTZWritable tstz = (reuse == null ? new TimestampLocalTZWritable() :
+            (TimestampLocalTZWritable) reuse);
+        byte[] data = new byte[TimestampLocalTZWritable.BINARY_SORTABLE_LENGTH];
         for (int i = 0; i < data.length; i++) {
           data[i] = buffer.read(invert);
         }
-        tstz.fromBinarySortable(data, 0);
+        // Across MR process boundary tz is normalized and stored in type
+        // and is not carried in data for each row.
+        tstz.fromBinarySortable(data, 0, ((TimestampLocalTZTypeInfo) type).timeZone());
         return tstz;
       case INTERVAL_YEAR_MONTH: {
         HiveIntervalYearMonthWritable i = reuse == null ? new HiveIntervalYearMonthWritable()
@@ -798,9 +801,9 @@ public class BinarySortableSerDe extends AbstractSerDe {
         serializeTimestampWritable(buffer, t, invert);
         return;
       }
-      case TIMESTAMPTZ: {
-        TimestampTZObjectInspector toi = (TimestampTZObjectInspector) poi;
-        TimestampTZWritable t = toi.getPrimitiveWritableObject(o);
+      case TIMESTAMPLOCALTZ: {
+        TimestampLocalTZObjectInspector toi = (TimestampLocalTZObjectInspector) poi;
+        TimestampLocalTZWritable t = toi.getPrimitiveWritableObject(o);
         serializeTimestampTZWritable(buffer, t, invert);
         return;
       }
@@ -975,7 +978,7 @@ public class BinarySortableSerDe extends AbstractSerDe {
   }
 
   public static void serializeTimestampTZWritable(
-      ByteStream.Output buffer, TimestampTZWritable t, boolean invert) {
+      ByteStream.Output buffer, TimestampLocalTZWritable t, boolean invert) {
     byte[] data = t.toBinarySortable();
     for (byte b : data) {
       writeByte(buffer, b, invert);

http://git-wip-us.apache.org/repos/asf/hive/blob/34eebff1/serde/src/java/org/apache/hadoop/hive/serde2/io/TimestampLocalTZWritable.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/io/TimestampLocalTZWritable.java b/serde/src/java/org/apache/hadoop/hive/serde2/io/TimestampLocalTZWritable.java
new file mode 100644
index 0000000..f0bb0ec
--- /dev/null
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/io/TimestampLocalTZWritable.java
@@ -0,0 +1,449 @@
+/**
+ * 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.serde2.io;
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.hive.common.type.TimestampTZ;
+import org.apache.hadoop.hive.serde2.ByteStream;
+import org.apache.hadoop.hive.serde2.lazybinary.LazyBinaryUtils;
+import org.apache.hadoop.io.WritableComparable;
+import org.apache.hadoop.io.WritableUtils;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.time.ZoneId;
+import java.util.Arrays;
+
+/**
+ * Writable for TimestampTZ. Copied from TimestampWritable.
+ * After we replace {@link java.sql.Timestamp} with {@link java.time.LocalDateTime} for Timestamp,
+ * it'll need a new Writable.
+ * All timestamp with time zone will be serialized 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 TimestampLocalTZWritable implements WritableComparable<TimestampLocalTZWritable> {
+
+  public static final byte[] nullBytes = {0x0, 0x0, 0x0, 0x0};
+  private static final int DECIMAL_OR_SECOND_VINT_FLAG = 1 << 31;
+  private static final long SEVEN_BYTE_LONG_SIGN_FLIP = 0xff80L << 48; // only need flip the MSB?
+
+  /**
+   * The maximum number of bytes required for a TimestampWritable
+   */
+  public static final int MAX_BYTES = 13;
+
+  public static final int BINARY_SORTABLE_LENGTH = 11;
+
+  private TimestampTZ timestampTZ = new TimestampTZ();
+  private ZoneId timeZone;
+
+  /**
+   * true if data is stored in timestamptz field rather than byte arrays.
+   * allows for lazy conversion to bytes when necessary
+   * false otherwise
+   */
+  private boolean bytesEmpty = true;
+  private boolean timestampTZEmpty = true;
+
+  /* Allow use of external byte[] for efficiency */
+  private byte[] currentBytes;
+  private final byte[] internalBytes = new byte[MAX_BYTES];
+  private byte[] externalBytes;
+  private int offset;
+
+  public TimestampLocalTZWritable() {
+    this.bytesEmpty = false;
+    this.currentBytes = internalBytes;
+    this.offset = 0;
+  }
+
+  public TimestampLocalTZWritable(byte[] bytes, int offset, ZoneId timeZone) {
+    set(bytes, offset, timeZone);
+  }
+
+  public TimestampLocalTZWritable(TimestampLocalTZWritable other) {
+    this(other.getBytes(), 0, other.getTimestampTZ().getZonedDateTime().getZone());
+  }
+
+  public TimestampLocalTZWritable(TimestampTZ tstz) {
+    set(tstz);
+  }
+
+  public void set(byte[] bytes, int offset, ZoneId timeZone) {
+    externalBytes = bytes;
+    this.offset = offset;
+    this.timeZone = timeZone;
+    bytesEmpty = false;
+    timestampTZEmpty = true;
+    currentBytes = externalBytes;
+  }
+
+  public void set(TimestampTZ tstz) {
+    if (tstz == null) {
+      timestampTZ.setZonedDateTime(null);
+      return;
+    }
+    timestampTZ = tstz;
+    timeZone = timestampTZ.getZonedDateTime().getZone();
+    bytesEmpty = true;
+    timestampTZEmpty = false;
+  }
+
+  public void set(TimestampLocalTZWritable t) {
+    if (t.bytesEmpty) {
+      set(t.getTimestampTZ());
+    } else if (t.currentBytes == t.externalBytes) {
+      set(t.currentBytes, t.offset, t.timeZone);
+    } else {
+      set(t.currentBytes, 0, t.timeZone);
+    }
+  }
+
+  public void setTimeZone(ZoneId timeZone) {
+    if (timestampTZ != null) {
+      timestampTZ.setZonedDateTime(
+          timestampTZ.getZonedDateTime().withZoneSameInstant(timeZone));
+    }
+    this.timeZone = timeZone;
+  }
+
+  public ZoneId getTimeZone() {
+    return timeZone;
+  }
+
+  public TimestampTZ getTimestampTZ() {
+    populateTimestampTZ();
+    return timestampTZ;
+  }
+
+  /**
+   * Used to create copies of objects
+   *
+   * @return a copy of the internal TimestampTZWritable byte[]
+   */
+  public byte[] getBytes() {
+    checkBytes();
+
+    int len = getTotalLength();
+    byte[] b = new byte[len];
+
+    System.arraycopy(currentBytes, offset, b, 0, len);
+    return b;
+  }
+
+  /**
+   * @return length of serialized TimestampTZWritable data. As a side effect, populates the internal
+   * byte array if empty.
+   */
+  private int getTotalLength() {
+    checkBytes();
+    return getTotalLength(currentBytes, offset);
+  }
+
+  /**
+   * The data of TimestampTZWritable can be stored either in a byte[]
+   * or in a TimestampTZ object. Calling this method ensures that the byte[]
+   * is populated from the TimestampTZ object if previously empty.
+   */
+  private void checkBytes() {
+    if (bytesEmpty) {
+      populateBytes();
+      offset = 0;
+      currentBytes = internalBytes;
+      bytesEmpty = false;
+    }
+  }
+
+  // Writes the TimestampTZ's serialized value to the internal byte array.
+  private void populateBytes() {
+    Arrays.fill(internalBytes, (byte) 0);
+
+    long seconds = timestampTZ.getEpochSecond();
+    int nanos = timestampTZ.getNanos();
+
+    boolean hasSecondVInt = seconds < 0 || seconds > Integer.MAX_VALUE;
+    boolean hasDecimal = setNanosBytes(nanos, internalBytes, offset + 4, hasSecondVInt);
+
+    int firstInt = (int) seconds;
+    if (hasDecimal || hasSecondVInt) {
+      firstInt |= DECIMAL_OR_SECOND_VINT_FLAG;
+    }
+    intToBytes(firstInt, internalBytes, offset);
+    if (hasSecondVInt) {
+      LazyBinaryUtils.writeVLongToByteArray(internalBytes,
+          offset + 4 + WritableUtils.decodeVIntSize(internalBytes[offset + 4]),
+          seconds >> 31);
+    }
+  }
+
+  private void populateTimestampTZ() {
+    if (timestampTZEmpty) {
+      if (bytesEmpty) {
+        throw new IllegalStateException("Bytes are empty");
+      }
+      long seconds = getSeconds(currentBytes, offset);
+      int nanos = hasDecimalOrSecondVInt(currentBytes[offset]) ? getNanos(currentBytes, offset + 4) : 0;
+      timestampTZ.set(seconds, nanos, timeZone);
+      timestampTZEmpty = false;
+    }
+  }
+
+  public long getSeconds() {
+    if (!timestampTZEmpty) {
+      return timestampTZ.getEpochSecond();
+    } else if (!bytesEmpty) {
+      return getSeconds(currentBytes, offset);
+    }
+    throw new IllegalStateException("Both timestamp and bytes are empty");
+  }
+
+  public int getNanos() {
+    if (!timestampTZEmpty) {
+      return timestampTZ.getNanos();
+    } else if (!bytesEmpty) {
+      return hasDecimalOrSecondVInt(currentBytes[offset]) ? getNanos(currentBytes, offset + 4) : 0;
+    }
+    throw new IllegalStateException("Both timestamp and bytes are empty");
+  }
+
+  @Override
+  public int compareTo(TimestampLocalTZWritable o) {
+    return getTimestampTZ().compareTo(o.getTimestampTZ());
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (o instanceof TimestampLocalTZWritable) {
+      return compareTo((TimestampLocalTZWritable) o) == 0;
+    }
+    return false;
+  }
+
+  @Override
+  public int hashCode() {
+    return getTimestampTZ().hashCode();
+  }
+
+  @Override
+  public String toString() {
+    populateTimestampTZ();
+    return timestampTZ.toString();
+  }
+
+  @Override
+  public void write(DataOutput dataOutput) throws IOException {
+    checkBytes();
+    dataOutput.write(currentBytes, offset, getTotalLength());
+  }
+
+  @Override
+  public void readFields(DataInput dataInput) throws IOException {
+    dataInput.readFully(internalBytes, 0, 4);
+    if (hasDecimalOrSecondVInt(internalBytes[0])) {
+      dataInput.readFully(internalBytes, 4, 1);
+      int len = (byte) WritableUtils.decodeVIntSize(internalBytes[4]);
+      if (len > 1) {
+        dataInput.readFully(internalBytes, 5, len - 1);
+      }
+
+      long vlong = LazyBinaryUtils.readVLongFromByteArray(internalBytes, 4);
+      Preconditions.checkState(vlong >= -1000000000 && vlong <= 999999999,
+          "Invalid nanos value for a TimestampTZWritable: " + vlong +
+              ", expected to be between -1000000000 and 999999999.");
+      if (vlong < 0) {
+        // This indicates there is a second VInt containing the additional bits of the seconds
+        // field.
+        dataInput.readFully(internalBytes, 4 + len, 1);
+        int secondVIntLen = (byte) WritableUtils.decodeVIntSize(internalBytes[4 + len]);
+        if (secondVIntLen > 1) {
+          dataInput.readFully(internalBytes, 5 + len, secondVIntLen - 1);
+        }
+      }
+    }
+    currentBytes = internalBytes;
+    offset = 0;
+    timestampTZEmpty = true;
+    bytesEmpty = false;
+  }
+
+  public byte[] toBinarySortable() {
+    byte[] b = new byte[BINARY_SORTABLE_LENGTH];
+    int nanos = getNanos();
+    // We flip the highest-order bit of the seven-byte representation of seconds to make negative
+    // values come before positive ones.
+    long seconds = getSeconds() ^ SEVEN_BYTE_LONG_SIGN_FLIP;
+    sevenByteLongToBytes(seconds, b, 0);
+    intToBytes(nanos, b, 7);
+    return b;
+  }
+
+  public void fromBinarySortable(byte[] bytes, int binSortOffset, ZoneId timeZone) {
+    // Flip the sign bit (and unused bits of the high-order byte) of the seven-byte long back.
+    long seconds = readSevenByteLong(bytes, binSortOffset) ^ SEVEN_BYTE_LONG_SIGN_FLIP;
+    int nanos = bytesToInt(bytes, binSortOffset + 7);
+    timestampTZ.set(seconds, nanos, timeZone);
+    timestampTZEmpty = false;
+    bytesEmpty = true;
+  }
+
+  public void writeToByteStream(ByteStream.RandomAccessOutput byteStream) {
+    checkBytes();
+    byteStream.write(currentBytes, offset, getTotalLength());
+  }
+
+  /**
+   * Given an integer representing nanoseconds, write its serialized
+   * value to the byte array b at offset
+   *
+   * @param nanos
+   * @param b
+   * @param offset
+   * @return
+   */
+  private static boolean setNanosBytes(int nanos, byte[] b, int offset, boolean hasSecondVInt) {
+    int decimal = 0;
+    if (nanos != 0) {
+      int counter = 0;
+      while (counter < 9) {
+        decimal *= 10;
+        decimal += nanos % 10;
+        nanos /= 10;
+        counter++;
+      }
+    }
+
+    if (hasSecondVInt || decimal != 0) {
+      // We use the sign of the reversed-nanoseconds field to indicate that there is a second VInt
+      // present.
+      LazyBinaryUtils.writeVLongToByteArray(b, offset, hasSecondVInt ? (-decimal - 1) : decimal);
+    }
+    return decimal != 0;
+  }
+
+  public static void setTimestampTZ(TimestampTZ t, byte[] bytes, int offset, ZoneId timeZone) {
+    long seconds = getSeconds(bytes, offset);
+    int nanos = hasDecimalOrSecondVInt(bytes[offset]) ? getNanos(bytes, offset + 4) : 0;
+    t.set(seconds, nanos, timeZone);
+  }
+
+  public static int getTotalLength(byte[] bytes, int offset) {
+    int len = 4;
+    if (hasDecimalOrSecondVInt(bytes[offset])) {
+      int firstVIntLen = WritableUtils.decodeVIntSize(bytes[offset + 4]);
+      len += firstVIntLen;
+      if (hasSecondVInt(bytes[offset + 4])) {
+        len += WritableUtils.decodeVIntSize(bytes[offset + 4 + firstVIntLen]);
+      }
+    }
+    return len;
+  }
+
+  public static long getSeconds(byte[] bytes, int offset) {
+    int firstVInt = bytesToInt(bytes, offset);
+    if (firstVInt >= 0 || !hasSecondVInt(bytes[offset + 4])) {
+      return firstVInt & ~DECIMAL_OR_SECOND_VINT_FLAG;
+    }
+    return ((long) (firstVInt & ~DECIMAL_OR_SECOND_VINT_FLAG)) |
+        (LazyBinaryUtils.readVLongFromByteArray(bytes,
+            offset + 4 + WritableUtils.decodeVIntSize(bytes[offset + 4])) << 31);
+  }
+
+  public static int getNanos(byte[] bytes, int offset) {
+    int val = (int) LazyBinaryUtils.readVLongFromByteArray(bytes, offset);
+    if (val < 0) {
+      val = -val - 1;
+    }
+    int len = (int) Math.floor(Math.log10(val)) + 1;
+
+    // Reverse the value
+    int tmp = 0;
+    while (val != 0) {
+      tmp *= 10;
+      tmp += val % 10;
+      val /= 10;
+    }
+    val = tmp;
+
+    if (len < 9) {
+      val *= Math.pow(10, 9 - len);
+    }
+    return val;
+  }
+
+  private static boolean hasDecimalOrSecondVInt(byte b) {
+    return b < 0;
+  }
+
+  private static boolean hasSecondVInt(byte b) {
+    return WritableUtils.isNegativeVInt(b);
+  }
+
+  /**
+   * Writes <code>value</code> into <code>dest</code> at <code>offset</code>
+   *
+   * @param value
+   * @param dest
+   * @param offset
+   */
+  private static void intToBytes(int value, byte[] dest, int offset) {
+    dest[offset] = (byte) ((value >> 24) & 0xFF);
+    dest[offset + 1] = (byte) ((value >> 16) & 0xFF);
+    dest[offset + 2] = (byte) ((value >> 8) & 0xFF);
+    dest[offset + 3] = (byte) (value & 0xFF);
+  }
+
+  /**
+   * Writes <code>value</code> into <code>dest</code> at <code>offset</code> as a seven-byte
+   * serialized long number.
+   */
+  private static void sevenByteLongToBytes(long value, byte[] dest, int offset) {
+    dest[offset] = (byte) ((value >> 48) & 0xFF);
+    dest[offset + 1] = (byte) ((value >> 40) & 0xFF);
+    dest[offset + 2] = (byte) ((value >> 32) & 0xFF);
+    dest[offset + 3] = (byte) ((value >> 24) & 0xFF);
+    dest[offset + 4] = (byte) ((value >> 16) & 0xFF);
+    dest[offset + 5] = (byte) ((value >> 8) & 0xFF);
+    dest[offset + 6] = (byte) (value & 0xFF);
+  }
+
+  /**
+   * @param bytes
+   * @param offset
+   * @return integer represented by the four bytes in <code>bytes</code>
+   * beginning at <code>offset</code>
+   */
+  private static int bytesToInt(byte[] bytes, int offset) {
+    return ((0xFF & bytes[offset]) << 24)
+        | ((0xFF & bytes[offset + 1]) << 16)
+        | ((0xFF & bytes[offset + 2]) << 8)
+        | (0xFF & bytes[offset + 3]);
+  }
+
+  private static long readSevenByteLong(byte[] bytes, int offset) {
+    // We need to shift everything 8 bits left and then shift back to populate the sign field.
+    return (((0xFFL & bytes[offset]) << 56)
+        | ((0xFFL & bytes[offset + 1]) << 48)
+        | ((0xFFL & bytes[offset + 2]) << 40)
+        | ((0xFFL & bytes[offset + 3]) << 32)
+        | ((0xFFL & bytes[offset + 4]) << 24)
+        | ((0xFFL & bytes[offset + 5]) << 16)
+        | ((0xFFL & bytes[offset + 6]) << 8)) >> 8;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/34eebff1/serde/src/java/org/apache/hadoop/hive/serde2/io/TimestampTZWritable.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/io/TimestampTZWritable.java b/serde/src/java/org/apache/hadoop/hive/serde2/io/TimestampTZWritable.java
deleted file mode 100644
index 8c3f8f6..0000000
--- a/serde/src/java/org/apache/hadoop/hive/serde2/io/TimestampTZWritable.java
+++ /dev/null
@@ -1,427 +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.serde2.io;
-
-import com.google.common.base.Preconditions;
-import org.apache.hadoop.hive.common.type.TimestampTZ;
-import org.apache.hadoop.hive.serde2.ByteStream;
-import org.apache.hadoop.hive.serde2.lazybinary.LazyBinaryUtils;
-import org.apache.hadoop.io.WritableComparable;
-import org.apache.hadoop.io.WritableUtils;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.util.Arrays;
-
-/**
- * Writable for TimestampTZ. Copied from TimestampWritable.
- * After we replace {@link java.sql.Timestamp} with {@link java.time.LocalDateTime} for Timestamp,
- * it'll need a new Writable.
- */
-public class TimestampTZWritable implements WritableComparable<TimestampTZWritable> {
-
-  public static final byte[] nullBytes = {0x0, 0x0, 0x0, 0x0};
-  private static final int DECIMAL_OR_SECOND_VINT_FLAG = 1 << 31;
-  private static final long SEVEN_BYTE_LONG_SIGN_FLIP = 0xff80L << 48; // only need flip the MSB?
-
-  /**
-   * The maximum number of bytes required for a TimestampWritable
-   */
-  public static final int MAX_BYTES = 13;
-
-  public static final int BINARY_SORTABLE_LENGTH = 11;
-
-  private TimestampTZ timestampTZ = new TimestampTZ();
-
-  /**
-   * true if data is stored in timestamptz field rather than byte arrays.
-   * allows for lazy conversion to bytes when necessary
-   * false otherwise
-   */
-  private boolean bytesEmpty = true;
-  private boolean timestampTZEmpty = true;
-
-  /* Allow use of external byte[] for efficiency */
-  private byte[] currentBytes;
-  private final byte[] internalBytes = new byte[MAX_BYTES];
-  private byte[] externalBytes;
-  private int offset;
-
-  public TimestampTZWritable() {
-    bytesEmpty = false;
-    currentBytes = internalBytes;
-    offset = 0;
-  }
-
-  public TimestampTZWritable(byte[] bytes, int offset) {
-    set(bytes, offset);
-  }
-
-  public TimestampTZWritable(TimestampTZWritable other) {
-    this(other.getBytes(), 0);
-  }
-
-  public TimestampTZWritable(TimestampTZ tstz) {
-    set(tstz);
-  }
-
-  public void set(byte[] bytes, int offset) {
-    externalBytes = bytes;
-    this.offset = offset;
-    bytesEmpty = false;
-    timestampTZEmpty = true;
-    currentBytes = externalBytes;
-  }
-
-  public void set(TimestampTZ tstz) {
-    if (tstz == null) {
-      timestampTZ.setZonedDateTime(null);
-      return;
-    }
-    timestampTZ = tstz;
-    bytesEmpty = true;
-    timestampTZEmpty = false;
-  }
-
-  public void set(TimestampTZWritable t) {
-    if (t.bytesEmpty) {
-      set(t.getTimestampTZ());
-    } else if (t.currentBytes == t.externalBytes) {
-      set(t.currentBytes, t.offset);
-    } else {
-      set(t.currentBytes, 0);
-    }
-  }
-
-  public TimestampTZ getTimestampTZ() {
-    populateTimestampTZ();
-    return timestampTZ;
-  }
-
-  /**
-   * Used to create copies of objects
-   *
-   * @return a copy of the internal TimestampTZWritable byte[]
-   */
-  public byte[] getBytes() {
-    checkBytes();
-
-    int len = getTotalLength();
-    byte[] b = new byte[len];
-
-    System.arraycopy(currentBytes, offset, b, 0, len);
-    return b;
-  }
-
-  /**
-   * @return length of serialized TimestampTZWritable data. As a side effect, populates the internal
-   * byte array if empty.
-   */
-  private int getTotalLength() {
-    checkBytes();
-    return getTotalLength(currentBytes, offset);
-  }
-
-  /**
-   * The data of TimestampTZWritable can be stored either in a byte[]
-   * or in a TimestampTZ object. Calling this method ensures that the byte[]
-   * is populated from the TimestampTZ object if previously empty.
-   */
-  private void checkBytes() {
-    if (bytesEmpty) {
-      populateBytes();
-      offset = 0;
-      currentBytes = internalBytes;
-      bytesEmpty = false;
-    }
-  }
-
-  // Writes the TimestampTZ's serialized value to the internal byte array.
-  private void populateBytes() {
-    Arrays.fill(internalBytes, (byte) 0);
-
-    long seconds = timestampTZ.getEpochSecond();
-    int nanos = timestampTZ.getNanos();
-
-    boolean hasSecondVInt = seconds < 0 || seconds > Integer.MAX_VALUE;
-    boolean hasDecimal = setNanosBytes(nanos, internalBytes, offset + 4, hasSecondVInt);
-
-    int firstInt = (int) seconds;
-    if (hasDecimal || hasSecondVInt) {
-      firstInt |= DECIMAL_OR_SECOND_VINT_FLAG;
-    }
-    intToBytes(firstInt, internalBytes, offset);
-    if (hasSecondVInt) {
-      LazyBinaryUtils.writeVLongToByteArray(internalBytes,
-          offset + 4 + WritableUtils.decodeVIntSize(internalBytes[offset + 4]),
-          seconds >> 31);
-    }
-  }
-
-  private void populateTimestampTZ() {
-    if (timestampTZEmpty) {
-      long seconds = getSeconds();
-      int nanos = getNanos();
-      timestampTZ.set(seconds, nanos);
-      timestampTZEmpty = false;
-    }
-  }
-
-  public long getSeconds() {
-    if (!timestampTZEmpty) {
-      return timestampTZ.getEpochSecond();
-    } else if (!bytesEmpty) {
-      return getSeconds(currentBytes, offset);
-    }
-    throw new IllegalStateException("Both timestamp and bytes are empty");
-  }
-
-  public int getNanos() {
-    if (!timestampTZEmpty) {
-      return timestampTZ.getNanos();
-    } else if (!bytesEmpty) {
-      return hasDecimalOrSecondVInt(currentBytes[offset]) ? getNanos(currentBytes, offset + 4) : 0;
-    }
-    throw new IllegalStateException("Both timestamp and bytes are empty");
-  }
-
-  @Override
-  public int compareTo(TimestampTZWritable o) {
-    return getTimestampTZ().compareTo(o.getTimestampTZ());
-  }
-
-  @Override
-  public boolean equals(Object o) {
-    if (o instanceof TimestampTZWritable) {
-      return compareTo((TimestampTZWritable) o) == 0;
-    }
-    return false;
-  }
-
-  @Override
-  public int hashCode() {
-    return getTimestampTZ().hashCode();
-  }
-
-  @Override
-  public String toString() {
-    populateTimestampTZ();
-    return timestampTZ.toString();
-  }
-
-  @Override
-  public void write(DataOutput dataOutput) throws IOException {
-    checkBytes();
-    dataOutput.write(currentBytes, offset, getTotalLength());
-  }
-
-  @Override
-  public void readFields(DataInput dataInput) throws IOException {
-    dataInput.readFully(internalBytes, 0, 4);
-    if (hasDecimalOrSecondVInt(internalBytes[0])) {
-      dataInput.readFully(internalBytes, 4, 1);
-      int len = (byte) WritableUtils.decodeVIntSize(internalBytes[4]);
-      if (len > 1) {
-        dataInput.readFully(internalBytes, 5, len - 1);
-      }
-
-      long vlong = LazyBinaryUtils.readVLongFromByteArray(internalBytes, 4);
-      Preconditions.checkState(vlong >= -1000000000 && vlong <= 999999999,
-          "Invalid nanos value for a TimestampTZWritable: " + vlong +
-              ", expected to be between -1000000000 and 999999999.");
-      if (vlong < 0) {
-        // This indicates there is a second VInt containing the additional bits of the seconds
-        // field.
-        dataInput.readFully(internalBytes, 4 + len, 1);
-        int secondVIntLen = (byte) WritableUtils.decodeVIntSize(internalBytes[4 + len]);
-        if (secondVIntLen > 1) {
-          dataInput.readFully(internalBytes, 5 + len, secondVIntLen - 1);
-        }
-      }
-    }
-    currentBytes = internalBytes;
-    offset = 0;
-    timestampTZEmpty = true;
-    bytesEmpty = false;
-  }
-
-  public byte[] toBinarySortable() {
-    byte[] b = new byte[BINARY_SORTABLE_LENGTH];
-    int nanos = getNanos();
-    // We flip the highest-order bit of the seven-byte representation of seconds to make negative
-    // values come before positive ones.
-    long seconds = getSeconds() ^ SEVEN_BYTE_LONG_SIGN_FLIP;
-    sevenByteLongToBytes(seconds, b, 0);
-    intToBytes(nanos, b, 7);
-    return b;
-  }
-
-  public void fromBinarySortable(byte[] bytes, int binSortOffset) {
-    // Flip the sign bit (and unused bits of the high-order byte) of the seven-byte long back.
-    long seconds = readSevenByteLong(bytes, binSortOffset) ^ SEVEN_BYTE_LONG_SIGN_FLIP;
-    int nanos = bytesToInt(bytes, binSortOffset + 7);
-    timestampTZ.set(seconds, nanos);
-    timestampTZEmpty = false;
-    bytesEmpty = true;
-  }
-
-  public void writeToByteStream(ByteStream.RandomAccessOutput byteStream) {
-    checkBytes();
-    byteStream.write(currentBytes, offset, getTotalLength());
-  }
-
-  /**
-   * Given an integer representing nanoseconds, write its serialized
-   * value to the byte array b at offset
-   *
-   * @param nanos
-   * @param b
-   * @param offset
-   * @return
-   */
-  private static boolean setNanosBytes(int nanos, byte[] b, int offset, boolean hasSecondVInt) {
-    int decimal = 0;
-    if (nanos != 0) {
-      int counter = 0;
-      while (counter < 9) {
-        decimal *= 10;
-        decimal += nanos % 10;
-        nanos /= 10;
-        counter++;
-      }
-    }
-
-    if (hasSecondVInt || decimal != 0) {
-      // We use the sign of the reversed-nanoseconds field to indicate that there is a second VInt
-      // present.
-      LazyBinaryUtils.writeVLongToByteArray(b, offset, hasSecondVInt ? (-decimal - 1) : decimal);
-    }
-    return decimal != 0;
-  }
-
-  public static void setTimestampTZ(TimestampTZ t, byte[] bytes, int offset) {
-    long seconds = getSeconds(bytes, offset);
-    int nanos = hasDecimalOrSecondVInt(bytes[offset]) ? getNanos(bytes, offset + 4) : 0;
-    t.set(seconds, nanos);
-  }
-
-  public static int getTotalLength(byte[] bytes, int offset) {
-    int len = 4;
-    if (hasDecimalOrSecondVInt(bytes[offset])) {
-      int firstVIntLen = WritableUtils.decodeVIntSize(bytes[offset + 4]);
-      len += firstVIntLen;
-      if (hasSecondVInt(bytes[offset + 4])) {
-        len += WritableUtils.decodeVIntSize(bytes[offset + 4 + firstVIntLen]);
-      }
-    }
-    return len;
-  }
-
-  public static long getSeconds(byte[] bytes, int offset) {
-    int firstVInt = bytesToInt(bytes, offset);
-    if (firstVInt >= 0 || !hasSecondVInt(bytes[offset + 4])) {
-      return firstVInt & ~DECIMAL_OR_SECOND_VINT_FLAG;
-    }
-    return ((long) (firstVInt & ~DECIMAL_OR_SECOND_VINT_FLAG)) |
-        (LazyBinaryUtils.readVLongFromByteArray(bytes,
-            offset + 4 + WritableUtils.decodeVIntSize(bytes[offset + 4])) << 31);
-  }
-
-  public static int getNanos(byte[] bytes, int offset) {
-    int val = (int) LazyBinaryUtils.readVLongFromByteArray(bytes, offset);
-    if (val < 0) {
-      val = -val - 1;
-    }
-    int len = (int) Math.floor(Math.log10(val)) + 1;
-
-    // Reverse the value
-    int tmp = 0;
-    while (val != 0) {
-      tmp *= 10;
-      tmp += val % 10;
-      val /= 10;
-    }
-    val = tmp;
-
-    if (len < 9) {
-      val *= Math.pow(10, 9 - len);
-    }
-    return val;
-  }
-
-  private static boolean hasDecimalOrSecondVInt(byte b) {
-    return b < 0;
-  }
-
-  private static boolean hasSecondVInt(byte b) {
-    return WritableUtils.isNegativeVInt(b);
-  }
-
-  /**
-   * Writes <code>value</code> into <code>dest</code> at <code>offset</code>
-   *
-   * @param value
-   * @param dest
-   * @param offset
-   */
-  private static void intToBytes(int value, byte[] dest, int offset) {
-    dest[offset] = (byte) ((value >> 24) & 0xFF);
-    dest[offset + 1] = (byte) ((value >> 16) & 0xFF);
-    dest[offset + 2] = (byte) ((value >> 8) & 0xFF);
-    dest[offset + 3] = (byte) (value & 0xFF);
-  }
-
-  /**
-   * Writes <code>value</code> into <code>dest</code> at <code>offset</code> as a seven-byte
-   * serialized long number.
-   */
-  private static void sevenByteLongToBytes(long value, byte[] dest, int offset) {
-    dest[offset] = (byte) ((value >> 48) & 0xFF);
-    dest[offset + 1] = (byte) ((value >> 40) & 0xFF);
-    dest[offset + 2] = (byte) ((value >> 32) & 0xFF);
-    dest[offset + 3] = (byte) ((value >> 24) & 0xFF);
-    dest[offset + 4] = (byte) ((value >> 16) & 0xFF);
-    dest[offset + 5] = (byte) ((value >> 8) & 0xFF);
-    dest[offset + 6] = (byte) (value & 0xFF);
-  }
-
-  /**
-   * @param bytes
-   * @param offset
-   * @return integer represented by the four bytes in <code>bytes</code>
-   * beginning at <code>offset</code>
-   */
-  private static int bytesToInt(byte[] bytes, int offset) {
-    return ((0xFF & bytes[offset]) << 24)
-        | ((0xFF & bytes[offset + 1]) << 16)
-        | ((0xFF & bytes[offset + 2]) << 8)
-        | (0xFF & bytes[offset + 3]);
-  }
-
-  private static long readSevenByteLong(byte[] bytes, int offset) {
-    // We need to shift everything 8 bits left and then shift back to populate the sign field.
-    return (((0xFFL & bytes[offset]) << 56)
-        | ((0xFFL & bytes[offset + 1]) << 48)
-        | ((0xFFL & bytes[offset + 2]) << 40)
-        | ((0xFFL & bytes[offset + 3]) << 32)
-        | ((0xFFL & bytes[offset + 4]) << 24)
-        | ((0xFFL & bytes[offset + 5]) << 16)
-        | ((0xFFL & bytes[offset + 6]) << 8)) >> 8;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/34eebff1/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazyFactory.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazyFactory.java b/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazyFactory.java
index f87ac76..3ed5b33 100644
--- a/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazyFactory.java
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazyFactory.java
@@ -46,7 +46,7 @@ import org.apache.hadoop.hive.serde2.lazy.objectinspector.primitive.LazyPrimitiv
 import org.apache.hadoop.hive.serde2.lazy.objectinspector.primitive.LazyShortObjectInspector;
 import org.apache.hadoop.hive.serde2.lazy.objectinspector.primitive.LazyStringObjectInspector;
 import org.apache.hadoop.hive.serde2.lazy.objectinspector.primitive.LazyTimestampObjectInspector;
-import org.apache.hadoop.hive.serde2.lazy.objectinspector.primitive.LazyTimestampTZObjectInspector;
+import org.apache.hadoop.hive.serde2.lazy.objectinspector.primitive.LazyTimestampLocalTZObjectInspector;
 import org.apache.hadoop.hive.serde2.lazy.objectinspector.primitive.LazyVoidObjectInspector;
 import org.apache.hadoop.hive.serde2.lazydio.LazyDioBinary;
 import org.apache.hadoop.hive.serde2.lazydio.LazyDioBoolean;
@@ -129,8 +129,8 @@ public final class LazyFactory {
       return new LazyDate((LazyDateObjectInspector) oi);
     case TIMESTAMP:
       return new LazyTimestamp((LazyTimestampObjectInspector) oi);
-    case TIMESTAMPTZ:
-      return new LazyTimestampTZ((LazyTimestampTZObjectInspector) oi);
+    case TIMESTAMPLOCALTZ:
+      return new LazyTimestampLocalTZ((LazyTimestampLocalTZObjectInspector) oi);
     case INTERVAL_YEAR_MONTH:
       return new LazyHiveIntervalYearMonth((LazyHiveIntervalYearMonthObjectInspector) oi);
     case INTERVAL_DAY_TIME:

http://git-wip-us.apache.org/repos/asf/hive/blob/34eebff1/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazySerDeParameters.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazySerDeParameters.java b/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazySerDeParameters.java
index ee4bb34..fe3f356 100644
--- a/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazySerDeParameters.java
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazySerDeParameters.java
@@ -23,8 +23,6 @@ import java.util.Arrays;
 import java.util.List;
 import java.util.Properties;
 
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.common.classification.InterfaceAudience.Public;
 import org.apache.hadoop.hive.common.classification.InterfaceStability.Stable;
@@ -33,11 +31,14 @@ import org.apache.hadoop.hive.serde.serdeConstants;
 import org.apache.hadoop.hive.serde2.SerDeException;
 import org.apache.hadoop.hive.serde2.SerDeUtils;
 import org.apache.hadoop.hive.serde2.lazy.objectinspector.primitive.LazyObjectInspectorParameters;
+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;
 import org.apache.hadoop.io.Text;
 import org.apache.hive.common.util.HiveStringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * SerDeParameters.
@@ -88,7 +89,7 @@ public class LazySerDeParameters implements LazyObjectInspectorParameters {
     lastColumnTakesRest = (lastColumnTakesRestString != null && lastColumnTakesRestString
         .equalsIgnoreCase("true"));
 
-    extractColumnInfo();
+    extractColumnInfo(job);
 
     // Create the LazyObject for storing the rows
     rowTypeInfo = TypeInfoFactory.getStructTypeInfo(columnNames, columnTypes);
@@ -133,7 +134,7 @@ public class LazySerDeParameters implements LazyObjectInspectorParameters {
    * Extracts and set column names and column types from the table properties
    * @throws SerDeException
    */
-  public void extractColumnInfo() throws SerDeException {
+  public void extractColumnInfo(Configuration conf) throws SerDeException {
     // Read the configuration parameters
     String columnNameProperty = tableProperties.getProperty(serdeConstants.LIST_COLUMNS);
     // NOTE: if "columns.types" is missing, all columns will be of String type
@@ -160,6 +161,16 @@ public class LazySerDeParameters implements LazyObjectInspectorParameters {
     }
 
     columnTypes = TypeInfoUtils.getTypeInfosFromTypeString(columnTypeProperty);
+    // Insert time-zone for timestamp type
+    if (conf != null) {
+      final TimestampLocalTZTypeInfo tsTZTypeInfo = new TimestampLocalTZTypeInfo(
+          conf.get(ConfVars.HIVE_LOCAL_TIME_ZONE.varname));
+      for (int i = 0; i < columnTypes.size(); i++) {
+        if (columnTypes.get(i) instanceof TimestampLocalTZTypeInfo) {
+          columnTypes.set(i, tsTZTypeInfo);
+        }
+      }
+    }
 
     if (columnNames.size() != columnTypes.size()) {
       throw new SerDeException(serdeName + ": columns has " + columnNames.size()

http://git-wip-us.apache.org/repos/asf/hive/blob/34eebff1/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazyTimestampLocalTZ.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazyTimestampLocalTZ.java b/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazyTimestampLocalTZ.java
new file mode 100644
index 0000000..ce5f202
--- /dev/null
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazyTimestampLocalTZ.java
@@ -0,0 +1,102 @@
+/**
+ * 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.serde2.lazy;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.io.UnsupportedEncodingException;
+import java.time.ZoneId;
+import java.time.format.DateTimeParseException;
+
+import org.apache.hadoop.hive.common.type.TimestampTZ;
+import org.apache.hadoop.hive.common.type.TimestampTZUtil;
+import org.apache.hadoop.hive.serde.serdeConstants;
+import org.apache.hadoop.hive.serde2.io.TimestampLocalTZWritable;
+import org.apache.hadoop.hive.serde2.lazy.objectinspector.primitive.LazyTimestampLocalTZObjectInspector;
+import org.apache.hadoop.hive.serde2.typeinfo.TimestampLocalTZTypeInfo;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * LazyPrimitive for TimestampLocalTZ. Similar to LazyTimestamp.
+ */
+public class LazyTimestampLocalTZ extends
+    LazyPrimitive<LazyTimestampLocalTZObjectInspector, TimestampLocalTZWritable> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(LazyTimestampLocalTZ.class);
+
+  private ZoneId timeZone;
+
+  public LazyTimestampLocalTZ(LazyTimestampLocalTZObjectInspector lazyTimestampTZObjectInspector) {
+    super(lazyTimestampTZObjectInspector);
+    TimestampLocalTZTypeInfo typeInfo = (TimestampLocalTZTypeInfo) oi.getTypeInfo();
+    if (typeInfo == null) {
+      throw new RuntimeException("TimestampLocalTZ type used without type params");
+    }
+    timeZone = typeInfo.timeZone();
+    data = new TimestampLocalTZWritable();
+  }
+
+  public LazyTimestampLocalTZ(LazyTimestampLocalTZ copy) {
+    super(copy);
+    timeZone = copy.timeZone;
+    data = new TimestampLocalTZWritable(copy.data);
+  }
+
+  @Override
+  public void init(ByteArrayRef bytes, int start, int length) {
+    String s = null;
+    if (!LazyUtils.isDateMaybe(bytes.getData(), start, length)) {
+      isNull = true;
+      return;
+    }
+
+    TimestampTZ t = null;
+    try {
+      s = new String(bytes.getData(), start, length, "US-ASCII");
+      if (s.equals("NULL")) {
+        isNull = true;
+        logExceptionMessage(bytes, start, length,
+            serdeConstants.TIMESTAMPLOCALTZ_TYPE_NAME.toUpperCase());
+      } else {
+        t = TimestampTZUtil.parse(s, timeZone);
+        isNull = false;
+      }
+    } catch (UnsupportedEncodingException e) {
+      isNull = true;
+      LOG.error("Unsupported encoding found ", e);
+    } catch (DateTimeParseException e) {
+      isNull = true;
+      logExceptionMessage(bytes, start, length, serdeConstants.TIMESTAMPLOCALTZ_TYPE_NAME.toUpperCase());
+    }
+    data.set(t);
+  }
+
+  @Override
+  public TimestampLocalTZWritable getWritableObject() {
+    return data;
+  }
+
+  public static void writeUTF8(OutputStream out, TimestampLocalTZWritable i) throws IOException {
+    if (i == null) {
+      out.write(TimestampLocalTZWritable.nullBytes);
+    } else {
+      out.write(i.toString().getBytes("US-ASCII"));
+    }
+  }
+}


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

Posted by jc...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/34eebff1/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazyTimestampTZ.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazyTimestampTZ.java b/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazyTimestampTZ.java
deleted file mode 100644
index df5c586..0000000
--- a/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazyTimestampTZ.java
+++ /dev/null
@@ -1,91 +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.serde2.lazy;
-
-import org.apache.hadoop.hive.common.type.TimestampTZ;
-import org.apache.hadoop.hive.serde.serdeConstants;
-import org.apache.hadoop.hive.serde2.io.TimestampTZWritable;
-import org.apache.hadoop.hive.serde2.lazy.objectinspector.primitive.LazyTimestampTZObjectInspector;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.io.OutputStream;
-import java.io.UnsupportedEncodingException;
-import java.time.format.DateTimeParseException;
-
-/**
- * LazyPrimitive for TimestampTZ. Similar to LazyTimestamp.
- */
-public class LazyTimestampTZ extends
-    LazyPrimitive<LazyTimestampTZObjectInspector, TimestampTZWritable> {
-
-  private static final Logger LOG = LoggerFactory.getLogger(LazyTimestampTZ.class);
-
-  public LazyTimestampTZ(LazyTimestampTZObjectInspector lazyTimestampTZObjectInspector) {
-    super(lazyTimestampTZObjectInspector);
-    data = new TimestampTZWritable();
-  }
-
-  public LazyTimestampTZ(LazyTimestampTZ copy) {
-    super(copy);
-    data = new TimestampTZWritable(copy.data);
-  }
-
-  @Override
-  public void init(ByteArrayRef bytes, int start, int length) {
-    String s = null;
-    if (!LazyUtils.isDateMaybe(bytes.getData(), start, length)) {
-      isNull = true;
-      return;
-    }
-
-    TimestampTZ t = null;
-    try {
-      s = new String(bytes.getData(), start, length, "US-ASCII");
-      if (s.equals("NULL")) {
-        isNull = true;
-        logExceptionMessage(bytes, start, length,
-            serdeConstants.TIMESTAMPTZ_TYPE_NAME.toUpperCase());
-      } else {
-        t = TimestampTZ.parse(s);
-        isNull = false;
-      }
-    } catch (UnsupportedEncodingException e) {
-      isNull = true;
-      LOG.error("Unsupported encoding found ", e);
-    } catch (DateTimeParseException e) {
-      isNull = true;
-      logExceptionMessage(bytes, start, length, serdeConstants.TIMESTAMPTZ_TYPE_NAME.toUpperCase());
-    }
-    data.set(t);
-  }
-
-  @Override
-  public TimestampTZWritable getWritableObject() {
-    return data;
-  }
-
-  public static void writeUTF8(OutputStream out, TimestampTZWritable i) throws IOException {
-    if (i == null) {
-      out.write(TimestampTZWritable.nullBytes);
-    } else {
-      out.write(i.toString().getBytes("US-ASCII"));
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/34eebff1/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazyUtils.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazyUtils.java b/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazyUtils.java
index c811753..80b3de2 100644
--- a/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazyUtils.java
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazyUtils.java
@@ -47,7 +47,7 @@ import org.apache.hadoop.hive.serde2.objectinspector.primitive.LongObjectInspect
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.ShortObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.StringObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.TimestampObjectInspector;
-import org.apache.hadoop.hive.serde2.objectinspector.primitive.TimestampTZObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.TimestampLocalTZObjectInspector;
 import org.apache.hadoop.io.BytesWritable;
 import org.apache.hadoop.io.Text;
 
@@ -307,8 +307,8 @@ public final class LazyUtils {
           ((TimestampObjectInspector) oi).getPrimitiveWritableObject(o));
       break;
     }
-    case TIMESTAMPTZ: {
-      LazyTimestampTZ.writeUTF8(out, ((TimestampTZObjectInspector) oi).
+    case TIMESTAMPLOCALTZ: {
+      LazyTimestampLocalTZ.writeUTF8(out, ((TimestampLocalTZObjectInspector) oi).
           getPrimitiveWritableObject(o));
       break;
     }

http://git-wip-us.apache.org/repos/asf/hive/blob/34eebff1/serde/src/java/org/apache/hadoop/hive/serde2/lazy/objectinspector/primitive/LazyPrimitiveObjectInspectorFactory.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/lazy/objectinspector/primitive/LazyPrimitiveObjectInspectorFactory.java b/serde/src/java/org/apache/hadoop/hive/serde2/lazy/objectinspector/primitive/LazyPrimitiveObjectInspectorFactory.java
index 6d1ee1e..e673f11 100644
--- a/serde/src/java/org/apache/hadoop/hive/serde2/lazy/objectinspector/primitive/LazyPrimitiveObjectInspectorFactory.java
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/lazy/objectinspector/primitive/LazyPrimitiveObjectInspectorFactory.java
@@ -27,6 +27,7 @@ import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector.Pr
 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.VarcharTypeInfo;
 
@@ -71,8 +72,6 @@ public final class LazyPrimitiveObjectInspectorFactory {
       new LazyDateObjectInspector();
   public static final LazyTimestampObjectInspector LAZY_TIMESTAMP_OBJECT_INSPECTOR =
       new LazyTimestampObjectInspector();
-  public static final LazyTimestampTZObjectInspector LAZY_TIMESTAMPTZ_OBJECT_INSPECTOR =
-      new LazyTimestampTZObjectInspector();
   public static final LazyHiveIntervalYearMonthObjectInspector LAZY_INTERVAL_YEAR_MONTH_OBJECT_INSPECTOR =
       new LazyHiveIntervalYearMonthObjectInspector();
   public static final LazyHiveIntervalDayTimeObjectInspector LAZY_INTERVAL_DAY_TIME_OBJECT_INSPECTOR =
@@ -113,8 +112,6 @@ public final class LazyPrimitiveObjectInspectorFactory {
         LAZY_DATE_OBJECT_INSPECTOR);
     cachedPrimitiveLazyObjectInspectors.put(TypeInfoFactory.getPrimitiveTypeInfo(serdeConstants.TIMESTAMP_TYPE_NAME),
         LAZY_TIMESTAMP_OBJECT_INSPECTOR);
-    cachedPrimitiveLazyObjectInspectors.put(TypeInfoFactory.getPrimitiveTypeInfo(
-        serdeConstants.TIMESTAMPTZ_TYPE_NAME), LAZY_TIMESTAMPTZ_OBJECT_INSPECTOR);
     cachedPrimitiveLazyObjectInspectors.put(TypeInfoFactory.getPrimitiveTypeInfo(serdeConstants.INTERVAL_YEAR_MONTH_TYPE_NAME),
         LAZY_INTERVAL_YEAR_MONTH_OBJECT_INSPECTOR);
     cachedPrimitiveLazyObjectInspectors.put(TypeInfoFactory.getPrimitiveTypeInfo(serdeConstants.INTERVAL_DAY_TIME_TYPE_NAME),
@@ -175,6 +172,9 @@ public final class LazyPrimitiveObjectInspectorFactory {
     case DECIMAL:
       poi = new LazyHiveDecimalObjectInspector((DecimalTypeInfo)typeInfo);
       break;
+    case TIMESTAMPLOCALTZ:
+      poi = new LazyTimestampLocalTZObjectInspector((TimestampLocalTZTypeInfo)typeInfo);
+      break;
     default:
       throw new RuntimeException(
           "Primitve type " + typeInfo.getPrimitiveCategory() + " should not take parameters");

http://git-wip-us.apache.org/repos/asf/hive/blob/34eebff1/serde/src/java/org/apache/hadoop/hive/serde2/lazy/objectinspector/primitive/LazyTimestampLocalTZObjectInspector.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/lazy/objectinspector/primitive/LazyTimestampLocalTZObjectInspector.java b/serde/src/java/org/apache/hadoop/hive/serde2/lazy/objectinspector/primitive/LazyTimestampLocalTZObjectInspector.java
new file mode 100644
index 0000000..d81eca8
--- /dev/null
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/lazy/objectinspector/primitive/LazyTimestampLocalTZObjectInspector.java
@@ -0,0 +1,52 @@
+/**
+ * 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.serde2.lazy.objectinspector.primitive;
+
+import org.apache.hadoop.hive.common.type.TimestampTZ;
+import org.apache.hadoop.hive.serde2.io.TimestampLocalTZWritable;
+import org.apache.hadoop.hive.serde2.lazy.LazyTimestampLocalTZ;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.TimestampLocalTZObjectInspector;
+import org.apache.hadoop.hive.serde2.typeinfo.TimestampLocalTZTypeInfo;
+
+public class LazyTimestampLocalTZObjectInspector
+    extends AbstractPrimitiveLazyObjectInspector<TimestampLocalTZWritable>
+    implements TimestampLocalTZObjectInspector {
+
+  protected LazyTimestampLocalTZObjectInspector(TimestampLocalTZTypeInfo typeInfo) {
+    super(typeInfo);
+  }
+
+  @Override
+  public TimestampTZ getPrimitiveJavaObject(Object o) {
+    if (o == null) {
+      return null;
+    }
+
+    TimestampTZ t = ((LazyTimestampLocalTZ) o).getWritableObject().getTimestampTZ();
+    TimestampLocalTZTypeInfo timestampTZTypeInfo = (TimestampLocalTZTypeInfo) typeInfo;
+    if (!t.getZonedDateTime().getZone().equals(timestampTZTypeInfo.timeZone())) {
+      t.setZonedDateTime(t.getZonedDateTime().withZoneSameInstant(timestampTZTypeInfo.timeZone()));
+    }
+    return t;
+  }
+
+  @Override
+  public Object copyObject(Object o) {
+    return o == null ? null : new LazyTimestampLocalTZ((LazyTimestampLocalTZ) o);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/34eebff1/serde/src/java/org/apache/hadoop/hive/serde2/lazy/objectinspector/primitive/LazyTimestampTZObjectInspector.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/lazy/objectinspector/primitive/LazyTimestampTZObjectInspector.java b/serde/src/java/org/apache/hadoop/hive/serde2/lazy/objectinspector/primitive/LazyTimestampTZObjectInspector.java
deleted file mode 100644
index 7336385..0000000
--- a/serde/src/java/org/apache/hadoop/hive/serde2/lazy/objectinspector/primitive/LazyTimestampTZObjectInspector.java
+++ /dev/null
@@ -1,43 +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.serde2.lazy.objectinspector.primitive;
-
-import org.apache.hadoop.hive.common.type.TimestampTZ;
-import org.apache.hadoop.hive.serde2.io.TimestampTZWritable;
-import org.apache.hadoop.hive.serde2.lazy.LazyTimestampTZ;
-import org.apache.hadoop.hive.serde2.objectinspector.primitive.TimestampTZObjectInspector;
-import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
-
-public class LazyTimestampTZObjectInspector
-    extends AbstractPrimitiveLazyObjectInspector<TimestampTZWritable>
-    implements TimestampTZObjectInspector {
-
-  public LazyTimestampTZObjectInspector() {
-    super(TypeInfoFactory.timestampTZTypeInfo);
-  }
-
-  @Override
-  public TimestampTZ getPrimitiveJavaObject(Object o) {
-    return o == null ? null : ((LazyTimestampTZ) o).getWritableObject().getTimestampTZ();
-  }
-
-  @Override
-  public Object copyObject(Object o) {
-    return o == null ? null : new LazyTimestampTZ((LazyTimestampTZ) o);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/34eebff1/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/LazyBinaryFactory.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/LazyBinaryFactory.java b/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/LazyBinaryFactory.java
index 16609bb..a916ab9 100644
--- a/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/LazyBinaryFactory.java
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/LazyBinaryFactory.java
@@ -44,7 +44,7 @@ import org.apache.hadoop.hive.serde2.objectinspector.primitive.WritableShortObje
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.WritableStringObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.WritableDateObjectInspector;
 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.objectinspector.primitive.WritableVoidObjectInspector;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
 
@@ -87,8 +87,8 @@ public final class LazyBinaryFactory {
       return new LazyBinaryDate((WritableDateObjectInspector) oi);
     case TIMESTAMP:
       return new LazyBinaryTimestamp((WritableTimestampObjectInspector) oi);
-    case TIMESTAMPTZ:
-      return new LazyBinaryTimestampTZ((WritableTimestampTZObjectInspector) oi);
+    case TIMESTAMPLOCALTZ:
+      return new LazyBinaryTimestampLocalTZ((WritableTimestampLocalTZObjectInspector) oi);
     case INTERVAL_YEAR_MONTH:
       return new LazyBinaryHiveIntervalYearMonth((WritableHiveIntervalYearMonthObjectInspector) oi);
     case INTERVAL_DAY_TIME:

http://git-wip-us.apache.org/repos/asf/hive/blob/34eebff1/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/LazyBinarySerDe.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/LazyBinarySerDe.java b/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/LazyBinarySerDe.java
index 268f374..7ab2083 100644
--- a/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/LazyBinarySerDe.java
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/LazyBinarySerDe.java
@@ -24,8 +24,8 @@ import java.util.List;
 import java.util.Map;
 import java.util.Properties;
 
-import org.apache.hadoop.hive.serde2.io.TimestampTZWritable;
-import org.apache.hadoop.hive.serde2.objectinspector.primitive.TimestampTZObjectInspector;
+import org.apache.hadoop.hive.serde2.io.TimestampLocalTZWritable;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.TimestampLocalTZObjectInspector;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -515,8 +515,8 @@ public class LazyBinarySerDe extends AbstractSerDe {
         t.writeToByteStream(byteStream);
         return;
       }
-      case TIMESTAMPTZ: {
-        TimestampTZWritable t = ((TimestampTZObjectInspector) poi).getPrimitiveWritableObject(obj);
+      case TIMESTAMPLOCALTZ: {
+        TimestampLocalTZWritable t = ((TimestampLocalTZObjectInspector) poi).getPrimitiveWritableObject(obj);
         t.writeToByteStream(byteStream);
         return;
       }

http://git-wip-us.apache.org/repos/asf/hive/blob/34eebff1/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/LazyBinaryTimestampLocalTZ.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/LazyBinaryTimestampLocalTZ.java b/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/LazyBinaryTimestampLocalTZ.java
new file mode 100644
index 0000000..926adc7
--- /dev/null
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/LazyBinaryTimestampLocalTZ.java
@@ -0,0 +1,43 @@
+/**
+ * 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.serde2.lazybinary;
+
+import java.time.ZoneId;
+
+import org.apache.hadoop.hive.serde2.io.TimestampLocalTZWritable;
+import org.apache.hadoop.hive.serde2.lazy.ByteArrayRef;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.WritableTimestampLocalTZObjectInspector;
+import org.apache.hadoop.hive.serde2.typeinfo.TimestampLocalTZTypeInfo;
+
+public class LazyBinaryTimestampLocalTZ extends
+    LazyBinaryPrimitive<WritableTimestampLocalTZObjectInspector, TimestampLocalTZWritable> {
+
+  private ZoneId timeZone;
+
+  public LazyBinaryTimestampLocalTZ(WritableTimestampLocalTZObjectInspector oi) {
+    super(oi);
+    TimestampLocalTZTypeInfo typeInfo = (TimestampLocalTZTypeInfo) oi.getTypeInfo();
+    this.timeZone = typeInfo.timeZone();
+    this.data = new TimestampLocalTZWritable();
+  }
+
+  @Override
+  public void init(ByteArrayRef bytes, int start, int length) {
+    data.set(bytes.getData(), start, timeZone);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/34eebff1/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/LazyBinaryTimestampTZ.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/LazyBinaryTimestampTZ.java b/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/LazyBinaryTimestampTZ.java
deleted file mode 100644
index 6d9ca6e..0000000
--- a/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/LazyBinaryTimestampTZ.java
+++ /dev/null
@@ -1,36 +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.serde2.lazybinary;
-
-import org.apache.hadoop.hive.serde2.io.TimestampTZWritable;
-import org.apache.hadoop.hive.serde2.lazy.ByteArrayRef;
-import org.apache.hadoop.hive.serde2.objectinspector.primitive.WritableTimestampTZObjectInspector;
-
-public class LazyBinaryTimestampTZ extends
-    LazyBinaryPrimitive<WritableTimestampTZObjectInspector, TimestampTZWritable> {
-
-  public LazyBinaryTimestampTZ(WritableTimestampTZObjectInspector oi) {
-    super(oi);
-    data = new TimestampTZWritable();
-  }
-
-  @Override
-  public void init(ByteArrayRef bytes, int start, int length) {
-    data.set(bytes.getData(), start);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/34eebff1/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/LazyBinaryUtils.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/LazyBinaryUtils.java b/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/LazyBinaryUtils.java
index 4c67259..3aa5688 100644
--- a/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/LazyBinaryUtils.java
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/LazyBinaryUtils.java
@@ -22,7 +22,7 @@ import java.util.List;
 import java.util.concurrent.ConcurrentHashMap;
 
 import org.apache.hadoop.hive.serde2.ByteStream.RandomAccessOutput;
-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.lazybinary.objectinspector.LazyBinaryObjectInspectorFactory;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
@@ -205,9 +205,9 @@ public final class LazyBinaryUtils {
         recordInfo.elementOffset = 0;
         recordInfo.elementSize = TimestampWritable.getTotalLength(bytes, offset);
         break;
-      case TIMESTAMPTZ:
+      case TIMESTAMPLOCALTZ:
         recordInfo.elementOffset = 0;
-        recordInfo.elementSize = TimestampTZWritable.getTotalLength(bytes, offset);
+        recordInfo.elementSize = TimestampLocalTZWritable.getTotalLength(bytes, offset);
         break;
       case INTERVAL_YEAR_MONTH:
         recordInfo.elementOffset = 0;

http://git-wip-us.apache.org/repos/asf/hive/blob/34eebff1/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/ObjectInspectorConverters.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/ObjectInspectorConverters.java b/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/ObjectInspectorConverters.java
index 7025ebf..7921de8 100644
--- a/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/ObjectInspectorConverters.java
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/ObjectInspectorConverters.java
@@ -42,10 +42,9 @@ import org.apache.hadoop.hive.serde2.objectinspector.primitive.SettableIntObject
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.SettableLongObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.SettableShortObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.SettableTimestampObjectInspector;
-import org.apache.hadoop.hive.serde2.objectinspector.primitive.SettableTimestampTZObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.SettableTimestampLocalTZObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.VoidObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.WritableStringObjectInspector;
-import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo;
 
 /**
  * ObjectInspectorConverters.
@@ -128,9 +127,10 @@ public final class ObjectInspectorConverters {
       return new PrimitiveObjectInspectorConverter.TimestampConverter(
           inputOI,
           (SettableTimestampObjectInspector) outputOI);
-    case TIMESTAMPTZ:
-      return new PrimitiveObjectInspectorConverter.TimestampTZConverter(inputOI,
-          (SettableTimestampTZObjectInspector) outputOI);
+    case TIMESTAMPLOCALTZ:
+      return new PrimitiveObjectInspectorConverter.TimestampLocalTZConverter(
+          inputOI,
+          (SettableTimestampLocalTZObjectInspector) outputOI);
     case INTERVAL_YEAR_MONTH:
       return new PrimitiveObjectInspectorConverter.HiveIntervalYearMonthConverter(
           inputOI,

http://git-wip-us.apache.org/repos/asf/hive/blob/34eebff1/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/ObjectInspectorUtils.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/ObjectInspectorUtils.java b/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/ObjectInspectorUtils.java
index 0d6fd4a..698ebe1 100644
--- a/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/ObjectInspectorUtils.java
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/ObjectInspectorUtils.java
@@ -27,9 +27,9 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 
-import org.apache.hadoop.hive.serde2.io.TimestampTZWritable;
-import org.apache.hadoop.hive.serde2.objectinspector.primitive.SettableTimestampTZObjectInspector;
-import org.apache.hadoop.hive.serde2.objectinspector.primitive.TimestampTZObjectInspector;
+import org.apache.hadoop.hive.serde2.io.TimestampLocalTZWritable;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.SettableTimestampLocalTZObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.TimestampLocalTZObjectInspector;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.hive.serde.serdeConstants;
@@ -416,7 +416,7 @@ public final class ObjectInspectorUtils {
         if (loi.getPrimitiveCategory() == PrimitiveObjectInspector.PrimitiveCategory.TIMESTAMP) {
           result = PrimitiveObjectInspectorFactory.javaTimestampObjectInspector.copyObject(result);
         } else if (loi.getPrimitiveCategory() ==
-            PrimitiveObjectInspector.PrimitiveCategory.TIMESTAMPTZ) {
+            PrimitiveObjectInspector.PrimitiveCategory.TIMESTAMPLOCALTZ) {
           result = PrimitiveObjectInspectorFactory.javaTimestampTZObjectInspector.
               copyObject(result);
         }
@@ -701,8 +701,8 @@ public final class ObjectInspectorUtils {
         TimestampWritable t = ((TimestampObjectInspector) poi)
             .getPrimitiveWritableObject(o);
         return t.hashCode();
-      case TIMESTAMPTZ:
-        TimestampTZWritable tstz = ((TimestampTZObjectInspector) poi).getPrimitiveWritableObject(o);
+      case TIMESTAMPLOCALTZ:
+        TimestampLocalTZWritable tstz = ((TimestampLocalTZObjectInspector) poi).getPrimitiveWritableObject(o);
         return tstz.hashCode();
       case INTERVAL_YEAR_MONTH:
         HiveIntervalYearMonthWritable intervalYearMonth = ((HiveIntervalYearMonthObjectInspector) poi)
@@ -965,10 +965,10 @@ public final class ObjectInspectorUtils {
             .getPrimitiveWritableObject(o2);
         return t1.compareTo(t2);
       }
-      case TIMESTAMPTZ: {
-        TimestampTZWritable tstz1 = ((TimestampTZObjectInspector) poi1).
+      case TIMESTAMPLOCALTZ: {
+        TimestampLocalTZWritable tstz1 = ((TimestampLocalTZObjectInspector) poi1).
             getPrimitiveWritableObject(o1);
-        TimestampTZWritable tstz2 = ((TimestampTZObjectInspector) poi2).
+        TimestampLocalTZWritable tstz2 = ((TimestampLocalTZObjectInspector) poi2).
             getPrimitiveWritableObject(o2);
         return tstz1.compareTo(tstz2);
       }
@@ -1339,8 +1339,8 @@ public final class ObjectInspectorUtils {
       return oi instanceof SettableDateObjectInspector;
     case TIMESTAMP:
       return oi instanceof SettableTimestampObjectInspector;
-    case TIMESTAMPTZ:
-      return oi instanceof SettableTimestampTZObjectInspector;
+    case TIMESTAMPLOCALTZ:
+      return oi instanceof SettableTimestampLocalTZObjectInspector;
     case INTERVAL_YEAR_MONTH:
       return oi instanceof SettableHiveIntervalYearMonthObjectInspector;
     case INTERVAL_DAY_TIME:

http://git-wip-us.apache.org/repos/asf/hive/blob/34eebff1/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/PrimitiveObjectInspector.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/PrimitiveObjectInspector.java b/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/PrimitiveObjectInspector.java
index f95dd37..3c58f06 100644
--- a/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/PrimitiveObjectInspector.java
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/PrimitiveObjectInspector.java
@@ -35,7 +35,7 @@ public interface PrimitiveObjectInspector extends ObjectInspector {
    */
   public static enum PrimitiveCategory {
     VOID, BOOLEAN, BYTE, SHORT, INT, LONG, FLOAT, DOUBLE, STRING,
-    DATE, TIMESTAMP, TIMESTAMPTZ, BINARY, DECIMAL, VARCHAR, CHAR,
+    DATE, TIMESTAMP, TIMESTAMPLOCALTZ, BINARY, DECIMAL, VARCHAR, CHAR,
     INTERVAL_YEAR_MONTH, INTERVAL_DAY_TIME, UNKNOWN
   };
 

http://git-wip-us.apache.org/repos/asf/hive/blob/34eebff1/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/JavaTimestampLocalTZObjectInspector.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/JavaTimestampLocalTZObjectInspector.java b/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/JavaTimestampLocalTZObjectInspector.java
new file mode 100644
index 0000000..5cd9746
--- /dev/null
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/JavaTimestampLocalTZObjectInspector.java
@@ -0,0 +1,103 @@
+/**
+ * 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.serde2.objectinspector.primitive;
+
+import org.apache.hadoop.hive.common.type.TimestampTZ;
+import org.apache.hadoop.hive.serde2.io.TimestampLocalTZWritable;
+import org.apache.hadoop.hive.serde2.typeinfo.TimestampLocalTZTypeInfo;
+
+public class JavaTimestampLocalTZObjectInspector
+    extends AbstractPrimitiveJavaObjectInspector implements SettableTimestampLocalTZObjectInspector {
+
+  public JavaTimestampLocalTZObjectInspector() {
+  }
+
+  public JavaTimestampLocalTZObjectInspector(TimestampLocalTZTypeInfo typeInfo) {
+    super(typeInfo);
+  }
+
+  @Override
+  public Object set(Object o, byte[] bytes, int offset) {
+    TimestampLocalTZWritable.setTimestampTZ(
+        (TimestampTZ) o, bytes, offset, ((TimestampLocalTZTypeInfo) typeInfo).timeZone());
+    return o;
+  }
+
+  @Override
+  public Object set(Object o, TimestampTZ t) {
+    if (t == null) {
+      return null;
+    }
+    ((TimestampTZ) o).setZonedDateTime(
+        t.getZonedDateTime().withZoneSameInstant(((TimestampLocalTZTypeInfo) typeInfo).timeZone()));
+    return o;
+  }
+
+  @Override
+  public Object set(Object o, TimestampLocalTZWritable t) {
+    if (t == null) {
+      return null;
+    }
+    ((TimestampTZ) o).setZonedDateTime(
+       t.getTimestampTZ().getZonedDateTime().withZoneSameInstant(((TimestampLocalTZTypeInfo) typeInfo).timeZone()));
+    return o;
+  }
+
+  @Override
+  public Object create(byte[] bytes, int offset) {
+    TimestampTZ t = new TimestampTZ();
+    TimestampLocalTZWritable.setTimestampTZ(
+        t, bytes, offset, ((TimestampLocalTZTypeInfo) typeInfo).timeZone());
+    return t;
+  }
+
+  @Override
+  public Object create(TimestampTZ t) {
+    return t;
+  }
+
+  @Override
+  public TimestampLocalTZWritable getPrimitiveWritableObject(Object o) {
+    if (o == null) {
+      return null;
+    }
+
+    TimestampTZ t = (TimestampTZ) o;
+    TimestampLocalTZTypeInfo timestampTZTypeInfo = (TimestampLocalTZTypeInfo) typeInfo;
+    if (!t.getZonedDateTime().getZone().equals(timestampTZTypeInfo.timeZone())) {
+      t.setZonedDateTime(
+          t.getZonedDateTime().withZoneSameInstant(timestampTZTypeInfo.timeZone()));
+    }
+    return new TimestampLocalTZWritable(t);
+  }
+
+  @Override
+  public TimestampTZ getPrimitiveJavaObject(Object o) {
+    if (o == null) {
+      return null;
+    }
+
+    TimestampTZ t = (TimestampTZ) o;
+    TimestampLocalTZTypeInfo timestampTZTypeInfo = (TimestampLocalTZTypeInfo) typeInfo;
+    if (!t.getZonedDateTime().getZone().equals(timestampTZTypeInfo.timeZone())) {
+      t.setZonedDateTime(
+          t.getZonedDateTime().withZoneSameInstant(timestampTZTypeInfo.timeZone()));
+    }
+    return t;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/34eebff1/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/JavaTimestampTZObjectInspector.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/JavaTimestampTZObjectInspector.java b/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/JavaTimestampTZObjectInspector.java
deleted file mode 100644
index 32b9c69..0000000
--- a/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/JavaTimestampTZObjectInspector.java
+++ /dev/null
@@ -1,76 +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.serde2.objectinspector.primitive;
-
-import org.apache.hadoop.hive.common.type.TimestampTZ;
-import org.apache.hadoop.hive.serde2.io.TimestampTZWritable;
-import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
-
-public class JavaTimestampTZObjectInspector
-    extends AbstractPrimitiveJavaObjectInspector implements SettableTimestampTZObjectInspector {
-
-  JavaTimestampTZObjectInspector() {
-    super(TypeInfoFactory.timestampTZTypeInfo);
-  }
-
-  @Override
-  public Object set(Object o, byte[] bytes, int offset) {
-    TimestampTZWritable.setTimestampTZ((TimestampTZ) o, bytes, offset);
-    return o;
-  }
-
-  @Override
-  public Object set(Object o, TimestampTZ t) {
-    if (t == null) {
-      return null;
-    }
-    ((TimestampTZ) o).set(t.getEpochSecond(), t.getNanos());
-    return o;
-  }
-
-  @Override
-  public Object set(Object o, TimestampTZWritable t) {
-    if (t == null) {
-      return null;
-    }
-    ((TimestampTZ) o).set(t.getSeconds(), t.getNanos());
-    return o;
-  }
-
-  @Override
-  public Object create(byte[] bytes, int offset) {
-    TimestampTZ t = new TimestampTZ();
-    TimestampTZWritable.setTimestampTZ(t, bytes, offset);
-    return t;
-  }
-
-  @Override
-  public Object create(TimestampTZ t) {
-    return new TimestampTZ(t.getZonedDateTime());
-  }
-
-  @Override
-  public TimestampTZWritable getPrimitiveWritableObject(Object o) {
-    return o == null ? null : new TimestampTZWritable((TimestampTZ) o);
-  }
-
-  @Override
-  public TimestampTZ getPrimitiveJavaObject(Object o) {
-    return o == null ? null : (TimestampTZ) o;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/34eebff1/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/PrimitiveObjectInspectorConverter.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/PrimitiveObjectInspectorConverter.java b/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/PrimitiveObjectInspectorConverter.java
index d4b7a32..66d47d7 100644
--- a/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/PrimitiveObjectInspectorConverter.java
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/PrimitiveObjectInspectorConverter.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.hive.serde2.objectinspector.primitive;
 
 import java.sql.Date;
 import java.sql.Timestamp;
+import java.time.ZoneId;
 
 import org.apache.hadoop.hive.common.type.HiveChar;
 import org.apache.hadoop.hive.common.type.HiveDecimal;
@@ -28,12 +29,11 @@ import org.apache.hadoop.hive.common.type.HiveIntervalDayTime;
 import org.apache.hadoop.hive.common.type.HiveVarchar;
 import org.apache.hadoop.hive.common.type.TimestampTZ;
 import org.apache.hadoop.hive.serde2.ByteStream;
-import org.apache.hadoop.hive.serde2.io.HiveCharWritable;
-import org.apache.hadoop.hive.serde2.io.HiveVarcharWritable;
 import org.apache.hadoop.hive.serde2.lazy.LazyInteger;
 import org.apache.hadoop.hive.serde2.lazy.LazyLong;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorConverters.Converter;
 import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector;
+import org.apache.hadoop.hive.serde2.typeinfo.TimestampLocalTZTypeInfo;
 import org.apache.hadoop.io.BytesWritable;
 import org.apache.hadoop.io.Text;
 
@@ -293,16 +293,19 @@ public class PrimitiveObjectInspectorConverter {
     }
   }
 
-  public static class TimestampTZConverter implements Converter {
+  public static class TimestampLocalTZConverter implements Converter {
     final PrimitiveObjectInspector inputOI;
-    final SettableTimestampTZObjectInspector outputOI;
+    final SettableTimestampLocalTZObjectInspector outputOI;
     final Object r;
+    final ZoneId timeZone;
 
-    public TimestampTZConverter(PrimitiveObjectInspector inputOI,
-        SettableTimestampTZObjectInspector outputOI) {
+    public TimestampLocalTZConverter(
+        PrimitiveObjectInspector inputOI,
+        SettableTimestampLocalTZObjectInspector outputOI) {
       this.inputOI = inputOI;
       this.outputOI = outputOI;
-      r = outputOI.create(new TimestampTZ());
+      this.r = outputOI.create(new TimestampTZ());
+      this.timeZone = ((TimestampLocalTZTypeInfo) outputOI.getTypeInfo()).timeZone();
     }
 
     @Override
@@ -311,7 +314,7 @@ public class PrimitiveObjectInspectorConverter {
         return null;
       }
 
-      return outputOI.set(r, PrimitiveObjectInspectorUtils.getTimestampTZ(input, inputOI));
+      return outputOI.set(r, PrimitiveObjectInspectorUtils.getTimestampLocalTZ(input, inputOI, timeZone));
     }
   }
 
@@ -489,8 +492,8 @@ public class PrimitiveObjectInspectorConverter {
         t.set(((TimestampObjectInspector) inputOI)
             .getPrimitiveWritableObject(input).toString());
         return t;
-      case TIMESTAMPTZ:
-        t.set(((TimestampTZObjectInspector) inputOI).getPrimitiveWritableObject(input).toString());
+      case TIMESTAMPLOCALTZ:
+        t.set(((TimestampLocalTZObjectInspector) inputOI).getPrimitiveWritableObject(input).toString());
         return t;
       case INTERVAL_YEAR_MONTH:
         t.set(((HiveIntervalYearMonthObjectInspector) inputOI)

http://git-wip-us.apache.org/repos/asf/hive/blob/34eebff1/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/PrimitiveObjectInspectorFactory.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/PrimitiveObjectInspectorFactory.java b/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/PrimitiveObjectInspectorFactory.java
index 2425c30..2445e08 100644
--- a/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/PrimitiveObjectInspectorFactory.java
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/PrimitiveObjectInspectorFactory.java
@@ -32,7 +32,7 @@ import org.apache.hadoop.hive.serde2.io.HiveIntervalDayTimeWritable;
 import org.apache.hadoop.hive.serde2.io.HiveIntervalYearMonthWritable;
 import org.apache.hadoop.hive.serde2.io.HiveVarcharWritable;
 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.objectinspector.ConstantObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector;
@@ -41,6 +41,7 @@ import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectIn
 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.VarcharTypeInfo;
 import org.apache.hadoop.io.BooleanWritable;
@@ -88,8 +89,8 @@ public final class PrimitiveObjectInspectorFactory {
       new WritableDateObjectInspector();
   public static final WritableTimestampObjectInspector writableTimestampObjectInspector =
       new WritableTimestampObjectInspector();
-  public static final WritableTimestampTZObjectInspector writableTimestampTZObjectInspector =
-      new WritableTimestampTZObjectInspector();
+  public static final WritableTimestampLocalTZObjectInspector writableTimestampTZObjectInspector =
+      new WritableTimestampLocalTZObjectInspector(TypeInfoFactory.timestampLocalTZTypeInfo);
   public static final WritableHiveIntervalYearMonthObjectInspector writableHiveIntervalYearMonthObjectInspector =
       new WritableHiveIntervalYearMonthObjectInspector();
   public static final WritableHiveIntervalDayTimeObjectInspector writableHiveIntervalDayTimeObjectInspector =
@@ -127,8 +128,7 @@ public final class PrimitiveObjectInspectorFactory {
         writableDateObjectInspector);
     cachedPrimitiveWritableInspectorCache.put(TypeInfoFactory.getPrimitiveTypeInfo(serdeConstants.TIMESTAMP_TYPE_NAME),
         writableTimestampObjectInspector);
-    cachedPrimitiveWritableInspectorCache.put(TypeInfoFactory.getPrimitiveTypeInfo(
-        serdeConstants.TIMESTAMPTZ_TYPE_NAME), writableTimestampTZObjectInspector);
+    cachedPrimitiveWritableInspectorCache.put(TypeInfoFactory.timestampLocalTZTypeInfo, writableTimestampTZObjectInspector);
     cachedPrimitiveWritableInspectorCache.put(TypeInfoFactory.getPrimitiveTypeInfo(serdeConstants.INTERVAL_YEAR_MONTH_TYPE_NAME),
         writableHiveIntervalYearMonthObjectInspector);
     cachedPrimitiveWritableInspectorCache.put(TypeInfoFactory.getPrimitiveTypeInfo(serdeConstants.INTERVAL_DAY_TIME_TYPE_NAME),
@@ -154,8 +154,7 @@ public final class PrimitiveObjectInspectorFactory {
     primitiveCategoryToWritableOI.put(PrimitiveCategory.VOID, writableVoidObjectInspector);
     primitiveCategoryToWritableOI.put(PrimitiveCategory.DATE, writableDateObjectInspector);
     primitiveCategoryToWritableOI.put(PrimitiveCategory.TIMESTAMP, writableTimestampObjectInspector);
-    primitiveCategoryToWritableOI.put(PrimitiveCategory.TIMESTAMPTZ,
-        writableTimestampTZObjectInspector);
+    primitiveCategoryToWritableOI.put(PrimitiveCategory.TIMESTAMPLOCALTZ, writableTimestampTZObjectInspector);
     primitiveCategoryToWritableOI.put(PrimitiveCategory.INTERVAL_YEAR_MONTH, writableHiveIntervalYearMonthObjectInspector);
     primitiveCategoryToWritableOI.put(PrimitiveCategory.INTERVAL_DAY_TIME, writableHiveIntervalDayTimeObjectInspector);
     primitiveCategoryToWritableOI.put(PrimitiveCategory.BINARY, writableBinaryObjectInspector);
@@ -188,8 +187,8 @@ public final class PrimitiveObjectInspectorFactory {
       new JavaDateObjectInspector();
   public static final JavaTimestampObjectInspector javaTimestampObjectInspector =
       new JavaTimestampObjectInspector();
-  public static final JavaTimestampTZObjectInspector javaTimestampTZObjectInspector =
-      new JavaTimestampTZObjectInspector();
+  public static final JavaTimestampLocalTZObjectInspector javaTimestampTZObjectInspector =
+      new JavaTimestampLocalTZObjectInspector(TypeInfoFactory.timestampLocalTZTypeInfo);
   public static final JavaHiveIntervalYearMonthObjectInspector javaHiveIntervalYearMonthObjectInspector =
       new JavaHiveIntervalYearMonthObjectInspector();
   public static final JavaHiveIntervalDayTimeObjectInspector javaHiveIntervalDayTimeObjectInspector =
@@ -227,8 +226,7 @@ public final class PrimitiveObjectInspectorFactory {
         javaDateObjectInspector);
     cachedPrimitiveJavaInspectorCache.put(TypeInfoFactory.getPrimitiveTypeInfo(serdeConstants.TIMESTAMP_TYPE_NAME),
         javaTimestampObjectInspector);
-    cachedPrimitiveJavaInspectorCache.put(TypeInfoFactory.timestampTZTypeInfo,
-        javaTimestampTZObjectInspector);
+    cachedPrimitiveJavaInspectorCache.put(TypeInfoFactory.timestampLocalTZTypeInfo, javaTimestampTZObjectInspector);
     cachedPrimitiveJavaInspectorCache.put(TypeInfoFactory.getPrimitiveTypeInfo(serdeConstants.INTERVAL_YEAR_MONTH_TYPE_NAME),
         javaHiveIntervalYearMonthObjectInspector);
     cachedPrimitiveJavaInspectorCache.put(TypeInfoFactory.getPrimitiveTypeInfo(serdeConstants.INTERVAL_DAY_TIME_TYPE_NAME),
@@ -254,7 +252,7 @@ public final class PrimitiveObjectInspectorFactory {
     primitiveCategoryToJavaOI.put(PrimitiveCategory.VOID, javaVoidObjectInspector);
     primitiveCategoryToJavaOI.put(PrimitiveCategory.DATE, javaDateObjectInspector);
     primitiveCategoryToJavaOI.put(PrimitiveCategory.TIMESTAMP, javaTimestampObjectInspector);
-    primitiveCategoryToJavaOI.put(PrimitiveCategory.TIMESTAMPTZ, javaTimestampTZObjectInspector);
+    primitiveCategoryToJavaOI.put(PrimitiveCategory.TIMESTAMPLOCALTZ, javaTimestampTZObjectInspector);
     primitiveCategoryToJavaOI.put(PrimitiveCategory.INTERVAL_YEAR_MONTH, javaHiveIntervalYearMonthObjectInspector);
     primitiveCategoryToJavaOI.put(PrimitiveCategory.INTERVAL_DAY_TIME, javaHiveIntervalDayTimeObjectInspector);
     primitiveCategoryToJavaOI.put(PrimitiveCategory.BINARY, javaByteArrayObjectInspector);
@@ -297,6 +295,9 @@ public final class PrimitiveObjectInspectorFactory {
     case VARCHAR:
       result = new WritableHiveVarcharObjectInspector((VarcharTypeInfo)typeInfo);
       break;
+    case TIMESTAMPLOCALTZ:
+      result = new WritableTimestampLocalTZObjectInspector((TimestampLocalTZTypeInfo)typeInfo);
+      break;
     case DECIMAL:
       result = new WritableHiveDecimalObjectInspector((DecimalTypeInfo)typeInfo);
       break;
@@ -348,8 +349,8 @@ public final class PrimitiveObjectInspectorFactory {
       return new WritableConstantDateObjectInspector((DateWritable)value);
     case TIMESTAMP:
       return new WritableConstantTimestampObjectInspector((TimestampWritable)value);
-    case TIMESTAMPTZ:
-      return new WritableConstantTimestampTZObjectInspector((TimestampTZWritable) value);
+    case TIMESTAMPLOCALTZ:
+      return new WritableConstantTimestampLocalTZObjectInspector((TimestampLocalTZTypeInfo)typeInfo, (TimestampLocalTZWritable) value);
     case INTERVAL_YEAR_MONTH:
       return new WritableConstantHiveIntervalYearMonthObjectInspector((HiveIntervalYearMonthWritable) value);
     case INTERVAL_DAY_TIME:
@@ -402,6 +403,9 @@ public final class PrimitiveObjectInspectorFactory {
     case VARCHAR:
       result = new JavaHiveVarcharObjectInspector((VarcharTypeInfo)typeInfo);
       break;
+    case TIMESTAMPLOCALTZ:
+      result = new JavaTimestampLocalTZObjectInspector((TimestampLocalTZTypeInfo)typeInfo);
+      break;
     case DECIMAL:
       result = new JavaHiveDecimalObjectInspector((DecimalTypeInfo)typeInfo);
       break;

http://git-wip-us.apache.org/repos/asf/hive/blob/34eebff1/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/PrimitiveObjectInspectorUtils.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/PrimitiveObjectInspectorUtils.java b/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/PrimitiveObjectInspectorUtils.java
index 886c298..084d40b 100644
--- a/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/PrimitiveObjectInspectorUtils.java
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/PrimitiveObjectInspectorUtils.java
@@ -24,14 +24,15 @@ import java.io.IOException;
 import java.nio.charset.CharacterCodingException;
 import java.sql.Date;
 import java.sql.Timestamp;
+import java.time.DateTimeException;
+import java.time.ZoneId;
 import java.util.HashMap;
 import java.util.Map;
 
 import org.apache.hadoop.hive.common.type.TimestampTZ;
+import org.apache.hadoop.hive.common.type.TimestampTZUtil;
 import org.apache.hadoop.hive.ql.util.TimestampUtils;
-import org.apache.hadoop.hive.serde2.io.TimestampTZWritable;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
+import org.apache.hadoop.hive.serde2.io.TimestampLocalTZWritable;
 import org.apache.hadoop.hive.common.type.HiveChar;
 import org.apache.hadoop.hive.common.type.HiveDecimal;
 import org.apache.hadoop.hive.common.type.HiveIntervalYearMonth;
@@ -72,7 +73,6 @@ import org.apache.hadoop.io.WritableUtils;
  * ObjectInspector to return to the caller of SerDe2.getObjectInspector().
  */
 public final class PrimitiveObjectInspectorUtils {
-  private static final Logger LOG = LoggerFactory.getLogger(PrimitiveObjectInspectorUtils.class);
 
   /**
    * TypeEntry stores information about a Hive Primitive TypeInfo.
@@ -229,8 +229,8 @@ public final class PrimitiveObjectInspectorUtils {
       PrimitiveCategory.TIMESTAMP, serdeConstants.TIMESTAMP_TYPE_NAME, null,
       Timestamp.class, TimestampWritable.class);
   public static final PrimitiveTypeEntry timestampTZTypeEntry = new PrimitiveTypeEntry(
-      PrimitiveCategory.TIMESTAMPTZ, serdeConstants.TIMESTAMPTZ_TYPE_NAME, null,
-      TimestampTZ.class, TimestampTZWritable.class);
+      PrimitiveCategory.TIMESTAMPLOCALTZ, serdeConstants.TIMESTAMPLOCALTZ_TYPE_NAME, null,
+      TimestampTZ.class, TimestampLocalTZWritable.class);
   public static final PrimitiveTypeEntry intervalYearMonthTypeEntry = new PrimitiveTypeEntry(
       PrimitiveCategory.INTERVAL_YEAR_MONTH, serdeConstants.INTERVAL_YEAR_MONTH_TYPE_NAME, null,
       HiveIntervalYearMonth.class, HiveIntervalYearMonthWritable.class);
@@ -445,9 +445,9 @@ public final class PrimitiveObjectInspectorUtils {
       return ((TimestampObjectInspector) oi1).getPrimitiveWritableObject(o1)
           .equals(((TimestampObjectInspector) oi2).getPrimitiveWritableObject(o2));
     }
-    case TIMESTAMPTZ: {
-      return ((TimestampTZObjectInspector) oi1).getPrimitiveWritableObject(o1).equals(
-          ((TimestampTZObjectInspector) oi2).getPrimitiveWritableObject(o2));
+    case TIMESTAMPLOCALTZ: {
+      return ((TimestampLocalTZObjectInspector) oi1).getPrimitiveWritableObject(o1).equals(
+          ((TimestampLocalTZObjectInspector) oi2).getPrimitiveWritableObject(o2));
     }
     case INTERVAL_YEAR_MONTH: {
       return ((HiveIntervalYearMonthObjectInspector) oi1).getPrimitiveWritableObject(o1)
@@ -880,8 +880,8 @@ public final class PrimitiveObjectInspectorUtils {
     case TIMESTAMP:
       result = ((TimestampObjectInspector) oi).getPrimitiveWritableObject(o).toString();
       break;
-    case TIMESTAMPTZ:
-      result = ((TimestampTZObjectInspector) oi).getPrimitiveWritableObject(o).toString();
+    case TIMESTAMPLOCALTZ:
+      result = ((TimestampLocalTZObjectInspector) oi).getPrimitiveWritableObject(o).toString();
       break;
     case INTERVAL_YEAR_MONTH:
       result = ((HiveIntervalYearMonthObjectInspector) oi).getPrimitiveWritableObject(o).toString();
@@ -1051,16 +1051,26 @@ public final class PrimitiveObjectInspectorUtils {
       try {
         result = Date.valueOf(s);
       } catch (IllegalArgumentException e) {
-        result = null;
+        Timestamp ts = getTimestampFromString(s);
+        if (ts != null) {
+          result = new Date(ts.getTime());
+        } else {
+          result = null;
+        }
       }
       break;
     case CHAR:
     case VARCHAR: {
+      String val = getString(o, oi).trim();
       try {
-        String val = getString(o, oi).trim();
         result = Date.valueOf(val);
       } catch (IllegalArgumentException e) {
-        result = null;
+        Timestamp ts = getTimestampFromString(val);
+        if (ts != null) {
+          result = new Date(ts.getTime());
+        } else {
+          result = null;
+        }
       }
       break;
     }
@@ -1071,7 +1081,7 @@ public final class PrimitiveObjectInspectorUtils {
       result = DateWritable.timeToDate(
           ((TimestampObjectInspector) oi).getPrimitiveWritableObject(o).getSeconds());
       break;
-    case TIMESTAMPTZ:
+    case TIMESTAMPLOCALTZ:
       String tstz = oi.getPrimitiveWritableObject(o).toString();
       int divSpace = tstz.indexOf(' ');
       if (divSpace == -1) {
@@ -1148,7 +1158,7 @@ public final class PrimitiveObjectInspectorUtils {
     case TIMESTAMP:
       result = ((TimestampObjectInspector) inputOI).getPrimitiveWritableObject(o).getTimestamp();
       break;
-    case TIMESTAMPTZ:
+    case TIMESTAMPLOCALTZ:
       String tstz = inputOI.getPrimitiveWritableObject(o).toString();
       int index = tstz.indexOf(" ");
       index = tstz.indexOf(" ", index + 1);
@@ -1164,7 +1174,44 @@ public final class PrimitiveObjectInspectorUtils {
     return result;
   }
 
-  public static TimestampTZ getTimestampTZ(Object o, PrimitiveObjectInspector oi) {
+  static Timestamp getTimestampFromString(String s) {
+    Timestamp result;
+    s = s.trim();
+    s = trimNanoTimestamp(s);
+
+    int firstSpace = s.indexOf(' ');
+    if (firstSpace < 0) {
+      s = s.concat(" 00:00:00");
+    }
+    try {
+      result = Timestamp.valueOf(s);
+    } catch (IllegalArgumentException e) {
+      // Let's try to parse it as timestamp with time zone and transform
+      try {
+        result = Timestamp.from(TimestampTZUtil.parse(s).getZonedDateTime().toInstant());
+      } catch (DateTimeException e2) {
+        result = null;
+      }
+    }
+    return result;
+  }
+
+  private static String trimNanoTimestamp(String s) {
+    int firstSpace = s.indexOf(' ');
+    // Throw away extra if more than 9 decimal places
+    int periodIdx = s.indexOf(".");
+    if (periodIdx != -1) {
+      int secondSpace = firstSpace < 0 ? -1 : s.indexOf(' ', firstSpace + 1);
+      int maxLength = secondSpace == -1 ? s.length() : secondSpace;
+      if (maxLength - periodIdx > 9) {
+        s = s.substring(0, periodIdx + 10).concat(s.substring(maxLength, s.length()));
+      }
+    }
+    return s;
+  }
+
+  public static TimestampTZ getTimestampLocalTZ(Object o, PrimitiveObjectInspector oi,
+          ZoneId timeZone) {
     if (o == null) {
       return null;
     }
@@ -1172,57 +1219,33 @@ public final class PrimitiveObjectInspectorUtils {
     case VOID: {
       return null;
     }
-    // The resulting timestamp with time zone will have TZ in UTC
-    // instead of the original TZ in the string representation.
     case STRING: {
       StringObjectInspector soi = (StringObjectInspector) oi;
       String s = soi.getPrimitiveJavaObject(o).trim();
-      return TimestampTZ.parseOrNull(s);
+      return TimestampTZUtil.parseOrNull(trimNanoTimestamp(s), timeZone);
     }
     case CHAR:
     case VARCHAR: {
       String s = getString(o, oi).trim();
-      return TimestampTZ.parseOrNull(s);
+      return TimestampTZUtil.parseOrNull(trimNanoTimestamp(s), timeZone);
     }
     case DATE: {
       Date date = ((DateObjectInspector) oi).getPrimitiveWritableObject(o).get();
-      return TimestampTZ.convert(date);
+      return TimestampTZUtil.convert(date, timeZone);
     }
     case TIMESTAMP: {
       Timestamp ts = ((TimestampObjectInspector) oi).getPrimitiveWritableObject(o).getTimestamp();
-      return TimestampTZ.convert(ts);
+      return TimestampTZUtil.convert(ts, timeZone);
     }
-    case TIMESTAMPTZ: {
-      return ((TimestampTZObjectInspector) oi).getPrimitiveWritableObject(o).getTimestampTZ();
+    case TIMESTAMPLOCALTZ: {
+      return ((TimestampLocalTZObjectInspector) oi).getPrimitiveWritableObject(o).getTimestampTZ();
     }
     default:
-      throw new RuntimeException("Cannot convert to " + serdeConstants.TIMESTAMPTZ_TYPE_NAME +
+      throw new RuntimeException("Cannot convert to " + serdeConstants.TIMESTAMPLOCALTZ_TYPE_NAME +
           " from: " + oi.getTypeName());
     }
   }
 
-  static Timestamp getTimestampFromString(String s) {
-    Timestamp result;
-    s = s.trim();
-
-    // Throw away extra if more than 9 decimal places
-    int periodIdx = s.indexOf(".");
-    if (periodIdx != -1) {
-      if (s.length() - periodIdx > 9) {
-        s = s.substring(0, periodIdx + 10);
-      }
-    }
-    if (s.indexOf(' ') < 0) {
-      s = s.concat(" 00:00:00");
-    }
-    try {
-      result = Timestamp.valueOf(s);
-    } catch (IllegalArgumentException e) {
-      result = null;
-    }
-    return result;
-  }
-
   public static HiveIntervalYearMonth getHiveIntervalYearMonth(Object o, PrimitiveObjectInspector oi) {
     if (o == null) {
       return null;
@@ -1331,7 +1354,7 @@ public final class PrimitiveObjectInspectorUtils {
         return PrimitiveGrouping.BOOLEAN_GROUP;
       case TIMESTAMP:
       case DATE:
-      case TIMESTAMPTZ:
+      case TIMESTAMPLOCALTZ:
         return PrimitiveGrouping.DATE_GROUP;
       case INTERVAL_YEAR_MONTH:
       case INTERVAL_DAY_TIME:

http://git-wip-us.apache.org/repos/asf/hive/blob/34eebff1/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/SettableTimestampLocalTZObjectInspector.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/SettableTimestampLocalTZObjectInspector.java b/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/SettableTimestampLocalTZObjectInspector.java
new file mode 100644
index 0000000..d10a361
--- /dev/null
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/SettableTimestampLocalTZObjectInspector.java
@@ -0,0 +1,34 @@
+/**
+ * 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.serde2.objectinspector.primitive;
+
+import org.apache.hadoop.hive.common.type.TimestampTZ;
+import org.apache.hadoop.hive.serde2.io.TimestampLocalTZWritable;
+
+public interface SettableTimestampLocalTZObjectInspector extends TimestampLocalTZObjectInspector {
+
+  Object set(Object o, byte[] bytes, int offset);
+
+  Object set(Object o, TimestampTZ t);
+
+  Object set(Object o, TimestampLocalTZWritable t);
+
+  Object create(byte[] bytes, int offset);
+
+  Object create(TimestampTZ t);
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/34eebff1/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/SettableTimestampTZObjectInspector.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/SettableTimestampTZObjectInspector.java b/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/SettableTimestampTZObjectInspector.java
deleted file mode 100644
index 032bc38..0000000
--- a/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/SettableTimestampTZObjectInspector.java
+++ /dev/null
@@ -1,34 +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.serde2.objectinspector.primitive;
-
-import org.apache.hadoop.hive.common.type.TimestampTZ;
-import org.apache.hadoop.hive.serde2.io.TimestampTZWritable;
-
-public interface SettableTimestampTZObjectInspector extends TimestampTZObjectInspector {
-
-  Object set(Object o, byte[] bytes, int offset);
-
-  Object set(Object o, TimestampTZ t);
-
-  Object set(Object o, TimestampTZWritable t);
-
-  Object create(byte[] bytes, int offset);
-
-  Object create(TimestampTZ t);
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/34eebff1/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/TimestampLocalTZObjectInspector.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/TimestampLocalTZObjectInspector.java b/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/TimestampLocalTZObjectInspector.java
new file mode 100644
index 0000000..6962a46
--- /dev/null
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/TimestampLocalTZObjectInspector.java
@@ -0,0 +1,29 @@
+/**
+ * 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.serde2.objectinspector.primitive;
+
+import org.apache.hadoop.hive.common.type.TimestampTZ;
+import org.apache.hadoop.hive.serde2.io.TimestampLocalTZWritable;
+import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector;
+
+public interface TimestampLocalTZObjectInspector extends PrimitiveObjectInspector {
+
+  TimestampLocalTZWritable getPrimitiveWritableObject(Object o);
+
+  TimestampTZ getPrimitiveJavaObject(Object o);
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/34eebff1/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/TimestampTZObjectInspector.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/TimestampTZObjectInspector.java b/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/TimestampTZObjectInspector.java
deleted file mode 100644
index d142d38..0000000
--- a/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/TimestampTZObjectInspector.java
+++ /dev/null
@@ -1,29 +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.serde2.objectinspector.primitive;
-
-import org.apache.hadoop.hive.common.type.TimestampTZ;
-import org.apache.hadoop.hive.serde2.io.TimestampTZWritable;
-import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector;
-
-public interface TimestampTZObjectInspector extends PrimitiveObjectInspector {
-
-  TimestampTZWritable getPrimitiveWritableObject(Object o);
-
-  TimestampTZ getPrimitiveJavaObject(Object o);
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/34eebff1/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/WritableConstantTimestampLocalTZObjectInspector.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/WritableConstantTimestampLocalTZObjectInspector.java b/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/WritableConstantTimestampLocalTZObjectInspector.java
new file mode 100644
index 0000000..1713e77
--- /dev/null
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/WritableConstantTimestampLocalTZObjectInspector.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
+ * <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.serde2.objectinspector.primitive;
+
+import org.apache.hadoop.hive.serde2.io.TimestampLocalTZWritable;
+import org.apache.hadoop.hive.serde2.objectinspector.ConstantObjectInspector;
+import org.apache.hadoop.hive.serde2.typeinfo.TimestampLocalTZTypeInfo;
+
+public class WritableConstantTimestampLocalTZObjectInspector
+    extends WritableTimestampLocalTZObjectInspector implements ConstantObjectInspector {
+
+  private TimestampLocalTZWritable value;
+
+  public WritableConstantTimestampLocalTZObjectInspector(
+      TimestampLocalTZTypeInfo typeInfo,
+      TimestampLocalTZWritable value) {
+    super(typeInfo);
+    this.value = value;
+  }
+
+  @Override
+  public Object getWritableConstantValue() {
+    return value;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/34eebff1/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/WritableConstantTimestampTZObjectInspector.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/WritableConstantTimestampTZObjectInspector.java b/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/WritableConstantTimestampTZObjectInspector.java
deleted file mode 100644
index 5805ce8..0000000
--- a/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/WritableConstantTimestampTZObjectInspector.java
+++ /dev/null
@@ -1,36 +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.serde2.objectinspector.primitive;
-
-import org.apache.hadoop.hive.serde2.io.TimestampTZWritable;
-import org.apache.hadoop.hive.serde2.objectinspector.ConstantObjectInspector;
-
-public class WritableConstantTimestampTZObjectInspector
-    extends WritableTimestampTZObjectInspector implements ConstantObjectInspector {
-
-  private TimestampTZWritable value;
-
-  public WritableConstantTimestampTZObjectInspector(TimestampTZWritable value) {
-    this.value = value;
-  }
-
-  @Override
-  public Object getWritableConstantValue() {
-    return value;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/34eebff1/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/WritableTimestampLocalTZObjectInspector.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/WritableTimestampLocalTZObjectInspector.java b/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/WritableTimestampLocalTZObjectInspector.java
new file mode 100644
index 0000000..d555bd3
--- /dev/null
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/WritableTimestampLocalTZObjectInspector.java
@@ -0,0 +1,107 @@
+/**
+ * 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.serde2.objectinspector.primitive;
+
+import org.apache.hadoop.hive.common.type.TimestampTZ;
+import org.apache.hadoop.hive.serde2.io.TimestampLocalTZWritable;
+import org.apache.hadoop.hive.serde2.typeinfo.TimestampLocalTZTypeInfo;
+
+public class WritableTimestampLocalTZObjectInspector extends
+    AbstractPrimitiveWritableObjectInspector implements SettableTimestampLocalTZObjectInspector {
+
+  public WritableTimestampLocalTZObjectInspector() {
+  }
+
+  public WritableTimestampLocalTZObjectInspector(TimestampLocalTZTypeInfo typeInfo) {
+    super(typeInfo);
+  }
+
+  @Override
+  public TimestampLocalTZWritable getPrimitiveWritableObject(Object o) {
+    if (o == null) {
+      return null;
+    }
+    TimestampLocalTZWritable t = (TimestampLocalTZWritable) o;
+    TimestampLocalTZTypeInfo timestampTZTypeInfo = (TimestampLocalTZTypeInfo) typeInfo;
+    if (!t.getTimestampTZ().getZonedDateTime().getZone().equals(timestampTZTypeInfo.timeZone())) {
+      t.setTimeZone(timestampTZTypeInfo.timeZone());
+    }
+    return t;
+  }
+
+  @Override
+  public TimestampTZ getPrimitiveJavaObject(Object o) {
+    if (o == null) {
+      return null;
+    }
+    TimestampLocalTZWritable t = (TimestampLocalTZWritable) o;
+    TimestampLocalTZTypeInfo timestampTZTypeInfo = (TimestampLocalTZTypeInfo) typeInfo;
+    if (!t.getTimestampTZ().getZonedDateTime().getZone().equals(timestampTZTypeInfo.timeZone())) {
+      t.setTimeZone(timestampTZTypeInfo.timeZone());
+    }
+    return t.getTimestampTZ();
+  }
+
+  @Override
+  public Object set(Object o, byte[] bytes, int offset) {
+    ((TimestampLocalTZWritable) o).set(bytes, offset, ((TimestampLocalTZTypeInfo) typeInfo).timeZone());
+    return o;
+  }
+
+  @Override
+  public Object set(Object o, TimestampTZ t) {
+    if (t == null) {
+      return null;
+    }
+    TimestampLocalTZTypeInfo timestampTZTypeInfo = (TimestampLocalTZTypeInfo) typeInfo;
+    if (!t.getZonedDateTime().getZone().equals(timestampTZTypeInfo.timeZone())) {
+      t.setZonedDateTime(t.getZonedDateTime().withZoneSameInstant(timestampTZTypeInfo.timeZone()));
+    }
+    ((TimestampLocalTZWritable) o).set(t);
+    return o;
+  }
+
+  @Override
+  public Object set(Object o, TimestampLocalTZWritable t) {
+    if (t == null) {
+      return null;
+    }
+    TimestampLocalTZTypeInfo timestampTZTypeInfo = (TimestampLocalTZTypeInfo) typeInfo;
+    if (!t.getTimestampTZ().getZonedDateTime().getZone().equals(timestampTZTypeInfo.timeZone())) {
+      t.getTimestampTZ().setZonedDateTime(
+          t.getTimestampTZ().getZonedDateTime().withZoneSameInstant(timestampTZTypeInfo.timeZone()));
+    }
+    ((TimestampLocalTZWritable) o).set(t);
+    return o;
+  }
+
+  @Override
+  public Object create(byte[] bytes, int offset) {
+    return new TimestampLocalTZWritable(bytes, offset, ((TimestampLocalTZTypeInfo) typeInfo).timeZone());
+  }
+
+  @Override
+  public Object create(TimestampTZ t) {
+    return new TimestampLocalTZWritable(t);
+  }
+
+  @Override
+  public Object copyObject(Object o) {
+    return o == null ? null : new TimestampLocalTZWritable((TimestampLocalTZWritable) o);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/34eebff1/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/WritableTimestampTZObjectInspector.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/WritableTimestampTZObjectInspector.java b/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/WritableTimestampTZObjectInspector.java
deleted file mode 100644
index 0b622c1..0000000
--- a/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/WritableTimestampTZObjectInspector.java
+++ /dev/null
@@ -1,79 +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.serde2.objectinspector.primitive;
-
-import org.apache.hadoop.hive.common.type.TimestampTZ;
-import org.apache.hadoop.hive.serde2.io.TimestampTZWritable;
-import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
-
-public class WritableTimestampTZObjectInspector extends
-    AbstractPrimitiveWritableObjectInspector implements SettableTimestampTZObjectInspector {
-
-  public WritableTimestampTZObjectInspector() {
-    super(TypeInfoFactory.timestampTZTypeInfo);
-  }
-
-  @Override
-  public TimestampTZWritable getPrimitiveWritableObject(Object o) {
-    return o == null ? null : (TimestampTZWritable) o;
-  }
-
-  @Override
-  public Object set(Object o, byte[] bytes, int offset) {
-    ((TimestampTZWritable) o).set(bytes, offset);
-    return o;
-  }
-
-  @Override
-  public Object set(Object o, TimestampTZ t) {
-    if (t == null) {
-      return null;
-    }
-    ((TimestampTZWritable) o).set(t);
-    return o;
-  }
-
-  @Override
-  public Object set(Object o, TimestampTZWritable t) {
-    if (t == null) {
-      return null;
-    }
-    ((TimestampTZWritable) o).set(t);
-    return o;
-  }
-
-  @Override
-  public Object create(byte[] bytes, int offset) {
-    return new TimestampTZWritable(bytes, offset);
-  }
-
-  @Override
-  public Object create(TimestampTZ t) {
-    return new TimestampTZWritable(t);
-  }
-
-  @Override
-  public TimestampTZ getPrimitiveJavaObject(Object o) {
-    return o == null ? null : ((TimestampTZWritable) o).getTimestampTZ();
-  }
-
-  @Override
-  public Object copyObject(Object o) {
-    return o == null ? null : new TimestampTZWritable((TimestampTZWritable) o);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/34eebff1/serde/src/java/org/apache/hadoop/hive/serde2/thrift/Type.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/thrift/Type.java b/serde/src/java/org/apache/hadoop/hive/serde2/thrift/Type.java
index 809f45b..3b56c12 100644
--- a/serde/src/java/org/apache/hadoop/hive/serde2/thrift/Type.java
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/thrift/Type.java
@@ -72,9 +72,9 @@ public enum Type {
   TIMESTAMP_TYPE("TIMESTAMP",
       java.sql.Types.TIMESTAMP,
       TTypeId.TIMESTAMP_TYPE),
-  TIMESTAMPTZ_TYPE(serdeConstants.TIMESTAMPTZ_TYPE_NAME.toUpperCase(),
-      Types.TIMESTAMP_WITH_TIMEZONE,
-      TTypeId.TIMESTAMPTZ_TYPE),
+  TIMESTAMPLOCALTZ_TYPE(serdeConstants.TIMESTAMPLOCALTZ_TYPE_NAME.toUpperCase(),
+      java.sql.Types.OTHER,
+      TTypeId.TIMESTAMPLOCALTZ_TYPE),
   INTERVAL_YEAR_MONTH_TYPE("INTERVAL_YEAR_MONTH",
       java.sql.Types.OTHER,
       TTypeId.INTERVAL_YEAR_MONTH_TYPE),
@@ -230,8 +230,8 @@ public enum Type {
       case TIMESTAMP: {
         return Type.TIMESTAMP_TYPE;
       }
-      case TIMESTAMPTZ: {
-        return Type.TIMESTAMPTZ_TYPE;
+      case TIMESTAMPLOCALTZ: {
+        return Type.TIMESTAMPLOCALTZ_TYPE;
       }
       case INTERVAL_YEAR_MONTH: {
         return Type.INTERVAL_YEAR_MONTH_TYPE;

http://git-wip-us.apache.org/repos/asf/hive/blob/34eebff1/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/TimestampLocalTZTypeInfo.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/TimestampLocalTZTypeInfo.java b/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/TimestampLocalTZTypeInfo.java
new file mode 100644
index 0000000..e19d7b0
--- /dev/null
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/TimestampLocalTZTypeInfo.java
@@ -0,0 +1,104 @@
+/**
+ * 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.serde2.typeinfo;
+
+import java.time.ZoneId;
+import java.util.Objects;
+
+import org.apache.hadoop.hive.common.type.TimestampTZUtil;
+import org.apache.hadoop.hive.serde.serdeConstants;
+
+public class TimestampLocalTZTypeInfo extends PrimitiveTypeInfo {
+  private static final long serialVersionUID = 1L;
+
+  private ZoneId timeZone;
+
+  public TimestampLocalTZTypeInfo() {
+    super(serdeConstants.TIMESTAMPLOCALTZ_TYPE_NAME);
+  }
+
+  public TimestampLocalTZTypeInfo(String timeZoneStr) {
+    super(serdeConstants.TIMESTAMPLOCALTZ_TYPE_NAME);
+    this.timeZone = TimestampTZUtil.parseTimeZone(timeZoneStr);
+  }
+
+  @Override
+  public String getTypeName() {
+    return serdeConstants.TIMESTAMPLOCALTZ_TYPE_NAME;
+  }
+
+  @Override
+  public void setTypeName(String typeName) {
+    // No need to set type name, it should always be timestamplocaltz
+    return;
+  }
+
+  @Override
+  public boolean equals(Object other) {
+    if (this == other) {
+      return true;
+    }
+    if (other == null || getClass() != other.getClass()) {
+      return false;
+    }
+
+    TimestampLocalTZTypeInfo dti = (TimestampLocalTZTypeInfo) other;
+
+    return this.timeZone().equals(dti.timeZone());
+  }
+
+  /**
+   * Generate the hashCode for this TypeInfo.
+   */
+  @Override
+  public int hashCode() {
+    return Objects.hash(typeName, timeZone);
+  }
+
+  @Override
+  public String toString() {
+    return getQualifiedName();
+  }
+
+  @Override
+  public String getQualifiedName() {
+    return getQualifiedName(timeZone);
+  }
+
+  public static String getQualifiedName(ZoneId timeZone) {
+    StringBuilder sb = new StringBuilder(serdeConstants.TIMESTAMPLOCALTZ_TYPE_NAME);
+    sb.append("('");
+    sb.append(timeZone);
+    sb.append("')");
+    return sb.toString();
+  }
+
+  public ZoneId timeZone() {
+    return timeZone;
+  }
+
+  public ZoneId getTimeZone() {
+    return timeZone;
+  }
+
+  public void setTimeZone(ZoneId timeZone) {
+    this.timeZone = timeZone;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/34eebff1/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/TypeInfoFactory.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/TypeInfoFactory.java b/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/TypeInfoFactory.java
index b0e0bf2..66f5f8f 100644
--- a/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/TypeInfoFactory.java
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/TypeInfoFactory.java
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.hive.serde2.typeinfo;
 
+import java.time.ZoneId;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.concurrent.ConcurrentHashMap;
@@ -55,8 +56,6 @@ public final class TypeInfoFactory {
   public static final PrimitiveTypeInfo shortTypeInfo = new PrimitiveTypeInfo(serdeConstants.SMALLINT_TYPE_NAME);
   public static final PrimitiveTypeInfo dateTypeInfo = new PrimitiveTypeInfo(serdeConstants.DATE_TYPE_NAME);
   public static final PrimitiveTypeInfo timestampTypeInfo = new PrimitiveTypeInfo(serdeConstants.TIMESTAMP_TYPE_NAME);
-  public static final PrimitiveTypeInfo timestampTZTypeInfo =
-      new PrimitiveTypeInfo(serdeConstants.TIMESTAMPTZ_TYPE_NAME);
   public static final PrimitiveTypeInfo intervalYearMonthTypeInfo = new PrimitiveTypeInfo(serdeConstants.INTERVAL_YEAR_MONTH_TYPE_NAME);
   public static final PrimitiveTypeInfo intervalDayTimeTypeInfo = new PrimitiveTypeInfo(serdeConstants.INTERVAL_DAY_TIME_TYPE_NAME);
   public static final PrimitiveTypeInfo binaryTypeInfo = new PrimitiveTypeInfo(serdeConstants.BINARY_TYPE_NAME);
@@ -67,6 +66,12 @@ public final class TypeInfoFactory {
   public static final DecimalTypeInfo decimalTypeInfo = new DecimalTypeInfo(HiveDecimal.SYSTEM_DEFAULT_PRECISION,
       HiveDecimal.SYSTEM_DEFAULT_SCALE);
 
+  /**
+   * A TimestampTZTypeInfo with system default time zone.
+   */
+  public static final TimestampLocalTZTypeInfo timestampLocalTZTypeInfo = new TimestampLocalTZTypeInfo(
+      ZoneId.systemDefault().getId());
+
   public static final PrimitiveTypeInfo unknownTypeInfo = new PrimitiveTypeInfo("unknown");
 
   // Map from type name (such as int or varchar(40) to the corresponding PrimitiveTypeInfo
@@ -87,7 +92,7 @@ public final class TypeInfoFactory {
     cachedPrimitiveTypeInfo.put(serdeConstants.SMALLINT_TYPE_NAME, shortTypeInfo);
     cachedPrimitiveTypeInfo.put(serdeConstants.DATE_TYPE_NAME, dateTypeInfo);
     cachedPrimitiveTypeInfo.put(serdeConstants.TIMESTAMP_TYPE_NAME, timestampTypeInfo);
-    cachedPrimitiveTypeInfo.put(serdeConstants.TIMESTAMPTZ_TYPE_NAME, timestampTZTypeInfo);
+    cachedPrimitiveTypeInfo.put(serdeConstants.TIMESTAMPLOCALTZ_TYPE_NAME, timestampLocalTZTypeInfo);
     cachedPrimitiveTypeInfo.put(serdeConstants.INTERVAL_YEAR_MONTH_TYPE_NAME, intervalYearMonthTypeInfo);
     cachedPrimitiveTypeInfo.put(serdeConstants.INTERVAL_DAY_TIME_TYPE_NAME, intervalDayTimeTypeInfo);
     cachedPrimitiveTypeInfo.put(serdeConstants.BINARY_TYPE_NAME, binaryTypeInfo);
@@ -158,6 +163,11 @@ public final class TypeInfoFactory {
         }
         return new DecimalTypeInfo(Integer.valueOf(parts.typeParams[0]),
             Integer.valueOf(parts.typeParams[1]));
+      case TIMESTAMPLOCALTZ:
+        if (parts.typeParams.length != 1) {
+          return null;
+        }
+        return new TimestampLocalTZTypeInfo(parts.typeParams[0]);
       default:
         return null;
     }
@@ -178,6 +188,11 @@ public final class TypeInfoFactory {
     return (DecimalTypeInfo) getPrimitiveTypeInfo(fullName);
   };
 
+  public static TimestampLocalTZTypeInfo getTimestampTZTypeInfo(ZoneId defaultTimeZone) {
+    String fullName = TimestampLocalTZTypeInfo.getQualifiedName(defaultTimeZone);
+    return (TimestampLocalTZTypeInfo) getPrimitiveTypeInfo(fullName);
+  };
+
   public static TypeInfo getPrimitiveTypeInfoFromPrimitiveWritable(
       Class<?> clazz) {
     String typeName = PrimitiveObjectInspectorUtils