You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@impala.apache.org by ta...@apache.org on 2018/10/04 21:52:43 UTC

[3/4] impala git commit: Revert "IMPALA-7595: Revert "IMPALA-7521: Speed up sub-second unix time->TimestampValue conversions""

Revert "IMPALA-7595: Revert "IMPALA-7521: Speed up sub-second unix time->TimestampValue conversions""

IMPALA-7595 added proper handling for invalid time-of-day values
in Parquet, so the DCHECK mentioned in IMPALA-7595 will no longer
be hit. This means that IMPALA-7521 can be committed again without
causing problems.

This reverts commit f8b472ee6442e31a867a6dd6aaac22cc44291d41.

Change-Id: Ibab04bc6ad09db331220312ed21d90622fdfc41b
Reviewed-on: http://gerrit.cloudera.org:8080/11573
Reviewed-by: Impala Public Jenkins <im...@cloudera.com>
Tested-by: Impala Public Jenkins <im...@cloudera.com>


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

Branch: refs/heads/master
Commit: d301600a85f399863d89c281a9a1dc3091d52fcc
Parents: e6bbe4e
Author: Csaba Ringhofer <cs...@cloudera.com>
Authored: Wed Oct 3 14:51:15 2018 +0200
Committer: Impala Public Jenkins <im...@cloudera.com>
Committed: Thu Oct 4 03:40:44 2018 +0000

----------------------------------------------------------------------
 .../benchmarks/convert-timestamp-benchmark.cc   | 183 ++++++++++++++++++-
 be/src/exec/data-source-scan-node.cc            |   2 +-
 be/src/exec/hdfs-orc-scanner.cc                 |   1 +
 be/src/exprs/decimal-operators-ir.cc            |   1 +
 be/src/exprs/expr-test.cc                       |   6 +-
 be/src/runtime/timestamp-test.cc                | 129 +++++++++++--
 be/src/runtime/timestamp-value.cc               |  43 +----
 be/src/runtime/timestamp-value.h                |  64 +++----
 be/src/runtime/timestamp-value.inline.h         |  50 ++++-
 9 files changed, 371 insertions(+), 108 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/impala/blob/d301600a/be/src/benchmarks/convert-timestamp-benchmark.cc
----------------------------------------------------------------------
diff --git a/be/src/benchmarks/convert-timestamp-benchmark.cc b/be/src/benchmarks/convert-timestamp-benchmark.cc
index 89ba857..dd40f2f 100644
--- a/be/src/benchmarks/convert-timestamp-benchmark.cc
+++ b/be/src/benchmarks/convert-timestamp-benchmark.cc
@@ -101,9 +101,28 @@ UnixTimeToLocalPtime:      Function  iters/ms   10%ile   50%ile   90%ile     10%
 UnixTimeToUtcPtime:        Function  iters/ms   10%ile   50%ile   90%ile     10%ile     50%ile     90%ile
                                                                          (relative) (relative) (relative)
 ---------------------------------------------------------------------------------------------------------
-                            (glibc)               18.8     19.2     19.5         1X         1X         1X
-                      (Google/CCTZ)                  8     8.14     8.27     0.425X     0.425X     0.424X
-                        (fast path)               29.5     30.2     30.8      1.57X      1.58X      1.58X
+                            (glibc)                 17     17.6     17.9         1X         1X         1X
+                      (Google/CCTZ)               6.45     6.71     6.81     0.379X     0.382X      0.38X
+                        (fast path)               25.1       26     26.4      1.47X      1.48X      1.48X
+                        (day split)               48.6     50.3     51.3      2.85X      2.87X      2.86X
+
+UtcFromUnixTimeMicros:     Function  iters/ms   10%ile   50%ile   90%ile     10%ile     50%ile     90%ile
+                                                                         (relative) (relative) (relative)
+---------------------------------------------------------------------------------------------------------
+                  (sec split (old))               17.9     18.7     19.1         1X         1X         1X
+                        (day split)                111      116      118      6.21X      6.19X      6.19X
+
+FromUnixTimeNanos:         Function  iters/ms   10%ile   50%ile   90%ile     10%ile     50%ile     90%ile
+                                                                         (relative) (relative) (relative)
+---------------------------------------------------------------------------------------------------------
+                  (sec split (old))               18.7     19.5     19.8         1X         1X         1X
+                  (sec split (new))                104      108      110      5.58X      5.55X      5.57X
+
+FromSubsecondUnixTime:     Function  iters/ms   10%ile   50%ile   90%ile     10%ile     50%ile     90%ile
+                                                                         (relative) (relative) (relative)
+---------------------------------------------------------------------------------------------------------
+                              (old)               18.7     18.7     18.7         1X         1X         1X
+                              (new)               73.5     74.1     74.1      3.94X      3.96X      3.96X
 
 Number of threads: 8
 
@@ -236,14 +255,17 @@ void bail_if_results_dont_match(const vector<const vector<T>*>& test_result_vec)
 
   auto b = test_result_vec.begin();
   for (auto it = b + 1; it != test_result_vec.end(); ++it) {
-    if (**b != **it) {
-      cerr << "Results don't match.";
-      exit(1);
+    auto& references = **b;
+    auto& results = **it;
+    for (int i = 0; i < references.size(); ++i) {
+      if (references[i] != results[i]) {
+        cerr << "Results don't match: " << references[i] << " vs " << results[i] << endl;
+        exit(1);
+      }
     }
   }
 }
 
-
 //
 // Test UtcToUnixTime (boost is expected to be the fastest, followed by CCTZ and glibc)
 //
@@ -540,6 +562,68 @@ boost::posix_time::ptime cctz_optimized_unix_time_to_utc_ptime(const time_t& uni
   return cctz_unix_time_to_utc_ptime(unix_time);
 }
 
+boost::posix_time::ptime split_unix_time_to_utc_ptime(const time_t& unix_time) {
+  int64_t time = unix_time;
+  int32_t days = TimestampValue::SplitTime<24*60*60>(&time);
+
+  return boost::posix_time::ptime(
+      boost::gregorian::date(1970, 1, 1) + boost::gregorian::date_duration(days),
+      boost::posix_time::nanoseconds(time*NANOS_PER_SEC));
+}
+
+TimestampValue sec_split_utc_from_unix_time_micros(const int64_t& unix_time_micros) {
+  int64_t ts_seconds = unix_time_micros / MICROS_PER_SEC;
+  int64_t micros_part = unix_time_micros - (ts_seconds * MICROS_PER_SEC);
+  boost::posix_time::ptime temp = cctz_optimized_unix_time_to_utc_ptime(ts_seconds);
+  temp += boost::posix_time::microseconds(micros_part);
+  return TimestampValue(temp);
+}
+
+TimestampValue day_split_utc_from_unix_time_micros(const int64_t& unix_time_micros) {
+  static const boost::gregorian::date EPOCH(1970,1,1);
+  int64_t micros = unix_time_micros;
+  int32_t days = TimestampValue::SplitTime<24LL*60*60*MICROS_PER_SEC>(&micros);
+
+  return TimestampValue(
+      EPOCH + boost::gregorian::date_duration(days),
+      boost::posix_time::nanoseconds(micros*1000));
+}
+
+struct SplitNanoAndSecond {
+  int64_t seconds;
+  int64_t nanos;
+};
+
+TimestampValue old_split_utc_from_unix_time_nanos(const SplitNanoAndSecond& unix_time) {
+  boost::posix_time::ptime temp =
+      cctz_optimized_unix_time_to_utc_ptime(unix_time.seconds);
+  temp += boost::posix_time::nanoseconds(unix_time.nanos);
+  return TimestampValue(temp);
+}
+
+TimestampValue new_split_utc_from_unix_time_nanos(const SplitNanoAndSecond& unix_time) {
+  // The TimestampValue version is used as it is hard to reproduce the same logic without
+  // accessing private members.
+  return TimestampValue::FromUnixTimeNanos(unix_time.seconds, unix_time.nanos,
+      TimezoneDatabase::GetUtcTimezone());
+}
+
+TimestampValue from_subsecond_unix_time_old(const double& unix_time) {
+  const double ONE_BILLIONTH = 0.000000001;
+  const time_t unix_time_whole = unix_time;
+  boost::posix_time::ptime temp =
+      cctz_optimized_unix_time_to_utc_ptime(unix_time_whole);
+  temp += boost::posix_time::nanoseconds((unix_time - unix_time_whole) / ONE_BILLIONTH);
+  return TimestampValue(temp);
+}
+
+TimestampValue from_subsecond_unix_time_new(const double& unix_time) {
+  const double ONE_BILLIONTH = 0.000000001;
+  int64_t unix_time_whole = unix_time;
+  int64_t nanos = (unix_time - unix_time_whole) / ONE_BILLIONTH;
+  return TimestampValue::FromUnixTimeNanos(
+      unix_time_whole, nanos, TimezoneDatabase::GetUtcTimezone());
+}
 
 //
 // Test ToUtc (CCTZ is expected to be faster than boost)
@@ -749,6 +833,10 @@ int main(int argc, char* argv[]) {
       time_t,
       boost::posix_time::ptime,
       fastpath_unix_time_to_utc_ptime> fastpath_unix_time_to_utc_ptime_data = time_data;
+  TestData<
+      time_t,
+      boost::posix_time::ptime,
+      split_unix_time_to_utc_ptime> split_unix_time_to_utc_ptime_data = time_data;
 
   glibc_unix_time_to_utc_ptime_data.add_to_benchmark(bm_unix_time_to_utc_ptime,
       "(glibc)");
@@ -756,12 +844,91 @@ int main(int argc, char* argv[]) {
       "(Google/CCTZ)");
   fastpath_unix_time_to_utc_ptime_data.add_to_benchmark(bm_unix_time_to_utc_ptime,
       "(fast path)");
+  split_unix_time_to_utc_ptime_data.add_to_benchmark(bm_unix_time_to_utc_ptime,
+      "(day split)");
   cout << bm_unix_time_to_utc_ptime.Measure() << endl;
 
   bail_if_results_dont_match(vector<const vector<boost::posix_time::ptime>*>{
       &glibc_unix_time_to_utc_ptime_data.result(),
       &cctz_unix_time_to_utc_ptime_data.result(),
-      &fastpath_unix_time_to_utc_ptime_data.result()});
+      &fastpath_unix_time_to_utc_ptime_data.result(),
+      &split_unix_time_to_utc_ptime_data.result()});
+
+  // Benchmark UtcFromUnixTimeMicros improvement in IMPALA-7417.
+  vector<time_t> microsec_data;
+  for (int i = 0; i < tsvalue_data.size(); ++i) {
+    const TimestampValue& tsvalue = tsvalue_data[i];
+    time_t unix_time;
+    tsvalue.ToUnixTime(TimezoneDatabase::GetUtcTimezone(), &unix_time);
+    int micros = (i * 1001) % MICROS_PER_SEC; // add some sub-second part
+    microsec_data.push_back(unix_time * MICROS_PER_SEC + micros);
+  }
+
+  Benchmark bm_utc_from_unix_time_micros("UtcFromUnixTimeMicros");
+  TestData<int64_t, TimestampValue, sec_split_utc_from_unix_time_micros>
+      sec_split_utc_from_unix_time_micros_data = microsec_data;
+  TestData<int64_t, TimestampValue, day_split_utc_from_unix_time_micros>
+      day_split_utc_from_unix_time_micros_data = microsec_data;
+
+  sec_split_utc_from_unix_time_micros_data.add_to_benchmark(bm_utc_from_unix_time_micros,
+      "(sec split (old))");
+  day_split_utc_from_unix_time_micros_data.add_to_benchmark(bm_utc_from_unix_time_micros,
+      "(day split)");
+  cout << bm_utc_from_unix_time_micros.Measure() << endl;
+
+  bail_if_results_dont_match(vector<const vector<TimestampValue>*>{
+      &sec_split_utc_from_unix_time_micros_data.result(),
+      &day_split_utc_from_unix_time_micros_data.result()});
+
+  // Benchmark FromUnixTimeNanos improvement in IMPALA-7417.
+  vector<SplitNanoAndSecond> nanosec_data;
+  for (int i = 0; i < tsvalue_data.size(); ++i) {
+    const TimestampValue& tsvalue = tsvalue_data[i];
+    time_t unix_time;
+    tsvalue.ToUnixTime(TimezoneDatabase::GetUtcTimezone(), &unix_time);
+    int nanos = (i * 1001) % NANOS_PER_SEC; // add some sub-second part
+    nanosec_data.push_back(SplitNanoAndSecond {unix_time, nanos} );
+  }
+
+  Benchmark bm_utc_from_unix_time_nanos("FromUnixTimeNanos");
+  TestData<SplitNanoAndSecond, TimestampValue, old_split_utc_from_unix_time_nanos>
+      old_split_utc_from_unix_time_nanos_data = nanosec_data;
+  TestData<SplitNanoAndSecond, TimestampValue, new_split_utc_from_unix_time_nanos>
+      new_split_utc_from_unix_time_nanos_data = nanosec_data;
+
+  old_split_utc_from_unix_time_nanos_data.add_to_benchmark(bm_utc_from_unix_time_nanos,
+      "(sec split (old))");
+  new_split_utc_from_unix_time_nanos_data.add_to_benchmark(bm_utc_from_unix_time_nanos,
+      "(sec split (new))");
+  cout << bm_utc_from_unix_time_nanos.Measure() << endl;
+
+  bail_if_results_dont_match(vector<const vector<TimestampValue>*>{
+      &old_split_utc_from_unix_time_nanos_data.result(),
+      &new_split_utc_from_unix_time_nanos_data.result()});
+
+  // Benchmark FromSubsecondUnixTime before and after IMPALA-7417.
+  vector<double> double_data;
+  for (int i = 0; i < tsvalue_data.size(); ++i) {
+    const TimestampValue& tsvalue = tsvalue_data[i];
+    time_t unix_time;
+    tsvalue.ToUnixTime(TimezoneDatabase::GetUtcTimezone(), &unix_time);
+    double nanos = (i * 1001) % NANOS_PER_SEC; // add some sub-second part
+    double_data.push_back((double)unix_time + nanos / NANOS_PER_SEC);
+  }
+
+  Benchmark from_subsecond_unix_time("FromSubsecondUnixTime");
+  TestData<double, TimestampValue, from_subsecond_unix_time_old>
+      from_subsecond_unix_time_old_data = double_data;
+  TestData<double, TimestampValue, from_subsecond_unix_time_new>
+      from_subsecond_unix_time_new_data = double_data;
+
+  from_subsecond_unix_time_old_data.add_to_benchmark(from_subsecond_unix_time, "(old)");
+  from_subsecond_unix_time_new_data.add_to_benchmark(from_subsecond_unix_time, "(new)");
+  cout << from_subsecond_unix_time.Measure() << endl;
+
+  bail_if_results_dont_match(vector<const vector<TimestampValue>*>{
+      &from_subsecond_unix_time_old_data.result(),
+      &from_subsecond_unix_time_new_data.result()});
 
   // If number of threads is specified, run multithreaded tests.
   int num_of_threads = (argc < 2) ? 0 : atoi(argv[1]);

http://git-wip-us.apache.org/repos/asf/impala/blob/d301600a/be/src/exec/data-source-scan-node.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/data-source-scan-node.cc b/be/src/exec/data-source-scan-node.cc
index fa11d05..10f0c7d 100644
--- a/be/src/exec/data-source-scan-node.cc
+++ b/be/src/exec/data-source-scan-node.cc
@@ -29,7 +29,7 @@
 #include "runtime/runtime-state.h"
 #include "runtime/row-batch.h"
 #include "runtime/string-value.h"
-#include "runtime/timestamp-value.h"
+#include "runtime/timestamp-value.inline.h"
 #include "runtime/tuple-row.h"
 #include "util/jni-util.h"
 #include "util/periodic-counter-updater.h"

http://git-wip-us.apache.org/repos/asf/impala/blob/d301600a/be/src/exec/hdfs-orc-scanner.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/hdfs-orc-scanner.cc b/be/src/exec/hdfs-orc-scanner.cc
index 55b3b71..fa36660 100644
--- a/be/src/exec/hdfs-orc-scanner.cc
+++ b/be/src/exec/hdfs-orc-scanner.cc
@@ -24,6 +24,7 @@
 #include "runtime/exec-env.h"
 #include "runtime/io/request-context.h"
 #include "runtime/runtime-filter.inline.h"
+#include "runtime/timestamp-value.inline.h"
 #include "runtime/tuple-row.h"
 #include "util/decompress.h"
 

http://git-wip-us.apache.org/repos/asf/impala/blob/d301600a/be/src/exprs/decimal-operators-ir.cc
----------------------------------------------------------------------
diff --git a/be/src/exprs/decimal-operators-ir.cc b/be/src/exprs/decimal-operators-ir.cc
index 0a41f97..c987b9d 100644
--- a/be/src/exprs/decimal-operators-ir.cc
+++ b/be/src/exprs/decimal-operators-ir.cc
@@ -26,6 +26,7 @@
 #include "exprs/scalar-expr.h"
 #include "exprs/timezone_db.h"
 #include "runtime/decimal-value.inline.h"
+#include "runtime/timestamp-value.inline.h"
 #include "util/decimal-util.h"
 #include "util/string-parser.h"
 

http://git-wip-us.apache.org/repos/asf/impala/blob/d301600a/be/src/exprs/expr-test.cc
----------------------------------------------------------------------
diff --git a/be/src/exprs/expr-test.cc b/be/src/exprs/expr-test.cc
index 7168b03..c926ff0 100644
--- a/be/src/exprs/expr-test.cc
+++ b/be/src/exprs/expr-test.cc
@@ -172,13 +172,15 @@ class ScopedTimeZoneOverride {
 
 // Enable FLAGS_use_local_tz_for_unix_timestamp_conversions for the duration of the scope.
 class ScopedLocalUnixTimestampConversionOverride {
+ bool original_;
  public:
   ScopedLocalUnixTimestampConversionOverride() {
+    original_ = FLAGS_use_local_tz_for_unix_timestamp_conversions;
     FLAGS_use_local_tz_for_unix_timestamp_conversions = true;
   }
 
   ~ScopedLocalUnixTimestampConversionOverride() {
-    FLAGS_use_local_tz_for_unix_timestamp_conversions = false;
+    FLAGS_use_local_tz_for_unix_timestamp_conversions = original_;
   }
 };
 
@@ -6548,6 +6550,7 @@ TEST_F(ExprTest, TimestampFunctions) {
   // Test that now() and current_timestamp() are reasonable.
   {
     ScopedTimeZoneOverride time_zone("PST8PDT");
+    ScopedLocalUnixTimestampConversionOverride use_local;
     const Timezone& local_tz = time_zone.GetTimezone();
 
     const TimestampValue start_time =
@@ -6573,6 +6576,7 @@ TEST_F(ExprTest, TimestampFunctions) {
   // Test cast(unix_timestamp() as timestamp).
   {
     ScopedTimeZoneOverride time_zone("PST8PDT");
+    ScopedLocalUnixTimestampConversionOverride use_local;
     const Timezone& local_tz = time_zone.GetTimezone();
 
     // UNIX_TIMESTAMP() has second precision so the comparison start time is shifted back

http://git-wip-us.apache.org/repos/asf/impala/blob/d301600a/be/src/runtime/timestamp-test.cc
----------------------------------------------------------------------
diff --git a/be/src/runtime/timestamp-test.cc b/be/src/runtime/timestamp-test.cc
index 3f3f01d..4e5b4b9 100644
--- a/be/src/runtime/timestamp-test.cc
+++ b/be/src/runtime/timestamp-test.cc
@@ -233,6 +233,66 @@ void TestTimestampTokens(vector<TimestampToken>* toks, int year, int month,
   } while (next_permutation(toks->begin(), toks->end()));
 }
 
+// Checks that FromSubsecondUnixTime gives the correct result. Conversion to double
+// is lossy so the result is expected to be within a certain range of 'expected'.
+// The fraction part of 'nanos' can express sub-nanoseconds - the current logic is to
+// truncate to the next whole nanosecond (towards 0).
+void TestFromDoubleUnixTime(
+    int64_t seconds, int64_t millis, double nanos, const TimestampValue& expected) {
+  auto tz = TimezoneDatabase::GetUtcTimezone();
+  TimestampValue from_double = TimestampValue::FromSubsecondUnixTime(
+      1.0 * seconds + 0.001 * millis + nanos / NANOS_PER_SEC, tz);
+
+  if (!expected.HasDate()) EXPECT_FALSE(from_double.HasDate());
+  else {
+    // Conversion to double is lossy so the timestamp can be a bit different.
+    int64_t expected_rounded_to_micros, from_double_rounded_to_micros;
+    EXPECT_TRUE(expected.UtcToUnixTimeMicros(&expected_rounded_to_micros));
+    EXPECT_TRUE(from_double.UtcToUnixTimeMicros(&from_double_rounded_to_micros));
+    // The difference can be more than a microsec in case of timestamps far from 1970.
+    int64_t MARGIN_OF_ERROR = 8;
+    EXPECT_LT(abs(expected_rounded_to_micros - from_double_rounded_to_micros),
+        MARGIN_OF_ERROR);
+  }
+}
+
+// Checks that all sub-second From*UnixTime gives the same result and that the result
+// is the same as 'expected'.
+// If 'expected' is nullptr then the result is expected to be invalid (out of range).
+void TestFromSubSecondFunctions(int64_t seconds, int64_t millis, const char* expected) {
+  auto tz = TimezoneDatabase::GetUtcTimezone();
+
+  TimestampValue from_millis =
+      TimestampValue::UtcFromUnixTimeMillis(seconds * 1000 + millis);
+  if (expected == nullptr) EXPECT_FALSE(from_millis.HasDate());
+  else EXPECT_EQ(expected, from_millis.ToString());
+
+  EXPECT_EQ(from_millis, TimestampValue::UtcFromUnixTimeMicros(
+      seconds * MICROS_PER_SEC + millis * 1000));
+  EXPECT_EQ(from_millis, TimestampValue::FromUnixTimeMicros(
+      seconds * MICROS_PER_SEC + millis * 1000, tz));
+
+  // Check the same timestamp shifted with some sub-nanosecs.
+  vector<double> sub_nanosec_offsets =
+      {0.0, 0.1, 0.9, 0.000001, 0.999999, 2.2250738585072020e-308};
+  for (double sub_nanos: sub_nanosec_offsets) {
+    TestFromDoubleUnixTime(seconds, millis, sub_nanos, from_millis);
+    TestFromDoubleUnixTime(seconds, millis, -sub_nanos, from_millis);
+  }
+
+  // Test FromUnixTimeNanos with shifted sec + subsec pairs.
+  vector<int64_t> signs = {-1, 1};
+  vector<int64_t> offsets = {0, 1, 2, 60, 60*60, 24*60*60};
+  for (int64_t sign: signs) {
+    for (int64_t offset: offsets) {
+      int64_t shifted_seconds = seconds + sign * offset;
+      int64_t shifted_nanos = (millis - 1000 * sign * offset) * 1000 * 1000;
+      EXPECT_EQ(from_millis,
+          TimestampValue::FromUnixTimeNanos(shifted_seconds, shifted_nanos, tz));
+    }
+  }
+}
+
 TEST(TimestampTest, Basic) {
   // Fix current time to determine the behavior parsing 2-digit year format
   // Set it to 03/01 to test 02/29 edge cases.
@@ -671,12 +731,9 @@ TEST(TimestampTest, Basic) {
   // Sub-second FromUnixTime functions incorrectly accepted the last second of 1399
   // as valid, because validation logic checked the nearest second rounded towards 0
   // (IMPALA-5664).
-  EXPECT_FALSE(TimestampValue::FromSubsecondUnixTime(
-      MIN_DATE_AS_UNIX_TIME - 0.1, utc_tz).HasDate());
-  EXPECT_FALSE(TimestampValue::UtcFromUnixTimeMicros(
-      MIN_DATE_AS_UNIX_TIME * MICROS_PER_SEC - 2000).HasDate());
-  EXPECT_FALSE(TimestampValue::FromUnixTimeNanos(
-      MIN_DATE_AS_UNIX_TIME, -NANOS_PER_MICRO * 100, utc_tz).HasDate());
+  TestFromSubSecondFunctions(MIN_DATE_AS_UNIX_TIME, -100, nullptr);
+  TestFromSubSecondFunctions(MIN_DATE_AS_UNIX_TIME, 100,
+      "1400-01-01 00:00:00.100000000");
 
   // Test the max supported date that can be represented in seconds.
   const int64_t MAX_DATE_AS_UNIX_TIME = 253402300799;
@@ -719,20 +776,9 @@ TEST(TimestampTest, Basic) {
   EXPECT_FALSE(too_late.HasTime());
 
   // Checking sub-second FromUnixTime functions near 10000.01.01
-  EXPECT_TRUE(TimestampValue::FromSubsecondUnixTime(
-      MAX_DATE_AS_UNIX_TIME + 0.99, utc_tz).HasDate());
-  EXPECT_FALSE(TimestampValue::FromSubsecondUnixTime(
-      MAX_DATE_AS_UNIX_TIME + 1, utc_tz).HasDate());
-
-  EXPECT_TRUE(TimestampValue::UtcFromUnixTimeMicros(
-      MAX_DATE_AS_UNIX_TIME * MICROS_PER_SEC + MICROS_PER_SEC - 1).HasDate());
-  EXPECT_FALSE(TimestampValue::UtcFromUnixTimeMicros(
-      MAX_DATE_AS_UNIX_TIME * MICROS_PER_SEC + MICROS_PER_SEC).HasDate());
-
-  EXPECT_TRUE(TimestampValue::FromUnixTimeNanos(
-      MAX_DATE_AS_UNIX_TIME, NANOS_PER_SEC - 1, utc_tz).HasDate());
-  EXPECT_FALSE(TimestampValue::FromUnixTimeNanos(
-      MAX_DATE_AS_UNIX_TIME, NANOS_PER_SEC, utc_tz).HasDate());
+  TestFromSubSecondFunctions(MAX_DATE_AS_UNIX_TIME, MILLIS_PER_SEC - 1,
+      "9999-12-31 23:59:59.999000000");
+  TestFromSubSecondFunctions(MAX_DATE_AS_UNIX_TIME, MILLIS_PER_SEC, nullptr);
 
   // Regression tests for IMPALA-1676, Unix times overflow int32 during year 2038
   EXPECT_EQ("2038-01-19 03:14:08",
@@ -807,6 +853,49 @@ TEST(TimestampTest, Basic) {
       TimestampValue::FromSubsecondUnixTime(0.008, utc_tz).ToString());
 }
 
+// Test subsecond unix time conversion for non edge cases.
+TEST(TimestampTest, SubSecond) {
+  // Test with millisec precision.
+  TestFromSubSecondFunctions(0, 0, "1970-01-01 00:00:00");
+  TestFromSubSecondFunctions(0, 100, "1970-01-01 00:00:00.100000000");
+  TestFromSubSecondFunctions(0, 1100, "1970-01-01 00:00:01.100000000");
+  TestFromSubSecondFunctions(0, 24*60*60*1000, "1970-01-02 00:00:00");
+  TestFromSubSecondFunctions(0, 2*24*60*60*1000 + 100, "1970-01-03 00:00:00.100000000");
+
+  TestFromSubSecondFunctions(0, -100, "1969-12-31 23:59:59.900000000");
+  TestFromSubSecondFunctions(0, -1100, "1969-12-31 23:59:58.900000000");
+  TestFromSubSecondFunctions(0, -24*60*60*1000, "1969-12-31 00:00:00");
+  TestFromSubSecondFunctions(0, -2*24*60*60*1000 + 100, "1969-12-30 00:00:00.100000000");
+
+  TestFromSubSecondFunctions(-1, 0, "1969-12-31 23:59:59");
+  TestFromSubSecondFunctions(-1, 100, "1969-12-31 23:59:59.100000000");
+  TestFromSubSecondFunctions(-1, 1100, "1970-01-01 00:00:00.100000000");
+  TestFromSubSecondFunctions(-1, 24*60*60*1000, "1970-01-01 23:59:59");
+  TestFromSubSecondFunctions(-1, 2*24*60*60*1000 + 100, "1970-01-02 23:59:59.100000000");
+
+  TestFromSubSecondFunctions(-1, -100, "1969-12-31 23:59:58.900000000");
+  TestFromSubSecondFunctions(-1, -1100, "1969-12-31 23:59:57.900000000");
+  TestFromSubSecondFunctions(-1, -24*60*60*1000, "1969-12-30 23:59:59");
+  TestFromSubSecondFunctions(-1, -2*24*60*60*1000 + 100, "1969-12-29 23:59:59.100000000");
+
+  // A few test with sub-millisec precision.
+  auto tz = TimezoneDatabase::GetUtcTimezone();
+  EXPECT_EQ("1970-01-01 00:00:00.000001000",
+      TimestampValue::UtcFromUnixTimeMicros(1).ToString());
+  EXPECT_EQ("1969-12-31 23:59:59.999999000",
+      TimestampValue::UtcFromUnixTimeMicros(-1).ToString());
+
+  EXPECT_EQ("1970-01-01 00:00:00.000001000",
+      TimestampValue::FromUnixTimeMicros(1, tz).ToString());
+  EXPECT_EQ("1969-12-31 23:59:59.999999000",
+      TimestampValue::FromUnixTimeMicros(-1, tz).ToString());
+
+  EXPECT_EQ("1970-01-01 00:00:00.000000001",
+      TimestampValue::FromUnixTimeNanos(0, 1, tz).ToString());
+  EXPECT_EQ("1969-12-31 23:59:59.999999999",
+      TimestampValue::FromUnixTimeNanos(0, -1, tz).ToString());
+}
+
 }
 
 IMPALA_TEST_MAIN();

http://git-wip-us.apache.org/repos/asf/impala/blob/d301600a/be/src/runtime/timestamp-value.cc
----------------------------------------------------------------------
diff --git a/be/src/runtime/timestamp-value.cc b/be/src/runtime/timestamp-value.cc
index a3631e5..14c57b9 100644
--- a/be/src/runtime/timestamp-value.cc
+++ b/be/src/runtime/timestamp-value.cc
@@ -142,7 +142,7 @@ void TimestampValue::LocalToUtc(const Timezone& local_tz) {
     SetToInvalidDateTime();
   } else {
     int64_t nanos = time_.fractional_seconds();
-    *this = UnixTimeToUtcPtime(TimePointToUnixTime(from_cl.pre));
+    *this = UtcFromUnixTimeTicks<1>(TimePointToUnixTime(from_cl.pre));
     // Time-zone conversion rules don't affect fractional seconds, leave them intact.
     time_ += nanoseconds(nanos);
   }
@@ -152,10 +152,8 @@ ostream& operator<<(ostream& os, const TimestampValue& timestamp_value) {
   return os << timestamp_value.ToString();
 }
 
-/// Return a ptime representation of the given Unix time (seconds since the Unix epoch).
-/// The time zone of the resulting ptime is 'local_tz'. This is called by UnixTimeToPtime.
-ptime TimestampValue::UnixTimeToLocalPtime(time_t unix_time,
-    const Timezone& local_tz) {
+TimestampValue TimestampValue::UnixTimeToLocal(
+    time_t unix_time, const Timezone& local_tz) {
   cctz::time_point<cctz::sys_seconds> from_tp = UnixTimeToTimePoint(unix_time);
   cctz::civil_second to_cs = cctz::convert(from_tp, local_tz);
   // boost::gregorian::date() throws boost::gregorian::bad_year if year is not in the
@@ -163,44 +161,17 @@ ptime TimestampValue::UnixTimeToLocalPtime(time_t unix_time,
   if (UNLIKELY(IsDateOutOfRange(to_cs))) {
     return ptime(not_a_date_time);
   } else {
-    return ptime(
+    return TimestampValue(
         boost::gregorian::date(to_cs.year(), to_cs.month(), to_cs.day()),
         boost::posix_time::time_duration(to_cs.hour(), to_cs.minute(), to_cs.second()));
   }
 }
 
-/// Return a ptime representation of the given Unix time (seconds since the Unix epoch).
-/// The time zone of the resulting ptime is UTC.
-/// In order to avoid a serious performance degredation using CCTZ, this function uses
-/// boost to convert the time_t to a ptime. Unfortunately, because the boost conversion
-/// relies on time_duration to represent the time_t and internally
-/// time_duration stores nanosecond precision ticks, the 'fast path' conversion using
-/// boost can only handle a limited range of dates (appx years 1677-2622, while Impala
-/// supports years 1400-9999). For dates outside this range, the conversion will instead
-/// use the CCTZ function convert which supports those dates. This is called by
-/// UnixTimeToPtime.
-ptime TimestampValue::UnixTimeToUtcPtime(time_t unix_time) {
-  // Minimum Unix time that can be converted with from_time_t: 1677-Sep-21 00:12:44
-  const int64_t MIN_BOOST_CONVERT_UNIX_TIME = -9223372036;
-  // Maximum Unix time that can be converted with from_time_t: 2262-Apr-11 23:47:16
-  const int64_t MAX_BOOST_CONVERT_UNIX_TIME = 9223372036;
-  if (LIKELY(unix_time >= MIN_BOOST_CONVERT_UNIX_TIME &&
-             unix_time <= MAX_BOOST_CONVERT_UNIX_TIME)) {
-    try {
-      return from_time_t(unix_time);
-    } catch (std::exception&) {
-      return ptime(not_a_date_time);
-    }
-  }
-
-  return UnixTimeToLocalPtime(unix_time, TimezoneDatabase::GetUtcTimezone());
-}
-
-ptime TimestampValue::UnixTimeToPtime(time_t unix_time, const Timezone& local_tz) {
+TimestampValue TimestampValue::FromUnixTime(time_t unix_time, const Timezone& local_tz) {
   if (FLAGS_use_local_tz_for_unix_timestamp_conversions) {
-    return UnixTimeToLocalPtime(unix_time, local_tz);
+    return UnixTimeToLocal(unix_time, local_tz);
   } else {
-    return UnixTimeToUtcPtime(unix_time);
+    return UtcFromUnixTimeTicks<1>(unix_time);
   }
 }
 

http://git-wip-us.apache.org/repos/asf/impala/blob/d301600a/be/src/runtime/timestamp-value.h
----------------------------------------------------------------------
diff --git a/be/src/runtime/timestamp-value.h b/be/src/runtime/timestamp-value.h
index 840a5ae..6d87d9b 100644
--- a/be/src/runtime/timestamp-value.h
+++ b/be/src/runtime/timestamp-value.h
@@ -97,22 +97,15 @@ class TimestampValue {
   /// Return the corresponding timestamp in the 'local_tz' time zone if
   /// FLAGS_use_local_tz_for_unix_timestamp_conversions is true. Otherwise, return the
   /// corresponding timestamp in UTC.
-  static TimestampValue FromUnixTime(time_t unix_time, const Timezone& local_tz) {
-    return TimestampValue(UnixTimeToPtime(unix_time, local_tz));
-  }
+  static TimestampValue FromUnixTime(time_t unix_time, const Timezone& local_tz);
 
   /// Same as FromUnixTime() above, but adds the specified number of nanoseconds to the
   /// resulting TimestampValue. Handles negative nanoseconds and the case where
   /// abs(nanos) >= 1e9.
   static TimestampValue FromUnixTimeNanos(time_t unix_time, int64_t nanos,
-      const Timezone& local_tz) {
-    boost::posix_time::ptime temp = UnixTimeToPtime(unix_time, local_tz);
-    temp += boost::posix_time::nanoseconds(nanos);
-    return TimestampValue(temp);
-  }
+      const Timezone& local_tz);
 
-  /// Return the corresponding timestamp in 'local_tz' time zone for the Unix time
-  /// specified in microseconds.
+  /// Same as FromUnixTime(), but expects the time in microseconds.
   static TimestampValue FromUnixTimeMicros(int64_t unix_time_micros,
       const Timezone& local_tz);
 
@@ -120,17 +113,15 @@ class TimestampValue {
   /// microseconds.
   static TimestampValue UtcFromUnixTimeMicros(int64_t unix_time_micros);
 
+  /// Return the corresponding timestamp in UTC for the Unix time specified in
+  /// milliseconds.
+  static TimestampValue UtcFromUnixTimeMillis(int64_t unix_time_millis);
+
   /// Returns a TimestampValue  in 'local_tz' time zone where the integer part of the
   /// specified 'unix_time' specifies the number of seconds (see above), and the
   /// fractional part is converted to nanoseconds and added to the resulting
   /// TimestampValue.
-  static TimestampValue FromSubsecondUnixTime(double unix_time,
-      const Timezone& local_tz) {
-    const time_t unix_time_whole = unix_time;
-    boost::posix_time::ptime temp = UnixTimeToPtime(unix_time_whole, local_tz);
-    temp += boost::posix_time::nanoseconds((unix_time - unix_time_whole) / ONE_BILLIONTH);
-    return TimestampValue(temp);
-  }
+  static TimestampValue FromSubsecondUnixTime(double unix_time, const Timezone& local_tz);
 
   /// Returns a TimestampValue converted from a TimestampVal. The caller must ensure
   /// the TimestampVal does not represent a NULL.
@@ -245,7 +236,7 @@ class TimestampValue {
   void LocalToUtc(const Timezone& local_tz);
 
   void set_date(const boost::gregorian::date d) { date_ = d; Validate(); }
-  void set_time(const boost::posix_time::time_duration t) { time_ = t; }
+  void set_time(const boost::posix_time::time_duration t) { time_ = t; Validate(); }
   const boost::gregorian::date& date() const { return date_; }
   const boost::posix_time::time_duration& time() const { return time_; }
 
@@ -286,6 +277,19 @@ class TimestampValue {
     return HashUtil::Hash(&date_, sizeof(date_), hash);
   }
 
+  /// Divides 'ticks' with 'GRANULARITY' (truncated towards negative infinity) and
+  /// sets 'ticks' to the remainder.
+  template <int64_t GRANULARITY>
+  inline static int64_t SplitTime(int64_t* ticks) {
+    int64_t result = *ticks / GRANULARITY;
+    *ticks %= GRANULARITY;
+    if (*ticks < 0) {
+      result--;
+      *ticks += GRANULARITY;
+    }
+    return result;
+  }
+
   static const char* LLVM_CLASS_NAME;
 
  private:
@@ -294,7 +298,6 @@ class TimestampValue {
   /// Used when converting a time with fractional seconds which are stored as in integer
   /// to a Unix time stored as a double.
   static const double ONE_BILLIONTH;
-
   /// Boost ptime leaves a gap in the structure, so we swap the order to make it
   /// 12 contiguous bytes.  We then must convert to and from the boost ptime data type.
   /// See IMP-87 for more information on why using ptime with the 4 byte gap is
@@ -313,24 +316,21 @@ class TimestampValue {
   }
 
   /// Sets both date and time to invalid if date is outside the valid range.
+  /// Time's validity is only checked in debug builds.
+  /// TODO: This could be also checked in release, but I am a bit afraid that it would
+  ///       affect performance and probably break some scenarios that are
+  ///       currently working more or less correctly.
   inline void Validate() {
     if (HasDate() && UNLIKELY(!IsValidDate(date_))) SetToInvalidDateTime();
+    else if (HasTime()) DCHECK(IsValidTime(time_));
   }
 
-  /// Return a ptime representation of the given Unix time (seconds since the Unix epoch).
-  /// The time zone of the resulting ptime is determined by
-  /// FLAGS_use_local_tz_for_unix_timestamp_conversions. If the flag is true, the value
-  /// will be in the 'local_tz' time zone. If the flag is false, the value will be in UTC.
-  static boost::posix_time::ptime UnixTimeToPtime(time_t unix_time,
-      const Timezone& local_tz);
-
-  /// Same as the above, but the time zone of the resulting ptime is always in the
-  /// 'local_tz' time zone.
-  static boost::posix_time::ptime UnixTimeToLocalPtime(time_t unix_time,
-      const Timezone& local_tz);
+  /// Converts 'unix_time' (in UTC seconds) to TimestampValue in timezone 'local_tz'.
+  static TimestampValue UnixTimeToLocal(time_t unix_time, const Timezone& local_tz);
 
-  /// Same as the above, but the time zone of the resulting ptime is always in UTC.
-  static boost::posix_time::ptime UnixTimeToUtcPtime(time_t unix_time);
+  /// Converts 'unix_time_ticks'/TICKS_PER_SEC seconds to TimestampValue.
+  template <int32_t TICKS_PER_SEC>
+  static TimestampValue UtcFromUnixTimeTicks(int64_t unix_time_ticks);
 };
 
 /// This function must be called 'hash_value' to be picked up by boost.

http://git-wip-us.apache.org/repos/asf/impala/blob/d301600a/be/src/runtime/timestamp-value.inline.h
----------------------------------------------------------------------
diff --git a/be/src/runtime/timestamp-value.inline.h b/be/src/runtime/timestamp-value.inline.h
index e049147..9c4b156 100644
--- a/be/src/runtime/timestamp-value.inline.h
+++ b/be/src/runtime/timestamp-value.inline.h
@@ -30,23 +30,53 @@
 
 namespace impala {
 
+template <int32_t TICKS_PER_SEC>
+inline TimestampValue TimestampValue::UtcFromUnixTimeTicks(int64_t unix_time_ticks) {
+  static const boost::gregorian::date EPOCH(1970,1,1);
+  int64_t days = SplitTime<(uint64_t)TICKS_PER_SEC*24*60*60>(&unix_time_ticks);
+
+  return TimestampValue(EPOCH + boost::gregorian::date_duration(days),
+      boost::posix_time::nanoseconds(unix_time_ticks*(NANOS_PER_SEC/TICKS_PER_SEC)));
+}
+
 inline TimestampValue TimestampValue::UtcFromUnixTimeMicros(int64_t unix_time_micros) {
-  int64_t ts_seconds = unix_time_micros / MICROS_PER_SEC;
-  int64_t micros_part = unix_time_micros - (ts_seconds * MICROS_PER_SEC);
-  boost::posix_time::ptime temp = UnixTimeToUtcPtime(ts_seconds);
-  temp += boost::posix_time::microseconds(micros_part);
-  return TimestampValue(temp);
+  return UtcFromUnixTimeTicks<MICROS_PER_SEC>(unix_time_micros);
 }
 
 inline TimestampValue TimestampValue::FromUnixTimeMicros(int64_t unix_time_micros,
     const Timezone& local_tz) {
-  int64_t ts_seconds = unix_time_micros / MICROS_PER_SEC;
-  int64_t micros_part = unix_time_micros - (ts_seconds * MICROS_PER_SEC);
-  boost::posix_time::ptime temp = UnixTimeToLocalPtime(ts_seconds, local_tz);
-  temp += boost::posix_time::microseconds(micros_part);
-  return TimestampValue(temp);
+  int64_t ts_seconds = SplitTime<MICROS_PER_SEC>(&unix_time_micros);
+  TimestampValue result = FromUnixTime(ts_seconds, local_tz);
+  if (result.HasDate()) result.time_ += boost::posix_time::microseconds(unix_time_micros);
+  return result;
 }
 
+inline TimestampValue TimestampValue::UtcFromUnixTimeMillis(int64_t unix_time_millis) {
+  return UtcFromUnixTimeTicks<MILLIS_PER_SEC>(unix_time_millis);
+}
+
+inline TimestampValue TimestampValue::FromSubsecondUnixTime(
+    double unix_time, const Timezone& local_tz) {
+  int64_t unix_time_whole = unix_time;
+  int64_t nanos = (unix_time - unix_time_whole) / ONE_BILLIONTH;
+  return FromUnixTimeNanos(unix_time_whole, nanos, local_tz);
+}
+
+inline TimestampValue TimestampValue::FromUnixTimeNanos(time_t unix_time, int64_t nanos,
+    const Timezone& local_tz) {
+  unix_time += SplitTime<NANOS_PER_SEC>(&nanos);
+  TimestampValue result = FromUnixTime(unix_time, local_tz);
+  // 'nanos' is guaranteed to be between [0,NANOS_PER_SEC) at this point, so the
+  // next addition cannot change the day or step to a different timezone.
+  if (result.HasDate()) {
+    DCHECK_GE(nanos, 0);
+    DCHECK_LT(nanos, NANOS_PER_SEC);
+    result.time_ += boost::posix_time::nanoseconds(nanos);
+  }
+  return result;
+}
+
+
 /// Interpret 'this' as a timestamp in UTC and convert to unix time.
 /// Returns false if the conversion failed ('unix_time' will be undefined), otherwise
 /// true.