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

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

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