You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by sp...@apache.org on 2017/05/17 14:42:52 UTC

[4/4] hive git commit: Revert "HIVE-12767: Implement table property to address Parquet int96 timestamp bug (Barna Zsombor Klara and Sergio Pena, reviewed by Ryan Blue)"

Revert "HIVE-12767: Implement table property to address Parquet int96 timestamp bug (Barna Zsombor Klara and Sergio Pena, reviewed by Ryan Blue)"

This reverts commit 84fdc1c7c8ff0922aa44f829dbfa9659935c503e.


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

Branch: refs/heads/branch-2.3
Commit: 32fd02b7c3910bc2971c80c9a9262a1488e4be80
Parents: d69758c
Author: Sergio Pena <se...@cloudera.com>
Authored: Wed May 17 09:34:08 2017 -0500
Committer: Sergio Pena <se...@cloudera.com>
Committed: Wed May 17 09:34:08 2017 -0500

----------------------------------------------------------------------
 .../org/apache/hadoop/hive/conf/HiveConf.java   |   7 +-
 data/files/impala_int96_timestamp.parq          | Bin 241 -> 0 bytes
 .../benchmark/storage/ColumnarStorageBench.java |   3 +-
 .../org/apache/hadoop/hive/ql/exec/DDLTask.java |  19 -
 .../io/parquet/MapredParquetOutputFormat.java   |  21 -
 .../ql/io/parquet/ParquetRecordReaderBase.java  |  59 +-
 .../ql/io/parquet/convert/ETypeConverter.java   |  24 +-
 .../parquet/read/DataWritableReadSupport.java   |  21 +-
 .../read/ParquetRecordReaderWrapper.java        |  26 +-
 .../ql/io/parquet/serde/ParquetTableUtils.java  |  23 -
 .../ql/io/parquet/timestamp/NanoTimeUtils.java  | 231 +++-----
 .../vector/VectorizedParquetRecordReader.java   |  39 +-
 .../vector/VectorizedPrimitiveColumnReader.java |  18 +-
 .../parquet/write/DataWritableWriteSupport.java |  14 +-
 .../ql/io/parquet/write/DataWritableWriter.java |  10 +-
 .../ql/io/parquet/TestDataWritableWriter.java   |  65 +--
 .../io/parquet/TestVectorizedColumnReader.java  |   2 +-
 .../parquet/VectorizedColumnReaderTestBase.java |   3 +-
 .../io/parquet/convert/TestETypeConverter.java  | 124 -----
 .../serde/TestParquetTimestampUtils.java        | 263 +++++++++
 .../io/parquet/timestamp/TestNanoTimeUtils.java | 233 --------
 .../TestParquetTimestampConverter.java          | 264 ---------
 .../clientpositive/parquet_int96_timestamp.q    |  64 ---
 .../parquet_timestamp_conversion.q              |  13 -
 .../parquet_int96_timestamp.q.out               | 535 -------------------
 .../parquet_timestamp_conversion.q.out          |  68 ---
 26 files changed, 419 insertions(+), 1730 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/32fd02b7/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
----------------------------------------------------------------------
diff --git a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
index 330bd3f..073c087 100644
--- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
+++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
@@ -1329,12 +1329,9 @@ public class HiveConf extends Configuration {
         "Maximum fraction of heap that can be used by Parquet file writers in one task.\n" +
         "It is for avoiding OutOfMemory error in tasks. Work with Parquet 1.6.0 and above.\n" +
         "This config parameter is defined in Parquet, so that it does not start with 'hive.'."),
-    @Deprecated
     HIVE_PARQUET_TIMESTAMP_SKIP_CONVERSION("hive.parquet.timestamp.skip.conversion", true,
-        "Current Hive implementation of parquet stores timestamps to UTC, this flag allows skipping of the conversion" +
-            "on reading parquet files from other tools"),
-    PARQUET_INT96_DEFAULT_UTC_WRITE_ZONE("parquet.mr.int96.enable.utc.write.zone", false,
-        "Enable this variable to use UTC as the default timezone for new Parquet tables."),
+      "Current Hive implementation of parquet stores timestamps to UTC, this flag allows skipping of the conversion" +
+      "on reading parquet files from other tools"),
     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." ),

http://git-wip-us.apache.org/repos/asf/hive/blob/32fd02b7/data/files/impala_int96_timestamp.parq
----------------------------------------------------------------------
diff --git a/data/files/impala_int96_timestamp.parq b/data/files/impala_int96_timestamp.parq
deleted file mode 100644
index d67dd14..0000000
Binary files a/data/files/impala_int96_timestamp.parq and /dev/null differ

http://git-wip-us.apache.org/repos/asf/hive/blob/32fd02b7/itests/hive-jmh/src/main/java/org/apache/hive/benchmark/storage/ColumnarStorageBench.java
----------------------------------------------------------------------
diff --git a/itests/hive-jmh/src/main/java/org/apache/hive/benchmark/storage/ColumnarStorageBench.java b/itests/hive-jmh/src/main/java/org/apache/hive/benchmark/storage/ColumnarStorageBench.java
index 781c4b9..a14b790 100644
--- a/itests/hive-jmh/src/main/java/org/apache/hive/benchmark/storage/ColumnarStorageBench.java
+++ b/itests/hive-jmh/src/main/java/org/apache/hive/benchmark/storage/ColumnarStorageBench.java
@@ -62,7 +62,6 @@ import org.apache.hadoop.mapred.RecordReader;
 import org.apache.hadoop.mapreduce.InputSplit;
 import org.apache.hadoop.mapreduce.Job;
 import org.apache.parquet.hadoop.ParquetInputFormat;
-import org.apache.parquet.hadoop.ParquetInputSplit;
 import org.apache.parquet.hadoop.api.ReadSupport;
 import org.apache.parquet.hadoop.example.GroupReadSupport;
 import org.openjdk.jmh.annotations.Param;
@@ -339,7 +338,7 @@ public class ColumnarStorageBench {
       Job vectorJob = new Job(conf, "read vector");
       ParquetInputFormat.setInputPaths(vectorJob, inputPath);
       ParquetInputFormat parquetInputFormat = new ParquetInputFormat(GroupReadSupport.class);
-      ParquetInputSplit split = (ParquetInputSplit) parquetInputFormat.getSplits(vectorJob).get(0);
+      InputSplit split = (InputSplit) parquetInputFormat.getSplits(vectorJob).get(0);
       initialVectorizedRowBatchCtx(conf);
       return new VectorizedParquetRecordReader(split, new JobConf(conf));
     }

http://git-wip-us.apache.org/repos/asf/hive/blob/32fd02b7/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
index 2b13477..b339f97 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
@@ -117,7 +117,6 @@ import org.apache.hadoop.hive.ql.io.merge.MergeFileWork;
 import org.apache.hadoop.hive.ql.io.orc.OrcInputFormat;
 import org.apache.hadoop.hive.ql.io.orc.OrcSerde;
 import org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe;
-import org.apache.hadoop.hive.ql.io.parquet.serde.ParquetTableUtils;
 import org.apache.hadoop.hive.ql.io.rcfile.truncate.ColumnTruncateTask;
 import org.apache.hadoop.hive.ql.io.rcfile.truncate.ColumnTruncateWork;
 import org.apache.hadoop.hive.ql.lockmgr.DbLockManager;
@@ -4341,18 +4340,6 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
       }
     }
 
-    // If PARQUET_INT96_DEFAULT_UTC_WRITE_ZONE is set to True, then set new Parquet tables timezone
-    // to UTC by default (only if the table property is not set)
-    if (tbl.getSerializationLib().equals(ParquetHiveSerDe.class.getName())) {
-      SessionState ss = SessionState.get();
-      if (ss.getConf().getBoolVar(ConfVars.PARQUET_INT96_DEFAULT_UTC_WRITE_ZONE)) {
-        String parquetTimezone = tbl.getProperty(ParquetTableUtils.PARQUET_INT96_WRITE_ZONE_PROPERTY);
-        if (parquetTimezone == null || parquetTimezone.isEmpty()) {
-          tbl.setProperty(ParquetTableUtils.PARQUET_INT96_WRITE_ZONE_PROPERTY, ParquetTableUtils.PARQUET_INT96_NO_ADJUSTMENT_ZONE);
-        }
-      }
-    }
-
     // create the table
     if (crtTbl.getReplaceMode()){
       // replace-mode creates are really alters using CreateTableDesc.
@@ -4474,12 +4461,6 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
       if (paramsStr != null) {
         retainer.addAll(Arrays.asList(paramsStr.split(",")));
       }
-
-      // Retain Parquet INT96 write zone property to keep Parquet timezone bugfixes.
-      if (params.get(ParquetTableUtils.PARQUET_INT96_WRITE_ZONE_PROPERTY) != null) {
-        retainer.add(ParquetTableUtils.PARQUET_INT96_WRITE_ZONE_PROPERTY);
-      }
-
       if (!retainer.isEmpty()) {
         params.keySet().retainAll(retainer);
       } else {

http://git-wip-us.apache.org/repos/asf/hive/blob/32fd02b7/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/MapredParquetOutputFormat.java
----------------------------------------------------------------------
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 26f1e75..379a913 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
@@ -18,10 +18,7 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
 import java.util.Properties;
-import java.util.TimeZone;
 
-import org.apache.hadoop.hive.ql.io.parquet.serde.ParquetTableUtils;
-import org.apache.parquet.Strings;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.fs.FileSystem;
@@ -116,7 +113,6 @@ public class MapredParquetOutputFormat extends FileOutputFormat<NullWritable, Pa
     }
 
     DataWritableWriteSupport.setSchema(HiveSchemaConverter.convert(columnNames, columnTypes), jobConf);
-    DataWritableWriteSupport.setTimeZone(getParquetWriterTimeZone(tableProperties), jobConf);
 
     return getParquerRecordWriterWrapper(realOutputFormat, jobConf, finalOutPath.toString(),
             progress,tableProperties);
@@ -132,21 +128,4 @@ public class MapredParquetOutputFormat extends FileOutputFormat<NullWritable, Pa
     return new ParquetRecordWriterWrapper(realOutputFormat, jobConf, finalOutPath.toString(),
             progress,tableProperties);
   }
-
-  private TimeZone getParquetWriterTimeZone(Properties tableProperties) {
-    // PARQUET_INT96_WRITE_ZONE_PROPERTY is a table property used to detect what timezone
-    // conversion to use when writing Parquet timestamps.
-    String timeZoneID =
-        tableProperties.getProperty(ParquetTableUtils.PARQUET_INT96_WRITE_ZONE_PROPERTY);
-    if (!Strings.isNullOrEmpty(timeZoneID)) {
-      if (!Arrays.asList(TimeZone.getAvailableIDs()).contains(timeZoneID)) {
-        throw new IllegalStateException("Unexpected timezone id found for parquet int96 conversion: " + timeZoneID);
-      }
-      return TimeZone.getTimeZone(timeZoneID);
-    }
-
-    // If no timezone is defined in table properties, then adjust timestamps using
-    // PARQUET_INT96_NO_ADJUSTMENT_ZONE timezone
-    return TimeZone.getTimeZone(ParquetTableUtils.PARQUET_INT96_NO_ADJUSTMENT_ZONE);
-  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/32fd02b7/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/ParquetRecordReaderBase.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/ParquetRecordReaderBase.java b/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/ParquetRecordReaderBase.java
index 8e33b7d..167f9b6 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/ParquetRecordReaderBase.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/ParquetRecordReaderBase.java
@@ -14,12 +14,10 @@
 package org.apache.hadoop.hive.ql.io.parquet;
 
 import com.google.common.base.Strings;
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.ql.io.parquet.read.DataWritableReadSupport;
 import org.apache.hadoop.hive.ql.io.parquet.read.ParquetFilterPredicateConverter;
-import org.apache.hadoop.hive.ql.io.parquet.serde.ParquetTableUtils;
 import org.apache.hadoop.hive.ql.io.sarg.ConvertAstToSearchArg;
 import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
 import org.apache.hadoop.hive.serde2.SerDeStats;
@@ -28,7 +26,6 @@ import org.apache.hadoop.mapred.JobConf;
 import org.apache.parquet.filter2.compat.FilterCompat;
 import org.apache.parquet.filter2.compat.RowGroupFilter;
 import org.apache.parquet.filter2.predicate.FilterPredicate;
-import org.apache.parquet.format.converter.ParquetMetadataConverter;
 import org.apache.parquet.hadoop.ParquetFileReader;
 import org.apache.parquet.hadoop.ParquetInputFormat;
 import org.apache.parquet.hadoop.ParquetInputSplit;
@@ -44,15 +41,14 @@ import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
 import java.util.ArrayList;
-import java.util.Arrays;
 import java.util.List;
-import java.util.TimeZone;
 
 public class ParquetRecordReaderBase {
   public static final Logger LOG = LoggerFactory.getLogger(ParquetRecordReaderBase.class);
 
   protected Path file;
   protected ProjectionPusher projectionPusher;
+  protected boolean skipTimestampConversion = false;
   protected SerDeStats serDeStats;
   protected JobConf jobConf;
 
@@ -74,11 +70,6 @@ public class ParquetRecordReaderBase {
     final JobConf conf
   ) throws IOException {
     ParquetInputSplit split;
-
-    if (oldSplit == null) {
-      return null;
-    }
-
     if (oldSplit instanceof FileSplit) {
       final Path finalPath = ((FileSplit) oldSplit).getPath();
       jobConf = projectionPusher.pushProjectionsAndFilters(conf, finalPath.getParent());
@@ -131,6 +122,9 @@ public class ParquetRecordReaderBase {
         filtedBlocks = splitGroup;
       }
 
+      if (HiveConf.getBoolVar(conf, HiveConf.ConfVars.HIVE_PARQUET_TIMESTAMP_SKIP_CONVERSION)) {
+        skipTimestampConversion = !Strings.nullToEmpty(fileMetaData.getCreatedBy()).startsWith("parquet-mr");
+      }
       split = new ParquetInputSplit(finalPath,
         splitStart,
         splitLength,
@@ -146,51 +140,6 @@ public class ParquetRecordReaderBase {
     }
   }
 
-  /**
-   * Sets the TimeZone conversion for Parquet timestamp columns.
-   *
-   * @param configuration Configuration object where to get and set the TimeZone conversion
-   * @param finalPath     path to the parquet file
-   */
-  protected void setTimeZoneConversion(Configuration configuration, Path finalPath) {
-    ParquetMetadata parquetMetadata;
-    String timeZoneID;
-
-    try {
-      parquetMetadata = ParquetFileReader.readFooter(configuration, finalPath,
-          ParquetMetadataConverter.NO_FILTER);
-    } catch (IOException e) {
-      // If an error occurred while reading the file, then we just skip the TimeZone setting.
-      // This error will probably occur on any other part of the code.
-      LOG.debug("Could not read parquet file footer at " + finalPath + ". Cannot determine " +
-          "parquet file timezone", e);
-      return;
-    }
-
-    boolean skipConversion = HiveConf.getBoolVar(configuration,
-        HiveConf.ConfVars.HIVE_PARQUET_TIMESTAMP_SKIP_CONVERSION);
-    FileMetaData fileMetaData = parquetMetadata.getFileMetaData();
-    if (!Strings.nullToEmpty(fileMetaData.getCreatedBy()).startsWith("parquet-mr") ||
-        skipConversion) {
-      // Impala writes timestamp values using GMT only. We should not try to convert Impala
-      // files to other type of timezones.
-      timeZoneID = ParquetTableUtils.PARQUET_INT96_NO_ADJUSTMENT_ZONE;
-    } else {
-      // TABLE_PARQUET_INT96_TIMEZONE is a table property used to detect what timezone conversion
-      // to use when reading Parquet timestamps.
-      timeZoneID = configuration.get(ParquetTableUtils.PARQUET_INT96_WRITE_ZONE_PROPERTY,
-          ParquetTableUtils.PARQUET_INT96_NO_ADJUSTMENT_ZONE);
-
-      if (!Arrays.asList(TimeZone.getAvailableIDs()).contains(timeZoneID)) {
-          throw new IllegalStateException("Unexpected timezone id found for parquet int96 conversion: " + timeZoneID);
-      }
-    }
-
-    // 'timeZoneID' should be valid, since we did not throw exception above
-    configuration.set(ParquetTableUtils.PARQUET_INT96_WRITE_ZONE_PROPERTY,
-        TimeZone.getTimeZone(timeZoneID).getID());
-  }
-
   public FilterCompat.Filter setFilter(final JobConf conf, MessageType schema) {
     SearchArgument sarg = ConvertAstToSearchArg.createFromConf(conf);
     if (sarg == null) {

http://git-wip-us.apache.org/repos/asf/hive/blob/32fd02b7/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/convert/ETypeConverter.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/convert/ETypeConverter.java b/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/convert/ETypeConverter.java
index f4ad083..76d93b8 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/convert/ETypeConverter.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/convert/ETypeConverter.java
@@ -16,11 +16,9 @@ package org.apache.hadoop.hive.ql.io.parquet.convert;
 import java.math.BigDecimal;
 import java.sql.Timestamp;
 import java.util.ArrayList;
-import java.util.Calendar;
 import java.util.Map;
-import java.util.TimeZone;
 
-import org.apache.hadoop.hive.ql.io.parquet.serde.ParquetTableUtils;
+import org.apache.hadoop.hive.conf.HiveConf;
 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.serde.serdeConstants;
@@ -38,7 +36,6 @@ import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Writable;
 
-import org.apache.parquet.Strings;
 import org.apache.parquet.column.Dictionary;
 import org.apache.parquet.io.api.Binary;
 import org.apache.parquet.io.api.PrimitiveConverter;
@@ -196,21 +193,16 @@ public enum ETypeConverter {
   ETIMESTAMP_CONVERTER(TimestampWritable.class) {
     @Override
     PrimitiveConverter getConverter(final PrimitiveType type, final int index, final ConverterParent parent, TypeInfo hiveTypeInfo) {
-      Map<String, String> metadata = parent.getMetadata();
-
-      // This variable must be initialized only once to keep good read performance while doing conversion of timestamps values.
-      final Calendar calendar;
-      if (Strings.isNullOrEmpty(metadata.get(ParquetTableUtils.PARQUET_INT96_WRITE_ZONE_PROPERTY))) {
-        // Local time should be used if timezone is not available.
-        calendar = Calendar.getInstance();
-      } else {
-        calendar = Calendar.getInstance(TimeZone.getTimeZone(metadata.get(ParquetTableUtils.PARQUET_INT96_WRITE_ZONE_PROPERTY)));
-      }
-
       return new BinaryConverter<TimestampWritable>(type, parent, index) {
         @Override
         protected TimestampWritable convert(Binary binary) {
-          Timestamp ts = NanoTimeUtils.getTimestamp(NanoTime.fromBinary(binary), calendar);
+          NanoTime nt = NanoTime.fromBinary(binary);
+          Map<String, String> metadata = parent.getMetadata();
+          //Current Hive parquet timestamp implementation stores it in UTC, but other components do not do that.
+          //If this file written by current Hive implementation itself, we need to do the reverse conversion, else skip the conversion.
+          boolean skipConversion = Boolean.parseBoolean(
+              metadata.get(HiveConf.ConfVars.HIVE_PARQUET_TIMESTAMP_SKIP_CONVERSION.varname));
+          Timestamp ts = NanoTimeUtils.getTimestamp(nt, skipConversion);
           return new TimestampWritable(ts);
         }
       };

http://git-wip-us.apache.org/repos/asf/hive/blob/32fd02b7/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/read/DataWritableReadSupport.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/read/DataWritableReadSupport.java b/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/read/DataWritableReadSupport.java
index 65178cf..604cbbc 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/read/DataWritableReadSupport.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/read/DataWritableReadSupport.java
@@ -16,6 +16,7 @@ package org.apache.hadoop.hive.ql.io.parquet.read;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.List;
 import java.util.ListIterator;
 import java.util.Map;
@@ -26,7 +27,6 @@ import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.ql.io.IOConstants;
 import org.apache.hadoop.hive.ql.io.parquet.convert.DataWritableRecordConverter;
 import org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe;
-import org.apache.hadoop.hive.ql.io.parquet.serde.ParquetTableUtils;
 import org.apache.hadoop.hive.ql.metadata.VirtualColumn;
 import org.apache.hadoop.hive.ql.optimizer.FieldNode;
 import org.apache.hadoop.hive.ql.optimizer.NestedColumnFieldPruningUtils;
@@ -59,9 +59,9 @@ import org.apache.parquet.schema.Types;
  *
  */
 public class DataWritableReadSupport extends ReadSupport<ArrayWritable> {
+
   public static final String HIVE_TABLE_AS_PARQUET_SCHEMA = "HIVE_TABLE_SCHEMA";
   public static final String PARQUET_COLUMN_INDEX_ACCESS = "parquet.column.index.access";
-
   private TypeInfo hiveTypeInfo;
   /**
    * From a string which columns names (including hive column), return a list
@@ -349,11 +349,6 @@ public class DataWritableReadSupport extends ReadSupport<ArrayWritable> {
     Map<String, String> contextMetadata = new HashMap<String, String>();
     boolean indexAccess = configuration.getBoolean(PARQUET_COLUMN_INDEX_ACCESS, false);
 
-    // Adds the PARQUET_INT96_WRITE_ZONE_PROPERTY value to the metadata object so that it passes the timezone
-    // to the Parquet readers. PARQUET_INT96_WRITE_ZONE_PROPERTY is set on ParquetRecordReaderWrapper.
-    contextMetadata.put(ParquetTableUtils.PARQUET_INT96_WRITE_ZONE_PROPERTY,
-        configuration.get(ParquetTableUtils.PARQUET_INT96_WRITE_ZONE_PROPERTY));
-
     if (columnNames != null) {
       List<String> columnNamesList = getColumnNames(columnNames);
       String columnTypes = configuration.get(IOConstants.COLUMNS_TYPES);
@@ -407,6 +402,16 @@ public class DataWritableReadSupport extends ReadSupport<ArrayWritable> {
   public RecordMaterializer<ArrayWritable> prepareForRead(final Configuration configuration,
       final Map<String, String> keyValueMetaData, final MessageType fileSchema,
           final org.apache.parquet.hadoop.api.ReadSupport.ReadContext readContext) {
-    return new DataWritableRecordConverter(readContext.getRequestedSchema(), readContext.getReadSupportMetadata(), hiveTypeInfo);
+    final Map<String, String> metadata = readContext.getReadSupportMetadata();
+    if (metadata == null) {
+      throw new IllegalStateException("ReadContext not initialized properly. " +
+        "Don't know the Hive Schema.");
+    }
+    String key = HiveConf.ConfVars.HIVE_PARQUET_TIMESTAMP_SKIP_CONVERSION.varname;
+    if (!metadata.containsKey(key)) {
+      metadata.put(key, String.valueOf(HiveConf.getBoolVar(
+        configuration, HiveConf.ConfVars.HIVE_PARQUET_TIMESTAMP_SKIP_CONVERSION)));
+    }
+    return new DataWritableRecordConverter(readContext.getRequestedSchema(), metadata, hiveTypeInfo);
   }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/32fd02b7/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/read/ParquetRecordReaderWrapper.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/read/ParquetRecordReaderWrapper.java b/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/read/ParquetRecordReaderWrapper.java
index 66fca1a..ac430a6 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/read/ParquetRecordReaderWrapper.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/read/ParquetRecordReaderWrapper.java
@@ -16,13 +16,10 @@ package org.apache.hadoop.hive.ql.io.parquet.read;
 import java.io.IOException;
 
 import org.apache.hadoop.hive.ql.io.parquet.ParquetRecordReaderBase;
-import org.apache.hadoop.mapred.FileSplit;
-import org.apache.hadoop.mapred.InputSplit;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.RecordReader;
-import org.apache.hadoop.mapred.Reporter;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.ql.io.IOConstants;
 import org.apache.hadoop.hive.ql.io.StatsProvidingRecordReader;
 import org.apache.hadoop.hive.ql.io.parquet.ProjectionPusher;
@@ -30,6 +27,10 @@ import org.apache.hadoop.hive.serde2.SerDeStats;
 import org.apache.hadoop.io.ArrayWritable;
 import org.apache.hadoop.io.NullWritable;
 import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.mapred.InputSplit;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.RecordReader;
+import org.apache.hadoop.mapred.Reporter;
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
 import org.apache.hadoop.mapreduce.TaskAttemptID;
 
@@ -79,14 +80,13 @@ public class ParquetRecordReaderWrapper extends ParquetRecordReaderBase
     }
 
     // create a TaskInputOutputContext
-    // TODO: This line is left due to incorrect Predicate push down results (parquet_ppd_char,parquet_ppd_varchar).
-    // The problem is that Parquet PPD is set on getSplit() function called above, but the old code used this
-    // line to overwrite such configuration. I'm adding a fix to timestamp issues only, so we should follow up
-    // this issue in another JIRA.
-    JobConf conf = new JobConf(oldJobConf);
-
-    // Set the TimeZone conversion in case the file has timestamp columns.
-    setTimeZoneConversion(conf, ((FileSplit)oldSplit).getPath());
+    Configuration conf = jobConf;
+    if (skipTimestampConversion ^ HiveConf.getBoolVar(
+        conf, HiveConf.ConfVars.HIVE_PARQUET_TIMESTAMP_SKIP_CONVERSION)) {
+      conf = new JobConf(oldJobConf);
+      HiveConf.setBoolVar(conf,
+        HiveConf.ConfVars.HIVE_PARQUET_TIMESTAMP_SKIP_CONVERSION, skipTimestampConversion);
+    }
 
     final TaskAttemptContext taskContext = ContextUtil.newTaskAttemptContext(conf, taskAttemptID);
     if (split != null) {

http://git-wip-us.apache.org/repos/asf/hive/blob/32fd02b7/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/serde/ParquetTableUtils.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/serde/ParquetTableUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/serde/ParquetTableUtils.java
deleted file mode 100644
index b339cc4..0000000
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/serde/ParquetTableUtils.java
+++ /dev/null
@@ -1,23 +0,0 @@
-/**
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hive.ql.io.parquet.serde;
-
-public class ParquetTableUtils {
-  // Parquet table properties
-  public static final String PARQUET_INT96_WRITE_ZONE_PROPERTY = "parquet.mr.int96.write.zone";
-
-  // This is not a TimeZone we convert into and print out, rather a delta, an adjustment we use.
-  // More precisely the lack of an adjustment in case of UTC
-  public static final String PARQUET_INT96_NO_ADJUSTMENT_ZONE = "UTC";
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/32fd02b7/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/timestamp/NanoTimeUtils.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/timestamp/NanoTimeUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/timestamp/NanoTimeUtils.java
index 5dc8088..3fd75d2 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/timestamp/NanoTimeUtils.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/timestamp/NanoTimeUtils.java
@@ -16,161 +16,98 @@ package org.apache.hadoop.hive.ql.io.parquet.timestamp;
 import java.sql.Timestamp;
 import java.util.Calendar;
 import java.util.GregorianCalendar;
-import java.util.Objects;
 import java.util.TimeZone;
 import java.util.concurrent.TimeUnit;
 
 import jodd.datetime.JDateTime;
-import org.apache.hadoop.hive.ql.io.parquet.serde.ParquetTableUtils;
 
 /**
  * Utilities for converting from java.sql.Timestamp to parquet timestamp.
  * This utilizes the Jodd library.
  */
 public class NanoTimeUtils {
-  private static final long NANOS_PER_HOUR = TimeUnit.HOURS.toNanos(1);
-  private static final long NANOS_PER_MINUTE = TimeUnit.MINUTES.toNanos(1);
-  private static final long NANOS_PER_SECOND = TimeUnit.SECONDS.toNanos(1);
-  private static final long NANOS_PER_DAY = TimeUnit.DAYS.toNanos(1);
-
-  private static final ThreadLocal<Calendar> parquetUTCCalendar = new ThreadLocal<Calendar>();
-  private static final ThreadLocal<Calendar> parquetLocalCalendar = new ThreadLocal<Calendar>();
-
-  private static Calendar getUTCCalendar() {
-    //Calendar.getInstance calculates the current-time needlessly, so cache an instance.
-    if (parquetUTCCalendar.get() == null) {
-      parquetUTCCalendar.set(Calendar.getInstance(TimeZone.getTimeZone("UTC")));
-    }
-    return parquetUTCCalendar.get();
-  }
-
-  private static Calendar getLocalCalendar() {
-    if (parquetLocalCalendar.get() == null) {
-      parquetLocalCalendar.set(Calendar.getInstance());
-    }
-    return parquetLocalCalendar.get();
-  }
-
-  public static Calendar getCalendar(boolean skipConversion) {
-    Calendar calendar = skipConversion ? Calendar.getInstance(TimeZone.getTimeZone("UTC"))
-        : Calendar.getInstance();
-    calendar.clear(); // Reset all fields before reusing this instance
-    return calendar;
-  }
-
-  @Deprecated
-  public static NanoTime getNanoTime(Timestamp ts, boolean skipConversion) {
-    return getNanoTime(ts, getCalendar(skipConversion));
-  }
-
-  /**
-   * Constructs a julian date from the floating time Timestamp.
-   * If the timezone of the calendar is different from the current local
-   * timezone, then the timestamp value will be adjusted.
-   * Possible adjustments:
-   *   - UTC Ts -> Local Ts copied to TableTZ Calendar -> UTC Ts -> JD
-   * @param ts floating time timestamp to store
-   * @param calendar timezone used to adjust the timestamp for parquet
-   * @return adjusted julian date
-   */
-  public static NanoTime getNanoTime(Timestamp ts, Calendar calendar) {
-
-    Calendar localCalendar = getLocalCalendar();
-    localCalendar.setTimeInMillis(ts.getTime());
-
-    Calendar adjustedCalendar = copyToCalendarWithTZ(localCalendar, calendar);
-
-    Calendar utcCalendar = getUTCCalendar();
-    utcCalendar.setTimeInMillis(adjustedCalendar.getTimeInMillis());
-
-    int year = utcCalendar.get(Calendar.YEAR);
-    if (utcCalendar.get(Calendar.ERA) == GregorianCalendar.BC) {
-      year = 1 - year;
-    }
-    JDateTime jDateTime = new JDateTime(year,
-        utcCalendar.get(Calendar.MONTH) + 1,  //java calendar index starting at 1.
-        utcCalendar.get(Calendar.DAY_OF_MONTH));
-    int days = jDateTime.getJulianDayNumber();
-
-    long hour = utcCalendar.get(Calendar.HOUR_OF_DAY);
-    long minute = utcCalendar.get(Calendar.MINUTE);
-    long second = utcCalendar.get(Calendar.SECOND);
-    long nanos = ts.getNanos();
-    long nanosOfDay = nanos + NANOS_PER_SECOND * second + NANOS_PER_MINUTE * minute +
-        NANOS_PER_HOUR * hour;
-
-    return new NanoTime(days, nanosOfDay);
-  }
-
-  @Deprecated
-  public static Timestamp getTimestamp(NanoTime nt, boolean skipConversion) {
-    return getTimestamp(nt, getCalendar(skipConversion));
-  }
-
-  /**
-   * Constructs a floating time Timestamp from the julian date contained in NanoTime.
-   * If the timezone of the calendar is different from the current local
-   * timezone, then the timestamp value will be adjusted.
-   * Possible adjustments:
-   *   - JD -> UTC Ts -> TableTZ Calendar copied to LocalTZ Calendar -> UTC Ts
-   * @param nt stored julian date
-   * @param calendar timezone used to adjust the timestamp for parquet
-   * @return floating time represented as a timestamp. Guaranteed to display
-   * the same when formatted using the current local timezone as with the local
-   * timezone at the time it was stored.
-   */
-  public static Timestamp getTimestamp(NanoTime nt, Calendar calendar) {
-    int julianDay = nt.getJulianDay();
-    long nanosOfDay = nt.getTimeOfDayNanos();
-
-    long remainder = nanosOfDay;
-    julianDay += remainder / NANOS_PER_DAY;
-    remainder %= NANOS_PER_DAY;
-    if (remainder < 0) {
-      remainder += NANOS_PER_DAY;
-      julianDay--;
-    }
-
-    JDateTime jDateTime = new JDateTime((double) julianDay);
-
-    Calendar utcCalendar = getUTCCalendar();
-    utcCalendar.clear();
-    utcCalendar.set(Calendar.YEAR, jDateTime.getYear());
-    utcCalendar.set(Calendar.MONTH, jDateTime.getMonth() - 1); //java calendar index starting at 1.
-    utcCalendar.set(Calendar.DAY_OF_MONTH, jDateTime.getDay());
-
-    int hour = (int) (remainder / (NANOS_PER_HOUR));
-    remainder = remainder % (NANOS_PER_HOUR);
-    int minutes = (int) (remainder / (NANOS_PER_MINUTE));
-    remainder = remainder % (NANOS_PER_MINUTE);
-    int seconds = (int) (remainder / (NANOS_PER_SECOND));
-    long nanos = remainder % NANOS_PER_SECOND;
-
-    utcCalendar.set(Calendar.HOUR_OF_DAY, hour);
-    utcCalendar.set(Calendar.MINUTE, minutes);
-    utcCalendar.set(Calendar.SECOND, seconds);
-
-    calendar.setTimeInMillis(utcCalendar.getTimeInMillis());
-
-    Calendar adjusterCalendar = copyToCalendarWithTZ(calendar, Calendar.getInstance());
-
-    Timestamp ts = new Timestamp(adjusterCalendar.getTimeInMillis());
-    ts.setNanos((int) nanos);
-    return ts;
-  }
-
-  private static Calendar copyToCalendarWithTZ(Calendar from, Calendar to) {
-    if(from.getTimeZone().getID().equals(to.getTimeZone().getID())) {
-      return from;
-    } else {
-      to.set(Calendar.ERA, from.get(Calendar.ERA));
-      to.set(Calendar.YEAR, from.get(Calendar.YEAR));
-      to.set(Calendar.MONTH, from.get(Calendar.MONTH));
-      to.set(Calendar.DAY_OF_MONTH, from.get(Calendar.DAY_OF_MONTH));
-      to.set(Calendar.HOUR_OF_DAY, from.get(Calendar.HOUR_OF_DAY));
-      to.set(Calendar.MINUTE, from.get(Calendar.MINUTE));
-      to.set(Calendar.SECOND, from.get(Calendar.SECOND));
-      return to;
-    }
-  }
+   static final long NANOS_PER_HOUR = TimeUnit.HOURS.toNanos(1);
+   static final long NANOS_PER_MINUTE = TimeUnit.MINUTES.toNanos(1);
+   static final long NANOS_PER_SECOND = TimeUnit.SECONDS.toNanos(1);
+   static final long NANOS_PER_DAY = TimeUnit.DAYS.toNanos(1);
+
+   private static final ThreadLocal<Calendar> parquetGMTCalendar = new ThreadLocal<Calendar>();
+   private static final ThreadLocal<Calendar> parquetLocalCalendar = new ThreadLocal<Calendar>();
+
+   private static Calendar getGMTCalendar() {
+     //Calendar.getInstance calculates the current-time needlessly, so cache an instance.
+     if (parquetGMTCalendar.get() == null) {
+       parquetGMTCalendar.set(Calendar.getInstance(TimeZone.getTimeZone("GMT")));
+     }
+     return parquetGMTCalendar.get();
+   }
+
+   private static Calendar getLocalCalendar() {
+     if (parquetLocalCalendar.get() == null) {
+       parquetLocalCalendar.set(Calendar.getInstance());
+     }
+     return parquetLocalCalendar.get();
+   }
+
+   public static Calendar getCalendar(boolean skipConversion) {
+     Calendar calendar = skipConversion ? getLocalCalendar() : getGMTCalendar();
+     calendar.clear(); // Reset all fields before reusing this instance
+     return calendar;
+   }
+
+   public static NanoTime getNanoTime(Timestamp ts, boolean skipConversion) {
+
+     Calendar calendar = getCalendar(skipConversion);
+     calendar.setTime(ts);
+     int year = calendar.get(Calendar.YEAR);
+     if (calendar.get(Calendar.ERA) == GregorianCalendar.BC) {
+       year = 1 - year;
+     }
+     JDateTime jDateTime = new JDateTime(year,
+       calendar.get(Calendar.MONTH) + 1,  //java calendar index starting at 1.
+       calendar.get(Calendar.DAY_OF_MONTH));
+     int days = jDateTime.getJulianDayNumber();
+
+     long hour = calendar.get(Calendar.HOUR_OF_DAY);
+     long minute = calendar.get(Calendar.MINUTE);
+     long second = calendar.get(Calendar.SECOND);
+     long nanos = ts.getNanos();
+     long nanosOfDay = nanos + NANOS_PER_SECOND * second + NANOS_PER_MINUTE * minute +
+         NANOS_PER_HOUR * hour;
+
+     return new NanoTime(days, nanosOfDay);
+   }
+
+   public static Timestamp getTimestamp(NanoTime nt, boolean skipConversion) {
+     int julianDay = nt.getJulianDay();
+     long nanosOfDay = nt.getTimeOfDayNanos();
+
+     long remainder = nanosOfDay;
+     julianDay += remainder / NANOS_PER_DAY;
+     remainder %= NANOS_PER_DAY;
+     if (remainder < 0) {
+       remainder += NANOS_PER_DAY;
+       julianDay--;
+     }
+
+     JDateTime jDateTime = new JDateTime((double) julianDay);
+     Calendar calendar = getCalendar(skipConversion);
+     calendar.set(Calendar.YEAR, jDateTime.getYear());
+     calendar.set(Calendar.MONTH, jDateTime.getMonth() - 1); //java calendar index starting at 1.
+     calendar.set(Calendar.DAY_OF_MONTH, jDateTime.getDay());
+
+     int hour = (int) (remainder / (NANOS_PER_HOUR));
+     remainder = remainder % (NANOS_PER_HOUR);
+     int minutes = (int) (remainder / (NANOS_PER_MINUTE));
+     remainder = remainder % (NANOS_PER_MINUTE);
+     int seconds = (int) (remainder / (NANOS_PER_SECOND));
+     long nanos = remainder % NANOS_PER_SECOND;
+
+     calendar.set(Calendar.HOUR_OF_DAY, hour);
+     calendar.set(Calendar.MINUTE, minutes);
+     calendar.set(Calendar.SECOND, seconds);
+     Timestamp ts = new Timestamp(calendar.getTimeInMillis());
+     ts.setNanos((int) nanos);
+     return ts;
+   }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/32fd02b7/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/vector/VectorizedParquetRecordReader.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/vector/VectorizedParquetRecordReader.java b/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/vector/VectorizedParquetRecordReader.java
index 312cdac..b6a1a7a 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/vector/VectorizedParquetRecordReader.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/vector/VectorizedParquetRecordReader.java
@@ -21,15 +21,14 @@ import org.apache.hadoop.hive.ql.io.IOConstants;
 import org.apache.hadoop.hive.ql.io.parquet.ParquetRecordReaderBase;
 import org.apache.hadoop.hive.ql.io.parquet.ProjectionPusher;
 import org.apache.hadoop.hive.ql.io.parquet.read.DataWritableReadSupport;
-import org.apache.hadoop.hive.ql.io.parquet.serde.ParquetTableUtils;
 import org.apache.hadoop.hive.serde2.ColumnProjectionUtils;
 import org.apache.hadoop.hive.serde2.SerDeStats;
 import org.apache.hadoop.hive.serde2.typeinfo.StructTypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
 import org.apache.hadoop.io.NullWritable;
-import org.apache.hadoop.mapred.FileSplit;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.RecordReader;
+import org.apache.hadoop.mapreduce.InputSplit;
 import org.apache.parquet.ParquetRuntimeException;
 import org.apache.parquet.column.ColumnDescriptor;
 import org.apache.parquet.column.page.PageReadStore;
@@ -98,15 +97,12 @@ public class VectorizedParquetRecordReader extends ParquetRecordReaderBase
 
   @VisibleForTesting
   public VectorizedParquetRecordReader(
-      ParquetInputSplit inputSplit,
-      JobConf conf) {
+    InputSplit inputSplit,
+    JobConf conf) {
     try {
       serDeStats = new SerDeStats();
       projectionPusher = new ProjectionPusher();
-      if (inputSplit != null) {
-        initialize(inputSplit, conf);
-        setTimeZoneConversion(jobConf, inputSplit.getPath());
-      }
+      initialize(inputSplit, conf);
       colsToInclude = ColumnProjectionUtils.getReadColumnIDs(conf);
       rbCtx = Utilities.getVectorizedRowBatchCtx(conf);
     } catch (Throwable e) {
@@ -121,11 +117,7 @@ public class VectorizedParquetRecordReader extends ParquetRecordReaderBase
     try {
       serDeStats = new SerDeStats();
       projectionPusher = new ProjectionPusher();
-      ParquetInputSplit inputSplit = getSplit(oldInputSplit, conf);
-      if (inputSplit != null) {
-        initialize(inputSplit, conf);
-        setTimeZoneConversion(jobConf, ((FileSplit) oldInputSplit).getPath());
-      }
+      initialize(getSplit(oldInputSplit, conf), conf);
       colsToInclude = ColumnProjectionUtils.getReadColumnIDs(conf);
       rbCtx = Utilities.getVectorizedRowBatchCtx(conf);
     } catch (Throwable e) {
@@ -135,12 +127,16 @@ public class VectorizedParquetRecordReader extends ParquetRecordReaderBase
   }
 
   public void initialize(
-      ParquetInputSplit split,
-      JobConf configuration) throws IOException, InterruptedException {
-
+    InputSplit oldSplit,
+    JobConf configuration) throws IOException, InterruptedException {
+    // the oldSplit may be null during the split phase
+    if (oldSplit == null) {
+      return;
+    }
     jobConf = configuration;
     ParquetMetadata footer;
     List<BlockMetaData> blocks;
+    ParquetInputSplit split = (ParquetInputSplit) oldSplit;
     boolean indexAccess =
       configuration.getBoolean(DataWritableReadSupport.PARQUET_COLUMN_INDEX_ACCESS, false);
     this.file = split.getPath();
@@ -291,18 +287,17 @@ public class VectorizedParquetRecordReader extends ParquetRecordReaderBase
     List<ColumnDescriptor> columns = requestedSchema.getColumns();
     List<Type> types = requestedSchema.getFields();
     columnReaders = new VectorizedColumnReader[columns.size()];
-    String timeZoneId = jobConf.get(ParquetTableUtils.PARQUET_INT96_WRITE_ZONE_PROPERTY);
 
     if (!ColumnProjectionUtils.isReadAllColumns(jobConf) && !indexColumnsWanted.isEmpty()) {
       for (int i = 0; i < types.size(); ++i) {
         columnReaders[i] =
           buildVectorizedParquetReader(columnTypesList.get(indexColumnsWanted.get(i)), types.get(i),
-            pages, requestedSchema.getColumns(), timeZoneId, 0);
+            pages, requestedSchema.getColumns(), skipTimestampConversion, 0);
       }
     } else {
       for (int i = 0; i < types.size(); ++i) {
         columnReaders[i] = buildVectorizedParquetReader(columnTypesList.get(i), types.get(i), pages,
-          requestedSchema.getColumns(), timeZoneId, 0);
+          requestedSchema.getColumns(), skipTimestampConversion, 0);
       }
     }
 
@@ -331,7 +326,7 @@ public class VectorizedParquetRecordReader extends ParquetRecordReaderBase
     Type type,
     PageReadStore pages,
     List<ColumnDescriptor> columnDescriptors,
-    String conversionTimeZone,
+    boolean skipTimestampConversion,
     int depth) throws IOException {
     List<ColumnDescriptor> descriptors =
       getAllColumnDescriptorByType(depth, type, columnDescriptors);
@@ -342,7 +337,7 @@ public class VectorizedParquetRecordReader extends ParquetRecordReaderBase
           "Failed to find related Parquet column descriptor with type " + type);
       } else {
         return new VectorizedPrimitiveColumnReader(descriptors.get(0),
-          pages.getPageReader(descriptors.get(0)), conversionTimeZone, type);
+          pages.getPageReader(descriptors.get(0)), skipTimestampConversion, type);
       }
     case STRUCT:
       StructTypeInfo structTypeInfo = (StructTypeInfo) typeInfo;
@@ -352,7 +347,7 @@ public class VectorizedParquetRecordReader extends ParquetRecordReaderBase
       for (int i = 0; i < fieldTypes.size(); i++) {
         VectorizedColumnReader r =
           buildVectorizedParquetReader(fieldTypes.get(i), types.get(i), pages, descriptors,
-              conversionTimeZone, depth + 1);
+            skipTimestampConversion, depth + 1);
         if (r != null) {
           fieldReaders.add(r);
         } else {

http://git-wip-us.apache.org/repos/asf/hive/blob/32fd02b7/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/vector/VectorizedPrimitiveColumnReader.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/vector/VectorizedPrimitiveColumnReader.java b/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/vector/VectorizedPrimitiveColumnReader.java
index c27e7d9..3d5c6e6 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/vector/VectorizedPrimitiveColumnReader.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/vector/VectorizedPrimitiveColumnReader.java
@@ -13,7 +13,6 @@
  */
 package org.apache.hadoop.hive.ql.io.parquet.vector;
 
-import com.google.common.base.Strings;
 import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.ColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.DecimalColumnVector;
@@ -46,8 +45,6 @@ import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.nio.ByteOrder;
 import java.sql.Timestamp;
-import java.util.Calendar;
-import java.util.TimeZone;
 
 import static org.apache.parquet.column.ValuesType.DEFINITION_LEVEL;
 import static org.apache.parquet.column.ValuesType.REPETITION_LEVEL;
@@ -61,7 +58,7 @@ public class VectorizedPrimitiveColumnReader implements VectorizedColumnReader {
 
   private static final Logger LOG = LoggerFactory.getLogger(VectorizedPrimitiveColumnReader.class);
 
-  private String conversionTimeZone;
+  private boolean skipTimestampConversion = false;
 
   /**
    * Total number of values read.
@@ -111,13 +108,13 @@ public class VectorizedPrimitiveColumnReader implements VectorizedColumnReader {
   public VectorizedPrimitiveColumnReader(
     ColumnDescriptor descriptor,
     PageReader pageReader,
-    String conversionTimeZone,
+    boolean skipTimestampConversion,
     Type type) throws IOException {
     this.descriptor = descriptor;
     this.type = type;
     this.pageReader = pageReader;
     this.maxDefLevel = descriptor.getMaxDefinitionLevel();
-    this.conversionTimeZone = conversionTimeZone;
+    this.skipTimestampConversion = skipTimestampConversion;
 
     DictionaryPage dictionaryPage = pageReader.readDictionaryPage();
     if (dictionaryPage != null) {
@@ -414,20 +411,13 @@ public class VectorizedPrimitiveColumnReader implements VectorizedColumnReader {
       }
       break;
     case INT96:
-      final Calendar calendar;
-      if (Strings.isNullOrEmpty(this.conversionTimeZone)) {
-        // Local time should be used if no timezone is specified
-        calendar = Calendar.getInstance();
-      } else {
-        calendar = Calendar.getInstance(TimeZone.getTimeZone(this.conversionTimeZone));
-      }
       for (int i = rowId; i < rowId + num; ++i) {
         ByteBuffer buf = dictionary.decodeToBinary((int) dictionaryIds.vector[i]).toByteBuffer();
         buf.order(ByteOrder.LITTLE_ENDIAN);
         long timeOfDayNanos = buf.getLong();
         int julianDay = buf.getInt();
         NanoTime nt = new NanoTime(julianDay, timeOfDayNanos);
-        Timestamp ts = NanoTimeUtils.getTimestamp(nt, calendar);
+        Timestamp ts = NanoTimeUtils.getTimestamp(nt, skipTimestampConversion);
         ((TimestampColumnVector) column).set(i, ts);
       }
       break;

http://git-wip-us.apache.org/repos/asf/hive/blob/32fd02b7/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/write/DataWritableWriteSupport.java
----------------------------------------------------------------------
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 71a78cf..f4621e5 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
@@ -14,7 +14,6 @@
 package org.apache.hadoop.hive.ql.io.parquet.write;
 
 import java.util.HashMap;
-import java.util.TimeZone;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.serde2.io.ParquetHiveRecord;
@@ -32,11 +31,9 @@ import org.apache.parquet.schema.MessageTypeParser;
 public class DataWritableWriteSupport extends WriteSupport<ParquetHiveRecord> {
 
   public static final String PARQUET_HIVE_SCHEMA = "parquet.hive.schema";
-  private static final String PARQUET_TIMEZONE_CONVERSION = "parquet.hive.timezone";
 
   private DataWritableWriter writer;
   private MessageType schema;
-  private TimeZone timeZone;
 
   public static void setSchema(final MessageType schema, final Configuration configuration) {
     configuration.set(PARQUET_HIVE_SCHEMA, schema.toString());
@@ -46,24 +43,15 @@ public class DataWritableWriteSupport extends WriteSupport<ParquetHiveRecord> {
     return MessageTypeParser.parseMessageType(configuration.get(PARQUET_HIVE_SCHEMA));
   }
 
-  public static void setTimeZone(final TimeZone timeZone, final Configuration configuration) {
-    configuration.set(PARQUET_TIMEZONE_CONVERSION, timeZone.getID());
-  }
-
-  public static TimeZone getTimeZone(final Configuration configuration) {
-    return TimeZone.getTimeZone(configuration.get(PARQUET_TIMEZONE_CONVERSION));
-  }
-
   @Override
   public WriteContext init(final Configuration configuration) {
     schema = getSchema(configuration);
-    timeZone = getTimeZone(configuration);
     return new WriteContext(schema, new HashMap<String, String>());
   }
 
   @Override
   public void prepareForWrite(final RecordConsumer recordConsumer) {
-    writer = new DataWritableWriter(recordConsumer, schema, timeZone);
+    writer = new DataWritableWriter(recordConsumer, schema);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hive/blob/32fd02b7/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/write/DataWritableWriter.java
----------------------------------------------------------------------
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 a400fa2..6b7b50a 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
@@ -49,10 +49,8 @@ import org.apache.parquet.schema.Type;
 
 import java.sql.Date;
 import java.sql.Timestamp;
-import java.util.Calendar;
 import java.util.List;
 import java.util.Map;
-import java.util.TimeZone;
 
 /**
  *
@@ -64,16 +62,14 @@ public class DataWritableWriter {
   private static final Logger LOG = LoggerFactory.getLogger(DataWritableWriter.class);
   protected final RecordConsumer recordConsumer;
   private final GroupType schema;
-  private final TimeZone timeZone;
 
   /* This writer will be created when writing the first row in order to get
   information about how to inspect the record data.  */
   private DataWriter messageWriter;
 
-  public DataWritableWriter(final RecordConsumer recordConsumer, final GroupType schema, final TimeZone timeZone) {
+  public DataWritableWriter(final RecordConsumer recordConsumer, final GroupType schema) {
     this.recordConsumer = recordConsumer;
     this.schema = schema;
-    this.timeZone = timeZone;
   }
 
   /**
@@ -492,17 +488,15 @@ public class DataWritableWriter {
 
   private class TimestampDataWriter implements DataWriter {
     private TimestampObjectInspector inspector;
-    private Calendar calendar;
 
     public TimestampDataWriter(TimestampObjectInspector inspector) {
       this.inspector = inspector;
-      this.calendar = Calendar.getInstance(timeZone);
     }
 
     @Override
     public void write(Object value) {
       Timestamp ts = inspector.getPrimitiveJavaObject(value);
-      recordConsumer.addBinary(NanoTimeUtils.getNanoTime(ts, calendar).toBinary());
+      recordConsumer.addBinary(NanoTimeUtils.getNanoTime(ts, false).toBinary());
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/32fd02b7/ql/src/test/org/apache/hadoop/hive/ql/io/parquet/TestDataWritableWriter.java
----------------------------------------------------------------------
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 74ec728..934ae9f 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
@@ -16,16 +16,13 @@ package org.apache.hadoop.hive.ql.io.parquet;
 import org.apache.hadoop.conf.Configuration;
 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.timestamp.NanoTimeUtils;
 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.TimestampWritable;
 import org.apache.hadoop.hive.serde2.typeinfo.StructTypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
@@ -48,14 +45,10 @@ import org.apache.parquet.schema.MessageType;
 import org.apache.parquet.schema.MessageTypeParser;
 
 import java.io.UnsupportedEncodingException;
-import java.sql.Timestamp;
 import java.util.ArrayList;
 import java.util.Arrays;
-import java.util.Calendar;
 import java.util.List;
 import java.util.Properties;
-import java.util.TimeZone;
-
 
 import static org.junit.Assert.*;
 import static org.mockito.Mockito.*;
@@ -111,10 +104,6 @@ public class TestDataWritableWriter {
     inOrder.verify(mockRecordConsumer).addBinary(Binary.fromString(value));
   }
 
-  private void addBinary(Binary value) {
-    inOrder.verify(mockRecordConsumer).addBinary(value);
-  }
-
   private void startGroup() {
     inOrder.verify(mockRecordConsumer).startGroup();
   }
@@ -147,10 +136,6 @@ public class TestDataWritableWriter {
     return new BooleanWritable(value);
   }
 
-  private TimestampWritable createTimestamp(Timestamp value) {
-    return new TimestampWritable(value);
-  }
-
   private BytesWritable createString(String value) throws UnsupportedEncodingException {
     return new BytesWritable(value.getBytes("UTF-8"));
   }
@@ -166,7 +151,7 @@ public class TestDataWritableWriter {
   private List<String> createHiveColumnsFrom(final String columnNamesStr) {
     List<String> columnNames;
     if (columnNamesStr.length() == 0) {
-      columnNames = new ArrayList<>();
+      columnNames = new ArrayList<String>();
     } else {
       columnNames = Arrays.asList(columnNamesStr.split(","));
     }
@@ -206,52 +191,12 @@ public class TestDataWritableWriter {
   }
 
   private void writeParquetRecord(String schema, ParquetHiveRecord record) throws SerDeException {
-    writeParquetRecord(schema, record, TimeZone.getTimeZone("GMT"));
-  }
-
-  private void writeParquetRecord(String schema, ParquetHiveRecord record, TimeZone timeZone) throws SerDeException {
     MessageType fileSchema = MessageTypeParser.parseMessageType(schema);
-    DataWritableWriter hiveParquetWriter = new DataWritableWriter(mockRecordConsumer, fileSchema, timeZone);
+    DataWritableWriter hiveParquetWriter = new DataWritableWriter(mockRecordConsumer, fileSchema);
     hiveParquetWriter.write(record);
   }
 
   @Test
-  public void testTimestampInt96() throws Exception {
-    String columnNames = "ts";
-    String columnTypes = "timestamp";
-
-    String fileSchema = "message hive_schema {\n"
-        + "  optional int96 ts;\n"
-        + "}\n";
-
-    ArrayWritable hiveRecord = createGroup(
-        createTimestamp(Timestamp.valueOf("2016-01-01 01:01:01"))
-    );
-
-    // Write record to Parquet format using CST timezone
-    writeParquetRecord(fileSchema, getParquetWritable(columnNames, columnTypes, hiveRecord), TimeZone.getTimeZone("CST"));
-
-    // Verify record was written correctly to Parquet
-    startMessage();
-      startField("ts", 0);
-        addBinary(NanoTimeUtils.getNanoTime(Timestamp.valueOf("2016-01-01 01:01:01"),
-            Calendar.getInstance(TimeZone.getTimeZone("CST"))).toBinary());
-      endField("ts", 0);
-    endMessage();
-
-    // Write record to Parquet format using PST timezone
-    writeParquetRecord(fileSchema, getParquetWritable(columnNames, columnTypes, hiveRecord), TimeZone.getTimeZone("PST"));
-
-    // Verify record was written correctly to Parquet
-    startMessage();
-      startField("ts", 0);
-        addBinary(NanoTimeUtils.getNanoTime(Timestamp.valueOf("2016-01-01 01:01:01"),
-          Calendar.getInstance(TimeZone.getTimeZone("PST"))).toBinary());
-      endField("ts", 0);
-    endMessage();
-  }
-
-  @Test
   public void testSimpleType() throws Exception {
     String columnNames = "int,double,boolean,float,string,tinyint,smallint,bigint";
     String columnTypes = "int,double,boolean,float,string,tinyint,smallint,bigint";

http://git-wip-us.apache.org/repos/asf/hive/blob/32fd02b7/ql/src/test/org/apache/hadoop/hive/ql/io/parquet/TestVectorizedColumnReader.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/io/parquet/TestVectorizedColumnReader.java b/ql/src/test/org/apache/hadoop/hive/ql/io/parquet/TestVectorizedColumnReader.java
index f4f6e88..670bfa6 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/io/parquet/TestVectorizedColumnReader.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/io/parquet/TestVectorizedColumnReader.java
@@ -106,7 +106,7 @@ public class TestVectorizedColumnReader extends VectorizedColumnReaderTestBase {
     HiveConf.setVar(conf, HiveConf.ConfVars.PLAN, "//tmp");
     initialVectorizedRowBatchCtx(conf);
     VectorizedParquetRecordReader reader =
-        new VectorizedParquetRecordReader((org.apache.hadoop.mapred.InputSplit)null, new JobConf(conf));
+        new VectorizedParquetRecordReader((InputSplit)null, new JobConf(conf));
     assertFalse(reader.next(reader.createKey(), reader.createValue()));
   }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/32fd02b7/ql/src/test/org/apache/hadoop/hive/ql/io/parquet/VectorizedColumnReaderTestBase.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/io/parquet/VectorizedColumnReaderTestBase.java b/ql/src/test/org/apache/hadoop/hive/ql/io/parquet/VectorizedColumnReaderTestBase.java
index f2d79cf..f537cee 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/io/parquet/VectorizedColumnReaderTestBase.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/io/parquet/VectorizedColumnReaderTestBase.java
@@ -47,7 +47,6 @@ import org.apache.hadoop.mapreduce.Job;
 import org.apache.parquet.example.data.Group;
 import org.apache.parquet.example.data.simple.SimpleGroupFactory;
 import org.apache.parquet.hadoop.ParquetInputFormat;
-import org.apache.parquet.hadoop.ParquetInputSplit;
 import org.apache.parquet.hadoop.ParquetWriter;
 import org.apache.parquet.hadoop.example.GroupReadSupport;
 import org.apache.parquet.hadoop.example.GroupWriteSupport;
@@ -223,7 +222,7 @@ public class VectorizedColumnReaderTestBase {
     Job vectorJob = new Job(conf, "read vector");
     ParquetInputFormat.setInputPaths(vectorJob, file);
     ParquetInputFormat parquetInputFormat = new ParquetInputFormat(GroupReadSupport.class);
-    ParquetInputSplit split = (ParquetInputSplit) parquetInputFormat.getSplits(vectorJob).get(0);
+    InputSplit split = (InputSplit) parquetInputFormat.getSplits(vectorJob).get(0);
     initialVectorizedRowBatchCtx(conf);
     return new VectorizedParquetRecordReader(split, new JobConf(conf));
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/32fd02b7/ql/src/test/org/apache/hadoop/hive/ql/io/parquet/convert/TestETypeConverter.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/io/parquet/convert/TestETypeConverter.java b/ql/src/test/org/apache/hadoop/hive/ql/io/parquet/convert/TestETypeConverter.java
deleted file mode 100644
index 2344d63..0000000
--- a/ql/src/test/org/apache/hadoop/hive/ql/io/parquet/convert/TestETypeConverter.java
+++ /dev/null
@@ -1,124 +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
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hive.ql.io.parquet.convert;
-
-import org.apache.hadoop.hive.ql.io.parquet.serde.ParquetTableUtils;
-import org.apache.hadoop.hive.ql.io.parquet.timestamp.NanoTimeUtils;
-import org.apache.hadoop.hive.serde2.io.TimestampWritable;
-import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
-import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
-import org.apache.hadoop.io.Writable;
-import org.apache.parquet.io.api.PrimitiveConverter;
-import org.apache.parquet.schema.PrimitiveType;
-import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName;
-import org.apache.parquet.schema.Type;
-import org.junit.Before;
-import org.junit.Test;
-
-import java.sql.Timestamp;
-import java.util.Calendar;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.TimeZone;
-
-import static org.junit.Assert.assertEquals;
-
-public class TestETypeConverter {
-
-  private ConverterParentHelper parent;
-  private Timestamp ts;
-
-  @Before
-  public void init() {
-    parent = new ConverterParentHelper();
-    ts = Timestamp.valueOf("2011-01-01 01:01:01.111111111");
-  }
-  /**
-   * This class helps to compare a Writable value pushed to the ConverterParent class.
-   */
-  private class ConverterParentHelper implements ConverterParent {
-    private Writable value;
-    private Map<String, String> metadata = new HashMap<>();
-
-    /**
-     * The set() method is called from within addXXXX() PrimitiveConverter methods.
-     */
-    @Override
-    public void set(int index, Writable value) {
-      this.value = value;
-    }
-
-    @Override
-    public Map<String, String> getMetadata() {
-      return metadata;
-    }
-
-    public void assertWritableValue(Writable expected) {
-      assertEquals(expected.getClass(), value.getClass());
-      assertEquals("Writable value set to Parent is different than expected", expected, value);
-    }
-  }
-
-  private PrimitiveConverter getETypeConverter(ConverterParent parent, PrimitiveTypeName typeName, TypeInfo type) {
-    return ETypeConverter.getNewConverter(new PrimitiveType(Type.Repetition.REQUIRED, typeName, "field"), 0, parent, type);
-  }
-
-  @Test
-  public void testTimestampInt96ConverterLocal() {
-   PrimitiveConverter converter;
-
-    // Default timezone should be Localtime
-    converter = getETypeConverter(parent, PrimitiveTypeName.INT96, TypeInfoFactory.timestampTypeInfo);
-    converter.addBinary(NanoTimeUtils.getNanoTime(ts, Calendar.getInstance()).toBinary());
-    parent.assertWritableValue(new TimestampWritable(ts));
-  }
-
-  @Test
-  public void testTimestampInt96ConverterGMT() {
-    PrimitiveConverter converter;
-
-    parent.metadata.put(ParquetTableUtils.PARQUET_INT96_WRITE_ZONE_PROPERTY, "GMT");
-    converter = getETypeConverter(parent, PrimitiveTypeName.INT96, TypeInfoFactory.timestampTypeInfo);
-    converter.addBinary(NanoTimeUtils.getNanoTime(ts,
-        Calendar.getInstance(TimeZone.getTimeZone("GMT"))).toBinary());
-    parent.assertWritableValue(new TimestampWritable(ts));
-
-  }
-
-  @Test
-  public void testTimestampInt96ConverterChicago() {
-    PrimitiveConverter converter;
-
-    parent.metadata.put(ParquetTableUtils.PARQUET_INT96_WRITE_ZONE_PROPERTY, "America/Chicago");
-    converter = getETypeConverter(parent, PrimitiveTypeName.INT96, TypeInfoFactory.timestampTypeInfo);
-    converter.addBinary(NanoTimeUtils.getNanoTime(ts,
-        Calendar.getInstance(TimeZone.getTimeZone("America/Chicago"))).toBinary());
-    parent.assertWritableValue(new TimestampWritable(ts));
-  }
-
-  @Test
-  public void testTimestampInt96ConverterEtc() {
-    PrimitiveConverter converter;
-
-    parent.metadata.put(ParquetTableUtils.PARQUET_INT96_WRITE_ZONE_PROPERTY, "Etc/GMT-12");
-    converter = getETypeConverter(parent, PrimitiveTypeName.INT96, TypeInfoFactory.timestampTypeInfo);
-    converter.addBinary(NanoTimeUtils.getNanoTime(ts,
-        Calendar.getInstance(TimeZone.getTimeZone("Etc/GMT-12"))).toBinary());
-    parent.assertWritableValue(new TimestampWritable(ts));
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/32fd02b7/ql/src/test/org/apache/hadoop/hive/ql/io/parquet/serde/TestParquetTimestampUtils.java
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..ec6def5
--- /dev/null
+++ b/ql/src/test/org/apache/hadoop/hive/ql/io/parquet/serde/TestParquetTimestampUtils.java
@@ -0,0 +1,263 @@
+/**
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.ql.io.parquet.serde;
+
+import java.sql.Timestamp;
+import java.util.Calendar;
+import java.util.Date;
+import java.util.GregorianCalendar;
+import java.util.TimeZone;
+import java.util.concurrent.TimeUnit;
+
+import junit.framework.Assert;
+import junit.framework.TestCase;
+
+import org.apache.hadoop.hive.ql.io.parquet.timestamp.NanoTime;
+import org.apache.hadoop.hive.ql.io.parquet.timestamp.NanoTimeUtils;
+
+
+
+/**
+ * Tests util-libraries used for parquet-timestamp.
+ */
+public class TestParquetTimestampUtils extends TestCase {
+
+  public void testJulianDay() {
+    //check if May 23, 1968 is Julian Day 2440000
+    Calendar cal = Calendar.getInstance();
+    cal.set(Calendar.YEAR,  1968);
+    cal.set(Calendar.MONTH, Calendar.MAY);
+    cal.set(Calendar.DAY_OF_MONTH, 23);
+    cal.set(Calendar.HOUR_OF_DAY, 0);
+    cal.setTimeZone(TimeZone.getTimeZone("GMT"));
+
+    Timestamp ts = new Timestamp(cal.getTimeInMillis());
+    NanoTime nt = NanoTimeUtils.getNanoTime(ts, false);
+    Assert.assertEquals(nt.getJulianDay(), 2440000);
+
+    Timestamp tsFetched = NanoTimeUtils.getTimestamp(nt, false);
+    Assert.assertEquals(tsFetched, ts);
+
+    //check if 30 Julian Days between Jan 1, 2005 and Jan 31, 2005.
+    Calendar cal1 = Calendar.getInstance();
+    cal1.set(Calendar.YEAR,  2005);
+    cal1.set(Calendar.MONTH, Calendar.JANUARY);
+    cal1.set(Calendar.DAY_OF_MONTH, 1);
+    cal1.set(Calendar.HOUR_OF_DAY, 0);
+    cal1.setTimeZone(TimeZone.getTimeZone("GMT"));
+
+    Timestamp ts1 = new Timestamp(cal1.getTimeInMillis());
+    NanoTime nt1 = NanoTimeUtils.getNanoTime(ts1, false);
+
+    Timestamp ts1Fetched = NanoTimeUtils.getTimestamp(nt1, false);
+    Assert.assertEquals(ts1Fetched, ts1);
+
+    Calendar cal2 = Calendar.getInstance();
+    cal2.set(Calendar.YEAR,  2005);
+    cal2.set(Calendar.MONTH, Calendar.JANUARY);
+    cal2.set(Calendar.DAY_OF_MONTH, 31);
+    cal2.set(Calendar.HOUR_OF_DAY, 0);
+    cal2.setTimeZone(TimeZone.getTimeZone("UTC"));
+
+    Timestamp ts2 = new Timestamp(cal2.getTimeInMillis());
+    NanoTime nt2 = NanoTimeUtils.getNanoTime(ts2, false);
+
+    Timestamp ts2Fetched = NanoTimeUtils.getTimestamp(nt2, false);
+    Assert.assertEquals(ts2Fetched, ts2);
+    Assert.assertEquals(nt2.getJulianDay() - nt1.getJulianDay(), 30);
+
+    //check if 1464305 Julian Days between Jan 1, 2005 BC and Jan 31, 2005.
+    cal1 = Calendar.getInstance();
+    cal1.set(Calendar.ERA,  GregorianCalendar.BC);
+    cal1.set(Calendar.YEAR,  2005);
+    cal1.set(Calendar.MONTH, Calendar.JANUARY);
+    cal1.set(Calendar.DAY_OF_MONTH, 1);
+    cal1.set(Calendar.HOUR_OF_DAY, 0);
+    cal1.setTimeZone(TimeZone.getTimeZone("GMT"));
+
+    ts1 = new Timestamp(cal1.getTimeInMillis());
+    nt1 = NanoTimeUtils.getNanoTime(ts1, false);
+
+    ts1Fetched = NanoTimeUtils.getTimestamp(nt1, false);
+    Assert.assertEquals(ts1Fetched, ts1);
+
+    cal2 = Calendar.getInstance();
+    cal2.set(Calendar.YEAR,  2005);
+    cal2.set(Calendar.MONTH, Calendar.JANUARY);
+    cal2.set(Calendar.DAY_OF_MONTH, 31);
+    cal2.set(Calendar.HOUR_OF_DAY, 0);
+    cal2.setTimeZone(TimeZone.getTimeZone("UTC"));
+
+    ts2 = new Timestamp(cal2.getTimeInMillis());
+    nt2 = NanoTimeUtils.getNanoTime(ts2, false);
+
+    ts2Fetched = NanoTimeUtils.getTimestamp(nt2, false);
+    Assert.assertEquals(ts2Fetched, ts2);
+    Assert.assertEquals(nt2.getJulianDay() - nt1.getJulianDay(), 1464305);
+}
+
+  public void testNanos() {
+    //case 1: 01:01:01.0000000001
+    Calendar cal = Calendar.getInstance();
+    cal.set(Calendar.YEAR,  1968);
+    cal.set(Calendar.MONTH, Calendar.MAY);
+    cal.set(Calendar.DAY_OF_MONTH, 23);
+    cal.set(Calendar.HOUR_OF_DAY, 1);
+    cal.set(Calendar.MINUTE, 1);
+    cal.set(Calendar.SECOND, 1);
+    cal.setTimeZone(TimeZone.getTimeZone("GMT"));
+    Timestamp ts = new Timestamp(cal.getTimeInMillis());
+    ts.setNanos(1);
+
+    //(1*60*60 + 1*60 + 1) * 10e9 + 1
+    NanoTime nt = NanoTimeUtils.getNanoTime(ts, false);
+    Assert.assertEquals(nt.getTimeOfDayNanos(), 3661000000001L);
+
+    //case 2: 23:59:59.999999999
+    cal = Calendar.getInstance();
+    cal.set(Calendar.YEAR,  1968);
+    cal.set(Calendar.MONTH, Calendar.MAY);
+    cal.set(Calendar.DAY_OF_MONTH, 23);
+    cal.set(Calendar.HOUR_OF_DAY, 23);
+    cal.set(Calendar.MINUTE, 59);
+    cal.set(Calendar.SECOND, 59);
+    cal.setTimeZone(TimeZone.getTimeZone("GMT"));
+    ts = new Timestamp(cal.getTimeInMillis());
+    ts.setNanos(999999999);
+
+    //(23*60*60 + 59*60 + 59)*10e9 + 999999999
+    nt = NanoTimeUtils.getNanoTime(ts, false);
+    Assert.assertEquals(nt.getTimeOfDayNanos(), 86399999999999L);
+
+    //case 3: verify the difference.
+    Calendar cal2 = Calendar.getInstance();
+    cal2.set(Calendar.YEAR,  1968);
+    cal2.set(Calendar.MONTH, Calendar.MAY);
+    cal2.set(Calendar.DAY_OF_MONTH, 23);
+    cal2.set(Calendar.HOUR_OF_DAY, 0);
+    cal2.set(Calendar.MINUTE, 10);
+    cal2.set(Calendar.SECOND, 0);
+    cal2.setTimeZone(TimeZone.getTimeZone("GMT"));
+    Timestamp ts2 = new Timestamp(cal2.getTimeInMillis());
+    ts2.setNanos(10);
+
+    Calendar cal1 = Calendar.getInstance();
+    cal1.set(Calendar.YEAR,  1968);
+    cal1.set(Calendar.MONTH, Calendar.MAY);
+    cal1.set(Calendar.DAY_OF_MONTH, 23);
+    cal1.set(Calendar.HOUR_OF_DAY, 0);
+    cal1.set(Calendar.MINUTE, 0);
+    cal1.set(Calendar.SECOND, 0);
+    cal1.setTimeZone(TimeZone.getTimeZone("GMT"));
+    Timestamp ts1 = new Timestamp(cal1.getTimeInMillis());
+    ts1.setNanos(1);
+
+    NanoTime n2 = NanoTimeUtils.getNanoTime(ts2, false);
+    NanoTime n1 = NanoTimeUtils.getNanoTime(ts1, false);
+
+    Assert.assertEquals(n2.getTimeOfDayNanos() - n1.getTimeOfDayNanos(), 600000000009L);
+
+    NanoTime n3 = new NanoTime(n1.getJulianDay() - 1, n1.getTimeOfDayNanos() + TimeUnit.DAYS.toNanos(1));
+    Assert.assertEquals(ts1, NanoTimeUtils.getTimestamp(n3, false));
+    n3 = new NanoTime(n1.getJulianDay() + 3, n1.getTimeOfDayNanos() - TimeUnit.DAYS.toNanos(3));
+    Assert.assertEquals(ts1, NanoTimeUtils.getTimestamp(n3, false));
+  }
+
+  public void testTimezone() {
+    Calendar cal = Calendar.getInstance();
+    cal.set(Calendar.YEAR,  1968);
+    cal.set(Calendar.MONTH, Calendar.MAY);
+    cal.set(Calendar.DAY_OF_MONTH, 23);
+    cal.set(Calendar.HOUR_OF_DAY, 17);
+    cal.set(Calendar.MINUTE, 1);
+    cal.set(Calendar.SECOND, 1);
+    cal.setTimeZone(TimeZone.getTimeZone("US/Pacific"));
+    Timestamp ts = new Timestamp(cal.getTimeInMillis());
+    ts.setNanos(1);
+
+    /**
+     * 17:00 PDT = 00:00 GMT (daylight-savings)
+     * (0*60*60 + 1*60 + 1)*10e9 + 1 = 61000000001, or
+     *
+     * 17:00 PST = 01:00 GMT (if not daylight savings)
+     * (1*60*60 + 1*60 + 1)*10e9 + 1 = 3661000000001
+     */
+    NanoTime nt = NanoTimeUtils.getNanoTime(ts, false);
+    long timeOfDayNanos = nt.getTimeOfDayNanos();
+    Assert.assertTrue(timeOfDayNanos == 61000000001L || timeOfDayNanos == 3661000000001L);
+
+    //in both cases, this will be the next day in GMT
+    Assert.assertEquals(nt.getJulianDay(), 2440001);
+  }
+
+  public void testTimezoneValues() {
+    valueTest(false);
+  }
+
+  public void testTimezonelessValues() {
+    valueTest(true);
+  }
+
+  public void testTimezoneless() {
+    Timestamp ts1 = Timestamp.valueOf("2011-01-01 00:30:30.111111111");
+    NanoTime nt1 = NanoTimeUtils.getNanoTime(ts1, true);
+    Assert.assertEquals(nt1.getJulianDay(), 2455563);
+    Assert.assertEquals(nt1.getTimeOfDayNanos(), 1830111111111L);
+    Timestamp ts1Fetched = NanoTimeUtils.getTimestamp(nt1, true);
+    Assert.assertEquals(ts1Fetched.toString(), ts1.toString());
+
+    Timestamp ts2 = Timestamp.valueOf("2011-02-02 08:30:30.222222222");
+    NanoTime nt2 = NanoTimeUtils.getNanoTime(ts2, true);
+    Assert.assertEquals(nt2.getJulianDay(), 2455595);
+    Assert.assertEquals(nt2.getTimeOfDayNanos(), 30630222222222L);
+    Timestamp ts2Fetched = NanoTimeUtils.getTimestamp(nt2, true);
+    Assert.assertEquals(ts2Fetched.toString(), ts2.toString());
+  }
+
+  private void valueTest(boolean local) {
+    //exercise a broad range of timestamps close to the present.
+    verifyTsString("2011-01-01 01:01:01.111111111", local);
+    verifyTsString("2012-02-02 02:02:02.222222222", local);
+    verifyTsString("2013-03-03 03:03:03.333333333", local);
+    verifyTsString("2014-04-04 04:04:04.444444444", local);
+    verifyTsString("2015-05-05 05:05:05.555555555", local);
+    verifyTsString("2016-06-06 06:06:06.666666666", local);
+    verifyTsString("2017-07-07 07:07:07.777777777", local);
+    verifyTsString("2018-08-08 08:08:08.888888888", local);
+    verifyTsString("2019-09-09 09:09:09.999999999", local);
+    verifyTsString("2020-10-10 10:10:10.101010101", local);
+    verifyTsString("2021-11-11 11:11:11.111111111", local);
+    verifyTsString("2022-12-12 12:12:12.121212121", local);
+    verifyTsString("2023-01-02 13:13:13.131313131", local);
+    verifyTsString("2024-02-02 14:14:14.141414141", local);
+    verifyTsString("2025-03-03 15:15:15.151515151", local);
+    verifyTsString("2026-04-04 16:16:16.161616161", local);
+    verifyTsString("2027-05-05 17:17:17.171717171", local);
+    verifyTsString("2028-06-06 18:18:18.181818181", local);
+    verifyTsString("2029-07-07 19:19:19.191919191", local);
+    verifyTsString("2030-08-08 20:20:20.202020202", local);
+    verifyTsString("2031-09-09 21:21:21.212121212", local);
+
+    //test some extreme cases.
+    verifyTsString("9999-09-09 09:09:09.999999999", local);
+    verifyTsString("0001-01-01 00:00:00.0", local);
+  }
+
+  private void verifyTsString(String tsString, boolean local) {
+    Timestamp ts = Timestamp.valueOf(tsString);
+    NanoTime nt = NanoTimeUtils.getNanoTime(ts, local);
+    Timestamp tsFetched = NanoTimeUtils.getTimestamp(nt, local);
+    Assert.assertEquals(tsString, tsFetched.toString());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/32fd02b7/ql/src/test/org/apache/hadoop/hive/ql/io/parquet/timestamp/TestNanoTimeUtils.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/io/parquet/timestamp/TestNanoTimeUtils.java b/ql/src/test/org/apache/hadoop/hive/ql/io/parquet/timestamp/TestNanoTimeUtils.java
deleted file mode 100644
index 37cf0e2..0000000
--- a/ql/src/test/org/apache/hadoop/hive/ql/io/parquet/timestamp/TestNanoTimeUtils.java
+++ /dev/null
@@ -1,233 +0,0 @@
-/**
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hive.ql.io.parquet.timestamp;
-
-import org.junit.Assert;
-import org.junit.Test;
-import java.sql.Timestamp;
-import java.text.ParseException;
-import java.text.SimpleDateFormat;
-import java.util.Calendar;
-import java.util.Date;
-import java.util.TimeZone;
-
-public class TestNanoTimeUtils {
-
-  // 3:34:10.101010101 PM on 1 January 2000:
-  public static final int JAN_1_2000 = 2451545; // according to Wikipedia
-  public static final long PM_3_34_10_101010101 =
-      ((15L*60L+34L)*60L+10L)*1000000000L + 101010101L;
-  public static final NanoTime KNOWN_TIME = new NanoTime(
-      JAN_1_2000, PM_3_34_10_101010101);
-
-  public static final long KNOWN_IN_MILLIS = 946740850101L; // currentmillis.com
-
-  public static final TimeZone UTC = TimeZone.getTimeZone("UTC");
-  public static final TimeZone PST = TimeZone.getTimeZone("PST");
-  public static final TimeZone CST = TimeZone.getTimeZone("CST");
-  public static final TimeZone PLUS_6 = TimeZone.getTimeZone("GMT+6");
-  public static final TimeZone MINUS_6 = TimeZone.getTimeZone("GMT-6");
-
-  // From Spark's NanoTime implementation
-  public static final int JULIAN_DAY_OF_EPOCH = 2440588;
-  public static final long SECONDS_PER_DAY = 60 * 60 * 24L;
-  public static final long MICROS_PER_SECOND = 1000L * 1000L;
-
-  /**
-   * Returns the number of microseconds since epoch from Julian day
-   * and nanoseconds in a day
-   *
-   * This is Spark's NanoTime implementation
-   */
-  public long fromJulianDay(int julianDay, long nanoseconds) {
-    // use Long to avoid rounding errors
-    long seconds = (((long) julianDay) - JULIAN_DAY_OF_EPOCH) * SECONDS_PER_DAY;
-    return seconds * MICROS_PER_SECOND + nanoseconds / 1000L;
-  }
-
-  /**
-   * Returns a Calendar from number of micros since epoch.
-   *
-   * This is a reliable conversion from micros since epoch to local time.
-   */
-  public Calendar toCalendar(long timestamp_us, TimeZone zone) {
-    Calendar cal = Calendar.getInstance(zone);
-    cal.setTimeInMillis(timestamp_us / 1000L);
-    return cal;
-  }
-
-  @Test
-  public void testFromJulianDay() {
-    Assert.assertEquals(KNOWN_IN_MILLIS,
-        fromJulianDay(JAN_1_2000, PM_3_34_10_101010101) / 1000L);
-  }
-
-  @Test
-  public void testKnownTimestampWithFromJulianDay() {
-    Calendar known = toCalendar(fromJulianDay(
-        JAN_1_2000, PM_3_34_10_101010101), UTC);
-    Assert.assertEquals(2000, known.get(Calendar.YEAR));
-    Assert.assertEquals(Calendar.JANUARY, known.get(Calendar.MONTH));
-    Assert.assertEquals(1, known.get(Calendar.DAY_OF_MONTH));
-    Assert.assertEquals(15, known.get(Calendar.HOUR_OF_DAY));
-    Assert.assertEquals(34, known.get(Calendar.MINUTE));
-    Assert.assertEquals(10, known.get(Calendar.SECOND));
-
-    // can't validate nanos because Calendar calculations are done in millis
-  }
-
-  @Test
-  public void testKnownTimestampWithoutConversion() {
-    // a UTC calendar will produce the same values as not converting
-    Calendar calendar = toCalendar(fromJulianDay(
-        JAN_1_2000, PM_3_34_10_101010101), UTC);
-
-    Timestamp known = NanoTimeUtils.getTimestamp(
-        KNOWN_TIME, true /* skip conversion from UTC to local */ );
-
-    Assert.assertEquals(calendar.get(Calendar.YEAR) - 1900, known.getYear());
-    Assert.assertEquals(calendar.get(Calendar.MONTH), known.getMonth());
-    Assert.assertEquals(calendar.get(Calendar.DAY_OF_MONTH), known.getDate());
-    Assert.assertEquals(calendar.get(Calendar.HOUR_OF_DAY), known.getHours());
-    Assert.assertEquals(calendar.get(Calendar.MINUTE), known.getMinutes());
-    Assert.assertEquals(calendar.get(Calendar.SECOND), known.getSeconds());
-    Assert.assertEquals(101010101, known.getNanos());
-
-    NanoTime actualJD = NanoTimeUtils.getNanoTime(known, true);
-
-    Assert.assertEquals(actualJD.getJulianDay(), JAN_1_2000);
-    Assert.assertEquals(actualJD.getTimeOfDayNanos(), PM_3_34_10_101010101);
-  }
-
-  @Test
-  public void testKnownTimestampWithConversion() {
-    // a PST calendar will produce the same values when converting to local
-    Calendar calendar = toCalendar(fromJulianDay(
-        JAN_1_2000, PM_3_34_10_101010101), PST); // CHANGE ME IF LOCAL IS NOT PST
-
-    Timestamp known = NanoTimeUtils.getTimestamp(
-        KNOWN_TIME, false /* do not skip conversion from UTC to local */ );
-
-    Assert.assertEquals(calendar.get(Calendar.YEAR) - 1900, known.getYear());
-    Assert.assertEquals(calendar.get(Calendar.MONTH), known.getMonth());
-    Assert.assertEquals(calendar.get(Calendar.DAY_OF_MONTH), known.getDate());
-    Assert.assertEquals(calendar.get(Calendar.HOUR_OF_DAY), known.getHours());
-    Assert.assertEquals(calendar.get(Calendar.MINUTE), known.getMinutes());
-    Assert.assertEquals(calendar.get(Calendar.SECOND), known.getSeconds());
-    Assert.assertEquals(101010101, known.getNanos());
-
-    NanoTime actualJD = NanoTimeUtils.getNanoTime(known, false);
-
-    Assert.assertEquals(actualJD.getJulianDay(), JAN_1_2000);
-    Assert.assertEquals(actualJD.getTimeOfDayNanos(), PM_3_34_10_101010101);
-  }
-
-  @Test
-  public void testKnownWithZoneArgumentUTC() { // EXPECTED BEHAVIOR
-    // the UTC calendar should match the alternative implementation with UTC
-    Calendar calendar = toCalendar(fromJulianDay(
-        JAN_1_2000, PM_3_34_10_101010101), UTC);
-
-    Timestamp known = NanoTimeUtils.getTimestamp(
-        KNOWN_TIME, Calendar.getInstance(UTC));
-
-    Assert.assertEquals(calendar.get(Calendar.YEAR) - 1900, known.getYear());
-    Assert.assertEquals(calendar.get(Calendar.MONTH), known.getMonth());
-    Assert.assertEquals(calendar.get(Calendar.DAY_OF_MONTH), known.getDate());
-    Assert.assertEquals(calendar.get(Calendar.HOUR_OF_DAY), known.getHours());
-    Assert.assertEquals(calendar.get(Calendar.MINUTE), known.getMinutes());
-    Assert.assertEquals(calendar.get(Calendar.SECOND), known.getSeconds());
-    Assert.assertEquals(101010101, known.getNanos());
-
-    NanoTime actualJD = NanoTimeUtils.getNanoTime(known, Calendar.getInstance(UTC));
-
-    Assert.assertEquals(actualJD.getJulianDay(), JAN_1_2000);
-    Assert.assertEquals(actualJD.getTimeOfDayNanos(), PM_3_34_10_101010101);
-  }
-
-  @Test
-  public void testKnownWithZoneArgumentGMTP6() {
-    Calendar calendar = toCalendar(fromJulianDay(
-        JAN_1_2000, PM_3_34_10_101010101), PLUS_6);
-
-    Timestamp known = NanoTimeUtils.getTimestamp(
-        KNOWN_TIME, Calendar.getInstance(PLUS_6));
-
-    Assert.assertEquals(calendar.get(Calendar.YEAR) - 1900, known.getYear());
-    Assert.assertEquals(calendar.get(Calendar.MONTH), known.getMonth());
-    Assert.assertEquals(calendar.get(Calendar.DAY_OF_MONTH), known.getDate());
-    Assert.assertEquals(calendar.get(Calendar.HOUR_OF_DAY), known.getHours());
-    Assert.assertEquals(calendar.get(Calendar.MINUTE), known.getMinutes());
-    Assert.assertEquals(calendar.get(Calendar.SECOND), known.getSeconds());
-    Assert.assertEquals(101010101, known.getNanos());
-
-    NanoTime actualJD = NanoTimeUtils.getNanoTime(known, Calendar.getInstance(PLUS_6));
-
-    Assert.assertEquals(actualJD.getJulianDay(), JAN_1_2000);
-    Assert.assertEquals(actualJD.getTimeOfDayNanos(), PM_3_34_10_101010101);
-  }
-
-  @Test
-  public void testKnownWithZoneArgumentGMTM6() {
-    Calendar calendar = toCalendar(fromJulianDay(
-        JAN_1_2000, PM_3_34_10_101010101), MINUS_6);
-
-    Timestamp known = NanoTimeUtils.getTimestamp(
-        KNOWN_TIME, Calendar.getInstance(MINUS_6));
-
-    Assert.assertEquals(calendar.get(Calendar.YEAR) - 1900, known.getYear());
-    Assert.assertEquals(calendar.get(Calendar.MONTH), known.getMonth());
-    Assert.assertEquals(calendar.get(Calendar.DAY_OF_MONTH), known.getDate());
-    Assert.assertEquals(calendar.get(Calendar.HOUR_OF_DAY), known.getHours());
-    Assert.assertEquals(calendar.get(Calendar.MINUTE), known.getMinutes());
-    Assert.assertEquals(calendar.get(Calendar.SECOND), known.getSeconds());
-    Assert.assertEquals(101010101, known.getNanos());
-
-    NanoTime actualJD = NanoTimeUtils.getNanoTime(known, Calendar.getInstance(MINUS_6));
-
-    Assert.assertEquals(actualJD.getJulianDay(), JAN_1_2000);
-    Assert.assertEquals(actualJD.getTimeOfDayNanos(), PM_3_34_10_101010101);
-  }
-
-  @Test
-  public void testCompareDeprecatedTimeStampWithNewTimeStamp() {
-    Timestamp newTsLocal = NanoTimeUtils.getTimestamp(KNOWN_TIME, Calendar.getInstance());
-    Timestamp depTsLocal = NanoTimeUtils.getTimestamp(KNOWN_TIME, false);
-
-    Assert.assertEquals(newTsLocal, depTsLocal);
-
-    Timestamp newTsUTC = NanoTimeUtils.getTimestamp(KNOWN_TIME, Calendar.getInstance(TimeZone.getTimeZone("UTC")));
-    Timestamp depTsUTC = NanoTimeUtils.getTimestamp(KNOWN_TIME, true);
-
-    Assert.assertEquals(newTsUTC, depTsUTC);
-  }
-
-  @Test
-  public void testCompareDeprecatedNanoTimeWithNewNanoTime() throws ParseException {
-    Date d = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss.SSS").parse("2001-01-01 15:34:01.101");
-    Timestamp knownTimestamp = new Timestamp(d.getTime());
-
-    NanoTime newNTLocal = NanoTimeUtils.getNanoTime(knownTimestamp, Calendar.getInstance());
-    NanoTime depNTLocal = NanoTimeUtils.getNanoTime(knownTimestamp, false);
-
-    Assert.assertEquals(newNTLocal.getJulianDay(), depNTLocal.getJulianDay());
-    Assert.assertEquals(newNTLocal.getTimeOfDayNanos(), depNTLocal.getTimeOfDayNanos());
-
-    NanoTime newNTUTC = NanoTimeUtils.getNanoTime(knownTimestamp, Calendar.getInstance(TimeZone.getTimeZone("UTC")));
-    NanoTime depNTUTC = NanoTimeUtils.getNanoTime(knownTimestamp, true);
-
-    Assert.assertEquals(newNTUTC.getJulianDay(), depNTUTC.getJulianDay());
-    Assert.assertEquals(newNTUTC.getTimeOfDayNanos(), depNTUTC.getTimeOfDayNanos());
-  }
-}
\ No newline at end of file