This is an automated email from the ASF dual-hosted git repository.

kuczoram pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/hive.git


The following commit(s) were added to refs/heads/master by this push:
     new 6c3ee53  HIVE-21216: Write Parquet INT64 timestamp (Karen Coppage via 
Marta Kuczora)
6c3ee53 is described below

commit 6c3ee53443ee141c6dc1e6ab2d8a7755beae34be
Author: Karen Coppage <[email protected]>
AuthorDate: Fri Feb 21 22:19:39 2020 +0100

    HIVE-21216: Write Parquet INT64 timestamp (Karen Coppage via Marta Kuczora)
---
 .../java/org/apache/hadoop/hive/conf/HiveConf.java |   6 +
 .../ql/io/parquet/MapredParquetOutputFormat.java   |   2 +-
 .../ql/io/parquet/convert/HiveSchemaConverter.java |  64 +++++--
 .../parquet/timestamp/ParquetTimestampUtils.java   |  27 +++
 .../io/parquet/write/DataWritableWriteSupport.java |   4 +-
 .../ql/io/parquet/write/DataWritableWriter.java    |  50 ++++-
 .../ql/io/parquet/HiveParquetSchemaTestUtils.java  |  32 +++-
 .../hive/ql/io/parquet/TestDataWritableWriter.java |  53 +++++-
 .../ql/io/parquet/TestHiveSchemaConverter.java     |  44 +++--
 .../parquet/serde/TestParquetTimestampUtils.java   | 129 ++++++++++++-
 .../clientpositive/parquet_int64_timestamp.q       |  50 +++++
 .../parquet_int64_timestamp_int96_compatibility.q  |  50 +++++
 .../clientpositive/parquet_int64_timestamp.q.out   | 205 +++++++++++++++++++++
 ...rquet_int64_timestamp_int96_compatibility.q.out | 201 ++++++++++++++++++++
 14 files changed, 862 insertions(+), 55 deletions(-)

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 d0a552a..c51b367 100644
--- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
+++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
@@ -2032,6 +2032,12 @@ public class HiveConf extends Configuration {
     
HIVE_INT_TIMESTAMP_CONVERSION_IN_SECONDS("hive.int.timestamp.conversion.in.seconds",
 false,
         "Boolean/tinyint/smallint/int/bigint value is interpreted as 
milliseconds during the timestamp conversion.\n" +
         "Set this flag to true to interpret the value as seconds to be 
consistent with float/double." ),
+    HIVE_PARQUET_WRITE_INT64_TIMESTAMP("hive.parquet.write.int64.timestamp", 
false,
+        "Write parquet timestamps as int64/LogicalTypes instead of 
int96/OriginalTypes. Note:" +
+        "Timestamps will be time zone agnostic (NEVER converted to a different 
time zone)."),
+    HIVE_PARQUET_TIMESTAMP_TIME_UNIT("hive.parquet.timestamp.time.unit", 
"millis",
+        new StringSet("nanos", "micros", "millis"),
+        "Store parquet int64/LogicalTypes timestamps in this time unit."),
 
     HIVE_ORC_BASE_DELTA_RATIO("hive.exec.orc.base.delta.ratio", 8, "The ratio 
of base writer and\n" +
         "delta writer in terms of STRIPE_SIZE and BUFFER_SIZE."),
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/MapredParquetOutputFormat.java
 
b/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/MapredParquetOutputFormat.java
index ba235f7..4786305 100644
--- 
a/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/MapredParquetOutputFormat.java
+++ 
b/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/MapredParquetOutputFormat.java
@@ -118,7 +118,7 @@ public class MapredParquetOutputFormat extends 
FileOutputFormat<NullWritable, Pa
       columnTypes = 
TypeInfoUtils.getTypeInfosFromTypeString(columnTypeProperty);
     }
 
-    
DataWritableWriteSupport.setSchema(HiveSchemaConverter.convert(columnNames, 
columnTypes), jobConf);
+    
DataWritableWriteSupport.setSchema(HiveSchemaConverter.convert(columnNames, 
columnTypes, jobConf), jobConf);
 
     return getParquerRecordWriterWrapper(realOutputFormat, jobConf, 
finalOutPath.toString(),
             progress,tableProperties);
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/convert/HiveSchemaConverter.java
 
b/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/convert/HiveSchemaConverter.java
index 21bfb2e..a59c15e 100644
--- 
a/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/convert/HiveSchemaConverter.java
+++ 
b/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/convert/HiveSchemaConverter.java
@@ -15,6 +15,8 @@ package org.apache.hadoop.hive.ql.io.parquet.convert;
 
 import java.util.List;
 
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe;
 import org.apache.hadoop.hive.serde.serdeConstants;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector.Category;
@@ -35,28 +37,36 @@ import org.apache.parquet.schema.Types;
 
 public class HiveSchemaConverter {
 
-  public static MessageType convert(final List<String> columnNames, final 
List<TypeInfo> columnTypes) {
-    final MessageType schema = new MessageType("hive_schema", 
convertTypes(columnNames, columnTypes));
+  public static MessageType convert(final List<String> columnNames,
+      final List<TypeInfo> columnTypes) {
+    return convert(columnNames, columnTypes, null);
+  }
+
+  public static MessageType convert(final List<String> columnNames,
+      final List<TypeInfo> columnTypes, Configuration conf) {
+    final MessageType schema =
+        new MessageType("hive_schema", convertTypes(columnNames, columnTypes, 
conf));
     return schema;
   }
 
-  private static Type[] convertTypes(final List<String> columnNames, final 
List<TypeInfo> columnTypes) {
+  private static Type[] convertTypes(final List<String> columnNames,
+      final List<TypeInfo> columnTypes, Configuration conf) {
     if (columnNames.size() != columnTypes.size()) {
       throw new IllegalStateException("Mismatched Hive columns and types. Hive 
columns names" +
         " found : " + columnNames + " . And Hive types found : " + 
columnTypes);
     }
     final Type[] types = new Type[columnNames.size()];
     for (int i = 0; i < columnNames.size(); ++i) {
-      types[i] = convertType(columnNames.get(i), columnTypes.get(i));
+      types[i] = convertType(columnNames.get(i), columnTypes.get(i), conf);
     }
     return types;
   }
 
-  private static Type convertType(final String name, final TypeInfo typeInfo) {
-    return convertType(name, typeInfo, Repetition.OPTIONAL);
+  private static Type convertType(final String name, final TypeInfo typeInfo, 
Configuration conf) {
+    return convertType(name, typeInfo, conf, Repetition.OPTIONAL);
   }
 
-  private static Type convertType(final String name, final TypeInfo typeInfo,
+  private static Type convertType(final String name, final TypeInfo typeInfo, 
Configuration conf,
                                   final Repetition repetition) {
     if (typeInfo.getCategory().equals(Category.PRIMITIVE)) {
       if (typeInfo.equals(TypeInfoFactory.stringTypeInfo)) {
@@ -81,7 +91,23 @@ public class HiveSchemaConverter {
       } else if (typeInfo.equals(TypeInfoFactory.binaryTypeInfo)) {
         return Types.primitive(PrimitiveTypeName.BINARY, 
repetition).named(name);
       } else if (typeInfo.equals(TypeInfoFactory.timestampTypeInfo)) {
-        return Types.primitive(PrimitiveTypeName.INT96, 
repetition).named(name);
+        boolean useInt64;
+        String timeUnitVal;
+        if (conf != null) {
+          useInt64 = HiveConf.getBoolVar(conf, 
HiveConf.ConfVars.HIVE_PARQUET_WRITE_INT64_TIMESTAMP);
+          timeUnitVal = HiveConf.getVar(conf, 
HiveConf.ConfVars.HIVE_PARQUET_TIMESTAMP_TIME_UNIT);
+        } else { //use defaults
+          useInt64 = 
HiveConf.ConfVars.HIVE_PARQUET_WRITE_INT64_TIMESTAMP.defaultBoolVal;
+          timeUnitVal = 
HiveConf.ConfVars.HIVE_PARQUET_TIMESTAMP_TIME_UNIT.defaultStrVal;
+        }
+        if (useInt64) {
+          LogicalTypeAnnotation.TimeUnit timeUnit =
+              
LogicalTypeAnnotation.TimeUnit.valueOf(timeUnitVal.toUpperCase());
+          return Types.primitive(PrimitiveTypeName.INT64, repetition)
+              .as(LogicalTypeAnnotation.timestampType(false, 
timeUnit)).named(name);
+        } else {
+          return Types.primitive(PrimitiveTypeName.INT96, 
repetition).named(name);
+        }
       } else if (typeInfo.equals(TypeInfoFactory.voidTypeInfo)) {
         throw new UnsupportedOperationException("Void type not implemented");
       } else if (typeInfo.getTypeName().toLowerCase().startsWith(
@@ -108,11 +134,11 @@ public class HiveSchemaConverter {
         throw new IllegalArgumentException("Unknown type: " + typeInfo);
       }
     } else if (typeInfo.getCategory().equals(Category.LIST)) {
-      return convertArrayType(name, (ListTypeInfo) typeInfo);
+      return convertArrayType(name, (ListTypeInfo) typeInfo, conf);
     } else if (typeInfo.getCategory().equals(Category.STRUCT)) {
-      return convertStructType(name, (StructTypeInfo) typeInfo);
+      return convertStructType(name, (StructTypeInfo) typeInfo, conf);
     } else if (typeInfo.getCategory().equals(Category.MAP)) {
-      return convertMapType(name, (MapTypeInfo) typeInfo);
+      return convertMapType(name, (MapTypeInfo) typeInfo, conf);
     } else if (typeInfo.getCategory().equals(Category.UNION)) {
       throw new UnsupportedOperationException("Union type not implemented");
     } else {
@@ -122,30 +148,32 @@ public class HiveSchemaConverter {
 
   // An optional group containing a repeated anonymous group "bag", containing
   // 1 anonymous element "array_element"
-  private static GroupType convertArrayType(final String name, final 
ListTypeInfo typeInfo) {
+  private static GroupType convertArrayType(final String name, final 
ListTypeInfo typeInfo, final Configuration conf) {
     final TypeInfo subType = typeInfo.getListElementTypeInfo();
     GroupType groupType = 
Types.optionalGroup().as(LogicalTypeAnnotation.listType())
-        .addField(Types.repeatedGroup().addField(convertType("array_element", 
subType))
+        .addField(Types.repeatedGroup().addField(convertType("array_element", 
subType, conf))
             .named(ParquetHiveSerDe.ARRAY.toString()))
         .named(name);
     return groupType;
   }
 
   // An optional group containing multiple elements
-  private static GroupType convertStructType(final String name, final 
StructTypeInfo typeInfo) {
+  private static GroupType convertStructType(final String name, final 
StructTypeInfo typeInfo,
+      final Configuration conf) {
     final List<String> columnNames = typeInfo.getAllStructFieldNames();
     final List<TypeInfo> columnTypes = typeInfo.getAllStructFieldTypeInfos();
-    GroupType groupType = 
Types.optionalGroup().addFields(convertTypes(columnNames, 
columnTypes)).named(name);
+    GroupType groupType = 
Types.optionalGroup().addFields(convertTypes(columnNames, columnTypes, 
conf)).named(name);
     return groupType;
   }
 
   // An optional group containing a repeated anonymous group "map", containing
   // 2 elements: "key", "value"
-  private static GroupType convertMapType(final String name, final MapTypeInfo 
typeInfo) {
+  private static GroupType convertMapType(final String name, final MapTypeInfo 
typeInfo,
+      final Configuration conf) {
     final Type keyType = convertType(ParquetHiveSerDe.MAP_KEY.toString(),
-        typeInfo.getMapKeyTypeInfo(), Repetition.REQUIRED);
+        typeInfo.getMapKeyTypeInfo(), conf, Repetition.REQUIRED);
     final Type valueType = convertType(ParquetHiveSerDe.MAP_VALUE.toString(),
-        typeInfo.getMapValueTypeInfo());
+        typeInfo.getMapValueTypeInfo(), conf);
     return ConversionPatterns.mapType(Repetition.OPTIONAL, name, keyType, 
valueType);
   }
 }
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/timestamp/ParquetTimestampUtils.java
 
b/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/timestamp/ParquetTimestampUtils.java
index 9ce07e7..5368476 100644
--- 
a/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/timestamp/ParquetTimestampUtils.java
+++ 
b/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/timestamp/ParquetTimestampUtils.java
@@ -13,6 +13,7 @@
  */
 package org.apache.hadoop.hive.ql.io.parquet.timestamp;
 
+import java.math.BigInteger;
 import java.time.ZoneId;
 import java.time.ZoneOffset;
 
@@ -53,4 +54,30 @@ public class ParquetTimestampUtils {
     }
     return Timestamp.ofEpochSecond(seconds, nanoseconds, zone);
   }
+
+  /**
+   * Convert Timestamp to Long, interpreted by TimeUnit.
+   * Returns null if nanoseconds outside of range 1677-09-21 
00:12:43.145224192 -
+   * 2262-04-11 23:47:16.854775807 as this would cause overflow.
+   */
+  public static Long getInt64(Timestamp timestamp, TimeUnit timeUnit) {
+    switch (timeUnit) {
+    case NANOS:
+      try {
+        BigInteger nanos = BigInteger.valueOf(timestamp.toEpochSecond())
+            .multiply(BigInteger.valueOf(NANO))
+            .add(BigInteger.valueOf(timestamp.getNanos()));
+        return nanos.longValueExact();
+      } catch (ArithmeticException e) {
+        return null;
+      }
+    case MICROS:
+      long secondsInMilli = timestamp.toEpochSecond() * MICRO;
+      return secondsInMilli + timestamp.getNanos() / MILLI;
+    case MILLIS:
+      return timestamp.toEpochMilli();
+    default:
+      throw new IllegalArgumentException("Time unit not recognized");
+    }
+  }
 }
\ No newline at end of file
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/write/DataWritableWriteSupport.java
 
b/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/write/DataWritableWriteSupport.java
index f4212f44..d3b1982 100644
--- 
a/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/write/DataWritableWriteSupport.java
+++ 
b/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/write/DataWritableWriteSupport.java
@@ -41,6 +41,7 @@ public class DataWritableWriteSupport extends 
WriteSupport<ParquetHiveRecord> {
   private DataWritableWriter writer;
   private MessageType schema;
   private boolean defaultDateProleptic;
+  private Configuration conf;
 
   public static void setSchema(final MessageType schema, final Configuration 
configuration) {
     configuration.set(PARQUET_HIVE_SCHEMA, schema.toString());
@@ -52,6 +53,7 @@ public class DataWritableWriteSupport extends 
WriteSupport<ParquetHiveRecord> {
 
   @Override
   public WriteContext init(final Configuration configuration) {
+    conf = configuration;
     schema = getSchema(configuration);
     Map<String, String> metaData = new HashMap<>();
     metaData.put(WRITER_TIMEZONE, TimeZone.getDefault().toZoneId().toString());
@@ -63,7 +65,7 @@ public class DataWritableWriteSupport extends 
WriteSupport<ParquetHiveRecord> {
 
   @Override
   public void prepareForWrite(final RecordConsumer recordConsumer) {
-    writer = new DataWritableWriter(recordConsumer, schema, 
defaultDateProleptic);
+    writer = new DataWritableWriter(recordConsumer, schema, 
defaultDateProleptic, conf);
   }
 
   @Override
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/write/DataWritableWriter.java
 
b/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/write/DataWritableWriter.java
index 1834008..41406aa 100644
--- 
a/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/write/DataWritableWriter.java
+++ 
b/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/write/DataWritableWriter.java
@@ -13,12 +13,15 @@
  */
 package org.apache.hadoop.hive.ql.io.parquet.write;
 
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.common.type.Date;
 import org.apache.hadoop.hive.common.type.HiveDecimal;
 import org.apache.hadoop.hive.common.type.Timestamp;
+import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe;
 import org.apache.hadoop.hive.ql.io.parquet.timestamp.NanoTimeUtils;
 import org.apache.hadoop.hive.common.type.CalendarUtils;
+import org.apache.hadoop.hive.ql.io.parquet.timestamp.ParquetTimestampUtils;
 import org.apache.hadoop.hive.serde2.io.DateWritableV2;
 import org.apache.hadoop.hive.serde2.io.ParquetHiveRecord;
 import org.apache.hadoop.hive.serde2.objectinspector.ListObjectInspector;
@@ -66,6 +69,7 @@ public class DataWritableWriter {
   protected final RecordConsumer recordConsumer;
   private final GroupType schema;
   private final boolean defaultDateProleptic;
+  private Configuration conf;
 
   /* This writer will be created when writing the first row in order to get
   information about how to inspect the record data.  */
@@ -78,6 +82,14 @@ public class DataWritableWriter {
     this.defaultDateProleptic = defaultDateProleptic;
   }
 
+       public DataWritableWriter(final RecordConsumer recordConsumer, final 
GroupType schema,
+                       final boolean defaultDateProleptic, final Configuration 
conf) {
+           this.recordConsumer = recordConsumer;
+    this.schema = schema;
+    this.defaultDateProleptic = defaultDateProleptic;
+    this.conf = conf;
+  }
+
   /**
    * It writes a record to Parquet.
    * @param record Contains the record that is going to be written.
@@ -201,10 +213,10 @@ public class DataWritableWriter {
       for (int i = 0; i < structFields.size(); i++) {
         StructField field = structFields.get(i);
         Object fieldValue = inspector.getStructFieldData(value, field);
+        DataWriter writer = structWriters[i];
 
-        if (fieldValue != null) {
+        if (fieldValue != null && isValidValue(fieldValue, writer)) {
           String fieldName = field.getFieldName();
-          DataWriter writer = structWriters[i];
 
           recordConsumer.startField(fieldName, i);
           writer.write(fieldValue);
@@ -212,6 +224,13 @@ public class DataWritableWriter {
         }
       }
     }
+
+    private boolean isValidValue(Object fieldValue, DataWriter writer) {
+      if (writer instanceof TimestampDataWriter) {
+        return ((TimestampDataWriter) writer).isValidTimestamp(fieldValue);
+      }
+      return true;
+    }
   }
 
   private class MessageDataWriter extends GroupDataWriter implements 
DataWriter {
@@ -494,15 +513,40 @@ public class DataWritableWriter {
 
   private class TimestampDataWriter implements DataWriter {
     private TimestampObjectInspector inspector;
+    boolean useInt64;
+    LogicalTypeAnnotation.TimeUnit timeUnit;
 
     public TimestampDataWriter(TimestampObjectInspector inspector) {
       this.inspector = inspector;
+      String timeUnitVal;
+      if (conf != null) {
+        useInt64 = HiveConf.getBoolVar(conf, 
HiveConf.ConfVars.HIVE_PARQUET_WRITE_INT64_TIMESTAMP);
+        timeUnitVal = HiveConf.getVar(conf, 
HiveConf.ConfVars.HIVE_PARQUET_TIMESTAMP_TIME_UNIT);
+      } else { //use defaults
+        useInt64 = 
HiveConf.ConfVars.HIVE_PARQUET_WRITE_INT64_TIMESTAMP.defaultBoolVal;
+        timeUnitVal = 
HiveConf.ConfVars.HIVE_PARQUET_TIMESTAMP_TIME_UNIT.defaultStrVal;
+      }
+      timeUnit = 
LogicalTypeAnnotation.TimeUnit.valueOf(timeUnitVal.toUpperCase());
     }
 
     @Override
     public void write(Object value) {
       Timestamp ts = inspector.getPrimitiveJavaObject(value);
-      recordConsumer.addBinary(NanoTimeUtils.getNanoTime(ts, 
false).toBinary());
+      if (useInt64) {
+        Long int64value = ParquetTimestampUtils.getInt64(ts, timeUnit);
+        recordConsumer.addLong(int64value);
+      } else {
+        recordConsumer.addBinary(NanoTimeUtils.getNanoTime(ts, 
false).toBinary());
+      }
+    }
+
+    boolean isValidTimestamp(Object fieldValue) {
+      // only check if int64 time unit is nanos
+      if (useInt64 && timeUnit == LogicalTypeAnnotation.TimeUnit.NANOS) {
+        Timestamp ts = inspector.getPrimitiveJavaObject(fieldValue);
+        return ParquetTimestampUtils.getInt64(ts, timeUnit) != null;
+      }
+      return true;
     }
   }
 
diff --git 
a/ql/src/test/org/apache/hadoop/hive/ql/io/parquet/HiveParquetSchemaTestUtils.java
 
b/ql/src/test/org/apache/hadoop/hive/ql/io/parquet/HiveParquetSchemaTestUtils.java
index 181894f..0d23d82 100644
--- 
a/ql/src/test/org/apache/hadoop/hive/ql/io/parquet/HiveParquetSchemaTestUtils.java
+++ 
b/ql/src/test/org/apache/hadoop/hive/ql/io/parquet/HiveParquetSchemaTestUtils.java
@@ -13,6 +13,7 @@
  */
 package org.apache.hadoop.hive.ql.io.parquet;
 
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.ql.io.parquet.convert.HiveSchemaConverter;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils;
@@ -56,13 +57,25 @@ public class HiveParquetSchemaTestUtils {
     return columnTypes;
   }
 
+  /**
+   * Only use if Configuration/HiveConf not needed for converting schema.
+   */
   public static void testConversion(
-    final String columnNamesStr,
-    final String columnsTypeStr,
-    final String actualSchema) throws Exception {
+      final String columnNamesStr,
+      final String columnsTypeStr,
+      final String actualSchema) throws Exception {
+    testConversion(columnNamesStr, columnsTypeStr, actualSchema, null);
+  }
+
+  public static void testConversion(
+      final String columnNamesStr,
+      final String columnsTypeStr,
+      final String actualSchema,
+      final Configuration conf) throws Exception {
     final List<String> columnNames = createHiveColumnsFrom(columnNamesStr);
     final List<TypeInfo> columnTypes = createHiveTypeInfoFrom(columnsTypeStr);
-    final MessageType messageTypeFound = 
HiveSchemaConverter.convert(columnNames, columnTypes);
+    final MessageType messageTypeFound = 
HiveSchemaConverter.convert(columnNames, columnTypes,
+        conf);
     final MessageType expectedMT = 
MessageTypeParser.parseMessageType(actualSchema);
     assertEquals("converting " + columnNamesStr + ": " + columnsTypeStr + " to 
" + actualSchema,
       expectedMT, messageTypeFound);
@@ -79,18 +92,19 @@ public class HiveParquetSchemaTestUtils {
   }
 
   public static void testLogicalTypeAnnotation(String hiveColumnType, String 
hiveColumnName,
-      LogicalTypeAnnotation expectedLogicalType) throws Exception {
+      LogicalTypeAnnotation expectedLogicalType, Configuration conf) throws 
Exception {
     Map<String, LogicalTypeAnnotation> expectedLogicalTypeForColumn = new 
HashMap<>();
     expectedLogicalTypeForColumn.put(hiveColumnName, expectedLogicalType);
-    testLogicalTypeAnnotations(hiveColumnName, hiveColumnType, 
expectedLogicalTypeForColumn);
+    testLogicalTypeAnnotations(hiveColumnName, hiveColumnType, 
expectedLogicalTypeForColumn, conf);
   }
 
   public static void testLogicalTypeAnnotations(final String hiveColumnNames,
-      final String hiveColumnTypes, final Map<String, LogicalTypeAnnotation> 
expectedLogicalTypes)
-      throws Exception {
+      final String hiveColumnTypes, final Map<String, LogicalTypeAnnotation> 
expectedLogicalTypes,
+      Configuration conf) throws Exception {
     final List<String> columnNames = createHiveColumnsFrom(hiveColumnNames);
     final List<TypeInfo> columnTypes = createHiveTypeInfoFrom(hiveColumnTypes);
-    final MessageType messageTypeFound = 
HiveSchemaConverter.convert(columnNames, columnTypes);
+    final MessageType messageTypeFound = 
HiveSchemaConverter.convert(columnNames, columnTypes,
+        conf);
     List<Type> actualFields = messageTypeFound.getFields();
     for (Type actualField : actualFields) {
       LogicalTypeAnnotation expectedLogicalType = 
expectedLogicalTypes.get(actualField.getName());
diff --git 
a/ql/src/test/org/apache/hadoop/hive/ql/io/parquet/TestDataWritableWriter.java 
b/ql/src/test/org/apache/hadoop/hive/ql/io/parquet/TestDataWritableWriter.java
index 01d9d2e..1c255ed 100644
--- 
a/ql/src/test/org/apache/hadoop/hive/ql/io/parquet/TestDataWritableWriter.java
+++ 
b/ql/src/test/org/apache/hadoop/hive/ql/io/parquet/TestDataWritableWriter.java
@@ -14,15 +14,18 @@
 package org.apache.hadoop.hive.ql.io.parquet;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.common.type.Timestamp;
+import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.ql.io.parquet.serde.ArrayWritableObjectInspector;
 import org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe;
 import org.apache.hadoop.hive.ql.io.parquet.write.DataWritableWriter;
-import org.apache.hadoop.hive.serde2.io.ByteWritable;
-import org.apache.hadoop.hive.serde2.io.ShortWritable;
-import org.apache.hadoop.hive.serde2.io.ParquetHiveRecord;
 import org.apache.hadoop.hive.serde2.SerDeException;
 import org.apache.hadoop.hive.serde2.SerDeUtils;
+import org.apache.hadoop.hive.serde2.io.ByteWritable;
 import org.apache.hadoop.hive.serde2.io.DoubleWritable;
+import org.apache.hadoop.hive.serde2.io.ParquetHiveRecord;
+import org.apache.hadoop.hive.serde2.io.ShortWritable;
+import org.apache.hadoop.hive.serde2.io.TimestampWritableV2;
 import org.apache.hadoop.hive.serde2.typeinfo.StructTypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
@@ -56,11 +59,13 @@ import static org.mockito.Mockito.*;
 public class TestDataWritableWriter {
   @Mock private RecordConsumer mockRecordConsumer;
   private InOrder inOrder;
+  private Configuration conf;
 
   @Before
   public void initMocks() {
     MockitoAnnotations.initMocks(this);
     inOrder = inOrder(mockRecordConsumer);
+    conf = new Configuration();
   }
 
   private void startMessage() {
@@ -84,7 +89,7 @@ public class TestDataWritableWriter {
     inOrder.verify(mockRecordConsumer).addInteger(value);
   }
 
-  private void addLong(int value) {
+  private void addLong(long value) {
     inOrder.verify(mockRecordConsumer).addLong(value);
   }
 
@@ -140,6 +145,10 @@ public class TestDataWritableWriter {
     return new BytesWritable(value.getBytes(StandardCharsets.UTF_8));
   }
 
+  private TimestampWritableV2 createTimestamp(String s) {
+    return new TimestampWritableV2(Timestamp.valueOf(s));
+  }
+
   private ArrayWritable createGroup(Writable...values) {
     return new ArrayWritable(Writable.class, values);
   }
@@ -185,14 +194,14 @@ public class TestDataWritableWriter {
     recordProperties.setProperty("columns.types", columnTypes);
 
     ParquetHiveSerDe serDe = new ParquetHiveSerDe();
-    SerDeUtils.initializeSerDe(serDe, new Configuration(), recordProperties, 
null);
+    SerDeUtils.initializeSerDe(serDe, conf, recordProperties, null);
 
     return new ParquetHiveRecord(serDe.deserialize(record), 
getObjectInspector(columnNames, columnTypes));
   }
 
   private void writeParquetRecord(String schema, ParquetHiveRecord record) 
throws SerDeException {
     MessageType fileSchema = MessageTypeParser.parseMessageType(schema);
-    DataWritableWriter hiveParquetWriter = new 
DataWritableWriter(mockRecordConsumer, fileSchema, false);
+    DataWritableWriter hiveParquetWriter = new 
DataWritableWriter(mockRecordConsumer, fileSchema, false, conf);
     hiveParquetWriter.write(record);
   }
 
@@ -256,6 +265,38 @@ public class TestDataWritableWriter {
   }
 
   @Test
+  public void testInt64Timestamp() throws Exception {
+    
conf.setBoolean(HiveConf.ConfVars.HIVE_PARQUET_WRITE_INT64_TIMESTAMP.varname, 
true);
+    verifyInt64Timestamp("nan", "1970-01-01 00:00:00.000000001", "nanos");
+    verifyInt64Timestamp("mic", "1970-01-01 00:00:00.000001", "micros");
+    verifyInt64Timestamp("mil", "1970-01-01 00:00:00.001", "millis");
+  }
+
+  private void verifyInt64Timestamp(String columnName, String timestampString, 
String timeUnit)
+      throws Exception {
+    conf.set(HiveConf.ConfVars.HIVE_PARQUET_TIMESTAMP_TIME_UNIT.varname, 
timeUnit);
+    String columnTypes = "timestamp";
+
+    String fileSchema = "message hive_schema {\n"
+        + "  optional int64 " + columnName + " (TIMESTAMP(" + 
timeUnit.toUpperCase() + ",false));\n"
+        + "}\n";
+
+    ArrayWritable hiveRecord = createGroup(
+        createTimestamp(timestampString)
+    );
+
+    // Write record to Parquet format
+    writeParquetRecord(fileSchema, getParquetWritable(columnName, columnTypes, 
hiveRecord));
+
+    // Verify record was written correctly to Parquet
+    startMessage();
+    startField(columnName, 0);
+    addLong(1L);
+    endField(columnName, 0);
+    endMessage();
+  }
+
+  @Test
   public void testStructType() throws Exception {
     String columnNames = "structCol";
     String columnTypes = "struct<a:int,b:double,c:boolean>";
diff --git 
a/ql/src/test/org/apache/hadoop/hive/ql/io/parquet/TestHiveSchemaConverter.java 
b/ql/src/test/org/apache/hadoop/hive/ql/io/parquet/TestHiveSchemaConverter.java
index dc80af1..0f3ca6e 100644
--- 
a/ql/src/test/org/apache/hadoop/hive/ql/io/parquet/TestHiveSchemaConverter.java
+++ 
b/ql/src/test/org/apache/hadoop/hive/ql/io/parquet/TestHiveSchemaConverter.java
@@ -23,6 +23,8 @@ import static 
org.apache.hadoop.hive.ql.io.parquet.HiveParquetSchemaTestUtils.te
 import java.util.ArrayList;
 import java.util.List;
 
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.ql.io.parquet.convert.HiveSchemaConverter;
 import org.apache.hadoop.hive.serde2.typeinfo.ListTypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
@@ -37,6 +39,8 @@ import org.junit.Test;
 
 public class TestHiveSchemaConverter {
 
+  Configuration conf = new Configuration();
+
   @Test
   public void testSimpleType() throws Exception {
     testConversion(
@@ -117,12 +121,14 @@ public class TestHiveSchemaConverter {
 
   @Test
   public void testTimestampType() throws Exception {
+    
conf.setBoolean(HiveConf.ConfVars.HIVE_PARQUET_WRITE_INT64_TIMESTAMP.varname, 
false);
     testConversion(
         "a",
         "timestamp",
         "message hive_schema {\n"
             + "  optional int96 a;\n"
-            + "}\n");
+            + "}\n",
+            conf);
   }
 
   @Test
@@ -192,6 +198,7 @@ public class TestHiveSchemaConverter {
 
   @Test
   public void testArrayTimestamp() throws Exception {
+    
conf.setBoolean(HiveConf.ConfVars.HIVE_PARQUET_WRITE_INT64_TIMESTAMP.varname, 
false);
     testConversion("arrayCol",
             "array<timestamp>",
             "message hive_schema {\n"
@@ -200,7 +207,8 @@ public class TestHiveSchemaConverter {
             + "      optional int96 array_element;\n"
             + "    }\n"
             + "  }\n"
-            + "}\n");
+            + "}\n",
+            conf);
   }
 
   @Test
@@ -287,13 +295,15 @@ public class TestHiveSchemaConverter {
 
   @Test
   public void testStructTimestamp() throws Exception {
+    
conf.setBoolean(HiveConf.ConfVars.HIVE_PARQUET_WRITE_INT64_TIMESTAMP.varname, 
false);
     testConversion("structCol",
             "struct<a:timestamp>",
             "message hive_schema {\n"
             + "  optional group structCol {\n"
             + "    optional int96 a;\n"
             + "  }\n"
-            + "}\n");
+            + "}\n",
+            conf);
   }
 
   @Test
@@ -392,19 +402,21 @@ public class TestHiveSchemaConverter {
 
   @Test
   public void testLogicalTypes() throws Exception {
-    testLogicalTypeAnnotation("string", "a", 
LogicalTypeAnnotation.stringType());
-    testLogicalTypeAnnotation("int", "a", null);
-    testLogicalTypeAnnotation("smallint", "a", 
LogicalTypeAnnotation.intType(16, true));
-    testLogicalTypeAnnotation("tinyint", "a", LogicalTypeAnnotation.intType(8, 
true));
-    testLogicalTypeAnnotation("bigint", "a", null);
-    testLogicalTypeAnnotation("double", "a", null);
-    testLogicalTypeAnnotation("float", "a", null);
-    testLogicalTypeAnnotation("boolean", "a", null);
-    testLogicalTypeAnnotation("binary", "a", null);
-    testLogicalTypeAnnotation("timestamp", "a", null);
-    testLogicalTypeAnnotation("char(3)", "a", 
LogicalTypeAnnotation.stringType());
-    testLogicalTypeAnnotation("varchar(30)", "a", 
LogicalTypeAnnotation.stringType());
-    testLogicalTypeAnnotation("decimal(7,2)", "a", 
LogicalTypeAnnotation.decimalType(2, 7));
+    
conf.setBoolean(HiveConf.ConfVars.HIVE_PARQUET_WRITE_INT64_TIMESTAMP.varname, 
true);
+    testLogicalTypeAnnotation("string", "a", 
LogicalTypeAnnotation.stringType(), conf);
+    testLogicalTypeAnnotation("int", "a", null, conf);
+    testLogicalTypeAnnotation("smallint", "a", 
LogicalTypeAnnotation.intType(16, true), conf);
+    testLogicalTypeAnnotation("tinyint", "a", LogicalTypeAnnotation.intType(8, 
true), conf);
+    testLogicalTypeAnnotation("bigint", "a", null, conf);
+    testLogicalTypeAnnotation("double", "a", null, conf);
+    testLogicalTypeAnnotation("float", "a", null, conf);
+    testLogicalTypeAnnotation("boolean", "a", null, conf);
+    testLogicalTypeAnnotation("binary", "a", null, conf);
+    testLogicalTypeAnnotation("timestamp", "a",
+        LogicalTypeAnnotation.timestampType(false, 
LogicalTypeAnnotation.TimeUnit.MILLIS), conf);
+    testLogicalTypeAnnotation("char(3)", "a", 
LogicalTypeAnnotation.stringType(), conf);
+    testLogicalTypeAnnotation("varchar(30)", "a", 
LogicalTypeAnnotation.stringType(), conf);
+    testLogicalTypeAnnotation("decimal(7,2)", "a", 
LogicalTypeAnnotation.decimalType(2, 7), conf);
   }
 
   @Test
diff --git 
a/ql/src/test/org/apache/hadoop/hive/ql/io/parquet/serde/TestParquetTimestampUtils.java
 
b/ql/src/test/org/apache/hadoop/hive/ql/io/parquet/serde/TestParquetTimestampUtils.java
index bc5e560..fd1b076 100644
--- 
a/ql/src/test/org/apache/hadoop/hive/ql/io/parquet/serde/TestParquetTimestampUtils.java
+++ 
b/ql/src/test/org/apache/hadoop/hive/ql/io/parquet/serde/TestParquetTimestampUtils.java
@@ -25,8 +25,9 @@ import org.apache.hadoop.hive.common.type.TimestampTZUtil;
 import org.apache.hadoop.hive.ql.io.parquet.timestamp.NanoTime;
 import org.apache.hadoop.hive.ql.io.parquet.timestamp.NanoTimeUtils;
 
+import org.apache.hadoop.hive.ql.io.parquet.timestamp.ParquetTimestampUtils;
+import org.apache.parquet.schema.LogicalTypeAnnotation;
 import org.junit.Assert;
-import junit.framework.TestCase;
 import org.junit.Test;
 
 
@@ -291,4 +292,130 @@ public class TestParquetTimestampUtils {
     Timestamp ts2 = TimestampTZUtil.convertTimestampToZone(ts, US_PACIFIC, 
NEW_YORK);
     Assert.assertTrue(Timestamp.valueOf("2018-01-01 03:00:00").equals(ts2));
   }
+
+  ///////////// INT64/LogicalTypes Timestamp tests /////////////
+
+  @Test
+  public void testInt64ExactValue() {
+
+    // basic value
+    Timestamp ts1 = Timestamp.valueOf("2011-01-01 00:30:00.000000001");
+    long millis = ParquetTimestampUtils.getInt64(ts1, 
LogicalTypeAnnotation.TimeUnit.MILLIS);
+    Assert.assertEquals(1293841800_000L, millis);
+    long micros = ParquetTimestampUtils.getInt64(ts1, 
LogicalTypeAnnotation.TimeUnit.MICROS);
+    Assert.assertEquals(1293841800000_000L, micros);
+    long nanos = ParquetTimestampUtils.getInt64(ts1, 
LogicalTypeAnnotation.TimeUnit.NANOS);
+    Assert.assertEquals(1293841800000_000_001L, nanos);
+
+    // test correct conversion to time units
+    Timestamp ts2 = Timestamp.valueOf("2011-01-01 00:30:00.55555");
+    millis = ParquetTimestampUtils.getInt64(ts2, 
LogicalTypeAnnotation.TimeUnit.MILLIS);
+    Assert.assertEquals(1293841800_555L, millis);
+    micros = ParquetTimestampUtils.getInt64(ts2, 
LogicalTypeAnnotation.TimeUnit.MICROS);
+    Assert.assertEquals(1293841800_555_550L, micros);
+    nanos = ParquetTimestampUtils.getInt64(ts2, 
LogicalTypeAnnotation.TimeUnit.NANOS);
+    Assert.assertEquals(1293841800_555_550_000L, nanos);
+
+    // test max nanos
+    Timestamp ts3 = Timestamp.valueOf("2018-12-31 23:59:59.999999999");
+    millis = ParquetTimestampUtils.getInt64(ts3, 
LogicalTypeAnnotation.TimeUnit.MILLIS);
+    Assert.assertEquals(1546300799_999L, millis);
+    micros = ParquetTimestampUtils.getInt64(ts3, 
LogicalTypeAnnotation.TimeUnit.MICROS);
+    Assert.assertEquals(1546300799_999_999L, micros);
+    nanos = ParquetTimestampUtils.getInt64(ts3, 
LogicalTypeAnnotation.TimeUnit.NANOS);
+    Assert.assertEquals(1546300799_999_999_999L, nanos);
+
+    // test pre-epoch date
+    Timestamp ts4 = Timestamp.valueOf("1968-01-31 00:30:00.000000001");
+    millis = ParquetTimestampUtils.getInt64(ts4, 
LogicalTypeAnnotation.TimeUnit.MILLIS);
+    Assert.assertEquals(-60564600_000L, millis);
+    micros = ParquetTimestampUtils.getInt64(ts4, 
LogicalTypeAnnotation.TimeUnit.MICROS);
+    Assert.assertEquals(-60564600000_000L, micros);
+    nanos = ParquetTimestampUtils.getInt64(ts4, 
LogicalTypeAnnotation.TimeUnit.NANOS);
+    Assert.assertEquals(-60564599999_999_999L, nanos);
+  }
+
+  @Test
+  public void testLegalInt64TimestampStrings() {
+    //exercise a broad range of timestamps close to the present.
+    verifyInt64TimestampValue("2011-01-01 01:01:01.111111111", true);
+    verifyInt64TimestampValue("2012-02-02 02:02:02.222222222", true);
+    verifyInt64TimestampValue("2013-03-03 03:03:03.333333333", true);
+    verifyInt64TimestampValue("2014-04-04 04:04:04.444444444", true);
+    verifyInt64TimestampValue("2015-05-05 05:05:05.555555555", true);
+    verifyInt64TimestampValue("2016-06-06 06:06:06.666666666", true);
+    verifyInt64TimestampValue("2017-07-07 07:07:07.777777777", true);
+    verifyInt64TimestampValue("2018-08-08 08:08:08.888888888", true);
+    verifyInt64TimestampValue("2019-09-09 09:09:09.999999999", true);
+    verifyInt64TimestampValue("2020-10-10 10:10:10.101010101", true);
+    verifyInt64TimestampValue("2021-11-11 11:11:11.111111111", true);
+    verifyInt64TimestampValue("2022-12-12 12:12:12.121212121", true);
+    verifyInt64TimestampValue("2023-01-02 13:13:13.131313131", true);
+    verifyInt64TimestampValue("2024-02-02 14:14:14.141414141", true);
+    verifyInt64TimestampValue("2025-03-03 15:15:15.151515151", true);
+    verifyInt64TimestampValue("2026-04-04 16:16:16.161616161", true);
+    verifyInt64TimestampValue("2027-05-05 17:17:17.171717171", true);
+    verifyInt64TimestampValue("2028-06-06 18:18:18.181818181", true);
+    verifyInt64TimestampValue("2029-07-07 19:19:19.191919191", true);
+    verifyInt64TimestampValue("2030-08-08 20:20:20.202020202", true);
+    verifyInt64TimestampValue("2031-09-09 21:21:21.212121212", true);
+
+    //test values around epoch
+    verifyInt64TimestampValue("1969-12-31 23:59:58.123456789", true);
+    verifyInt64TimestampValue("1969-12-31 23:59:59.999999999", true);
+    verifyInt64TimestampValue("1970-01-01 00:00:00.0", true);
+    verifyInt64TimestampValue("1970-01-01 00:00:00.000000001", true);
+
+    //test min and max values for nano
+    verifyInt64TimestampValue("1677-09-21 00:12:43.145224192", 
LogicalTypeAnnotation.TimeUnit.NANOS, true);
+    verifyInt64TimestampValue("2262-04-11 23:47:16.854775807", 
LogicalTypeAnnotation.TimeUnit.NANOS, true);
+
+    //test some extreme cases.
+    verifyInt64TimestampValue("0001-01-01 00:00:00.001001001", 
LogicalTypeAnnotation.TimeUnit.MILLIS, true);
+    verifyInt64TimestampValue("0001-01-01 00:00:00.001001001", 
LogicalTypeAnnotation.TimeUnit.MICROS, true);
+    verifyInt64TimestampValue("9999-09-09 09:09:09.999999999", 
LogicalTypeAnnotation.TimeUnit.MILLIS, true);
+    verifyInt64TimestampValue("9999-09-09 09:09:09.999999999", 
LogicalTypeAnnotation.TimeUnit.MICROS, true);
+  }
+
+  /**
+   * Timestamps 1 nanosecond out of nano range should return null.
+   */
+  @Test
+  public void testIllegalInt64TimestampStrings() {
+    verifyInt64TimestampValue("1677-09-21 00:12:43.145224191", 
LogicalTypeAnnotation.TimeUnit.NANOS, false);
+    verifyInt64TimestampValue("2262-04-11 23:47:16.854775808", 
LogicalTypeAnnotation.TimeUnit.NANOS, false);
+  }
+
+  private void verifyInt64TimestampValue(String tsString, boolean legal) {
+    verifyInt64TimestampValue(tsString, LogicalTypeAnnotation.TimeUnit.MILLIS, 
legal);
+    verifyInt64TimestampValue(tsString, LogicalTypeAnnotation.TimeUnit.MICROS, 
legal);
+    verifyInt64TimestampValue(tsString, LogicalTypeAnnotation.TimeUnit.NANOS, 
legal);
+  }
+
+  private void verifyInt64TimestampValue(String tsString, 
LogicalTypeAnnotation.TimeUnit timeUnit, boolean legal) {
+    Timestamp ts = truncateTimestampString(tsString, timeUnit);
+    String truncatedTsString = ts.toString();
+    Long int64Value = ParquetTimestampUtils.getInt64(ts, timeUnit);
+    if (legal) {
+      Timestamp tsFetched = ParquetTimestampUtils.getTimestamp(int64Value, 
timeUnit, false);
+      Assert.assertEquals(truncatedTsString, tsFetched.toString());
+    } else {
+      Assert.assertEquals(null, int64Value);
+    }
+  }
+
+  private Timestamp truncateTimestampString(String tsString, 
LogicalTypeAnnotation.TimeUnit timeUnit) {
+    Timestamp ts = Timestamp.valueOf(tsString);
+    switch (timeUnit) {
+    case MILLIS:
+      ts.setNanos(ts.getNanos() / 1_000_000 * 1_000_000);
+      break;
+    case MICROS:
+      ts.setNanos(ts.getNanos() / 1_000 * 1_000);
+      break;
+    default:
+      break;
+    }
+    return ts;
+  }
 }
diff --git a/ql/src/test/queries/clientpositive/parquet_int64_timestamp.q 
b/ql/src/test/queries/clientpositive/parquet_int64_timestamp.q
new file mode 100644
index 0000000..4de315c
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/parquet_int64_timestamp.q
@@ -0,0 +1,50 @@
+set hive.parquet.write.int64.timestamp=true;
+set time zone Asia/Singapore;
+
+--store timestamps as strings for copying into different schemas
+create table parquet_timestamp_staging (i int, s string);
+
+insert into parquet_timestamp_staging
+values
+(0, '0001-01-01'),
+(1, '1677-09-21 00:12:43.145224192'),
+(2, '1969-12-31 23:59:59.99999999999999999'),
+(3, '1970-01-01 00:00:00'),
+(4, '2013-09-27 01:36:18.000000001'),
+(5, '2018-01-02 13:14:15.678999'),
+(6, '2262-04-11 23:47:16.854775807'),
+(7, '9999-12-31 23:59:59.999999999999');
+
+create table parquet_int64_timestamp (i int, ts timestamp) stored as parquet;
+
+--test nanoseconds read/write
+set hive.parquet.timestamp.time.unit=nanos;
+insert into parquet_int64_timestamp select i, cast (s as timestamp) from 
parquet_timestamp_staging;
+select * from parquet_int64_timestamp order by i;
+
+--test microseconds read/write
+set hive.parquet.timestamp.time.unit=micros;
+insert into parquet_int64_timestamp select i + 10, cast (s as timestamp) from 
parquet_timestamp_staging;
+select * from parquet_int64_timestamp order by i;
+
+--test milliseconds read/write
+set hive.parquet.timestamp.time.unit=millis;
+insert into parquet_int64_timestamp select i + 20, cast (s as timestamp) from 
parquet_timestamp_staging;
+select * from parquet_int64_timestamp order by i;
+
+
+--time zone should not affect values, since timestamp is time zone agnostic
+set time zone America/Buenos_Aires;
+
+--test filters
+select * from parquet_int64_timestamp where ts >  '1969-12-31 23:59:59.9'
+                                            and
+                                            ts <  '1970-01-01 00:00:00.0' 
order by i;
+select * from parquet_int64_timestamp where ts <= '1970-01-01 00:00:00.0'
+                                            and
+                                            ts >= '1970-01-01 00:00:00.0' 
order by i;
+select * from parquet_int64_timestamp where ts =  '1970-01-01 00:00:00.0' 
order by i;
+
+select * from parquet_int64_timestamp where ts between cast ('1969-12-31 
23:59:59.9' as timestamp) and
+                                                       cast ('1970-01-01 
00:00:00' as timestamp)
+                                      order by i;
diff --git 
a/ql/src/test/queries/clientpositive/parquet_int64_timestamp_int96_compatibility.q
 
b/ql/src/test/queries/clientpositive/parquet_int64_timestamp_int96_compatibility.q
new file mode 100644
index 0000000..8499529
--- /dev/null
+++ 
b/ql/src/test/queries/clientpositive/parquet_int64_timestamp_int96_compatibility.q
@@ -0,0 +1,50 @@
+--store timestamps as strings for copying into different schemas
+create table parquet_timestamp_staging_2 (i int, s string);
+insert into parquet_timestamp_staging_2
+values
+(0, '0001-01-01'),
+(1, '1677-09-21 00:12:43.145224192'),
+(2, '1969-12-31 23:59:59.99999999999999999'),
+(3, '1970-01-01 00:00:00'),
+(4, '2013-09-27 01:36:18.000000001'),
+(5, '2018-01-02 13:14:15.678999'),
+(6, '2262-04-11 23:47:16.854775807'),
+(7, '9999-12-31 23:59:59.999999999999');
+
+
+--make int64 table with microsecond granularity
+set hive.parquet.write.int64.timestamp=true;
+set hive.parquet.timestamp.time.unit=micros;
+create table parquet_int64 (i int, ts timestamp) stored as parquet;
+insert into parquet_int64 select i, cast (s as timestamp) from 
parquet_timestamp_staging_2;
+
+--make int96 table
+set hive.parquet.write.int64.timestamp=false;
+create table parquet_int96 (i int, ts timestamp) stored as parquet;
+insert into parquet_int96 select i + 10, cast (s as timestamp) from 
parquet_timestamp_staging_2;
+
+--join int64 and int96 tables
+select parquet_int64.i, parquet_int64.ts, parquet_int96.ts
+    from parquet_int64
+    join parquet_int96
+    on parquet_int64.ts = parquet_int96.ts
+    order by parquet_int64.i;
+
+
+--create table with mixed int64/milli and int96 values
+set hive.parquet.write.int64.timestamp=true;
+set hive.parquet.timestamp.time.unit=millis;
+create table parquet_mixed_timestamp as select * from parquet_int64;
+set hive.parquet.write.int64.timestamp=false;
+insert into parquet_mixed_timestamp select i + 10, cast (s as timestamp) from 
parquet_timestamp_staging_2;
+
+select * from parquet_mixed_timestamp order by i;
+select * from parquet_mixed_timestamp where ts > cast ('2200-01-01 
00:00:00.00' as timestamp) order by i;
+select * from parquet_mixed_timestamp where ts < cast ('1900-12-31 
23:59:59.9999999999' as timestamp) order by i;
+select count(*) from parquet_mixed_timestamp where ts = cast ('1970-01-01 
00:00:00.00' as timestamp);
+--join mixed table and int64 table
+select *
+    from parquet_mixed_timestamp
+    join parquet_int64
+    on parquet_mixed_timestamp.ts = parquet_int64.ts
+    order by parquet_mixed_timestamp.i;
diff --git a/ql/src/test/results/clientpositive/parquet_int64_timestamp.q.out 
b/ql/src/test/results/clientpositive/parquet_int64_timestamp.q.out
new file mode 100644
index 0000000..c91be11
--- /dev/null
+++ b/ql/src/test/results/clientpositive/parquet_int64_timestamp.q.out
@@ -0,0 +1,205 @@
+PREHOOK: query: create table parquet_timestamp_staging (i int, s string)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@parquet_timestamp_staging
+POSTHOOK: query: create table parquet_timestamp_staging (i int, s string)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@parquet_timestamp_staging
+PREHOOK: query: insert into parquet_timestamp_staging
+values
+(0, '0001-01-01'),
+(1, '1677-09-21 00:12:43.145224192'),
+(2, '1969-12-31 23:59:59.99999999999999999'),
+(3, '1970-01-01 00:00:00'),
+(4, '2013-09-27 01:36:18.000000001'),
+(5, '2018-01-02 13:14:15.678999'),
+(6, '2262-04-11 23:47:16.854775807'),
+(7, '9999-12-31 23:59:59.999999999999')
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@parquet_timestamp_staging
+POSTHOOK: query: insert into parquet_timestamp_staging
+values
+(0, '0001-01-01'),
+(1, '1677-09-21 00:12:43.145224192'),
+(2, '1969-12-31 23:59:59.99999999999999999'),
+(3, '1970-01-01 00:00:00'),
+(4, '2013-09-27 01:36:18.000000001'),
+(5, '2018-01-02 13:14:15.678999'),
+(6, '2262-04-11 23:47:16.854775807'),
+(7, '9999-12-31 23:59:59.999999999999')
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@parquet_timestamp_staging
+POSTHOOK: Lineage: parquet_timestamp_staging.i SCRIPT []
+POSTHOOK: Lineage: parquet_timestamp_staging.s SCRIPT []
+PREHOOK: query: create table parquet_int64_timestamp (i int, ts timestamp) 
stored as parquet
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@parquet_int64_timestamp
+POSTHOOK: query: create table parquet_int64_timestamp (i int, ts timestamp) 
stored as parquet
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@parquet_int64_timestamp
+PREHOOK: query: insert into parquet_int64_timestamp select i, cast (s as 
timestamp) from parquet_timestamp_staging
+PREHOOK: type: QUERY
+PREHOOK: Input: default@parquet_timestamp_staging
+PREHOOK: Output: default@parquet_int64_timestamp
+POSTHOOK: query: insert into parquet_int64_timestamp select i, cast (s as 
timestamp) from parquet_timestamp_staging
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@parquet_timestamp_staging
+POSTHOOK: Output: default@parquet_int64_timestamp
+POSTHOOK: Lineage: parquet_int64_timestamp.i SIMPLE 
[(parquet_timestamp_staging)parquet_timestamp_staging.FieldSchema(name:i, 
type:int, comment:null), ]
+POSTHOOK: Lineage: parquet_int64_timestamp.ts EXPRESSION 
[(parquet_timestamp_staging)parquet_timestamp_staging.FieldSchema(name:s, 
type:string, comment:null), ]
+PREHOOK: query: select * from parquet_int64_timestamp order by i
+PREHOOK: type: QUERY
+PREHOOK: Input: default@parquet_int64_timestamp
+#### A masked pattern was here ####
+POSTHOOK: query: select * from parquet_int64_timestamp order by i
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@parquet_int64_timestamp
+#### A masked pattern was here ####
+0      NULL
+1      1677-09-21 00:12:43.145224192
+2      1969-12-31 23:59:59.999999999
+3      1970-01-01 00:00:00
+4      2013-09-27 01:36:18.000000001
+5      2018-01-02 13:14:15.678999
+6      2262-04-11 23:47:16.854775807
+7      NULL
+PREHOOK: query: insert into parquet_int64_timestamp select i + 10, cast (s as 
timestamp) from parquet_timestamp_staging
+PREHOOK: type: QUERY
+PREHOOK: Input: default@parquet_timestamp_staging
+PREHOOK: Output: default@parquet_int64_timestamp
+POSTHOOK: query: insert into parquet_int64_timestamp select i + 10, cast (s as 
timestamp) from parquet_timestamp_staging
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@parquet_timestamp_staging
+POSTHOOK: Output: default@parquet_int64_timestamp
+POSTHOOK: Lineage: parquet_int64_timestamp.i EXPRESSION 
[(parquet_timestamp_staging)parquet_timestamp_staging.FieldSchema(name:i, 
type:int, comment:null), ]
+POSTHOOK: Lineage: parquet_int64_timestamp.ts EXPRESSION 
[(parquet_timestamp_staging)parquet_timestamp_staging.FieldSchema(name:s, 
type:string, comment:null), ]
+PREHOOK: query: select * from parquet_int64_timestamp order by i
+PREHOOK: type: QUERY
+PREHOOK: Input: default@parquet_int64_timestamp
+#### A masked pattern was here ####
+POSTHOOK: query: select * from parquet_int64_timestamp order by i
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@parquet_int64_timestamp
+#### A masked pattern was here ####
+0      NULL
+1      1677-09-21 00:12:43.145224192
+2      1969-12-31 23:59:59.999999999
+3      1970-01-01 00:00:00
+4      2013-09-27 01:36:18.000000001
+5      2018-01-02 13:14:15.678999
+6      2262-04-11 23:47:16.854775807
+7      NULL
+10     0001-01-01 00:00:00
+11     1677-09-21 00:12:43.145224
+12     1969-12-31 23:59:59.999999
+13     1970-01-01 00:00:00
+14     2013-09-27 01:36:18
+15     2018-01-02 13:14:15.678999
+16     2262-04-11 23:47:16.854775
+17     9999-12-31 23:59:59.999999
+PREHOOK: query: insert into parquet_int64_timestamp select i + 20, cast (s as 
timestamp) from parquet_timestamp_staging
+PREHOOK: type: QUERY
+PREHOOK: Input: default@parquet_timestamp_staging
+PREHOOK: Output: default@parquet_int64_timestamp
+POSTHOOK: query: insert into parquet_int64_timestamp select i + 20, cast (s as 
timestamp) from parquet_timestamp_staging
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@parquet_timestamp_staging
+POSTHOOK: Output: default@parquet_int64_timestamp
+POSTHOOK: Lineage: parquet_int64_timestamp.i EXPRESSION 
[(parquet_timestamp_staging)parquet_timestamp_staging.FieldSchema(name:i, 
type:int, comment:null), ]
+POSTHOOK: Lineage: parquet_int64_timestamp.ts EXPRESSION 
[(parquet_timestamp_staging)parquet_timestamp_staging.FieldSchema(name:s, 
type:string, comment:null), ]
+PREHOOK: query: select * from parquet_int64_timestamp order by i
+PREHOOK: type: QUERY
+PREHOOK: Input: default@parquet_int64_timestamp
+#### A masked pattern was here ####
+POSTHOOK: query: select * from parquet_int64_timestamp order by i
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@parquet_int64_timestamp
+#### A masked pattern was here ####
+0      NULL
+1      1677-09-21 00:12:43.145224192
+2      1969-12-31 23:59:59.999999999
+3      1970-01-01 00:00:00
+4      2013-09-27 01:36:18.000000001
+5      2018-01-02 13:14:15.678999
+6      2262-04-11 23:47:16.854775807
+7      NULL
+10     0001-01-01 00:00:00
+11     1677-09-21 00:12:43.145224
+12     1969-12-31 23:59:59.999999
+13     1970-01-01 00:00:00
+14     2013-09-27 01:36:18
+15     2018-01-02 13:14:15.678999
+16     2262-04-11 23:47:16.854775
+17     9999-12-31 23:59:59.999999
+20     0001-01-01 00:00:00
+21     1677-09-21 00:12:43.145
+22     1969-12-31 23:59:59.999
+23     1970-01-01 00:00:00
+24     2013-09-27 01:36:18
+25     2018-01-02 13:14:15.678
+26     2262-04-11 23:47:16.854
+27     9999-12-31 23:59:59.999
+PREHOOK: query: select * from parquet_int64_timestamp where ts >  '1969-12-31 
23:59:59.9'
+                                            and
+                                            ts <  '1970-01-01 00:00:00.0' 
order by i
+PREHOOK: type: QUERY
+PREHOOK: Input: default@parquet_int64_timestamp
+#### A masked pattern was here ####
+POSTHOOK: query: select * from parquet_int64_timestamp where ts >  '1969-12-31 
23:59:59.9'
+                                            and
+                                            ts <  '1970-01-01 00:00:00.0' 
order by i
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@parquet_int64_timestamp
+#### A masked pattern was here ####
+2      1969-12-31 23:59:59.999999999
+12     1969-12-31 23:59:59.999999
+22     1969-12-31 23:59:59.999
+PREHOOK: query: select * from parquet_int64_timestamp where ts <= '1970-01-01 
00:00:00.0'
+                                            and
+                                            ts >= '1970-01-01 00:00:00.0' 
order by i
+PREHOOK: type: QUERY
+PREHOOK: Input: default@parquet_int64_timestamp
+#### A masked pattern was here ####
+POSTHOOK: query: select * from parquet_int64_timestamp where ts <= '1970-01-01 
00:00:00.0'
+                                            and
+                                            ts >= '1970-01-01 00:00:00.0' 
order by i
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@parquet_int64_timestamp
+#### A masked pattern was here ####
+3      1970-01-01 00:00:00
+13     1970-01-01 00:00:00
+23     1970-01-01 00:00:00
+PREHOOK: query: select * from parquet_int64_timestamp where ts =  '1970-01-01 
00:00:00.0' order by i
+PREHOOK: type: QUERY
+PREHOOK: Input: default@parquet_int64_timestamp
+#### A masked pattern was here ####
+POSTHOOK: query: select * from parquet_int64_timestamp where ts =  '1970-01-01 
00:00:00.0' order by i
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@parquet_int64_timestamp
+#### A masked pattern was here ####
+3      1970-01-01 00:00:00
+13     1970-01-01 00:00:00
+23     1970-01-01 00:00:00
+PREHOOK: query: select * from parquet_int64_timestamp where ts between cast 
('1969-12-31 23:59:59.9' as timestamp) and
+                                                       cast ('1970-01-01 
00:00:00' as timestamp)
+                                      order by i
+PREHOOK: type: QUERY
+PREHOOK: Input: default@parquet_int64_timestamp
+#### A masked pattern was here ####
+POSTHOOK: query: select * from parquet_int64_timestamp where ts between cast 
('1969-12-31 23:59:59.9' as timestamp) and
+                                                       cast ('1970-01-01 
00:00:00' as timestamp)
+                                      order by i
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@parquet_int64_timestamp
+#### A masked pattern was here ####
+2      1969-12-31 23:59:59.999999999
+3      1970-01-01 00:00:00
+12     1969-12-31 23:59:59.999999
+13     1970-01-01 00:00:00
+22     1969-12-31 23:59:59.999
+23     1970-01-01 00:00:00
diff --git 
a/ql/src/test/results/clientpositive/parquet_int64_timestamp_int96_compatibility.q.out
 
b/ql/src/test/results/clientpositive/parquet_int64_timestamp_int96_compatibility.q.out
new file mode 100644
index 0000000..72fc69a
--- /dev/null
+++ 
b/ql/src/test/results/clientpositive/parquet_int64_timestamp_int96_compatibility.q.out
@@ -0,0 +1,201 @@
+PREHOOK: query: create table parquet_timestamp_staging_2 (i int, s string)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@parquet_timestamp_staging_2
+POSTHOOK: query: create table parquet_timestamp_staging_2 (i int, s string)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@parquet_timestamp_staging_2
+PREHOOK: query: insert into parquet_timestamp_staging_2
+values
+(0, '0001-01-01'),
+(1, '1677-09-21 00:12:43.145224192'),
+(2, '1969-12-31 23:59:59.99999999999999999'),
+(3, '1970-01-01 00:00:00'),
+(4, '2013-09-27 01:36:18.000000001'),
+(5, '2018-01-02 13:14:15.678999'),
+(6, '2262-04-11 23:47:16.854775807'),
+(7, '9999-12-31 23:59:59.999999999999')
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@parquet_timestamp_staging_2
+POSTHOOK: query: insert into parquet_timestamp_staging_2
+values
+(0, '0001-01-01'),
+(1, '1677-09-21 00:12:43.145224192'),
+(2, '1969-12-31 23:59:59.99999999999999999'),
+(3, '1970-01-01 00:00:00'),
+(4, '2013-09-27 01:36:18.000000001'),
+(5, '2018-01-02 13:14:15.678999'),
+(6, '2262-04-11 23:47:16.854775807'),
+(7, '9999-12-31 23:59:59.999999999999')
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@parquet_timestamp_staging_2
+POSTHOOK: Lineage: parquet_timestamp_staging_2.i SCRIPT []
+POSTHOOK: Lineage: parquet_timestamp_staging_2.s SCRIPT []
+PREHOOK: query: create table parquet_int64 (i int, ts timestamp) stored as 
parquet
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@parquet_int64
+POSTHOOK: query: create table parquet_int64 (i int, ts timestamp) stored as 
parquet
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@parquet_int64
+PREHOOK: query: insert into parquet_int64 select i, cast (s as timestamp) from 
parquet_timestamp_staging_2
+PREHOOK: type: QUERY
+PREHOOK: Input: default@parquet_timestamp_staging_2
+PREHOOK: Output: default@parquet_int64
+POSTHOOK: query: insert into parquet_int64 select i, cast (s as timestamp) 
from parquet_timestamp_staging_2
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@parquet_timestamp_staging_2
+POSTHOOK: Output: default@parquet_int64
+POSTHOOK: Lineage: parquet_int64.i SIMPLE 
[(parquet_timestamp_staging_2)parquet_timestamp_staging_2.FieldSchema(name:i, 
type:int, comment:null), ]
+POSTHOOK: Lineage: parquet_int64.ts EXPRESSION 
[(parquet_timestamp_staging_2)parquet_timestamp_staging_2.FieldSchema(name:s, 
type:string, comment:null), ]
+PREHOOK: query: create table parquet_int96 (i int, ts timestamp) stored as 
parquet
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@parquet_int96
+POSTHOOK: query: create table parquet_int96 (i int, ts timestamp) stored as 
parquet
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@parquet_int96
+PREHOOK: query: insert into parquet_int96 select i + 10, cast (s as timestamp) 
from parquet_timestamp_staging_2
+PREHOOK: type: QUERY
+PREHOOK: Input: default@parquet_timestamp_staging_2
+PREHOOK: Output: default@parquet_int96
+POSTHOOK: query: insert into parquet_int96 select i + 10, cast (s as 
timestamp) from parquet_timestamp_staging_2
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@parquet_timestamp_staging_2
+POSTHOOK: Output: default@parquet_int96
+POSTHOOK: Lineage: parquet_int96.i EXPRESSION 
[(parquet_timestamp_staging_2)parquet_timestamp_staging_2.FieldSchema(name:i, 
type:int, comment:null), ]
+POSTHOOK: Lineage: parquet_int96.ts EXPRESSION 
[(parquet_timestamp_staging_2)parquet_timestamp_staging_2.FieldSchema(name:s, 
type:string, comment:null), ]
+PREHOOK: query: select parquet_int64.i, parquet_int64.ts, parquet_int96.ts
+    from parquet_int64
+    join parquet_int96
+    on parquet_int64.ts = parquet_int96.ts
+    order by parquet_int64.i
+PREHOOK: type: QUERY
+PREHOOK: Input: default@parquet_int64
+PREHOOK: Input: default@parquet_int96
+#### A masked pattern was here ####
+POSTHOOK: query: select parquet_int64.i, parquet_int64.ts, parquet_int96.ts
+    from parquet_int64
+    join parquet_int96
+    on parquet_int64.ts = parquet_int96.ts
+    order by parquet_int64.i
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@parquet_int64
+POSTHOOK: Input: default@parquet_int96
+#### A masked pattern was here ####
+0      0001-01-01 00:00:00     0001-01-01 00:00:00
+3      1970-01-01 00:00:00     1970-01-01 00:00:00
+5      2018-01-02 13:14:15.678999      2018-01-02 13:14:15.678999
+PREHOOK: query: create table parquet_mixed_timestamp as select * from 
parquet_int64
+PREHOOK: type: CREATETABLE_AS_SELECT
+PREHOOK: Input: default@parquet_int64
+PREHOOK: Output: database:default
+PREHOOK: Output: default@parquet_mixed_timestamp
+POSTHOOK: query: create table parquet_mixed_timestamp as select * from 
parquet_int64
+POSTHOOK: type: CREATETABLE_AS_SELECT
+POSTHOOK: Input: default@parquet_int64
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@parquet_mixed_timestamp
+POSTHOOK: Lineage: parquet_mixed_timestamp.i SIMPLE 
[(parquet_int64)parquet_int64.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: parquet_mixed_timestamp.ts SIMPLE 
[(parquet_int64)parquet_int64.FieldSchema(name:ts, type:timestamp, 
comment:null), ]
+PREHOOK: query: insert into parquet_mixed_timestamp select i + 10, cast (s as 
timestamp) from parquet_timestamp_staging_2
+PREHOOK: type: QUERY
+PREHOOK: Input: default@parquet_timestamp_staging_2
+PREHOOK: Output: default@parquet_mixed_timestamp
+POSTHOOK: query: insert into parquet_mixed_timestamp select i + 10, cast (s as 
timestamp) from parquet_timestamp_staging_2
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@parquet_timestamp_staging_2
+POSTHOOK: Output: default@parquet_mixed_timestamp
+POSTHOOK: Lineage: parquet_mixed_timestamp.i EXPRESSION 
[(parquet_timestamp_staging_2)parquet_timestamp_staging_2.FieldSchema(name:i, 
type:int, comment:null), ]
+POSTHOOK: Lineage: parquet_mixed_timestamp.ts EXPRESSION 
[(parquet_timestamp_staging_2)parquet_timestamp_staging_2.FieldSchema(name:s, 
type:string, comment:null), ]
+PREHOOK: query: select * from parquet_mixed_timestamp order by i
+PREHOOK: type: QUERY
+PREHOOK: Input: default@parquet_mixed_timestamp
+#### A masked pattern was here ####
+POSTHOOK: query: select * from parquet_mixed_timestamp order by i
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@parquet_mixed_timestamp
+#### A masked pattern was here ####
+0      0001-01-01 00:00:00
+1      1677-09-21 00:12:43.145224
+2      1969-12-31 23:59:59.999999
+3      1970-01-01 00:00:00
+4      2013-09-27 01:36:18
+5      2018-01-02 13:14:15.678999
+6      2262-04-11 23:47:16.854775
+7      9999-12-31 23:59:59.999999
+10     0001-01-01 00:00:00
+11     1677-09-21 00:12:43.145224192
+12     1969-12-31 23:59:59.999999999
+13     1970-01-01 00:00:00
+14     2013-09-27 01:36:18.000000001
+15     2018-01-02 13:14:15.678999
+16     2262-04-11 23:47:16.854775807
+17     9999-12-31 23:59:59.999999999
+PREHOOK: query: select * from parquet_mixed_timestamp where ts > cast 
('2200-01-01 00:00:00.00' as timestamp) order by i
+PREHOOK: type: QUERY
+PREHOOK: Input: default@parquet_mixed_timestamp
+#### A masked pattern was here ####
+POSTHOOK: query: select * from parquet_mixed_timestamp where ts > cast 
('2200-01-01 00:00:00.00' as timestamp) order by i
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@parquet_mixed_timestamp
+#### A masked pattern was here ####
+6      2262-04-11 23:47:16.854775
+7      9999-12-31 23:59:59.999999
+16     2262-04-11 23:47:16.854775807
+17     9999-12-31 23:59:59.999999999
+PREHOOK: query: select * from parquet_mixed_timestamp where ts < cast 
('1900-12-31 23:59:59.9999999999' as timestamp) order by i
+PREHOOK: type: QUERY
+PREHOOK: Input: default@parquet_mixed_timestamp
+#### A masked pattern was here ####
+POSTHOOK: query: select * from parquet_mixed_timestamp where ts < cast 
('1900-12-31 23:59:59.9999999999' as timestamp) order by i
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@parquet_mixed_timestamp
+#### A masked pattern was here ####
+0      0001-01-01 00:00:00
+1      1677-09-21 00:12:43.145224
+10     0001-01-01 00:00:00
+11     1677-09-21 00:12:43.145224192
+PREHOOK: query: select count(*) from parquet_mixed_timestamp where ts = cast 
('1970-01-01 00:00:00.00' as timestamp)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@parquet_mixed_timestamp
+#### A masked pattern was here ####
+POSTHOOK: query: select count(*) from parquet_mixed_timestamp where ts = cast 
('1970-01-01 00:00:00.00' as timestamp)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@parquet_mixed_timestamp
+#### A masked pattern was here ####
+2
+PREHOOK: query: select *
+    from parquet_mixed_timestamp
+    join parquet_int64
+    on parquet_mixed_timestamp.ts = parquet_int64.ts
+    order by parquet_mixed_timestamp.i
+PREHOOK: type: QUERY
+PREHOOK: Input: default@parquet_int64
+PREHOOK: Input: default@parquet_mixed_timestamp
+#### A masked pattern was here ####
+POSTHOOK: query: select *
+    from parquet_mixed_timestamp
+    join parquet_int64
+    on parquet_mixed_timestamp.ts = parquet_int64.ts
+    order by parquet_mixed_timestamp.i
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@parquet_int64
+POSTHOOK: Input: default@parquet_mixed_timestamp
+#### A masked pattern was here ####
+0      0001-01-01 00:00:00     0       0001-01-01 00:00:00
+1      1677-09-21 00:12:43.145224      1       1677-09-21 00:12:43.145224
+2      1969-12-31 23:59:59.999999      2       1969-12-31 23:59:59.999999
+3      1970-01-01 00:00:00     3       1970-01-01 00:00:00
+4      2013-09-27 01:36:18     4       2013-09-27 01:36:18
+5      2018-01-02 13:14:15.678999      5       2018-01-02 13:14:15.678999
+6      2262-04-11 23:47:16.854775      6       2262-04-11 23:47:16.854775
+7      9999-12-31 23:59:59.999999      7       9999-12-31 23:59:59.999999
+10     0001-01-01 00:00:00     0       0001-01-01 00:00:00
+13     1970-01-01 00:00:00     3       1970-01-01 00:00:00
+15     2018-01-02 13:14:15.678999      5       2018-01-02 13:14:15.678999

Reply via email to