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 2017/09/25 06:28:54 UTC

[2/3] incubator-impala git commit: IMPALA-5599: Fix for mis-use of TimestampValue

IMPALA-5599: Fix for mis-use of TimestampValue

The TimestampValue class is being used for non-database purposes
in many places, such as in log messages.

This change proposes to introduce APIs to convert Unix timetamps
into the corresponding date-time strings. We provide a series of
functions for different input time units, and also give the user
control over the precision of the output date-time string. APIs
are provided to format in UTC and local time zones. The new APIs
can be used to replace (or instead of) TimestampValue::ToString()
in those places where Unix timestamps are being converted to
strings for printing.

The current commit implements the APIs and replaces calls to
TimestampValue::ToString() in be/src/service.

A new unit test, time-test, has been added to the back-end tests.

Other uses of TimestampValue in be/src/service, such as to track
start and end times of queries, etc., will be analyzed and changed
as appropriate in a follow-up commit.

Change-Id: I9b0ae06f6d94968c87a199625aa3332b26988142
Reviewed-on: http://gerrit.cloudera.org:8080/8084
Reviewed-by: Dan Hecht <dh...@cloudera.com>
Tested-by: Impala Public Jenkins


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

Branch: refs/heads/master
Commit: d53f43b42c7c41f72d6a31505344f0393bdbe8a5
Parents: f53ce3b
Author: Zoram Thanga <zo...@cloudera.com>
Authored: Fri Sep 15 15:27:44 2017 -0700
Committer: Impala Public Jenkins <im...@gerrit.cloudera.org>
Committed: Sat Sep 23 02:46:01 2017 +0000

----------------------------------------------------------------------
 be/src/service/impala-server.cc |  19 +++---
 be/src/util/CMakeLists.txt      |   1 +
 be/src/util/time-test.cc        | 113 +++++++++++++++++++++++++++++++++++
 be/src/util/time.cc             | 108 +++++++++++++++++++++++++++++++++
 be/src/util/time.h              |  46 +++++++++++++-
 5 files changed, 274 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/d53f43b4/be/src/service/impala-server.cc
----------------------------------------------------------------------
diff --git a/be/src/service/impala-server.cc b/be/src/service/impala-server.cc
index 051cfaf..11de619 100644
--- a/be/src/service/impala-server.cc
+++ b/be/src/service/impala-server.cc
@@ -75,6 +75,7 @@
 #include "util/summary-util.h"
 #include "util/test-info.h"
 #include "util/uid-util.h"
+#include "util/time.h"
 
 #include "gen-cpp/Types_types.h"
 #include "gen-cpp/ImpalaService.h"
@@ -914,12 +915,10 @@ Status ImpalaServer::ExecuteInternal(
 
 void ImpalaServer::PrepareQueryContext(TQueryCtx* query_ctx) {
   query_ctx->__set_pid(getpid());
-  TimestampValue utc_timestamp = TimestampValue::UtcTime();
-  query_ctx->__set_utc_timestamp_string(utc_timestamp.ToString());
-  TimestampValue local_timestamp(utc_timestamp);
-  local_timestamp.UtcToLocal();
-  query_ctx->__set_now_string(local_timestamp.ToString());
-  query_ctx->__set_start_unix_millis(UnixMillis());
+  int64_t now_us = UnixMicros();
+  query_ctx->__set_utc_timestamp_string(ToUtcStringFromUnixMicros(now_us));
+  query_ctx->__set_now_string(ToStringFromUnixMicros(now_us));
+  query_ctx->__set_start_unix_millis(now_us / MICROS_PER_MILLI);
   query_ctx->__set_coord_address(ExecEnv::GetInstance()->backend_address());
 
   // Creating a random_generator every time is not free, but
@@ -1141,11 +1140,10 @@ Status ImpalaServer::GetSessionState(const TUniqueId& session_id,
     if (mark_active) {
       lock_guard<mutex> session_lock(i->second->lock);
       if (i->second->expired) {
-        int64_t last_time_s = i->second->last_accessed_ms / 1000;
         stringstream ss;
         ss << "Client session expired due to more than " << i->second->session_timeout
            << "s of inactivity (last activity was at: "
-           << TimestampValue::FromUnixTime(last_time_s).ToString() << ").";
+           << ToStringFromUnixMillis(i->second->last_accessed_ms) << ").";
         return Status(ss.str());
       }
       if (i->second->closed) return Status("Session is closed");
@@ -1843,7 +1841,7 @@ void ImpalaServer::RegisterSessionTimeout(int32_t session_timeout) {
         if (now - last_accessed_ms <= session_timeout_ms) continue;
         LOG(INFO) << "Expiring session: " << session_state.first << ", user:"
                   << session_state.second->connected_user << ", last active: "
-                  << TimestampValue::FromUnixTime(last_accessed_ms / 1000).ToString();
+                  << ToStringFromUnixMillis(last_accessed_ms);
         session_state.second->expired = true;
         ImpaladMetrics::NUM_SESSIONS_EXPIRED->Increment(1L);
         // Since expired is true, no more queries will be added to the inflight list.
@@ -1919,11 +1917,10 @@ void ImpalaServer::RegisterSessionTimeout(int32_t session_timeout) {
           }
         } else if (!query_state->is_active()) {
           // Otherwise time to expire this query
-          int64_t last_active_s = query_state->last_active_ms() / 1000;
           VLOG_QUERY
               << "Expiring query due to client inactivity: " << expiration_event->second
               << ", last activity was at: "
-              << TimestampValue::FromUnixTime(last_active_s).ToString();
+              << ToStringFromUnixMillis(query_state->last_active_ms());
           const string& err_msg = Substitute(
               "Query $0 expired due to client inactivity (timeout is $1)",
               PrintId(expiration_event->second),

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/d53f43b4/be/src/util/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/be/src/util/CMakeLists.txt b/be/src/util/CMakeLists.txt
index 3f18094..947ab78 100644
--- a/be/src/util/CMakeLists.txt
+++ b/be/src/util/CMakeLists.txt
@@ -133,5 +133,6 @@ ADD_BE_TEST(runtime-profile-test)
 ADD_BE_TEST(string-parser-test)
 ADD_BE_TEST(symbols-util-test)
 ADD_BE_TEST(thread-pool-test)
+ADD_BE_TEST(time-test)
 ADD_BE_TEST(uid-util-test)
 ADD_BE_TEST(webserver-test)

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/d53f43b4/be/src/util/time-test.cc
----------------------------------------------------------------------
diff --git a/be/src/util/time-test.cc b/be/src/util/time-test.cc
new file mode 100644
index 0000000..dde0d12
--- /dev/null
+++ b/be/src/util/time-test.cc
@@ -0,0 +1,113 @@
+// 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.
+
+#include <algorithm>
+#include <regex>
+#include <sstream>
+
+#include "util/time.h"
+#include "testutil/gtest-util.h"
+
+#include "common/names.h"
+
+using namespace std;
+
+namespace impala {
+  // Basic tests for the time formatting APIs in util/time.h.
+TEST(TimeTest, Basic) {
+  EXPECT_EQ("1970-01-01 00:00:00", ToUtcStringFromUnix(0));
+  EXPECT_EQ("1970-01-01 00:00:00.000", ToUtcStringFromUnixMillis(0));
+  EXPECT_EQ("1970-01-01 00:00:00.001", ToUtcStringFromUnixMillis(1));
+  EXPECT_EQ("1970-01-01 00:00:00", ToUtcStringFromUnixMillis(0, TimePrecision::Second));
+  EXPECT_EQ("1970-01-01 00:00:00.000000", ToUtcStringFromUnixMicros(0));
+  EXPECT_EQ("1970-01-01 00:00:00.000001", ToUtcStringFromUnixMicros(1));
+  EXPECT_EQ("1970-01-01 00:00:00.000", ToUtcStringFromUnixMicros(0,
+      TimePrecision::Millisecond));
+  EXPECT_EQ("1970-01-01 00:00:00", ToUtcStringFromUnixMicros(0, TimePrecision::Second));
+
+  EXPECT_EQ("1970-01-01 00:01:00", ToUtcStringFromUnix(60));
+  EXPECT_EQ("1970-01-01 01:00:00", ToUtcStringFromUnix(3600));
+  // Check we are handling negative times - time before the Unix epoch
+  EXPECT_EQ("1969-12-31 23:59:59", ToUtcStringFromUnix(-1));
+
+  // The earliest date-time that can be represented with the high-resolution
+  // chrono::system_clock is 1677-09-21 00:12:44 UTC, which is
+  // -9223372036854775808 (INT64_MIN) nanoseconds before the Unix epoch.
+  EXPECT_EQ("1677-09-21 00:12:44",
+      ToUtcStringFromUnix(INT64_MIN / NANOS_PER_SEC));
+  EXPECT_EQ("1677-09-21 00:12:44.854",
+      ToUtcStringFromUnixMillis(INT64_MIN / NANOS_PER_MICRO / MICROS_PER_MILLI));
+  EXPECT_EQ("1677-09-21 00:12:44.854775",
+      ToUtcStringFromUnixMicros(INT64_MIN / NANOS_PER_MICRO));
+
+  // The latest date-time that can be represented with the high-resoliution
+  // chrono::system_clock is 2262-04-11 23:47:16 UTC, which is
+  // 9223372036854775807 (INT64_MAX) nanoseconds since the Unix epoch.
+  EXPECT_EQ("2262-04-11 23:47:16",
+      ToUtcStringFromUnix(INT64_MAX / NANOS_PER_SEC));
+  EXPECT_EQ("2262-04-11 23:47:16.854",
+      ToUtcStringFromUnixMillis(INT64_MAX / NANOS_PER_MICRO / MICROS_PER_MILLI));
+  EXPECT_EQ("2262-04-11 23:47:16.854775",
+      ToUtcStringFromUnixMicros(INT64_MAX / NANOS_PER_MICRO));
+
+  EXPECT_EQ("1969-12-31 23:59:59.000", ToUtcStringFromUnixMillis(-1000));
+  EXPECT_EQ("1969-12-31 23:59:59.999", ToUtcStringFromUnixMillis(-1999));
+  EXPECT_EQ("1969-12-31 23:59:58.000", ToUtcStringFromUnixMillis(-2000));
+  EXPECT_EQ("1969-12-31 23:59:58.001", ToUtcStringFromUnixMillis(-2001));
+
+  EXPECT_EQ("1969-12-31 23:59:59.000000", ToUtcStringFromUnixMicros(-1000000));
+  EXPECT_EQ("1969-12-31 23:59:59.999999", ToUtcStringFromUnixMicros(-1999999));
+  EXPECT_EQ("1969-12-31 23:59:58.000001", ToUtcStringFromUnixMicros(-2000001));
+
+  // Unix time does not represent leap seconds. Test continuous roll-over of
+  // Unix time after 1998-12-31 23:59:59
+  EXPECT_EQ("1998-12-31 23:59:59", ToUtcStringFromUnix(915148799));
+  EXPECT_EQ("1999-01-01 00:00:00", ToUtcStringFromUnix(915148800));
+
+  // Check that for the same Unix time, our output string agrees with the output
+  // of strftime(3).
+  int64_t now_s = UnixMillis() / MILLIS_PER_SEC;
+  char now_buf[256];
+  strftime(now_buf, sizeof(now_buf), "%F %T", localtime(static_cast<time_t *>(&now_s)));
+  EXPECT_EQ(string(now_buf), ToStringFromUnix(now_s)) << "now_s=" << now_s;
+
+  strftime(now_buf, sizeof(now_buf), "%F %T", gmtime(static_cast<time_t *>(&now_s)));
+  EXPECT_EQ(string(now_buf), ToUtcStringFromUnix(now_s)) << "now_s=" << now_s;
+
+  // Check zero-padding of date-time string's fractional second part if input
+  // time's resolution is less than that requested by the caller.
+  smatch sm; // Place holder to be passed to regex_search() below.
+  string s1 = ToStringFromUnix(now_s, TimePrecision::Millisecond);
+  string s2 = ToUtcStringFromUnix(now_s, TimePrecision::Millisecond);
+  EXPECT_TRUE(regex_search(s1, sm, regex(R"(\.(000)$)")));
+  EXPECT_TRUE(regex_search(s2, sm, regex(R"(\.(000)$)")));
+
+  int64_t now_ms = UnixMillis();
+  s1 = ToStringFromUnixMillis(now_ms, TimePrecision::Microsecond);
+  s2 = ToUtcStringFromUnixMillis(now_ms, TimePrecision::Microsecond);
+  EXPECT_TRUE(regex_search(s1, sm, regex(R"(\.\d{3}(000)$)")));
+  EXPECT_TRUE(regex_search(s2, sm, regex(R"(\.\d{3}(000)$)")));
+
+  int64_t now_us = UnixMicros();
+  s1 = ToStringFromUnixMicros(now_us, TimePrecision::Nanosecond);
+  s2 = ToUtcStringFromUnixMicros(now_us, TimePrecision::Nanosecond);
+  EXPECT_TRUE(regex_search(s1, sm, regex(R"(\.\d{6}(000)$)")));
+  EXPECT_TRUE(regex_search(s2, sm, regex(R"(\.\d{6}(000)$)")));
+} // TEST
+} // namespace impala
+
+IMPALA_TEST_MAIN();

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/d53f43b4/be/src/util/time.cc
----------------------------------------------------------------------
diff --git a/be/src/util/time.cc b/be/src/util/time.cc
index e6530de..dd0587b 100644
--- a/be/src/util/time.cc
+++ b/be/src/util/time.cc
@@ -17,6 +17,9 @@
 
 #include <chrono>
 #include <thread>
+#include <iomanip>
+#include <sstream>
+#include <cstdlib>
 
 #include "util/time.h"
 
@@ -26,3 +29,108 @@ using namespace std;
 void impala::SleepForMs(const int64_t duration_ms) {
   this_thread::sleep_for(chrono::milliseconds(duration_ms));
 }
+
+// Convert the given time_point, 't', into a date-time string in the
+// UTC time zone if 'utc' is true, or the local time zone if it is false.
+// The returned string is of the form yyy-MM-dd HH::mm::SS.
+static string TimepointToString(const chrono::system_clock::time_point& t,
+    bool utc) {
+  char buf[256];
+  struct tm tmp;
+  auto input_time = chrono::system_clock::to_time_t(t);
+
+  // gcc 4.9 does not support C++14 get_time and put_time functions, so we're
+  // stuck with strftime() for now.
+  if (utc) {
+    strftime(buf, sizeof(buf), "%F %T", gmtime_r(&input_time, &tmp));
+  } else {
+    strftime(buf, sizeof(buf), "%F %T", localtime_r(&input_time, &tmp));
+  }
+  return string(buf);
+}
+
+// Format the sub-second part of the input time point object 't', at the
+// precision specified by 'p'. The returned string is meant to be appended to
+// the string returned by TimePointToString() above.
+// Note the use of abs(). This is to make sure we correctly format negative times,
+// i.e., times before the Unix epoch.
+static string FormatSubSecond(const chrono::system_clock::time_point& t,
+    TimePrecision p) {
+  stringstream ss;
+  auto frac = t.time_since_epoch();
+  if (p == TimePrecision::Millisecond) {
+    auto subsec = chrono::duration_cast<chrono::milliseconds>(frac) % MILLIS_PER_SEC;
+    ss << "." << std::setfill('0') << std::setw(3) << abs(subsec.count());
+  } else if (p == TimePrecision::Microsecond) {
+    auto subsec = chrono::duration_cast<chrono::microseconds>(frac) % MICROS_PER_SEC;
+    ss << "." << std::setfill('0') << std::setw(6) << abs(subsec.count());
+  } else if (p == TimePrecision::Nanosecond) {
+    auto subsec = chrono::duration_cast<chrono::nanoseconds>(frac) % NANOS_PER_SEC;
+    ss << "." << std::setfill('0') << std::setw(9) << abs(subsec.count());
+  } else {
+    // 1-second precision or unknown unit. Return empty string.
+    DCHECK_EQ(TimePrecision::Second, p);
+    ss << "";
+  }
+  return ss.str();
+}
+
+// Convert time point 't' into date-time string at precision 'p'.
+// Output string is in UTC time zone if 'utc' is true, else it is in the
+// local time zone.
+static string ToString(const chrono::system_clock::time_point& t, TimePrecision p,
+    bool utc)
+{
+  stringstream ss;
+  ss << TimepointToString(t, utc);
+  ss << FormatSubSecond(t, p);
+  return ss.str();
+}
+
+// Convenience function to convert Unix time, specified as seconds since
+// the Unix epoch, into a C++ time_point object.
+static chrono::system_clock::time_point TimepointFromUnix(int64_t s) {
+  return chrono::system_clock::time_point(chrono::seconds(s));
+}
+
+// Convenience function to convert Unix time, specified as milliseconds since
+// the Unix epoch, into a C++ time_point object.
+static chrono::system_clock::time_point TimepointFromUnixMillis(int64_t ms) {
+  return chrono::system_clock::time_point(chrono::milliseconds(ms));
+}
+
+// Convenience function to convert Unix time, specified as microseconds since
+// the Unix epoch, into a C++ time_point object.
+static chrono::system_clock::time_point TimepointFromUnixMicros(int64_t us) {
+  return chrono::system_clock::time_point(chrono::microseconds(us));
+}
+
+string impala::ToStringFromUnix(int64_t s, TimePrecision p) {
+  chrono::system_clock::time_point t = TimepointFromUnix(s);
+  return ToString(t, p, false);
+}
+
+string impala::ToUtcStringFromUnix(int64_t s, TimePrecision p) {
+  chrono::system_clock::time_point t = TimepointFromUnix(s);
+  return ToString(t, p, true);
+}
+
+string impala::ToStringFromUnixMillis(int64_t ms, TimePrecision p) {
+  chrono::system_clock::time_point t = TimepointFromUnixMillis(ms);
+  return ToString(t, p, false);
+}
+
+string impala::ToUtcStringFromUnixMillis(int64_t ms, TimePrecision p) {
+  chrono::system_clock::time_point t = TimepointFromUnixMillis(ms);
+  return ToString(t, p, true);
+}
+
+string impala::ToStringFromUnixMicros(int64_t us, TimePrecision p) {
+  chrono::system_clock::time_point t = TimepointFromUnixMicros(us);
+  return ToString(t, p, false);
+}
+
+string impala::ToUtcStringFromUnixMicros(int64_t us, TimePrecision p) {
+  chrono::system_clock::time_point t = TimepointFromUnixMicros(us);
+  return ToString(t, p, true);
+}

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/d53f43b4/be/src/util/time.h
----------------------------------------------------------------------
diff --git a/be/src/util/time.h b/be/src/util/time.h
index 3ec009a..5e9a4fd 100644
--- a/be/src/util/time.h
+++ b/be/src/util/time.h
@@ -20,6 +20,7 @@
 
 #include <stdint.h>
 #include <time.h>
+#include <string>
 
 #include "gutil/walltime.h"
 
@@ -48,7 +49,6 @@ inline int64_t MonotonicSeconds() {
   return GetMonoTimeMicros() / MICROS_PER_SEC;
 }
 
-
 /// Returns the number of milliseconds that have passed since the Unix epoch. This is
 /// affected by manual changes to the system clock but is more suitable for use across
 /// a cluster. For more accurate timings on the local host use the monotonic functions
@@ -57,9 +57,51 @@ inline int64_t UnixMillis() {
   return GetCurrentTimeMicros() / MICROS_PER_MILLI;
 }
 
+/// Returns the number of microseconds that have passed since the Unix epoch. This is
+/// affected by manual changes to the system clock but is more suitable for use across
+/// a cluster. For more accurate timings on the local host use the monotonic functions
+/// above.
+inline int64_t UnixMicros() {
+  return GetCurrentTimeMicros();
+}
+
 /// Sleeps the current thread for at least duration_ms milliseconds.
 void SleepForMs(const int64_t duration_ms);
 
-}
+// An enum class to use as precision argument for the ToString*() functions below
+enum TimePrecision {
+  Second,
+  Millisecond,
+  Microsecond,
+  Nanosecond
+};
+
+/// Converts the input Unix time, 's', specified in seconds since the Unix epoch, to a
+/// date-time string in the local time zone. The precision in the output date-time string
+/// is specified by the second argument, 'p'. The returned string is of the format
+/// yyyy-MM-dd HH:mm:SS[.ms[us[ns]]. It's worth noting that if the precision specified
+/// by 'p' is higher than that of the input timestamp, the part corresponding to
+/// 'p' in the fractional second part of the output will just be zero-padded.
+std::string ToStringFromUnix(int64_t s, TimePrecision p = TimePrecision::Second);
+
+/// Converts input seconds-since-epoch to date-time string in UTC time zone.
+std::string ToUtcStringFromUnix(int64_t s, TimePrecision p = TimePrecision::Second);
+
+/// Converts input milliseconds-since-epoch to date-time string in local time zone.
+std::string ToStringFromUnixMillis(int64_t ms,
+    TimePrecision p = TimePrecision::Millisecond);
+
+/// Converts input milliseconds-since-epoch to date-time string in UTC time zone.
+std::string ToUtcStringFromUnixMillis(int64_t ms,
+    TimePrecision p = TimePrecision::Millisecond);
+
+/// Converts input microseconds-since-epoch to date-time string in local time zone.
+std::string ToStringFromUnixMicros(int64_t us,
+    TimePrecision p = TimePrecision::Microsecond);
+
+/// Converts input microseconds-since-epoch to date-time string in UTC time zone.
+std::string ToUtcStringFromUnixMicros(int64_t us,
+    TimePrecision p = TimePrecision::Microsecond);
 
+} // namespace impala
 #endif