You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by mm...@apache.org on 2018/06/25 10:38:41 UTC

[33/33] hive git commit: Revert "HIVE-12192 : Hive should carry out timestamp computations in UTC (Jesus Camacho Rodriguez via Ashutosh Chauhan)"

Revert "HIVE-12192 : Hive should carry out timestamp computations in UTC (Jesus Camacho Rodriguez via Ashutosh Chauhan)"

This reverts commit b8fda81c44dd5950c09f277672691025af2369b0.


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

Branch: refs/heads/master
Commit: 33088de0efd40a456eb4b5a327d57620c11128ff
Parents: 2277661
Author: Matt McCline <mm...@hortonworks.com>
Authored: Mon Jun 25 05:37:43 2018 -0500
Committer: Matt McCline <mm...@hortonworks.com>
Committed: Mon Jun 25 05:37:43 2018 -0500

----------------------------------------------------------------------
 .../hive/accumulo/mr/TestHiveAccumuloTypes.java |  23 +-
 .../predicate/TestAccumuloRangeGenerator.java   |   2 +-
 .../apache/hadoop/hive/common/type/Date.java    | 181 -----
 .../hadoop/hive/common/type/Timestamp.java      | 235 ------
 .../hive/common/type/TimestampTZUtil.java       |  16 +-
 .../hadoop/hive/common/type/TimestampUtils.java | 171 -----
 .../org/apache/hadoop/hive/conf/HiveConf.java   |   2 +-
 .../org/apache/hive/common/util/DateParser.java |  24 +-
 .../org/apache/hive/common/util/DateUtils.java  |   2 -
 .../hive/common/util/TimestampParser.java       |  36 +-
 .../TestHiveDecimalOrcSerializationUtils.java   |   8 +
 .../hive/common/type/TestTimestampTZ.java       |   1 +
 .../apache/hive/common/util/TestDateParser.java |   7 +-
 .../hive/common/util/TestTimestampParser.java   |  27 +-
 .../hadoop/hive/druid/serde/DruidSerDe.java     |  77 +-
 .../hadoop/hive/druid/serde/TestDruidSerDe.java |  27 +-
 .../hadoop/hive/hbase/HBaseRowSerializer.java   |   2 +-
 .../hive/hbase/HiveHBaseInputFormatUtil.java    |   3 +-
 .../apache/hadoop/hive/hbase/LazyHBaseRow.java  |   4 +-
 .../test/results/positive/hbase_timestamp.q.out |  64 +-
 .../org/apache/hive/hcatalog/data/DataType.java |   4 +-
 .../apache/hive/hcatalog/data/HCatRecord.java   |   4 +-
 .../apache/hive/hcatalog/data/JsonSerDe.java    |   4 +-
 .../apache/hive/hcatalog/data/ReaderWriter.java |  15 +-
 .../hcatalog/data/TestDefaultHCatRecord.java    |   8 +-
 .../hive/hcatalog/data/TestJsonSerDe.java       |   6 +-
 .../hive/hcatalog/pig/HCatBaseStorer.java       |   8 +-
 .../apache/hive/hcatalog/pig/HCatLoader.java    |   1 -
 .../apache/hive/hcatalog/pig/PigHCatUtil.java   |   9 +-
 .../hcatalog/pig/AbstractHCatLoaderTest.java    |  20 +-
 .../hcatalog/pig/AbstractHCatStorerTest.java    |  16 +-
 .../vectorization/ColumnVectorGenUtil.java      |  11 +-
 .../apache/hive/jdbc/BaseJdbcWithMiniLlap.java  |   4 +-
 .../hive/jdbc/TestJdbcWithMiniLlapArrow.java    |   4 +-
 .../test/resources/testconfiguration.properties |   1 -
 .../hive/llap/io/TestChunkedInputStream.java    |   4 +-
 .../io/decode/GenericColumnVectorProducer.java  |   2 +-
 .../llap/io/decode/OrcEncodedDataConsumer.java  |   3 +-
 .../DTIColumnCompareScalar.txt                  |   4 +-
 .../DTIScalarCompareColumn.txt                  |   4 +-
 ...eColumnArithmeticIntervalYearMonthColumn.txt |  28 +-
 ...eColumnArithmeticIntervalYearMonthScalar.txt |  26 +-
 .../DateColumnArithmeticTimestampColumn.txt     |  14 +-
 .../DateColumnArithmeticTimestampScalar.txt     |  16 +-
 ...eScalarArithmeticIntervalYearMonthColumn.txt |  16 +-
 .../DateScalarArithmeticTimestampColumn.txt     |   6 +-
 .../FilterColumnBetweenDynamicValue.txt         |   2 +-
 ...terTimestampColumnCompareTimestampScalar.txt |   2 +-
 ...terTimestampScalarCompareTimestampColumn.txt |   2 +-
 ...ervalYearMonthColumnArithmeticDateColumn.txt |  28 +-
 ...ervalYearMonthColumnArithmeticDateScalar.txt |  16 +-
 ...YearMonthColumnArithmeticTimestampScalar.txt |   2 +-
 ...ervalYearMonthScalarArithmeticDateColumn.txt |  26 +-
 .../LongDoubleColumnCompareTimestampScalar.txt  |   2 +-
 .../TimestampColumnArithmeticDateColumn.txt     |  14 +-
 .../TimestampColumnArithmeticDateScalar.txt     |   6 +-
 ...TimestampColumnArithmeticTimestampScalar.txt |   2 +-
 .../TimestampColumnCompareTimestampScalar.txt   |   2 +-
 .../TimestampScalarArithmeticDateColumn.txt     |  16 +-
 ...pScalarArithmeticIntervalYearMonthColumn.txt |   2 +-
 ...TimestampScalarArithmeticTimestampColumn.txt |   2 +-
 .../TimestampScalarCompareTimestampColumn.txt   |   2 +-
 .../UDAFTemplates/VectorUDAFAvgTimestamp.txt    |   2 +-
 .../UDAFTemplates/VectorUDAFMinMaxTimestamp.txt |   2 +-
 .../hive/ql/exec/ColumnStatsUpdateTask.java     |   4 +-
 .../hadoop/hive/ql/exec/FunctionRegistry.java   |  14 +-
 .../hive/ql/exec/vector/TimestampUtils.java     |  29 +-
 .../hive/ql/exec/vector/VectorAssignRow.java    |  24 +-
 .../hive/ql/exec/vector/VectorBatchDebug.java   |   7 +-
 .../exec/vector/VectorColumnAssignFactory.java  |  19 +-
 .../ql/exec/vector/VectorDeserializeRow.java    |  14 +-
 .../hive/ql/exec/vector/VectorExtractRow.java   |  15 +-
 .../ql/exec/vector/VectorHashKeyWrapper.java    |   5 +-
 .../hive/ql/exec/vector/VectorSerializeRow.java |   6 +-
 .../ql/exec/vector/VectorizationContext.java    |  27 +-
 .../ql/exec/vector/VectorizedBatchUtil.java     |  41 +-
 .../ql/exec/vector/VectorizedRowBatchCtx.java   |  25 +-
 .../vector/expressions/CastDateToString.java    |  13 +-
 .../vector/expressions/CastDateToTimestamp.java |   4 +-
 .../exec/vector/expressions/CastLongToDate.java |   1 +
 .../vector/expressions/CastLongToTimestamp.java |   2 +
 .../CastMillisecondsLongToTimestamp.java        |   2 +
 .../vector/expressions/CastStringToDate.java    |  10 +-
 .../vector/expressions/CastTimestampToChar.java |  54 --
 .../vector/expressions/CastTimestampToDate.java |   4 +-
 .../expressions/CastTimestampToDecimal.java     |   7 +-
 .../expressions/CastTimestampToDouble.java      |   2 +
 .../expressions/CastTimestampToString.java      |  68 --
 .../expressions/CastTimestampToVarChar.java     |  55 --
 .../expressions/ConstantVectorExpression.java   |   3 +-
 .../expressions/DateColSubtractDateColumn.java  |  29 +-
 .../expressions/DateColSubtractDateScalar.java  |  22 +-
 .../DateScalarSubtractDateColumn.java           |  18 +-
 .../vector/expressions/DecimalColumnInList.java |   1 +
 .../DynamicValueVectorExpression.java           |   6 +-
 .../expressions/TimestampToStringUnaryUDF.java  |   2 +-
 .../expressions/VectorExpressionWriter.java     |   4 +-
 .../VectorExpressionWriterFactory.java          |  23 +-
 .../expressions/VectorUDFDateAddColCol.java     |  19 +-
 .../expressions/VectorUDFDateAddColScalar.java  |  11 +-
 .../expressions/VectorUDFDateAddScalarCol.java  |  19 +-
 .../expressions/VectorUDFDateDiffColCol.java    |  19 +-
 .../expressions/VectorUDFDateDiffColScalar.java |  12 +-
 .../expressions/VectorUDFDateDiffScalarCol.java |  10 +-
 .../vector/expressions/VectorUDFDateLong.java   |   8 +
 .../vector/expressions/VectorUDFDateString.java |  11 +
 .../expressions/VectorUDFDateTimestamp.java     |   9 +
 .../expressions/VectorUDFDayOfWeekString.java   |   4 +-
 .../VectorUDFTimestampFieldDate.java            |   8 +-
 .../VectorUDFTimestampFieldString.java          |   4 +-
 .../VectorUDFTimestampFieldTimestamp.java       |   4 +-
 .../expressions/VectorUDFUnixTimeStampDate.java |   6 +-
 .../VectorUDFUnixTimeStampString.java           |  12 +-
 .../ql/exec/vector/udf/VectorUDFAdaptor.java    |  20 +
 .../hadoop/hive/ql/io/BatchToRowReader.java     |  34 +-
 .../apache/hadoop/hive/ql/io/orc/OrcFile.java   |  12 -
 .../hadoop/hive/ql/io/orc/RecordReaderImpl.java |  33 +-
 .../hadoop/hive/ql/io/orc/WriterImpl.java       |  15 +-
 .../ql/io/parquet/convert/ETypeConverter.java   |  18 +-
 .../serde/primitive/ParquetStringInspector.java |   8 +-
 .../ql/io/parquet/timestamp/NanoTimeUtils.java  |   8 +-
 .../parquet/vector/ParquetDataColumnReader.java |   2 +-
 .../vector/ParquetDataColumnReaderFactory.java  |   2 +-
 .../vector/VectorizedPrimitiveColumnReader.java |   5 +-
 .../ql/io/parquet/write/DataWritableWriter.java |  12 +-
 .../formatting/MetaDataFormatUtils.java         |   4 +-
 .../hive/ql/optimizer/StatsOptimizer.java       |   4 +-
 .../calcite/translator/ExprNodeConverter.java   |   4 +-
 .../calcite/translator/RexNodeConverter.java    |  17 +-
 .../hive/ql/parse/BaseSemanticAnalyzer.java     |  20 +-
 .../hive/ql/parse/TypeCheckProcFactory.java     |   5 +-
 .../ql/stats/ColumnStatisticsObjTranslator.java |   6 +-
 .../apache/hadoop/hive/ql/udf/UDFDateFloor.java |  19 +-
 .../hadoop/hive/ql/udf/UDFDayOfMonth.java       | 136 ++--
 .../apache/hadoop/hive/ql/udf/UDFDayOfWeek.java |  29 +-
 .../hadoop/hive/ql/udf/UDFFromUnixTime.java     |   2 -
 .../org/apache/hadoop/hive/ql/udf/UDFHour.java  | 132 ++--
 .../apache/hadoop/hive/ql/udf/UDFMinute.java    | 132 ++--
 .../org/apache/hadoop/hive/ql/udf/UDFMonth.java | 135 ++--
 .../apache/hadoop/hive/ql/udf/UDFSecond.java    | 136 ++--
 .../apache/hadoop/hive/ql/udf/UDFToBoolean.java |   8 +-
 .../apache/hadoop/hive/ql/udf/UDFToByte.java    |   4 +-
 .../apache/hadoop/hive/ql/udf/UDFToDouble.java  |   4 +-
 .../apache/hadoop/hive/ql/udf/UDFToFloat.java   |   4 +-
 .../apache/hadoop/hive/ql/udf/UDFToInteger.java |   4 +-
 .../apache/hadoop/hive/ql/udf/UDFToLong.java    |   4 +-
 .../apache/hadoop/hive/ql/udf/UDFToShort.java   |   5 +-
 .../apache/hadoop/hive/ql/udf/UDFToString.java  |   8 +-
 .../hadoop/hive/ql/udf/UDFWeekOfYear.java       |  32 +-
 .../org/apache/hadoop/hive/ql/udf/UDFYear.java  | 135 ++--
 .../hadoop/hive/ql/udf/generic/BaseMaskUDF.java |   9 +-
 .../ql/udf/generic/GenericUDAFBloomFilter.java  |   9 +-
 .../ql/udf/generic/GenericUDAFComputeStats.java |  16 +-
 .../hadoop/hive/ql/udf/generic/GenericUDF.java  |  79 +-
 .../ql/udf/generic/GenericUDFAddMonths.java     |  39 +-
 .../ql/udf/generic/GenericUDFCurrentDate.java   |  15 +-
 .../udf/generic/GenericUDFCurrentTimestamp.java |  15 +-
 .../hive/ql/udf/generic/GenericUDFDate.java     |  19 +-
 .../hive/ql/udf/generic/GenericUDFDateAdd.java  |  21 +-
 .../hive/ql/udf/generic/GenericUDFDateDiff.java |  31 +-
 .../ql/udf/generic/GenericUDFDateFormat.java    |  24 +-
 .../hive/ql/udf/generic/GenericUDFDateSub.java  |   4 +-
 .../udf/generic/GenericUDFFromUtcTimestamp.java |  47 +-
 .../ql/udf/generic/GenericUDFInBloomFilter.java |   9 +-
 .../hive/ql/udf/generic/GenericUDFLastDay.java  |  24 +-
 .../hive/ql/udf/generic/GenericUDFMask.java     |  10 +-
 .../hive/ql/udf/generic/GenericUDFMaskHash.java |   3 +-
 .../ql/udf/generic/GenericUDFMonthsBetween.java |  29 +-
 .../hive/ql/udf/generic/GenericUDFNextDay.java  |  31 +-
 .../ql/udf/generic/GenericUDFOPDTIMinus.java    |  16 +-
 .../ql/udf/generic/GenericUDFOPDTIPlus.java     |  16 +-
 .../hive/ql/udf/generic/GenericUDFQuarter.java  |   8 +-
 .../hive/ql/udf/generic/GenericUDFReflect2.java |   6 +-
 .../ql/udf/generic/GenericUDFTimestamp.java     |   2 +-
 .../udf/generic/GenericUDFToUnixTimeStamp.java  |  10 +-
 .../hive/ql/udf/generic/GenericUDFTrunc.java    |  59 +-
 .../ql/udf/generic/GenericUDFUnixTimeStamp.java |   3 +-
 .../hive/ql/udf/ptf/ValueBoundaryScanner.java   |  11 +-
 .../hadoop/hive/ql/util/DateTimeMath.java       | 304 ++------
 .../hive/ql/exec/TestFunctionRegistry.java      |  16 +-
 .../hadoop/hive/ql/exec/TestUtilities.java      |   5 +-
 .../ql/exec/persistence/TestRowContainer.java   |   6 +-
 .../TestTimestampWritableAndColumnVector.java   |   4 +-
 .../exec/vector/TestVectorGroupByOperator.java  |  12 +-
 .../exec/vector/TestVectorizationContext.java   |   2 +-
 .../ql/exec/vector/VectorRandomRowSource.java   |  15 +-
 .../hive/ql/exec/vector/VectorVerifyFast.java   |  20 +-
 .../expressions/TestVectorDateExpressions.java  |  58 +-
 .../TestVectorExpressionWriters.java            |  16 +-
 .../TestVectorGenericDateExpressions.java       |  17 +-
 .../expressions/TestVectorMathFunctions.java    |  10 +-
 .../TestVectorTimestampExpressions.java         |  96 +--
 .../expressions/TestVectorTimestampExtract.java |  76 +-
 .../vector/expressions/TestVectorTypeCasts.java |  23 +-
 .../exec/vector/mapjoin/fast/VerifyFastRow.java |  21 +-
 .../vector/util/VectorizedRowGroupGenUtil.java  |   6 +-
 .../batchgen/VectorColumnGroupGenerator.java    |   6 +-
 .../io/arrow/TestArrowColumnarBatchSerDe.java   |  26 +-
 .../hive/ql/io/orc/TestInputOutputFormat.java   |   8 +-
 .../hadoop/hive/ql/io/orc/TestOrcFile.java      |  37 +-
 .../hive/ql/io/orc/TestOrcSerDeStats.java       |  16 +-
 .../hive/ql/io/orc/TestVectorizedORCReader.java |  29 +-
 .../parquet/VectorizedColumnReaderTestBase.java |  23 +-
 .../serde/TestParquetTimestampUtils.java        |  42 +-
 .../hive/ql/parse/TestSemanticAnalyzer.java     |   6 +-
 .../ql/udf/TestUDFDateFormatGranularity.java    |  39 +-
 .../ql/udf/generic/TestGenericUDFAddMonths.java |  52 +-
 .../hive/ql/udf/generic/TestGenericUDFDate.java |  38 +-
 .../ql/udf/generic/TestGenericUDFDateAdd.java   |  42 +-
 .../ql/udf/generic/TestGenericUDFDateDiff.java  |  26 +-
 .../udf/generic/TestGenericUDFDateFormat.java   |  44 +-
 .../ql/udf/generic/TestGenericUDFDateSub.java   |  42 +-
 .../generic/TestGenericUDFFromUtcTimestamp.java |   7 +-
 .../ql/udf/generic/TestGenericUDFGreatest.java  |  11 +-
 .../ql/udf/generic/TestGenericUDFLastDay.java   |  63 +-
 .../ql/udf/generic/TestGenericUDFLeast.java     |  11 +-
 .../generic/TestGenericUDFMonthsBetween.java    |  41 +-
 .../ql/udf/generic/TestGenericUDFNextDay.java   |  32 +-
 .../ql/udf/generic/TestGenericUDFNullif.java    |   8 +-
 .../ql/udf/generic/TestGenericUDFOPMinus.java   |  33 +-
 .../ql/udf/generic/TestGenericUDFOPPlus.java    |  57 +-
 .../ql/udf/generic/TestGenericUDFQuarter.java   |  39 +-
 .../ql/udf/generic/TestGenericUDFSortArray.java |  12 +-
 .../generic/TestGenericUDFToUnixTimestamp.java  |  25 +-
 .../ql/udf/generic/TestGenericUDFTrunc.java     | 171 ++---
 .../hadoop/hive/ql/util/TestDateTimeMath.java   |  26 +-
 .../test/queries/clientnegative/date_literal3.q |   3 +
 ql/src/test/queries/clientpositive/date_udf.q   |   2 +-
 .../queries/clientpositive/druid_timestamptz2.q |  60 --
 .../test/queries/clientpositive/localtimezone.q |  42 +-
 .../queries/clientpositive/localtimezone2.q     |  55 --
 .../queries/clientpositive/parquet_ppd_char.q   |   1 -
 .../clientpositive/parquet_vectorization_13.q   |  16 +-
 .../clientpositive/parquet_vectorization_7.q    |  16 +-
 .../queries/clientpositive/singletsinsertorc.q  |   6 -
 .../clientpositive/timestamp_comparison2.q      |   4 +-
 .../test/queries/clientpositive/timestamp_dst.q |   2 -
 .../test/queries/clientpositive/udf_reflect2.q  |   4 +-
 .../queries/clientpositive/vectorization_13.q   |  16 +-
 .../queries/clientpositive/vectorization_7.q    |  16 +-
 .../clientpositive/vectorization_decimal_date.q |   5 +-
 .../results/clientnegative/date_literal3.q.out  |   1 +
 .../beeline/udf_unix_timestamp.q.out            |   6 +-
 .../clientpositive/cast_on_constant.q.out       |   8 +-
 .../confirm_initial_tbl_stats.q.out             |   6 +-
 .../results/clientpositive/constprog_type.q.out |   4 +-
 .../test/results/clientpositive/date_udf.q.out  |   6 +-
 .../test/results/clientpositive/decimal_1.q.out |   2 +-
 .../test/results/clientpositive/decimal_2.q.out |   2 +-
 .../druid/druid_timestamptz2.q.out              | 308 --------
 ql/src/test/results/clientpositive/foldts.q.out |   2 +-
 .../clientpositive/infer_join_preds.q.out       |   2 +-
 .../results/clientpositive/interval_alt.q.out   |   2 +-
 .../clientpositive/interval_arithmetic.q.out    |   8 +-
 .../llap/acid_vectorization_original.q.out      |  14 +-
 .../llap/current_date_timestamp.q.out           |  34 +-
 .../llap/default_constraint.q.out               |   2 +-
 .../insert_values_orig_table_use_metadata.q.out |  24 +-
 .../clientpositive/llap/llap_uncompressed.q.out |   2 +-
 .../clientpositive/llap/orc_analyze.q.out       |  34 +-
 .../results/clientpositive/llap/orc_llap.q.out  |   4 +-
 .../clientpositive/llap/orc_merge11.q.out       |  44 +-
 .../clientpositive/llap/orc_merge5.q.out        |   4 +-
 .../clientpositive/llap/orc_merge6.q.out        |   8 +-
 .../clientpositive/llap/orc_merge7.q.out        |   4 +-
 .../clientpositive/llap/orc_ppd_date.q.out      |  50 +-
 .../clientpositive/llap/orc_ppd_decimal.q.out   | 100 +--
 .../clientpositive/llap/orc_ppd_timestamp.q.out |  48 +-
 .../clientpositive/llap/orc_ppd_varchar.q.out   |  32 +-
 .../clientpositive/llap/results_cache_2.q.out   |   6 +-
 .../llap/schema_evol_orc_acid_part.q.out        |   8 +-
 .../schema_evol_orc_acid_part_llap_io.q.out     |   8 +-
 .../llap/schema_evol_orc_acid_table.q.out       |   8 +-
 .../schema_evol_orc_acid_table_llap_io.q.out    |   8 +-
 .../schema_evol_orc_acidvec_part_llap_io.q.out  |   8 +-
 .../llap/schema_evol_orc_acidvec_table.q.out    |   8 +-
 .../schema_evol_orc_acidvec_table_llap_io.q.out |   8 +-
 .../llap/schema_evol_orc_nonvec_part.q.out      |   8 +-
 ...chema_evol_orc_nonvec_part_all_complex.q.out |  16 +-
 ...ol_orc_nonvec_part_all_complex_llap_io.q.out |  16 +-
 ...ema_evol_orc_nonvec_part_all_primitive.q.out |  30 +-
 ..._orc_nonvec_part_all_primitive_llap_io.q.out |  30 +-
 .../schema_evol_orc_nonvec_part_llap_io.q.out   |   8 +-
 .../llap/schema_evol_orc_nonvec_table.q.out     |   8 +-
 .../schema_evol_orc_nonvec_table_llap_io.q.out  |   8 +-
 .../llap/schema_evol_orc_vec_part.q.out         |   8 +-
 .../schema_evol_orc_vec_part_all_complex.q.out  |  16 +-
 ..._evol_orc_vec_part_all_complex_llap_io.q.out |  16 +-
 ...schema_evol_orc_vec_part_all_primitive.q.out |  30 +-
 ...vol_orc_vec_part_all_primitive_llap_io.q.out |  30 +-
 .../llap/schema_evol_orc_vec_table.q.out        |   8 +-
 .../schema_evol_orc_vec_table_llap_io.q.out     |   8 +-
 ...ma_evol_text_nonvec_part_all_primitive.q.out |  24 +-
 ...text_nonvec_part_all_primitive_llap_io.q.out |  24 +-
 ...chema_evol_text_vec_part_all_primitive.q.out |  24 +-
 ...ol_text_vec_part_all_primitive_llap_io.q.out |  24 +-
 ...ma_evol_text_vecrow_part_all_primitive.q.out |  24 +-
 .../clientpositive/llap/singletsinsertorc.q.out |  28 -
 .../clientpositive/llap/update_all_types.q.out  |   4 +-
 .../llap/vector_aggregate_9.q.out               |   2 +-
 .../clientpositive/llap/vector_between_in.q.out |  88 +--
 .../llap/vector_binary_join_groupby.q.out       |   2 +-
 .../llap/vector_case_when_2.q.out               | 118 +--
 .../clientpositive/llap/vector_data_types.q.out |   4 +-
 .../clientpositive/llap/vector_decimal_1.q.out  |   2 +-
 .../clientpositive/llap/vector_decimal_2.q.out  |   6 +-
 .../llap/vector_decimal_cast.q.out              |  40 +-
 .../llap/vector_decimal_expressions.q.out       |  40 +-
 .../clientpositive/llap/vector_interval_1.q.out |  10 +-
 .../clientpositive/llap/vector_interval_2.q.out |  12 +-
 .../llap/vector_interval_arithmetic.q.out       | 272 +++----
 .../llap/vector_partitioned_date_time.q.out     |  56 +-
 .../llap/vector_ptf_part_simple.q.out           | 108 +--
 .../llap/vector_udf_adaptor_1.q.out             |  20 +-
 .../clientpositive/llap/vectorization_13.q.out  |  40 +-
 .../clientpositive/llap/vectorization_7.q.out   |  40 +-
 .../llap/vectorization_decimal_date.q.out       |  34 +-
 .../llap/vectorization_short_regress.q.out      |   8 +-
 .../clientpositive/llap/vectorized_casts.q.out  |  54 +-
 .../llap/vectorized_date_funcs.q.out            | 554 +++++++-------
 .../llap/vectorized_timestamp.q.out             |  12 +-
 .../llap/vectorized_timestamp_funcs.q.out       |  58 +-
 .../llap/vectorized_timestamp_ints_casts.q.out  | 104 +--
 .../clientpositive/llap_uncompressed.q.out      |   2 +-
 .../results/clientpositive/localtimezone.q.out  |  84 +--
 .../results/clientpositive/localtimezone2.q.out | 148 ----
 .../results/clientpositive/orc_file_dump.q.out  |  12 +-
 .../results/clientpositive/orc_merge11.q.out    |  44 +-
 .../results/clientpositive/orc_merge12.q.out    | 644 ++++++++--------
 .../results/clientpositive/orc_merge5.q.out     |  24 +-
 .../results/clientpositive/orc_merge6.q.out     |  48 +-
 .../clientpositive/orc_merge_incompat1.q.out    |  10 +-
 .../clientpositive/orc_merge_incompat2.q.out    |  18 +-
 .../results/clientpositive/orc_ppd_char.q.out   |  36 +-
 .../parquet_vectorization_13.q.out              |  40 +-
 .../parquet_vectorization_7.q.out               |  40 +-
 .../parquet_vectorization_decimal_date.q.out    |  20 +-
 .../clientpositive/partition_timestamp.q.out    |  60 +-
 .../clientpositive/partition_timestamp2.q.out   | 250 +++----
 .../clientpositive/perf/spark/query12.q.out     |   2 +-
 .../clientpositive/perf/spark/query16.q.out     |   2 +-
 .../clientpositive/perf/spark/query20.q.out     |   2 +-
 .../clientpositive/perf/spark/query21.q.out     |   2 +-
 .../clientpositive/perf/spark/query32.q.out     |   4 +-
 .../clientpositive/perf/spark/query37.q.out     |   2 +-
 .../clientpositive/perf/spark/query40.q.out     |   2 +-
 .../clientpositive/perf/spark/query5.q.out      |   4 +-
 .../clientpositive/perf/spark/query77.q.out     |  10 +-
 .../clientpositive/perf/spark/query80.q.out     |   6 +-
 .../clientpositive/perf/spark/query82.q.out     |   2 +-
 .../clientpositive/perf/spark/query92.q.out     |   4 +-
 .../clientpositive/perf/spark/query94.q.out     |   2 +-
 .../clientpositive/perf/spark/query95.q.out     |   2 +-
 .../clientpositive/perf/spark/query98.q.out     |   2 +-
 .../clientpositive/perf/tez/query12.q.out       |   2 +-
 .../clientpositive/perf/tez/query16.q.out       |   2 +-
 .../clientpositive/perf/tez/query20.q.out       |   2 +-
 .../clientpositive/perf/tez/query21.q.out       |   2 +-
 .../clientpositive/perf/tez/query32.q.out       |   2 +-
 .../clientpositive/perf/tez/query37.q.out       |   2 +-
 .../clientpositive/perf/tez/query40.q.out       |   2 +-
 .../clientpositive/perf/tez/query5.q.out        |   2 +-
 .../clientpositive/perf/tez/query77.q.out       |   2 +-
 .../clientpositive/perf/tez/query80.q.out       |   2 +-
 .../clientpositive/perf/tez/query82.q.out       |   2 +-
 .../clientpositive/perf/tez/query92.q.out       |   2 +-
 .../clientpositive/perf/tez/query94.q.out       |   2 +-
 .../clientpositive/perf/tez/query95.q.out       |   2 +-
 .../clientpositive/perf/tez/query98.q.out       |   2 +-
 .../clientpositive/singletsinsertorc.q.out      |  28 -
 .../results/clientpositive/spark/date_udf.q.out |   6 +-
 .../clientpositive/spark/orc_merge5.q.out       |  32 +-
 .../clientpositive/spark/orc_merge6.q.out       |  48 +-
 .../clientpositive/spark/orc_merge7.q.out       |  34 +-
 .../clientpositive/spark/orc_merge8.q.out       |  10 +-
 .../clientpositive/spark/orc_merge9.q.out       |  16 +-
 .../spark/orc_merge_incompat1.q.out             |  22 +-
 .../spark/orc_merge_incompat2.q.out             |  34 +-
 .../spark/parquet_vectorization_13.q.out        |  40 +-
 .../spark/parquet_vectorization_7.q.out         |  40 +-
 .../parquet_vectorization_decimal_date.q.out    |  20 +-
 .../clientpositive/spark/timestamp_1.q.out      |  48 +-
 .../clientpositive/spark/timestamp_2.q.out      |  48 +-
 .../clientpositive/spark/timestamp_3.q.out      |   4 +-
 .../clientpositive/spark/timestamp_udf.q.out    |   4 +-
 .../spark/vector_between_in.q.out               |  88 +--
 .../spark/vector_data_types.q.out               |   4 +-
 .../clientpositive/spark/vectorization_13.q.out |  40 +-
 .../spark/vectorization_decimal_date.q.out      |  34 +-
 .../spark/vectorization_short_regress.q.out     |  28 +-
 .../spark/vectorized_timestamp_funcs.q.out      |  58 +-
 .../tez/acid_vectorization_original.q.out       | 740 -------------------
 .../tez/acid_vectorization_original_tez.q.out   |  38 +-
 .../clientpositive/tez/orc_merge12.q.out        | 644 ++++++++--------
 .../test/results/clientpositive/timestamp.q.out |   6 +-
 .../results/clientpositive/timestamp_1.q.out    |  48 +-
 .../results/clientpositive/timestamp_2.q.out    |  48 +-
 .../results/clientpositive/timestamp_3.q.out    |   4 +-
 .../clientpositive/timestamp_comparison2.q.out  |  12 +-
 .../results/clientpositive/timestamp_dst.q.out  |   9 -
 .../clientpositive/timestamp_formats.q.out      |  84 +--
 .../clientpositive/timestamp_ints_casts.q.out   | 104 +--
 .../clientpositive/timestamp_literal.q.out      |   2 +-
 .../results/clientpositive/timestamp_udf.q.out  |   4 +-
 .../results/clientpositive/timestamptz_3.q.out  |   4 +-
 .../results/clientpositive/typechangetest.q.out |  36 +-
 ql/src/test/results/clientpositive/udf5.q.out   |   4 +-
 .../clientpositive/udf_folder_constants.q.out   |   4 +-
 .../clientpositive/udf_from_utc_timestamp.q.out |   4 +-
 .../test/results/clientpositive/udf_mask.q.out  |   2 +-
 .../clientpositive/udf_mask_first_n.q.out       |   2 +-
 .../clientpositive/udf_mask_last_n.q.out        |   2 +-
 .../clientpositive/udf_mask_show_first_n.q.out  |   2 +-
 .../clientpositive/udf_mask_show_last_n.q.out   |   2 +-
 .../results/clientpositive/udf_reflect2.q.out   |  20 +-
 .../clientpositive/udf_to_unix_timestamp.q.out  |   6 +-
 .../clientpositive/udf_to_utc_timestamp.q.out   |   2 +-
 .../clientpositive/udf_unix_timestamp.q.out     |   6 +-
 .../clientpositive/update_all_types.q.out       |   4 +-
 .../clientpositive/vector_aggregate_9.q.out     |   2 +-
 .../vector_binary_join_groupby.q.out            |   2 +-
 .../clientpositive/vector_case_when_2.q.out     | 118 +--
 .../clientpositive/vector_data_types.q.out      |   4 +-
 .../clientpositive/vector_decimal_1.q.out       |   2 +-
 .../clientpositive/vector_decimal_cast.q.out    |  40 +-
 .../vector_decimal_expressions.q.out            |  40 +-
 .../clientpositive/vector_interval_1.q.out      |  10 +-
 .../vector_interval_arithmetic.q.out            | 272 +++----
 .../clientpositive/vectorization_13.q.out       |  40 +-
 .../clientpositive/vectorization_7.q.out        |  40 +-
 .../vectorization_decimal_date.q.out            |  34 +-
 .../clientpositive/vectorized_casts.q.out       |  54 +-
 .../clientpositive/vectorized_date_funcs.q.out  | 554 +++++++-------
 .../clientpositive/vectorized_timestamp.q.out   |  12 +-
 .../vectorized_timestamp_funcs.q.out            |  56 +-
 .../vectorized_timestamp_ints_casts.q.out       | 104 +--
 .../clientpositive/windowing_distinct.q.out     |  24 +-
 .../apache/hadoop/hive/serde2/JsonSerDe.java    |   7 +-
 .../hadoop/hive/serde2/RandomTypeUtil.java      | 189 -----
 .../apache/hadoop/hive/serde2/RegexSerDe.java   |   4 +-
 .../hive/serde2/avro/AvroDeserializer.java      |  10 +-
 .../hadoop/hive/serde2/avro/AvroSerializer.java |  13 +-
 .../binarysortable/BinarySortableSerDe.java     |  18 +-
 .../fast/BinarySortableDeserializeRead.java     |   4 +-
 .../fast/BinarySortableSerializeWrite.java      |  14 +-
 .../hive/serde2/fast/DeserializeRead.java       |  12 +-
 .../hadoop/hive/serde2/fast/SerializeWrite.java |   4 +-
 .../hadoop/hive/serde2/io/DateWritableV2.java   | 154 ----
 .../serde2/io/TimestampLocalTZWritable.java     |   4 +-
 .../hive/serde2/io/TimestampWritableV2.java     | 625 ----------------
 .../hadoop/hive/serde2/lazy/LazyDate.java       |  12 +-
 .../hadoop/hive/serde2/lazy/LazyTimestamp.java  |  16 +-
 .../hadoop/hive/serde2/lazy/VerifyLazy.java     |  14 +-
 .../lazy/fast/LazySimpleDeserializeRead.java    |   2 +-
 .../lazy/fast/LazySimpleSerializeWrite.java     |  18 +-
 .../primitive/LazyDateObjectInspector.java      |   9 +-
 .../primitive/LazyTimestampObjectInspector.java |   6 +-
 .../hive/serde2/lazybinary/LazyBinaryDate.java  |   8 +-
 .../hive/serde2/lazybinary/LazyBinarySerDe.java |  10 +-
 .../serde2/lazybinary/LazyBinarySerDe2.java     |   8 +-
 .../serde2/lazybinary/LazyBinaryTimestamp.java  |   8 +-
 .../hive/serde2/lazybinary/LazyBinaryUtils.java |   4 +-
 .../fast/LazyBinaryDeserializeRead.java         |   4 +-
 .../fast/LazyBinarySerializeWrite.java          |  14 +-
 .../objectinspector/ObjectInspectorUtils.java   |  18 +-
 .../primitive/DateObjectInspector.java          |   7 +-
 .../JavaConstantDateObjectInspector.java        |   7 +-
 .../JavaConstantTimestampObjectInspector.java   |   7 +-
 .../primitive/JavaDateObjectInspector.java      |  31 +-
 .../primitive/JavaTimestampObjectInspector.java |  40 +-
 .../PrimitiveObjectInspectorConverter.java      |   8 +-
 .../PrimitiveObjectInspectorFactory.java        |   8 +-
 .../PrimitiveObjectInspectorUtils.java          |  62 +-
 .../primitive/SettableDateObjectInspector.java  |  13 +-
 .../SettableTimestampObjectInspector.java       |  13 +-
 .../primitive/TimestampObjectInspector.java     |   7 +-
 .../WritableConstantDateObjectInspector.java    |   9 +-
 ...ritableConstantTimestampObjectInspector.java |   9 +-
 .../primitive/WritableDateObjectInspector.java  |  37 +-
 .../WritableHiveCharObjectInspector.java        |  10 +-
 .../WritableHiveVarcharObjectInspector.java     |   6 +-
 .../WritableTimestampObjectInspector.java       |  38 +-
 .../hive/serde2/SerdeRandomRowSource.java       |   5 +-
 .../apache/hadoop/hive/serde2/VerifyFast.java   |  20 +-
 .../hive/serde2/binarysortable/MyTestClass.java |   6 +-
 .../binarysortable/MyTestPrimitiveClass.java    |  17 +-
 .../hadoop/hive/serde2/io/TestDateWritable.java |  80 +-
 .../hive/serde2/io/TestTimestampWritable.java   |  84 +--
 .../hive/serde2/lazy/TestLazyPrimitive.java     |   5 +-
 .../serde2/lazybinary/MyTestClassBigger.java    |   7 +-
 .../serde2/lazybinary/MyTestClassSmaller.java   |   7 +-
 .../TestStandardObjectInspectors.java           |   6 +-
 .../TestPrimitiveObjectInspectorUtils.java      |  96 +--
 .../hive/metastore/MetaStoreDirectSql.java      |   9 +-
 .../hadoop/hive/metastore/parser/Filter.g       |   2 -
 .../hive/metastore/utils/MetaStoreUtils.java    |   2 -
 .../ql/exec/vector/TimestampColumnVector.java   |   2 +-
 .../hadoop/hive/serde2/io/DateWritable.java     |   1 -
 .../ql/exec/vector/TestStructColumnVector.java  |  22 +-
 .../apache/hadoop/hive/tools/GenVectorCode.java |  14 +-
 500 files changed, 6284 insertions(+), 9702 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/33088de0/accumulo-handler/src/test/org/apache/hadoop/hive/accumulo/mr/TestHiveAccumuloTypes.java
----------------------------------------------------------------------
diff --git a/accumulo-handler/src/test/org/apache/hadoop/hive/accumulo/mr/TestHiveAccumuloTypes.java b/accumulo-handler/src/test/org/apache/hadoop/hive/accumulo/mr/TestHiveAccumuloTypes.java
index 2eeb7de..926f572 100644
--- a/accumulo-handler/src/test/org/apache/hadoop/hive/accumulo/mr/TestHiveAccumuloTypes.java
+++ b/accumulo-handler/src/test/org/apache/hadoop/hive/accumulo/mr/TestHiveAccumuloTypes.java
@@ -22,7 +22,8 @@ import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
 import java.io.DataInputStream;
 import java.io.DataOutputStream;
-import java.time.LocalDateTime;
+import java.sql.Date;
+import java.sql.Timestamp;
 import java.util.Map.Entry;
 
 import org.apache.accumulo.core.client.BatchWriter;
@@ -38,16 +39,14 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.accumulo.AccumuloHiveConstants;
 import org.apache.hadoop.hive.accumulo.AccumuloHiveRow;
 import org.apache.hadoop.hive.accumulo.serde.AccumuloSerDeParameters;
-import org.apache.hadoop.hive.common.type.Date;
 import org.apache.hadoop.hive.common.type.HiveChar;
 import org.apache.hadoop.hive.common.type.HiveDecimal;
 import org.apache.hadoop.hive.common.type.HiveVarchar;
-import org.apache.hadoop.hive.common.type.Timestamp;
 import org.apache.hadoop.hive.serde.serdeConstants;
 import org.apache.hadoop.hive.serde2.ByteStream;
-import org.apache.hadoop.hive.serde2.io.DateWritableV2;
+import org.apache.hadoop.hive.serde2.io.DateWritable;
 import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
-import org.apache.hadoop.hive.serde2.io.TimestampWritableV2;
+import org.apache.hadoop.hive.serde2.io.TimestampWritable;
 import org.apache.hadoop.hive.serde2.lazy.ByteArrayRef;
 import org.apache.hadoop.hive.serde2.lazy.LazyBoolean;
 import org.apache.hadoop.hive.serde2.lazy.LazyByte;
@@ -235,17 +234,17 @@ public class TestHiveAccumuloTypes {
 
     // date
     baos.reset();
-    Date now = Date.ofEpochMilli(System.currentTimeMillis());
-    DateWritableV2 dateWritable = new DateWritableV2(now);
+    Date now = new Date(System.currentTimeMillis());
+    DateWritable dateWritable = new DateWritable(now);
     Date dateValue = dateWritable.get();
     dateWritable.write(out);
     m.put(cfBytes, "date".getBytes(), baos.toByteArray());
 
     // tiemestamp
     baos.reset();
-    Timestamp timestampValue = Timestamp.ofEpochMilli(System.currentTimeMillis());
+    Timestamp timestampValue = new Timestamp(now.getTime());
     ByteStream.Output output = new ByteStream.Output();
-    TimestampWritableV2 timestampWritable = new TimestampWritableV2(timestampValue);
+    TimestampWritable timestampWritable = new TimestampWritable(new Timestamp(now.getTime()));
     timestampWritable.write(new DataOutputStream(output));
     output.close();
     m.put(cfBytes, "timestamp".getBytes(), output.toByteArray());
@@ -588,8 +587,8 @@ public class TestHiveAccumuloTypes {
     m.put(cfBytes, "decimal".getBytes(), baos.toByteArray());
 
     // date
-    Date now = Date.ofEpochMilli(System.currentTimeMillis());
-    DateWritableV2 dateWritable = new DateWritableV2(now);
+    Date now = new Date(System.currentTimeMillis());
+    DateWritable dateWritable = new DateWritable(now);
     Date dateValue = dateWritable.get();
     baos.reset();
     JavaDateObjectInspector dateOI = (JavaDateObjectInspector) PrimitiveObjectInspectorFactory
@@ -599,7 +598,7 @@ public class TestHiveAccumuloTypes {
     m.put(cfBytes, "date".getBytes(), baos.toByteArray());
 
     // timestamp
-    Timestamp timestampValue = Timestamp.valueOf(LocalDateTime.now().toString());
+    Timestamp timestampValue = new Timestamp(now.getTime());
     baos.reset();
     JavaTimestampObjectInspector timestampOI = (JavaTimestampObjectInspector) PrimitiveObjectInspectorFactory
         .getPrimitiveJavaObjectInspector(TypeInfoFactory

http://git-wip-us.apache.org/repos/asf/hive/blob/33088de0/accumulo-handler/src/test/org/apache/hadoop/hive/accumulo/predicate/TestAccumuloRangeGenerator.java
----------------------------------------------------------------------
diff --git a/accumulo-handler/src/test/org/apache/hadoop/hive/accumulo/predicate/TestAccumuloRangeGenerator.java b/accumulo-handler/src/test/org/apache/hadoop/hive/accumulo/predicate/TestAccumuloRangeGenerator.java
index 9df2aad..5f3baab 100644
--- a/accumulo-handler/src/test/org/apache/hadoop/hive/accumulo/predicate/TestAccumuloRangeGenerator.java
+++ b/accumulo-handler/src/test/org/apache/hadoop/hive/accumulo/predicate/TestAccumuloRangeGenerator.java
@@ -25,7 +25,6 @@ import org.apache.hadoop.hive.accumulo.TestAccumuloDefaultIndexScanner;
 import org.apache.hadoop.hive.accumulo.columns.ColumnEncoding;
 import org.apache.hadoop.hive.accumulo.columns.HiveAccumuloRowIdColumnMapping;
 import org.apache.hadoop.hive.accumulo.serde.AccumuloSerDeParameters;
-import org.apache.hadoop.hive.common.type.Date;
 import org.apache.hadoop.hive.ql.lib.DefaultGraphWalker;
 import org.apache.hadoop.hive.ql.lib.DefaultRuleDispatcher;
 import org.apache.hadoop.hive.ql.lib.Dispatcher;
@@ -53,6 +52,7 @@ import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 
+import java.sql.Date;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;

http://git-wip-us.apache.org/repos/asf/hive/blob/33088de0/common/src/java/org/apache/hadoop/hive/common/type/Date.java
----------------------------------------------------------------------
diff --git a/common/src/java/org/apache/hadoop/hive/common/type/Date.java b/common/src/java/org/apache/hadoop/hive/common/type/Date.java
deleted file mode 100644
index 6ecfcf6..0000000
--- a/common/src/java/org/apache/hadoop/hive/common/type/Date.java
+++ /dev/null
@@ -1,181 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hive.common.type;
-
-import java.time.Instant;
-import java.time.LocalDate;
-import java.time.LocalDateTime;
-import java.time.ZoneOffset;
-import java.time.format.DateTimeFormatter;
-import java.time.format.DateTimeFormatterBuilder;
-import java.time.format.DateTimeParseException;
-import java.time.format.ResolverStyle;
-import java.time.format.SignStyle;
-
-import static java.time.temporal.ChronoField.DAY_OF_MONTH;
-import static java.time.temporal.ChronoField.MONTH_OF_YEAR;
-import static java.time.temporal.ChronoField.YEAR;
-
-/**
- * This is the internal type for Date.
- * The full qualified input format of Date is "yyyy-MM-dd".
- */
-public class Date implements Comparable<Date> {
-
-  private static final LocalDate EPOCH = LocalDate.of(1970, 1, 1);
-  private static final DateTimeFormatter PARSE_FORMATTER;
-  private static final DateTimeFormatter PRINT_FORMATTER;
-  static {
-    DateTimeFormatterBuilder builder = new DateTimeFormatterBuilder();
-    builder.appendValue(YEAR, 1, 10, SignStyle.NORMAL)
-        .appendLiteral('-')
-        .appendValue(MONTH_OF_YEAR, 1, 2, SignStyle.NORMAL)
-        .appendLiteral('-')
-        .appendValue(DAY_OF_MONTH, 1, 2, SignStyle.NORMAL);
-    PARSE_FORMATTER = builder.toFormatter().withResolverStyle(ResolverStyle.LENIENT);
-    builder = new DateTimeFormatterBuilder();
-    builder.append(DateTimeFormatter.ofPattern("yyyy-MM-dd"));
-    PRINT_FORMATTER = builder.toFormatter();
-  }
-
-  private LocalDate localDate;
-
-  private Date(LocalDate localDate) {
-    this.localDate = localDate != null ? localDate : EPOCH;
-  }
-
-  public Date() {
-    this(EPOCH);
-  }
-
-  public Date(Date d) {
-    this(d.localDate);
-  }
-
-  @Override
-  public String toString() {
-    return localDate.format(PRINT_FORMATTER);
-  }
-
-  public int hashCode() {
-    return localDate.hashCode();
-  }
-
-  @Override
-  public boolean equals(Object other) {
-    if (other instanceof Date) {
-      return compareTo((Date) other) == 0;
-    }
-    return false;
-  }
-
-  @Override
-  public int compareTo(Date o) {
-    return localDate.compareTo(o.localDate);
-  }
-
-  public int toEpochDay() {
-    return (int) localDate.toEpochDay();
-  }
-
-  public long toEpochSecond() {
-    return localDate.atStartOfDay().toEpochSecond(ZoneOffset.UTC);
-  }
-
-  public long toEpochMilli() {
-    return localDate.atStartOfDay().toInstant(ZoneOffset.UTC).toEpochMilli();
-  }
-
-  public void setYear(int year) {
-    localDate = localDate.withYear(year);
-  }
-
-  public void setMonth(int month) {
-    localDate = localDate.withMonth(month);
-  }
-
-  public void setDayOfMonth(int dayOfMonth) {
-    localDate = localDate.withDayOfMonth(dayOfMonth);
-  }
-
-  public void setTimeInDays(int epochDay) {
-    localDate = LocalDate.ofEpochDay(epochDay);
-  }
-
-  public void setTimeInMillis(long epochMilli) {
-    localDate = LocalDateTime.ofInstant(
-        Instant.ofEpochMilli(epochMilli), ZoneOffset.UTC).toLocalDate();
-  }
-
-  public static Date valueOf(String s) {
-    s = s.trim();
-    int idx = s.indexOf(" ");
-    if (idx != -1) {
-      s = s.substring(0, idx);
-    }
-    LocalDate localDate;
-    try {
-      localDate = LocalDate.parse(s, PARSE_FORMATTER);
-    } catch (DateTimeParseException e) {
-      throw new IllegalArgumentException("Cannot create date, parsing error");
-    }
-    return new Date(localDate);
-  }
-
-  public static Date ofEpochDay(int epochDay) {
-    return new Date(LocalDate.ofEpochDay(epochDay));
-  }
-
-  public static Date ofEpochMilli(long epochMilli) {
-    return new Date(LocalDateTime.ofInstant(
-        Instant.ofEpochMilli(epochMilli), ZoneOffset.UTC).toLocalDate());
-  }
-
-  public static Date of(int year, int month, int dayOfMonth) {
-    return new Date(LocalDate.of(year, month, dayOfMonth));
-  }
-
-  public int getYear() {
-    return localDate.getYear();
-  }
-
-  public int getMonth() {
-    return localDate.getMonthValue();
-  }
-
-  public int getDay() {
-    return localDate.getDayOfMonth();
-  }
-
-  public int lengthOfMonth() {
-    return localDate.lengthOfMonth();
-  }
-
-  public int getDayOfWeek() {
-    return localDate.getDayOfWeek().plus(1).getValue();
-  }
-
-  /**
-   * Return a copy of this object.
-   */
-  public Object clone() {
-    // LocalDateTime is immutable.
-    return new Date(this.localDate);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/33088de0/common/src/java/org/apache/hadoop/hive/common/type/Timestamp.java
----------------------------------------------------------------------
diff --git a/common/src/java/org/apache/hadoop/hive/common/type/Timestamp.java b/common/src/java/org/apache/hadoop/hive/common/type/Timestamp.java
deleted file mode 100644
index a8b7b6d..0000000
--- a/common/src/java/org/apache/hadoop/hive/common/type/Timestamp.java
+++ /dev/null
@@ -1,235 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hive.common.type;
-
-import java.time.Instant;
-import java.time.LocalDateTime;
-import java.time.ZoneOffset;
-import java.time.format.DateTimeFormatter;
-import java.time.format.DateTimeFormatterBuilder;
-import java.time.format.DateTimeParseException;
-import java.time.format.ResolverStyle;
-import java.time.format.SignStyle;
-import java.time.temporal.ChronoField;
-
-import static java.time.temporal.ChronoField.DAY_OF_MONTH;
-import static java.time.temporal.ChronoField.HOUR_OF_DAY;
-import static java.time.temporal.ChronoField.MINUTE_OF_HOUR;
-import static java.time.temporal.ChronoField.MONTH_OF_YEAR;
-import static java.time.temporal.ChronoField.SECOND_OF_MINUTE;
-import static java.time.temporal.ChronoField.YEAR;
-
-/**
- * This is the internal type for Timestamp.
- * The full qualified input format of Timestamp is
- * "yyyy-MM-dd HH:mm:ss[.SSS...]", where the time part is optional.
- * If time part is absent, a default '00:00:00.0' will be used.
- */
-public class Timestamp implements Comparable<Timestamp> {
-  
-  private static final LocalDateTime EPOCH = LocalDateTime.of(1970, 1, 1, 0, 0, 0);
-  private static final DateTimeFormatter PARSE_FORMATTER;
-  private static final DateTimeFormatter PRINT_FORMATTER;
-
-  static {
-    DateTimeFormatterBuilder builder = new DateTimeFormatterBuilder();
-    // Date part
-    builder.appendValue(YEAR, 1, 10, SignStyle.NORMAL)
-        .appendLiteral('-')
-        .appendValue(MONTH_OF_YEAR, 1, 2, SignStyle.NORMAL)
-        .appendLiteral('-')
-        .appendValue(DAY_OF_MONTH, 1, 2, SignStyle.NORMAL);
-    // Time part
-    builder
-        .optionalStart().appendLiteral(" ")
-        .appendValue(HOUR_OF_DAY, 1, 2, SignStyle.NORMAL)
-        .appendLiteral(':')
-        .appendValue(MINUTE_OF_HOUR, 1, 2, SignStyle.NORMAL)
-        .appendLiteral(':')
-        .appendValue(SECOND_OF_MINUTE, 1, 2, SignStyle.NORMAL)
-        .optionalStart().appendFraction(ChronoField.NANO_OF_SECOND, 1, 9, true).optionalEnd()
-        .optionalEnd();
-    PARSE_FORMATTER = builder.toFormatter().withResolverStyle(ResolverStyle.LENIENT);
-    builder = new DateTimeFormatterBuilder();
-    // Date and time parts
-    builder.append(DateTimeFormatter.ofPattern("yyyy-MM-dd HH:mm:ss"));
-    // Fractional part
-    builder.optionalStart().appendFraction(ChronoField.NANO_OF_SECOND, 0, 9, true).optionalEnd();
-    PRINT_FORMATTER = builder.toFormatter();
-  }
-
-  private LocalDateTime localDateTime;
-
-  /* Private constructor */
-  private Timestamp(LocalDateTime localDateTime) {
-    this.localDateTime = localDateTime != null ? localDateTime : EPOCH;
-  }
-
-  public Timestamp() {
-    this(EPOCH);
-  }
-
-  public Timestamp(Timestamp t) {
-    this(t.localDateTime);
-  }
-
-  public void set(Timestamp t) {
-    this.localDateTime = t != null ? t.localDateTime : EPOCH;
-  }
-
-  public String format(DateTimeFormatter formatter) {
-    return localDateTime.format(formatter);
-  }
-
-  @Override
-  public String toString() {
-    return localDateTime.format(PRINT_FORMATTER);
-  }
-
-  public int hashCode() {
-    return localDateTime.hashCode();
-  }
-
-  @Override
-  public boolean equals(Object other) {
-    if (other instanceof Timestamp) {
-      return compareTo((Timestamp) other) == 0;
-    }
-    return false;
-  }
-
-  @Override
-  public int compareTo(Timestamp o) {
-    return localDateTime.compareTo(o.localDateTime);
-  }
-
-  public long toEpochSecond() {
-    return localDateTime.toEpochSecond(ZoneOffset.UTC);
-  }
-
-  public void setTimeInSeconds(long epochSecond) {
-    setTimeInSeconds(epochSecond, 0);
-  }
-
-  public void setTimeInSeconds(long epochSecond, int nanos) {
-    localDateTime = LocalDateTime.ofEpochSecond(
-        epochSecond, nanos, ZoneOffset.UTC);
-  }
-
-  public long toEpochMilli() {
-    return localDateTime.toInstant(ZoneOffset.UTC).toEpochMilli();
-  }
-
-  public void setTimeInMillis(long epochMilli) {
-    localDateTime = LocalDateTime.ofInstant(
-        Instant.ofEpochMilli(epochMilli), ZoneOffset.UTC);
-  }
-
-  public void setTimeInMillis(long epochMilli, int nanos) {
-    localDateTime = LocalDateTime
-        .ofInstant(Instant.ofEpochMilli(epochMilli), ZoneOffset.UTC)
-        .withNano(nanos);
-  }
-
-  public int getNanos() {
-    return localDateTime.getNano();
-  }
-
-  public static Timestamp valueOf(String s) {
-    s = s.trim();
-    LocalDateTime localDateTime;
-    try {
-      localDateTime = LocalDateTime.parse(s, PARSE_FORMATTER);
-    } catch (DateTimeParseException e) {
-      // Try ISO-8601 format
-      try {
-        localDateTime = LocalDateTime.parse(s);
-      } catch (DateTimeParseException e2) {
-        throw new IllegalArgumentException("Cannot create timestamp, parsing error");
-      }
-    }
-    return new Timestamp(localDateTime);
-  }
-
-  public static Timestamp ofEpochSecond(long epochSecond) {
-    return ofEpochSecond(epochSecond, 0);
-  }
-
-  public static Timestamp ofEpochSecond(long epochSecond, int nanos) {
-    return new Timestamp(
-        LocalDateTime.ofEpochSecond(epochSecond, nanos, ZoneOffset.UTC));
-  }
-
-  public static Timestamp ofEpochMilli(long epochMilli) {
-    return new Timestamp(LocalDateTime
-        .ofInstant(Instant.ofEpochMilli(epochMilli), ZoneOffset.UTC));
-  }
-
-  public static Timestamp ofEpochMilli(long epochMilli, int nanos) {
-    return new Timestamp(LocalDateTime
-        .ofInstant(Instant.ofEpochMilli(epochMilli), ZoneOffset.UTC)
-        .withNano(nanos));
-  }
-
-  public void setNanos(int nanos) {
-    localDateTime = localDateTime.withNano(nanos);
-  }
-
-  public int getYear() {
-    return localDateTime.getYear();
-  }
-
-  public int getMonth() {
-    return localDateTime.getMonthValue();
-  }
-
-  public int getDay() {
-    return localDateTime.getDayOfMonth();
-  }
-
-  public int getHours() {
-    return localDateTime.getHour();
-  }
-
-  public int getMinutes() {
-    return localDateTime.getMinute();
-  }
-
-  public int getSeconds() {
-    return localDateTime.getSecond();
-  }
-
-  public int getDayOfWeek() {
-    return localDateTime.getDayOfWeek().plus(1).getValue();
-  }
-
-  /**
-   * Return a copy of this object.
-   */
-  public Object clone() {
-    // LocalDateTime is immutable.
-    return new Timestamp(this.localDateTime);
-  }
-
-  public java.sql.Timestamp toSqlTimestamp() {
-    java.sql.Timestamp ts = new java.sql.Timestamp(toEpochMilli());
-    ts.setNanos(getNanos());
-    return ts;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/33088de0/common/src/java/org/apache/hadoop/hive/common/type/TimestampTZUtil.java
----------------------------------------------------------------------
diff --git a/common/src/java/org/apache/hadoop/hive/common/type/TimestampTZUtil.java b/common/src/java/org/apache/hadoop/hive/common/type/TimestampTZUtil.java
index 213650c..90ffddb 100644
--- a/common/src/java/org/apache/hadoop/hive/common/type/TimestampTZUtil.java
+++ b/common/src/java/org/apache/hadoop/hive/common/type/TimestampTZUtil.java
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.hive.common.type;
 
+import java.sql.Timestamp;
 import java.text.DateFormat;
 import java.text.SimpleDateFormat;
 import java.time.DateTimeException;
@@ -30,6 +31,7 @@ import java.time.format.DateTimeParseException;
 import java.time.format.TextStyle;
 import java.time.temporal.ChronoField;
 import java.time.temporal.TemporalAccessor;
+import java.util.Date;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
@@ -43,6 +45,9 @@ public class TimestampTZUtil {
   private static final LocalTime DEFAULT_LOCAL_TIME = LocalTime.of(0, 0);
   private static final Pattern SINGLE_DIGIT_PATTERN = Pattern.compile("[\\+-]\\d:\\d\\d");
 
+  private static final ThreadLocal<DateFormat> CONVERT_FORMATTER =
+      ThreadLocal.withInitial(() -> new SimpleDateFormat("yyyy-MM-dd HH:mm:ss"));
+
   static final DateTimeFormatter FORMATTER;
   static {
     DateTimeFormatterBuilder builder = new DateTimeFormatterBuilder();
@@ -121,14 +126,11 @@ public class TimestampTZUtil {
     }
   }
 
-  // Converts Date to TimestampTZ.
+  // Converts Date to TimestampTZ. The conversion is done text-wise since
+  // Date/Timestamp should be treated as description of date/time.
   public static TimestampTZ convert(Date date, ZoneId defaultTimeZone) {
-    return parse(date.toString(), defaultTimeZone);
-  }
-
-  // Converts Timestamp to TimestampTZ.
-  public static TimestampTZ convert(Timestamp ts, ZoneId defaultTimeZone) {
-    return parse(ts.toString(), defaultTimeZone);
+    String s = date instanceof Timestamp ? date.toString() : CONVERT_FORMATTER.get().format(date);
+    return parse(s, defaultTimeZone);
   }
 
   public static ZoneId parseTimeZone(String timeZoneStr) {

http://git-wip-us.apache.org/repos/asf/hive/blob/33088de0/common/src/java/org/apache/hadoop/hive/common/type/TimestampUtils.java
----------------------------------------------------------------------
diff --git a/common/src/java/org/apache/hadoop/hive/common/type/TimestampUtils.java b/common/src/java/org/apache/hadoop/hive/common/type/TimestampUtils.java
deleted file mode 100644
index ab60db1..0000000
--- a/common/src/java/org/apache/hadoop/hive/common/type/TimestampUtils.java
+++ /dev/null
@@ -1,171 +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.common.type;
-
-import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
-
-import java.math.BigDecimal;
-import java.time.DateTimeException;
-
-/**
- * Utilities for Timestamps and the relevant conversions.
- */
-public class TimestampUtils {
-  public static final BigDecimal BILLION_BIG_DECIMAL = BigDecimal.valueOf(1000000000);
-
-  /**
-   * Convert the timestamp to a double measured in seconds.
-   * @return double representation of the timestamp, accurate to nanoseconds
-   */
-  public static double getDouble(Timestamp ts) {
-    long seconds = ts.toEpochSecond();
-    return seconds + ((double) ts.getNanos()) / 1000000000;
-  }
-
-  public static Timestamp doubleToTimestamp(double f) {
-    try {
-      long seconds = (long) f;
-
-      // We must ensure the exactness of the double's fractional portion.
-      // 0.6 as the fraction part will be converted to 0.59999... and
-      // significantly reduce the savings from binary serialization
-      BigDecimal bd = new BigDecimal(String.valueOf(f));
-
-      bd = bd.subtract(new BigDecimal(seconds)).multiply(new BigDecimal(1000000000));
-      int nanos = bd.intValue();
-
-      // Convert to millis
-      long millis = seconds * 1000;
-      if (nanos < 0) {
-        millis -= 1000;
-        nanos += 1000000000;
-      }
-
-      return Timestamp.ofEpochMilli(millis, nanos);
-    } catch (IllegalArgumentException | DateTimeException nfe) {
-      return null;
-    }
-  }
-
-  /**
-   * Take a HiveDecimal and return the timestamp representation where the fraction part is the
-   * nanoseconds and integer part is the number of seconds.
-   * @param dec
-   * @return
-   */
-  public static Timestamp decimalToTimestamp(HiveDecimal dec) {
-    try {
-      HiveDecimalWritable nanosWritable = new HiveDecimalWritable(dec);
-      nanosWritable.mutateFractionPortion();               // Clip off seconds portion.
-      nanosWritable.mutateScaleByPowerOfTen(9);            // Bring nanoseconds into integer portion.
-      if (!nanosWritable.isSet() || !nanosWritable.isInt()) {
-        return null;
-      }
-      int nanos = nanosWritable.intValue();
-      if (nanos < 0) {
-        nanos += 1000000000;
-      }
-      nanosWritable.setFromLong(nanos);
-
-      HiveDecimalWritable nanoInstant = new HiveDecimalWritable(dec);
-      nanoInstant.mutateScaleByPowerOfTen(9);
-
-      nanoInstant.mutateSubtract(nanosWritable);
-      nanoInstant.mutateScaleByPowerOfTen(-9);              // Back to seconds.
-      if (!nanoInstant.isSet() || !nanoInstant.isLong()) {
-        return null;
-      }
-      long millis = nanoInstant.longValue() * 1000;
-      return Timestamp.ofEpochMilli(millis, nanos);
-    } catch (IllegalArgumentException | DateTimeException nfe) {
-      // E.g. DateTimeException: Invalid value for Year (valid values -999999999 - 999999999)
-      return null;
-    }
-  }
-
-  /**
-   * Take a HiveDecimalWritable and return the timestamp representation where the fraction part
-   * is the nanoseconds and integer part is the number of seconds.
-   *
-   * This is a HiveDecimalWritable variation with supplied scratch objects.
-   * @param decWritable
-   * @param scratchDecWritable1
-   * @param scratchDecWritable2
-   * @return
-   */
-  public static Timestamp decimalToTimestamp(
-      HiveDecimalWritable decWritable,
-      HiveDecimalWritable scratchDecWritable1, HiveDecimalWritable scratchDecWritable2) {
-
-    HiveDecimalWritable nanosWritable = scratchDecWritable1;
-    nanosWritable.set(decWritable);
-    nanosWritable.mutateFractionPortion();               // Clip off seconds portion.
-    nanosWritable.mutateScaleByPowerOfTen(9);            // Bring nanoseconds into integer portion.
-    if (!nanosWritable.isSet() || !nanosWritable.isInt()) {
-      return null;
-    }
-    int nanos = nanosWritable.intValue();
-    if (nanos < 0) {
-      nanos += 1000000000;
-    }
-    nanosWritable.setFromLong(nanos);
-
-    HiveDecimalWritable nanoInstant = scratchDecWritable2;
-    nanoInstant.set(decWritable);
-    nanoInstant.mutateScaleByPowerOfTen(9);
-
-    nanoInstant.mutateSubtract(nanosWritable);
-    nanoInstant.mutateScaleByPowerOfTen(-9);              // Back to seconds.
-    if (!nanoInstant.isSet() || !nanoInstant.isLong()) {
-      return null;
-    }
-    long seconds = nanoInstant.longValue();
-
-    return Timestamp.ofEpochSecond(seconds, nanos);
-  }
-
-  public static Timestamp decimalToTimestamp(HiveDecimalV1 dec) {
-    try {
-      BigDecimal nanoInstant = dec.bigDecimalValue().multiply(BILLION_BIG_DECIMAL);
-      int nanos = nanoInstant.remainder(BILLION_BIG_DECIMAL).intValue();
-      if (nanos < 0) {
-        nanos += 1000000000;
-      }
-      long seconds =
-          nanoInstant.subtract(new BigDecimal(nanos)).divide(BILLION_BIG_DECIMAL).longValue();
-
-      return Timestamp.ofEpochSecond(seconds, nanos);
-    } catch (IllegalArgumentException | DateTimeException nfe) {
-      return null;
-    }
-  }
-
-  /**
-   * Rounds the number of milliseconds relative to the epoch down to the nearest whole number of
-   * seconds. 500 would round to 0, -500 would round to -1.
-   */
-  public static long millisToSeconds(long millis) {
-    if (millis >= 0) {
-      return millis / 1000;
-    } else {
-      return (millis - 999) / 1000;
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/33088de0/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 18f30ad..3691786 100644
--- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
+++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
@@ -1893,7 +1893,7 @@ 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.'."),
-    HIVE_PARQUET_TIMESTAMP_SKIP_CONVERSION("hive.parquet.timestamp.skip.conversion", false,
+    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"),
     HIVE_INT_TIMESTAMP_CONVERSION_IN_SECONDS("hive.int.timestamp.conversion.in.seconds", false,

http://git-wip-us.apache.org/repos/asf/hive/blob/33088de0/common/src/java/org/apache/hive/common/util/DateParser.java
----------------------------------------------------------------------
diff --git a/common/src/java/org/apache/hive/common/util/DateParser.java b/common/src/java/org/apache/hive/common/util/DateParser.java
index 5db14f1..949fdba 100644
--- a/common/src/java/org/apache/hive/common/util/DateParser.java
+++ b/common/src/java/org/apache/hive/common/util/DateParser.java
@@ -17,18 +17,24 @@
  */
 package org.apache.hive.common.util;
 
-import org.apache.hadoop.hive.common.type.Date;
+import java.sql.Date;
+import java.text.ParsePosition;
+import java.text.SimpleDateFormat;
 
 /**
  * Date parser class for Hive.
  */
 public class DateParser {
-
+  private final SimpleDateFormat formatter;
+  private final ParsePosition pos;
   public DateParser() {
- }
+    formatter = new SimpleDateFormat("yyyy-MM-dd");
+    // TODO: ideally, we should set formatter.setLenient(false);
+    pos = new ParsePosition(0);
+  }
 
   public Date parseDate(String strValue) {
-    Date result = new Date();
+    Date result = new Date(0);
     if (parseDate(strValue, result)) {
       return result;
     }
@@ -36,16 +42,12 @@ public class DateParser {
   }
 
   public boolean parseDate(String strValue, Date result) {
-    Date parsedVal;
-    try {
-      parsedVal = Date.valueOf(strValue);
-    } catch (IllegalArgumentException e) {
-      parsedVal = null;
-    }
+    pos.setIndex(0);
+    java.util.Date parsedVal = formatter.parse(strValue, pos);
     if (parsedVal == null) {
       return false;
     }
-    result.setTimeInMillis(parsedVal.toEpochMilli());
+    result.setTime(parsedVal.getTime());
     return true;
   }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/33088de0/common/src/java/org/apache/hive/common/util/DateUtils.java
----------------------------------------------------------------------
diff --git a/common/src/java/org/apache/hive/common/util/DateUtils.java b/common/src/java/org/apache/hive/common/util/DateUtils.java
index e70de28..65f3b94 100644
--- a/common/src/java/org/apache/hive/common/util/DateUtils.java
+++ b/common/src/java/org/apache/hive/common/util/DateUtils.java
@@ -20,7 +20,6 @@ package org.apache.hive.common.util;
 
 import java.math.BigDecimal;
 import java.text.SimpleDateFormat;
-import java.util.TimeZone;
 
 /**
  * DateUtils. Thread-safe class
@@ -33,7 +32,6 @@ public class DateUtils {
     protected SimpleDateFormat initialValue() {
       SimpleDateFormat simpleDateFormat = new SimpleDateFormat("yyyy-MM-dd");
       simpleDateFormat.setLenient(false);
-      simpleDateFormat.setTimeZone(TimeZone.getTimeZone("UTC"));
       return simpleDateFormat;
     }
   };

http://git-wip-us.apache.org/repos/asf/hive/blob/33088de0/common/src/java/org/apache/hive/common/util/TimestampParser.java
----------------------------------------------------------------------
diff --git a/common/src/java/org/apache/hive/common/util/TimestampParser.java b/common/src/java/org/apache/hive/common/util/TimestampParser.java
index d30ab88..f674b5d 100644
--- a/common/src/java/org/apache/hive/common/util/TimestampParser.java
+++ b/common/src/java/org/apache/hive/common/util/TimestampParser.java
@@ -18,18 +18,19 @@
 
 package org.apache.hive.common.util;
 
+import java.math.BigDecimal;
+import java.sql.Timestamp;
 import java.util.Arrays;
+import java.util.Iterator;
 import java.util.List;
 import java.util.Optional;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
-import org.apache.hadoop.hive.common.type.Timestamp;
 import org.joda.time.DateTime;
-import org.joda.time.LocalDateTime;
+import org.joda.time.IllegalInstantException;
 import org.joda.time.MutableDateTime;
 import org.joda.time.DateTimeFieldType;
-import org.joda.time.chrono.ISOChronology;
 import org.joda.time.format.DateTimeFormat;
 import org.joda.time.format.DateTimeFormatter;
 import org.joda.time.format.DateTimeFormatterBuilder;
@@ -51,8 +52,19 @@ public class TimestampParser {
 
   protected final static String[] stringArray = new String[] {};
   protected final static String millisFormatString = "millis";
-  protected final static DateTime startingDateValue =
-      new DateTime(1970, 1, 1, 0, 0, 0, 0, ISOChronology.getInstanceUTC());
+  @Nullable
+  private final static DateTime startingDateValue = makeStartingDateValue();
+
+  @Nullable
+  private static DateTime makeStartingDateValue() {
+    try {
+      return new DateTime(1970, 1, 1, 0, 0, 0, 0);
+    } catch (IllegalInstantException e) {
+      // 1970-01-01 00:00:00 did not exist in some zones. In these zones, we need to take different,
+      // less optimal parsing route.
+      return null;
+    }
+  }
 
   protected String[] formatStrings = null;
   protected DateTimeFormatter fmt = null;
@@ -114,24 +126,21 @@ public class TimestampParser {
 
     if (startingDateValue != null) {
       // reset value in case any date fields are missing from the date pattern
-      MutableDateTime mdt = new MutableDateTime(
-          startingDateValue, ISOChronology.getInstanceUTC());
+      MutableDateTime mdt = new MutableDateTime(startingDateValue);
 
       // Using parseInto() avoids throwing exception when parsing,
       // allowing fallback to default timestamp parsing if custom patterns fail.
       int ret = fmt.parseInto(mdt, strValue, 0);
       // Only accept parse results if we parsed the entire string
       if (ret == strValue.length()) {
-        return Optional.of(Timestamp.ofEpochMilli(mdt.getMillis()));
+        return Optional.of(new Timestamp(mdt.getMillis()));
       }
       return Optional.empty();
     }
 
     try {
-      LocalDateTime dt = fmt.parseLocalDateTime(strValue);
-      return Optional.of(
-          Timestamp.ofEpochMilli(
-              dt.toDateTime(ISOChronology.getInstanceUTC().getZone()).getMillis()));
+      DateTime dt = fmt.parseDateTime(strValue);
+      return Optional.of(new Timestamp(dt.getMillis()));
     } catch (IllegalArgumentException e) {
       return Optional.empty();
     }
@@ -172,8 +181,7 @@ public class TimestampParser {
 
       // Joda DateTime only has precision to millis, cut off any fractional portion
       long millis = Long.parseLong(matcher.group(1));
-      DateTime dt =
-          new DateTime(millis, ISOChronology.getInstanceUTC());
+      DateTime dt = new DateTime(millis);
       for (DateTimeFieldType field : dateTimeFields) {
         bucket.saveField(field, dt.get(field));
       }

http://git-wip-us.apache.org/repos/asf/hive/blob/33088de0/common/src/test/org/apache/hadoop/hive/common/type/TestHiveDecimalOrcSerializationUtils.java
----------------------------------------------------------------------
diff --git a/common/src/test/org/apache/hadoop/hive/common/type/TestHiveDecimalOrcSerializationUtils.java b/common/src/test/org/apache/hadoop/hive/common/type/TestHiveDecimalOrcSerializationUtils.java
index 1435339..72dce4d 100644
--- a/common/src/test/org/apache/hadoop/hive/common/type/TestHiveDecimalOrcSerializationUtils.java
+++ b/common/src/test/org/apache/hadoop/hive/common/type/TestHiveDecimalOrcSerializationUtils.java
@@ -17,18 +17,26 @@
  */
 package org.apache.hadoop.hive.common.type;
 
+import java.sql.Timestamp;
 import java.util.Random;
+import java.util.Arrays;
 import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
 import java.io.IOException;
 import java.math.BigDecimal;
 import java.math.BigInteger;
 
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritableV1;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
 import org.apache.orc.impl.SerializationUtils;
 import org.apache.hadoop.hive.common.type.RandomTypeUtil;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.StringExpr;
+import org.apache.hadoop.hive.ql.util.TimestampUtils;
 
 import com.google.code.tempusfugit.concurrency.annotations.*;
+import com.google.code.tempusfugit.concurrency.*;
 
 import org.junit.*;
 

http://git-wip-us.apache.org/repos/asf/hive/blob/33088de0/common/src/test/org/apache/hadoop/hive/common/type/TestTimestampTZ.java
----------------------------------------------------------------------
diff --git a/common/src/test/org/apache/hadoop/hive/common/type/TestTimestampTZ.java b/common/src/test/org/apache/hadoop/hive/common/type/TestTimestampTZ.java
index cd23abe..5a3f048 100644
--- a/common/src/test/org/apache/hadoop/hive/common/type/TestTimestampTZ.java
+++ b/common/src/test/org/apache/hadoop/hive/common/type/TestTimestampTZ.java
@@ -21,6 +21,7 @@ package org.apache.hadoop.hive.common.type;
 import org.junit.Assert;
 import org.junit.Test;
 
+import java.sql.Timestamp;
 import java.time.ZoneId;
 import java.time.format.DateTimeParseException;
 import java.util.TimeZone;

http://git-wip-us.apache.org/repos/asf/hive/blob/33088de0/common/src/test/org/apache/hive/common/util/TestDateParser.java
----------------------------------------------------------------------
diff --git a/common/src/test/org/apache/hive/common/util/TestDateParser.java b/common/src/test/org/apache/hive/common/util/TestDateParser.java
index 8c3a7a4..0553b3d 100644
--- a/common/src/test/org/apache/hive/common/util/TestDateParser.java
+++ b/common/src/test/org/apache/hive/common/util/TestDateParser.java
@@ -18,13 +18,13 @@
 package org.apache.hive.common.util;
 
 import static org.junit.Assert.*;
-
-import org.apache.hadoop.hive.common.type.Date;
 import org.junit.Test;
 
+import java.sql.Date;
+
 public class TestDateParser {
   DateParser parser = new DateParser();
-  Date date = new Date();
+  Date date = new Date(0);
 
   void checkValidCase(String strValue, Date expected) {
     Date dateValue = parser.parseDate(strValue);
@@ -57,6 +57,7 @@ public class TestDateParser {
     checkValidCase(" 1946-01-01", Date.valueOf("1946-01-01"));
     checkValidCase(" 2001-11-12 01:02:03", Date.valueOf("2001-11-12"));
 
+    // Current date parsing is lenient
     checkValidCase("2001-13-12", Date.valueOf("2002-01-12"));
     checkValidCase("2001-11-31", Date.valueOf("2001-12-01"));
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/33088de0/common/src/test/org/apache/hive/common/util/TestTimestampParser.java
----------------------------------------------------------------------
diff --git a/common/src/test/org/apache/hive/common/util/TestTimestampParser.java b/common/src/test/org/apache/hive/common/util/TestTimestampParser.java
index 00a7904..c982af6 100644
--- a/common/src/test/org/apache/hive/common/util/TestTimestampParser.java
+++ b/common/src/test/org/apache/hive/common/util/TestTimestampParser.java
@@ -18,10 +18,11 @@
 
 package org.apache.hive.common.util;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.fail;
+import java.sql.Timestamp;
+import java.util.Arrays;
+import java.util.List;
 
-import org.apache.hadoop.hive.common.type.Timestamp;
+import static org.junit.Assert.*;
 import org.junit.Test;
 
 public class TestTimestampParser {
@@ -64,11 +65,10 @@ public class TestTimestampParser {
             Timestamp.valueOf("1945-12-31 23:59:59.1234")),
         new ValidTimestampCase("1970-01-01 00:00:00",
             Timestamp.valueOf("1970-01-01 00:00:00")),
-        new ValidTimestampCase("1945-12-31T23:59:59",
-            Timestamp.valueOf("1945-12-31 23:59:59")),
     };
 
     String[] invalidCases = {
+        "1945-12-31T23:59:59",
         "12345",
     };
 
@@ -111,12 +111,11 @@ public class TestTimestampParser {
             Timestamp.valueOf("2001-01-01 00:00:00")),
         new ValidTimestampCase("1945-12-31 23:59:59.1234",
             Timestamp.valueOf("1945-12-31 23:59:59.1234")),
-        new ValidTimestampCase("1945-12-31T23:59:59.12345",
-            Timestamp.valueOf("1945-12-31 23:59:59.12345"))
     };
 
     String[] invalidCases = {
         "1945-12-31-23:59:59",
+        "1945-12-31T23:59:59.12345", // our pattern didn't specify 5 decimal places
         "12345",
     };
 
@@ -134,20 +133,19 @@ public class TestTimestampParser {
     TimestampParser tp = new TimestampParser(patterns);
 
     ValidTimestampCase[] validCases = {
-        new ValidTimestampCase("0", Timestamp.ofEpochMilli(0)),
-        new ValidTimestampCase("-1000000", Timestamp.ofEpochMilli(-1000000)),
-        new ValidTimestampCase("1420509274123", Timestamp.ofEpochMilli(1420509274123L)),
-        new ValidTimestampCase("1420509274123.456789", Timestamp.ofEpochMilli(1420509274123L)),
+        new ValidTimestampCase("0", new Timestamp(0)),
+        new ValidTimestampCase("-1000000", new Timestamp(-1000000)),
+        new ValidTimestampCase("1420509274123", new Timestamp(1420509274123L)),
+        new ValidTimestampCase("1420509274123.456789", new Timestamp(1420509274123L)),
 
         // Other format pattern should also work
         new ValidTimestampCase("1945-12-31T23:59:59",
             Timestamp.valueOf("1945-12-31 23:59:59")),
-        new ValidTimestampCase("1945-12-31T23:59:59.12345",
-            Timestamp.valueOf("1945-12-31 23:59:59.12345")),
     };
 
     String[] invalidCases = {
         "1945-12-31-23:59:59",
+        "1945-12-31T23:59:59.12345", // our pattern didn't specify 5 decimal places
         "1420509274123-",
     };
 
@@ -169,11 +167,10 @@ public class TestTimestampParser {
             Timestamp.valueOf("1970-01-01 05:06:00")),
         new ValidTimestampCase("05:06:07",
             Timestamp.valueOf("1970-05-06 00:00:07")),
-        new ValidTimestampCase("1945-12-31T23:59:59",
-            Timestamp.valueOf("1945-12-31 23:59:59")),
     };
 
     String[] invalidCases = {
+        "1945-12-31T23:59:59",
         "1945:12:31-",
         "12345",
     };

http://git-wip-us.apache.org/repos/asf/hive/blob/33088de0/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidSerDe.java
----------------------------------------------------------------------
diff --git a/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidSerDe.java b/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidSerDe.java
index f0e12a2..a9e7837 100644
--- a/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidSerDe.java
+++ b/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidSerDe.java
@@ -19,21 +19,6 @@ package org.apache.hadoop.hive.druid.serde;
 
 import com.fasterxml.jackson.core.type.TypeReference;
 import com.google.common.base.Function;
-import java.io.IOException;
-import java.io.InputStream;
-import java.time.Instant;
-import java.time.ZonedDateTime;
-import java.time.format.DateTimeFormatter;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Properties;
-import java.util.stream.Collectors;
-
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
 import io.druid.query.Druids;
@@ -42,10 +27,8 @@ import io.druid.query.metadata.metadata.ColumnAnalysis;
 import io.druid.query.metadata.metadata.SegmentAnalysis;
 import io.druid.query.metadata.metadata.SegmentMetadataQuery;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hive.common.type.Date;
 import org.apache.hadoop.hive.common.type.HiveChar;
 import org.apache.hadoop.hive.common.type.HiveVarchar;
-import org.apache.hadoop.hive.common.type.Timestamp;
 import org.apache.hadoop.hive.common.type.TimestampTZ;
 import org.apache.hadoop.hive.conf.Constants;
 import org.apache.hadoop.hive.conf.HiveConf;
@@ -59,13 +42,13 @@ import org.apache.hadoop.hive.serde2.SerDeException;
 import org.apache.hadoop.hive.serde2.SerDeSpec;
 import org.apache.hadoop.hive.serde2.SerDeStats;
 import org.apache.hadoop.hive.serde2.io.ByteWritable;
-import org.apache.hadoop.hive.serde2.io.DateWritableV2;
+import org.apache.hadoop.hive.serde2.io.DateWritable;
 import org.apache.hadoop.hive.serde2.io.DoubleWritable;
 import org.apache.hadoop.hive.serde2.io.HiveCharWritable;
 import org.apache.hadoop.hive.serde2.io.HiveVarcharWritable;
 import org.apache.hadoop.hive.serde2.io.ShortWritable;
 import org.apache.hadoop.hive.serde2.io.TimestampLocalTZWritable;
-import org.apache.hadoop.hive.serde2.io.TimestampWritableV2;
+import org.apache.hadoop.hive.serde2.io.TimestampWritable;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory;
 import org.apache.hadoop.hive.serde2.objectinspector.StructField;
@@ -99,6 +82,23 @@ import org.apache.hadoop.util.StringUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.io.IOException;
+import java.io.InputStream;
+import java.sql.Date;
+import java.sql.Timestamp;
+import java.time.Instant;
+import java.time.ZonedDateTime;
+import java.time.format.DateTimeFormatter;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Properties;
+import java.util.stream.Collectors;
+
 import static org.apache.hadoop.hive.druid.serde.DruidSerDeUtils.TIMESTAMP_FORMAT;
 import static org.joda.time.format.ISODateTimeFormat.dateOptionalTimeParser;
 
@@ -307,9 +307,9 @@ import static org.joda.time.format.ISODateTimeFormat.dateOptionalTimeParser;
       final Object res;
       switch (types[i].getPrimitiveCategory()) {
       case TIMESTAMP:
-        res = ((TimestampObjectInspector) fields.get(i).getFieldObjectInspector())
-            .getPrimitiveJavaObject(values.get(i)).toEpochMilli();
-          break;
+        res = ((TimestampObjectInspector) fields.get(i).getFieldObjectInspector()).getPrimitiveJavaObject(values.get(i))
+            .getTime();
+        break;
       case TIMESTAMPLOCALTZ:
         res = ((TimestampLocalTZObjectInspector) fields.get(i).getFieldObjectInspector())
             .getPrimitiveJavaObject(values.get(i)).getZonedDateTime().toInstant().toEpochMilli();
@@ -330,24 +330,22 @@ import static org.joda.time.format.ISODateTimeFormat.dateOptionalTimeParser;
         res = ((FloatObjectInspector) fields.get(i).getFieldObjectInspector()).get(values.get(i));
         break;
       case DOUBLE:
-        res = ((DoubleObjectInspector) fields.get(i).getFieldObjectInspector())
-            .get(values.get(i));
+        res = ((DoubleObjectInspector) fields.get(i).getFieldObjectInspector()).get(values.get(i));
         break;
       case CHAR:
-        res = ((HiveCharObjectInspector) fields.get(i).getFieldObjectInspector())
-            .getPrimitiveJavaObject(values.get(i)).getValue();
+        res = ((HiveCharObjectInspector) fields.get(i).getFieldObjectInspector()).getPrimitiveJavaObject(values.get(i))
+            .getValue();
         break;
       case VARCHAR:
-        res = ((HiveVarcharObjectInspector) fields.get(i).getFieldObjectInspector())
-            .getPrimitiveJavaObject(values.get(i)).getValue();
+        res =
+            ((HiveVarcharObjectInspector) fields.get(i).getFieldObjectInspector()).getPrimitiveJavaObject(values.get(i))
+                .getValue();
         break;
       case STRING:
-        res = ((StringObjectInspector) fields.get(i).getFieldObjectInspector())
-            .getPrimitiveJavaObject(values.get(i));
+        res = ((StringObjectInspector) fields.get(i).getFieldObjectInspector()).getPrimitiveJavaObject(values.get(i));
         break;
       case BOOLEAN:
-        res = ((BooleanObjectInspector) fields.get(i).getFieldObjectInspector())
-            .get(values.get(i));
+        res = ((BooleanObjectInspector) fields.get(i).getFieldObjectInspector()).get(values.get(i));
         break;
       default:
         throw new SerDeException("Unsupported type: " + types[i].getPrimitiveCategory());
@@ -362,7 +360,8 @@ import static org.joda.time.format.ISODateTimeFormat.dateOptionalTimeParser;
         fields.get(granularityFieldIndex).getFieldName().equals(Constants.DRUID_TIMESTAMP_GRANULARITY_COL_NAME));
     value.put(Constants.DRUID_TIMESTAMP_GRANULARITY_COL_NAME,
         ((TimestampObjectInspector) fields.get(granularityFieldIndex).getFieldObjectInspector())
-            .getPrimitiveJavaObject(values.get(granularityFieldIndex)).toEpochMilli());
+            .getPrimitiveJavaObject(values.get(granularityFieldIndex)).getTime()
+    );
     if (values.size() == columns.length + 2) {
       // Then partition number if any.
       final int partitionNumPos = granularityFieldIndex + 1;
@@ -396,11 +395,11 @@ import static org.joda.time.format.ISODateTimeFormat.dateOptionalTimeParser;
       switch (types[i].getPrimitiveCategory()) {
       case TIMESTAMP:
         if (value instanceof Number) {
-          output.add(new TimestampWritableV2(Timestamp.valueOf(
+          output.add(new TimestampWritable(Timestamp.valueOf(
               ZonedDateTime.ofInstant(Instant.ofEpochMilli(((Number) value).longValue()), tsTZTypeInfo.timeZone())
                   .format(DateTimeFormatter.ofPattern(TIMESTAMP_FORMAT)))));
         } else {
-          output.add(new TimestampWritableV2(Timestamp.valueOf((String) value)));
+          output.add(new TimestampWritable(Timestamp.valueOf((String) value)));
         }
 
         break;
@@ -418,14 +417,12 @@ import static org.joda.time.format.ISODateTimeFormat.dateOptionalTimeParser;
             ))));
         break;
       case DATE:
-        final DateWritableV2 dateWritable;
+        final DateWritable dateWritable;
         if (value instanceof Number) {
-          dateWritable = new DateWritableV2(
-              Date.ofEpochMilli((((Number) value).longValue())));
+          dateWritable = new DateWritable(new Date((((Number) value).longValue())));
         } else {
           // it is an extraction fn need to be parsed
-          dateWritable = new DateWritableV2(
-              Date.ofEpochMilli(dateOptionalTimeParser().parseDateTime((String) value).getMillis()));
+          dateWritable = new DateWritable(new Date(dateOptionalTimeParser().parseDateTime((String) value).getMillis()));
         }
         output.add(dateWritable);
         break;

http://git-wip-us.apache.org/repos/asf/hive/blob/33088de0/druid-handler/src/test/org/apache/hadoop/hive/druid/serde/TestDruidSerDe.java
----------------------------------------------------------------------
diff --git a/druid-handler/src/test/org/apache/hadoop/hive/druid/serde/TestDruidSerDe.java b/druid-handler/src/test/org/apache/hadoop/hive/druid/serde/TestDruidSerDe.java
index 060c654..922c1db 100644
--- a/druid-handler/src/test/org/apache/hadoop/hive/druid/serde/TestDruidSerDe.java
+++ b/druid-handler/src/test/org/apache/hadoop/hive/druid/serde/TestDruidSerDe.java
@@ -27,6 +27,7 @@ import java.io.ByteArrayInputStream;
 import java.io.IOException;
 import java.io.InputStream;
 import java.lang.reflect.InvocationTargetException;
+import java.sql.Timestamp;
 import java.time.Instant;
 import java.time.ZoneOffset;
 import java.util.ArrayList;
@@ -34,11 +35,20 @@ import java.util.List;
 import java.util.Map.Entry;
 import java.util.Properties;
 
+import com.fasterxml.jackson.core.type.TypeReference;
+import com.google.common.util.concurrent.SettableFuture;
+import com.metamx.http.client.HttpClient;
+import com.metamx.http.client.response.HttpResponseHandler;
+import io.druid.data.input.Row;
+import io.druid.query.Result;
+import io.druid.query.select.SelectResultValue;
+import io.druid.query.timeseries.TimeseriesResultValue;
+import io.druid.query.topn.TopNResultValue;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.common.type.HiveChar;
+import org.apache.hadoop.hive.common.type.HiveDecimal;
 import org.apache.hadoop.hive.common.type.HiveVarchar;
-import org.apache.hadoop.hive.common.type.Timestamp;
 import org.apache.hadoop.hive.common.type.TimestampTZ;
 import org.apache.hadoop.hive.conf.Constants;
 import org.apache.hadoop.hive.druid.DruidStorageHandlerUtils;
@@ -52,10 +62,11 @@ 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.HiveCharWritable;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
 import org.apache.hadoop.hive.serde2.io.HiveVarcharWritable;
 import org.apache.hadoop.hive.serde2.io.ShortWritable;
 import org.apache.hadoop.hive.serde2.io.TimestampLocalTZWritable;
-import org.apache.hadoop.hive.serde2.io.TimestampWritableV2;
+import org.apache.hadoop.hive.serde2.io.TimestampWritable;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory;
 import org.apache.hadoop.hive.serde2.objectinspector.StructField;
@@ -69,25 +80,17 @@ import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.NullWritable;
 import org.apache.hadoop.io.Text;
+
 import org.junit.Before;
 import org.junit.Test;
 
 import com.fasterxml.jackson.core.JsonParseException;
-import com.fasterxml.jackson.core.type.TypeReference;
 import com.fasterxml.jackson.databind.JsonMappingException;
 import com.google.common.base.Function;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Lists;
-import com.google.common.util.concurrent.SettableFuture;
-import com.metamx.http.client.HttpClient;
-import com.metamx.http.client.response.HttpResponseHandler;
 
-import io.druid.data.input.Row;
 import io.druid.query.Query;
-import io.druid.query.Result;
-import io.druid.query.select.SelectResultValue;
-import io.druid.query.timeseries.TimeseriesResultValue;
-import io.druid.query.topn.TopNResultValue;
 
 /**
  * Basic tests for Druid SerDe. The examples are taken from Druid 0.9.1.1
@@ -775,7 +778,7 @@ public class TestDruidSerDe {
       new IntWritable(1112123),
       new ShortWritable((short) 12),
       new ByteWritable((byte) 0),
-      new TimestampWritableV2(Timestamp.ofEpochSecond(1377907200L)) // granularity
+      new TimestampWritable(new Timestamp(1377907200000L)) // granularity
   };
   private static final DruidWritable DRUID_WRITABLE = new DruidWritable(
       ImmutableMap.<String, Object>builder()

http://git-wip-us.apache.org/repos/asf/hive/blob/33088de0/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HBaseRowSerializer.java
----------------------------------------------------------------------
diff --git a/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HBaseRowSerializer.java b/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HBaseRowSerializer.java
index 3805779..bc4e146 100644
--- a/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HBaseRowSerializer.java
+++ b/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HBaseRowSerializer.java
@@ -102,7 +102,7 @@ public class HBaseRowSerializer {
         timestamp = ((LongObjectInspector)inspector).get(value);
       } else {
         PrimitiveObjectInspector primitive = (PrimitiveObjectInspector) inspector;
-        timestamp = PrimitiveObjectInspectorUtils.getTimestamp(value, primitive).toEpochMilli();
+        timestamp = PrimitiveObjectInspectorUtils.getTimestamp(value, primitive).getTime();
       }
     }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/33088de0/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HiveHBaseInputFormatUtil.java
----------------------------------------------------------------------
diff --git a/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HiveHBaseInputFormatUtil.java b/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HiveHBaseInputFormatUtil.java
index 8b73bfb..05cc30a 100644
--- a/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HiveHBaseInputFormatUtil.java
+++ b/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HiveHBaseInputFormatUtil.java
@@ -35,6 +35,7 @@ import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hive.hbase.ColumnMappings.ColumnMapping;
 import org.apache.hadoop.hive.ql.exec.ExprNodeConstantEvaluator;
 import org.apache.hadoop.hive.ql.exec.SerializationUtilities;
+import org.apache.hadoop.hive.ql.exec.Utilities;
 import org.apache.hadoop.hive.ql.index.IndexSearchCondition;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.plan.TableScanDesc;
@@ -295,7 +296,7 @@ class HiveHBaseInputFormatUtil {
         timestamp = ((LongObjectInspector) inspector).get(value);
       } else {
         PrimitiveObjectInspector primitive = (PrimitiveObjectInspector) inspector;
-        timestamp = PrimitiveObjectInspectorUtils.getTimestamp(value, primitive).toEpochMilli();
+        timestamp = PrimitiveObjectInspectorUtils.getTimestamp(value, primitive).getTime();
       }
     } catch (HiveException e) {
       throw new IOException(e);

http://git-wip-us.apache.org/repos/asf/hive/blob/33088de0/hbase-handler/src/java/org/apache/hadoop/hive/hbase/LazyHBaseRow.java
----------------------------------------------------------------------
diff --git a/hbase-handler/src/java/org/apache/hadoop/hive/hbase/LazyHBaseRow.java b/hbase-handler/src/java/org/apache/hadoop/hive/hbase/LazyHBaseRow.java
index 2aeaa33..d94dbe8 100644
--- a/hbase-handler/src/java/org/apache/hadoop/hive/hbase/LazyHBaseRow.java
+++ b/hbase-handler/src/java/org/apache/hadoop/hive/hbase/LazyHBaseRow.java
@@ -23,7 +23,6 @@ import java.util.Arrays;
 import java.util.List;
 
 import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hive.common.type.Timestamp;
 import org.apache.hadoop.hive.hbase.ColumnMappings.ColumnMapping;
 import org.apache.hadoop.hive.hbase.struct.HBaseValueFactory;
 import org.apache.hadoop.hive.serde2.SerDeException;
@@ -162,8 +161,7 @@ public class LazyHBaseRow extends LazyStruct {
         }
         LazyObjectBase lz = fields[fieldID];
         if (lz instanceof LazyTimestamp) {
-          ((LazyTimestamp) lz).getWritableObject().set(
-              Timestamp.ofEpochMilli(timestamp));
+          ((LazyTimestamp) lz).getWritableObject().setTime(timestamp);
         } else {
           ((LazyLong) lz).getWritableObject().set(timestamp);
         }