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 2016/05/23 15:40:32 UTC

[06/17] incubator-impala git commit: IMPALA-3569: handle errors in timezone db initialization

IMPALA-3569: handle errors in timezone db initialization

Change-Id: I6b4d5e6b992ea023f801edb7b487e57f39920c03
Reviewed-on: http://gerrit.cloudera.org:8080/3125
Reviewed-by: Dan Hecht <dh...@cloudera.com>
Tested-by: Internal 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/1ccfc45d
Tree: http://git-wip-us.apache.org/repos/asf/incubator-impala/tree/1ccfc45d
Diff: http://git-wip-us.apache.org/repos/asf/incubator-impala/diff/1ccfc45d

Branch: refs/heads/master
Commit: 1ccfc45d41acabd5fc9330a9a8b4bb50284c3e99
Parents: 51869ea
Author: Tim Armstrong <ta...@cloudera.com>
Authored: Wed May 18 12:05:53 2016 -0700
Committer: Tim Armstrong <ta...@cloudera.com>
Committed: Mon May 23 08:40:19 2016 -0700

----------------------------------------------------------------------
 be/src/common/init.cc                  |  2 ++
 be/src/exprs/expr-test.cc              |  1 +
 be/src/exprs/timestamp-functions-ir.cc | 44 +++--------------------------
 be/src/exprs/timestamp-functions.h     |  6 ++--
 be/src/exprs/timezone_db.cc            | 32 +++++++++++++++++++++
 be/src/runtime/exec-env.cc             |  2 --
 be/src/runtime/exec-env.h              |  2 --
 7 files changed, 42 insertions(+), 47 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/1ccfc45d/be/src/common/init.cc
----------------------------------------------------------------------
diff --git a/be/src/common/init.cc b/be/src/common/init.cc
index 9a47c1c..b116059 100644
--- a/be/src/common/init.cc
+++ b/be/src/common/init.cc
@@ -21,6 +21,7 @@
 #include "common/status.h"
 #include "exec/kudu-util.h"
 #include "exprs/expr.h"
+#include "exprs/timestamp-functions.h"
 #include "gutil/atomicops.h"
 #include "util/cpu-info.h"
 #include "util/debug-util.h"
@@ -187,6 +188,7 @@ void impala::InitCommonRuntime(int argc, char** argv, bool init_jvm,
   AtomicOps_x86CPUFeaturesInit();
   impala::InitThreading();
   impala::TimestampParser::Init();
+  ABORT_IF_ERROR(impala::TimezoneDatabase::Initialize());
   ABORT_IF_ERROR(impala::InitAuth(argv[0]));
 
   // Initialize maintenance_thread after InitGoogleLoggingSafe and InitThreading.

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/1ccfc45d/be/src/exprs/expr-test.cc
----------------------------------------------------------------------
diff --git a/be/src/exprs/expr-test.cc b/be/src/exprs/expr-test.cc
index 349946a..09f60cb 100644
--- a/be/src/exprs/expr-test.cc
+++ b/be/src/exprs/expr-test.cc
@@ -34,6 +34,7 @@
 #include "exprs/like-predicate.h"
 #include "exprs/literal.h"
 #include "exprs/null-literal.h"
+#include "exprs/timestamp-functions.h"
 #include "gen-cpp/Exprs_types.h"
 #include "gen-cpp/hive_metastore_types.h"
 #include "rpc/thrift-client.h"

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/1ccfc45d/be/src/exprs/timestamp-functions-ir.cc
----------------------------------------------------------------------
diff --git a/be/src/exprs/timestamp-functions-ir.cc b/be/src/exprs/timestamp-functions-ir.cc
index 9808419..08992d9 100644
--- a/be/src/exprs/timestamp-functions-ir.cc
+++ b/be/src/exprs/timestamp-functions-ir.cc
@@ -33,8 +33,6 @@
 #include "udf/udf-internal.h"
 #include "runtime/runtime-state.h"
 
-#define TIMEZONE_DATABASE "be/files/date_time_zonespec.csv"
-
 #include "common/names.h"
 
 using boost::algorithm::iequals;
@@ -770,8 +768,8 @@ TimestampVal TimestampFunctions::FromUtc(FunctionContext* context,
   if (!ts_value.HasDateOrTime()) return TimestampVal::null();
 
   const StringValue& tz_string_value = StringValue::FromStringVal(tz_string_val);
-  time_zone_ptr timezone =
-      TimezoneDatabase::FindTimezone(tz_string_value.DebugString(), ts_value);
+  time_zone_ptr timezone = TimezoneDatabase::FindTimezone(
+      string(tz_string_value.ptr, tz_string_value.len), ts_value);
   if (timezone == NULL) {
     // This should return null. Hive just ignores it.
     stringstream ss;
@@ -799,8 +797,8 @@ TimestampVal TimestampFunctions::ToUtc(FunctionContext* context,
   if (!ts_value.HasDateOrTime()) return TimestampVal::null();
 
   const StringValue& tz_string_value = StringValue::FromStringVal(tz_string_val);
-  time_zone_ptr timezone =
-      TimezoneDatabase::FindTimezone(tz_string_value.DebugString(), ts_value);
+  time_zone_ptr timezone = TimezoneDatabase::FindTimezone(
+      string(tz_string_value.ptr, tz_string_value.len), ts_value);
   // This should raise some sort of error or at least null. Hive Just ignores it.
   if (timezone == NULL) {
     stringstream ss;
@@ -817,40 +815,6 @@ TimestampVal TimestampFunctions::ToUtc(FunctionContext* context,
   return return_val;
 }
 
-TimezoneDatabase::TimezoneDatabase() {
-  // Create a temporary file and write the timezone information.  The boost
-  // interface only loads this format from a file.  We don't want to raise
-  // an error here since this is done when the backend is created and this
-  // information might not actually get used by any queries.
-  char filestr[] = "/tmp/impala.tzdb.XXXXXXX";
-  FILE* file;
-  int fd;
-  if ((fd = mkstemp(filestr)) == -1) {
-    LOG(ERROR) << "Could not create temporary timezone file: " << filestr;
-    return;
-  }
-  if ((file = fopen(filestr, "w")) == NULL) {
-    unlink(filestr);
-    close(fd);
-    LOG(ERROR) << "Could not open temporary timezone file: " << filestr;
-    return;
-  }
-  if (fputs(TIMEZONE_DATABASE_STR, file) == EOF) {
-    unlink(filestr);
-    close(fd);
-    fclose(file);
-    LOG(ERROR) << "Could not load temporary timezone file: " << filestr;
-    return;
-  }
-  fclose(file);
-  tz_database_.load_from_file(string(filestr));
-  tz_region_list_ = tz_database_.region_list();
-  unlink(filestr);
-  close(fd);
-}
-
-TimezoneDatabase::~TimezoneDatabase() { }
-
 time_zone_ptr TimezoneDatabase::FindTimezone(const string& tz, const TimestampValue& tv) {
   // The backing database does not capture some subtleties, there are special cases
   if ((tv.date().year() < 2011 || (tv.date().year() == 2011 && tv.date().month() < 4)) &&

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/1ccfc45d/be/src/exprs/timestamp-functions.h
----------------------------------------------------------------------
diff --git a/be/src/exprs/timestamp-functions.h b/be/src/exprs/timestamp-functions.h
index 3e08eb3..23c0d10 100644
--- a/be/src/exprs/timestamp-functions.h
+++ b/be/src/exprs/timestamp-functions.h
@@ -17,8 +17,8 @@
 #define IMPALA_EXPRS_TIMESTAMP_FUNCTIONS_H
 
 #include <boost/date_time/local_time/local_time.hpp>
-#include <string>
 
+#include "common/status.h"
 #include "udf/udf.h"
 
 using namespace impala_udf;
@@ -186,8 +186,8 @@ class TimestampFunctions {
 /// Functions to load and access the timestamp database.
 class TimezoneDatabase {
  public:
-   TimezoneDatabase();
-   ~TimezoneDatabase();
+  /// Set up the static timezone database.
+  static Status Initialize();
 
   /// Converts the name of a timezone to a boost timezone object.
   /// Some countries change their timezones, the tiemstamp is required to correctly

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/1ccfc45d/be/src/exprs/timezone_db.cc
----------------------------------------------------------------------
diff --git a/be/src/exprs/timezone_db.cc b/be/src/exprs/timezone_db.cc
index e7c4301..b45ec73 100644
--- a/be/src/exprs/timezone_db.cc
+++ b/be/src/exprs/timezone_db.cc
@@ -14,11 +14,14 @@
 
 #include "timestamp-functions.h"
 
+#include "gutil/strings/substitute.h"
+
 #include "common/names.h"
 
 using boost::local_time::tz_database;
 using boost::local_time::time_zone_ptr;
 using boost::local_time::posix_time_zone;
+using strings::Substitute;
 
 namespace impala {
 
@@ -653,5 +656,34 @@ const char* TimezoneDatabase::TIMEZONE_DATABASE_STR = "\"ID\",\"STD ABBR\",\"STD
 \"WET\",\"WET\",\"Western European Time\",\"WEST\",\"Western European Summer Time\",\"+00:00:00\",\"+01:00:00\",\"-1;0;3\",\"+01:00:00\",\"-1;0;10\",\"+01:00:00\"\n\
 \"Zulu\",\"UTC\",\"Coordinated Universal Time\",\"\",\"\",\"+00:00:00\",\"+00:00:00\",\"\",\"\",\"\",\"\"";
 
+Status TimezoneDatabase::Initialize() {
+  // Create a temporary file and write the timezone information.  The boost
+  // interface only loads this format from a file.  We don't want to raise
+  // an error here since this is done when the backend is created and this
+  // information might not actually get used by any queries.
+  char filestr[] = "/tmp/impala.tzdb.XXXXXXX";
+  FILE* file;
+  int fd;
+  if ((fd = mkstemp(filestr)) == -1) {
+    return Status(Substitute("Could not create temporary timezone file: $0", filestr));
+  }
+  if ((file = fopen(filestr, "w")) == NULL) {
+    unlink(filestr);
+    close(fd);
+    return Status(Substitute("Could not open temporary timezone file: $0", filestr));
+  }
+  if (fputs(TIMEZONE_DATABASE_STR, file) == EOF) {
+    unlink(filestr);
+    close(fd);
+    fclose(file);
+    return Status(Substitute("Could not load temporary timezone file: $0", filestr));
+  }
+  fclose(file);
+  tz_database_.load_from_file(string(filestr));
+  tz_region_list_ = tz_database_.region_list();
+  unlink(filestr);
+  close(fd);
+  return Status::OK();
+}
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/1ccfc45d/be/src/runtime/exec-env.cc
----------------------------------------------------------------------
diff --git a/be/src/runtime/exec-env.cc b/be/src/runtime/exec-env.cc
index 046c049..07378e4 100644
--- a/be/src/runtime/exec-env.cc
+++ b/be/src/runtime/exec-env.cc
@@ -164,7 +164,6 @@ ExecEnv::ExecEnv()
             FLAGS_coordinator_rpc_threads, numeric_limits<int32_t>::max())),
     async_rpc_pool_(new CallableThreadPool("rpc-pool", "async-rpc-sender", 8, 10000)),
     enable_webserver_(FLAGS_enable_webserver),
-    tz_database_(TimezoneDatabase()),
     is_fe_tests_(false),
     backend_address_(MakeNetworkAddress(FLAGS_hostname, FLAGS_be_port)),
     is_pseudo_distributed_llama_(false) {
@@ -216,7 +215,6 @@ ExecEnv::ExecEnv(const string& hostname, int backend_port, int subscriber_port,
             FLAGS_coordinator_rpc_threads, numeric_limits<int32_t>::max())),
     async_rpc_pool_(new CallableThreadPool("rpc-pool", "async-rpc-sender", 8, 10000)),
     enable_webserver_(FLAGS_enable_webserver && webserver_port > 0),
-    tz_database_(TimezoneDatabase()),
     is_fe_tests_(false),
     backend_address_(MakeNetworkAddress(FLAGS_hostname, FLAGS_be_port)),
     is_pseudo_distributed_llama_(false) {

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/1ccfc45d/be/src/runtime/exec-env.h
----------------------------------------------------------------------
diff --git a/be/src/runtime/exec-env.h b/be/src/runtime/exec-env.h
index c52c68b..e405bf8 100644
--- a/be/src/runtime/exec-env.h
+++ b/be/src/runtime/exec-env.h
@@ -21,7 +21,6 @@
 #include <boost/thread/thread.hpp>
 
 #include "common/status.h"
-#include "exprs/timestamp-functions.h"
 #include "runtime/backend-client.h"
 #include "util/cgroups-mgr.h"
 #include "util/hdfs-bulk-ops.h" // For declaration of HdfsOpThreadPool
@@ -154,7 +153,6 @@ class ExecEnv {
 
  private:
   static ExecEnv* exec_env_;
-  TimezoneDatabase tz_database_;
   bool is_fe_tests_;
 
   /// Address of the Impala backend server instance