You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@impala.apache.org by mj...@apache.org on 2016/08/10 16:05:32 UTC

[3/3] incubator-impala git commit: IMPALA-1112: Remove some unncessary code from cross-compilation

IMPALA-1112: Remove some unncessary code from cross-compilation

This change stops including some boost library header files
which pulls in other unnecessary boost library header files.
This reduces the amount of cross-compiled code which needs
to be materialized during codegen.

This change also removes some UDF's Prepare() and Close()
functions and UDF functions fromUtc(), toUtc() and uuid()
from cross-compilation as they won't benefit from it.

With this change, the bitcode module reduces from 2.12 MB to 1.86MB.

Change-Id: I543809c69da0b4085a0e299b91cd550b274c46af
Reviewed-on: http://gerrit.cloudera.org:8080/3793
Reviewed-by: Michael Ho <kw...@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/27b3b4d4
Tree: http://git-wip-us.apache.org/repos/asf/incubator-impala/tree/27b3b4d4
Diff: http://git-wip-us.apache.org/repos/asf/incubator-impala/diff/27b3b4d4

Branch: refs/heads/master
Commit: 27b3b4d470c6309729faba0b3872abf85f716f78
Parents: 9ca292a
Author: Michael Ho <kw...@cloudera.com>
Authored: Wed Jul 27 10:11:55 2016 -0700
Committer: Internal Jenkins <cl...@gerrit.cloudera.org>
Committed: Wed Aug 10 10:07:16 2016 +0000

----------------------------------------------------------------------
 be/src/benchmarks/parse-timestamp-benchmark.cc |   2 +
 be/src/common/init.cc                          |   2 +-
 be/src/exprs/CMakeLists.txt                    |   2 +
 be/src/exprs/expr.cc                           |   1 +
 be/src/exprs/timestamp-functions-ir.cc         | 153 +----------------
 be/src/exprs/timestamp-functions.cc            | 176 ++++++++++++++++++++
 be/src/exprs/timestamp-functions.h             |  25 +--
 be/src/exprs/timezone_db.cc                    |   6 +-
 be/src/exprs/timezone_db.h                     |  54 ++++++
 be/src/exprs/utility-functions-ir.cc           |  41 +----
 be/src/exprs/utility-functions.cc              |  64 +++++++
 be/src/exprs/utility-functions.h               |   3 +
 be/src/runtime/mem-tracker.cc                  |   1 +
 be/src/runtime/timestamp-value.cc              |   2 +
 be/src/runtime/timestamp-value.h               |   8 +-
 be/src/service/impala-http-handler.cc          |   1 +
 be/src/statestore/statestore.cc                |   1 +
 be/src/util/debug-util.cc                      |   1 +
 18 files changed, 331 insertions(+), 212 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/27b3b4d4/be/src/benchmarks/parse-timestamp-benchmark.cc
----------------------------------------------------------------------
diff --git a/be/src/benchmarks/parse-timestamp-benchmark.cc b/be/src/benchmarks/parse-timestamp-benchmark.cc
index 8824d85..68f4eac 100644
--- a/be/src/benchmarks/parse-timestamp-benchmark.cc
+++ b/be/src/benchmarks/parse-timestamp-benchmark.cc
@@ -20,11 +20,13 @@
 #include <iostream>
 #include <vector>
 #include <sstream>
+#include <boost/date_time/posix_time/posix_time.hpp>
 #include "runtime/string-value.h"
 #include "runtime/timestamp-parse-util.h"
 #include "runtime/timestamp-value.h"
 #include "util/benchmark.h"
 #include "util/cpu-info.h"
+
 #include "common/names.h"
 
 namespace gregorian = boost::gregorian;

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/27b3b4d4/be/src/common/init.cc
----------------------------------------------------------------------
diff --git a/be/src/common/init.cc b/be/src/common/init.cc
index 97d601b..3b49f44 100644
--- a/be/src/common/init.cc
+++ b/be/src/common/init.cc
@@ -24,7 +24,7 @@
 #include "common/status.h"
 #include "exec/kudu-util.h"
 #include "exprs/expr.h"
-#include "exprs/timestamp-functions.h"
+#include "exprs/timezone_db.h"
 #include "gutil/atomicops.h"
 #include "util/cpu-info.h"
 #include "util/debug-util.h"

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/27b3b4d4/be/src/exprs/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/be/src/exprs/CMakeLists.txt b/be/src/exprs/CMakeLists.txt
index 27062ff..feec660 100644
--- a/be/src/exprs/CMakeLists.txt
+++ b/be/src/exprs/CMakeLists.txt
@@ -50,11 +50,13 @@ add_library(Exprs
   operators-ir.cc
   slot-ref.cc
   string-functions-ir.cc
+  timestamp-functions.cc
   timestamp-functions-ir.cc
   timezone_db.cc
   tuple-is-null-predicate.cc
   scalar-fn-call.cc
   udf-builtins-ir.cc
+  utility-functions.cc
   utility-functions-ir.cc
 )
 add_dependencies(Exprs thrift-deps gen_ir_descriptions)

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/27b3b4d4/be/src/exprs/expr.cc
----------------------------------------------------------------------
diff --git a/be/src/exprs/expr.cc b/be/src/exprs/expr.cc
index 71758ae..0e3610d 100644
--- a/be/src/exprs/expr.cc
+++ b/be/src/exprs/expr.cc
@@ -511,6 +511,7 @@ void Expr::InitBuiltinsDummy() {
   MathFunctions::Pi(NULL);
   StringFunctions::Length(NULL, StringVal::null());
   TimestampFunctions::Year(NULL, TimestampVal::null());
+  TimestampFunctions::UnixAndFromUnixPrepare(NULL, FunctionContext::FRAGMENT_LOCAL);
   UdfBuiltins::Pi(NULL);
   UtilityFunctions::Pid(NULL);
 }

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/27b3b4d4/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 33ace86..f250d5a 100644
--- a/be/src/exprs/timestamp-functions-ir.cc
+++ b/be/src/exprs/timestamp-functions-ir.cc
@@ -15,34 +15,25 @@
 // specific language governing permissions and limitations
 // under the License.
 
-#include <boost/date_time/posix_time/posix_time.hpp>
-#include <boost/date_time/gregorian/gregorian.hpp>
-#include <boost/date_time/time_zone_base.hpp>
-#include <boost/date_time/local_time/local_time.hpp>
-#include <boost/algorithm/string.hpp>
+#include "exprs/timestamp-functions.h"
+
+#include <boost/date_time/compiler_config.hpp>
+#include <boost/date_time/posix_time/posix_time_types.hpp>
+#include <boost/date_time/gregorian/gregorian_types.hpp>
 #include <ctime>
 #include <gutil/strings/substitute.h>
 
-#include "exprs/timestamp-functions.h"
-#include "exprs/expr.h"
 #include "exprs/anyval-util.h"
-
-#include "runtime/tuple-row.h"
+#include "runtime/string-value.inline.h"
 #include "runtime/timestamp-parse-util.h"
 #include "runtime/timestamp-value.h"
-#include "util/path-builder.h"
-#include "runtime/string-value.inline.h"
 #include "udf/udf.h"
 #include "udf/udf-internal.h"
-#include "runtime/runtime-state.h"
 
 #include "common/names.h"
 
-using boost::algorithm::iequals;
 using boost::gregorian::greg_month;
 using boost::gregorian::min_date_time;
-using boost::local_time::local_date_time;
-using boost::local_time::time_zone_ptr;
 using boost::posix_time::not_a_date_time;
 using boost::posix_time::ptime;
 using namespace impala_udf;
@@ -100,42 +91,6 @@ const int64_t TimestampFunctions::MAX_MILLI_INTERVAL =
 const int64_t TimestampFunctions::MAX_MICRO_INTERVAL =
     TimestampFunctions::MAX_MILLI_INTERVAL * 1000;
 
-void TimestampFunctions::UnixAndFromUnixPrepare(FunctionContext* context,
-    FunctionContext::FunctionStateScope scope) {
-  if (scope != FunctionContext::THREAD_LOCAL) return;
-  DateTimeFormatContext* dt_ctx = NULL;
-  if (context->IsArgConstant(1)) {
-    StringVal fmt_val = *reinterpret_cast<StringVal*>(context->GetConstantArg(1));
-    const StringValue& fmt_ref = StringValue::FromStringVal(fmt_val);
-    if (fmt_val.is_null || fmt_ref.len == 0) {
-      TimestampFunctions::ReportBadFormat(context, fmt_val, true);
-      return;
-    }
-    dt_ctx = new DateTimeFormatContext(fmt_ref.ptr, fmt_ref.len);
-    bool parse_result = TimestampParser::ParseFormatTokens(dt_ctx);
-    if (!parse_result) {
-      delete dt_ctx;
-      TimestampFunctions::ReportBadFormat(context, fmt_val, true);
-      return;
-    }
-  } else {
-    // If our format string is constant, then we benefit from it only being parsed once in
-    // the code above. If it's not constant, then we can reuse a context by resetting it.
-    // This is much cheaper vs alloc/dealloc'ing a context for each evaluation.
-    dt_ctx = new DateTimeFormatContext();
-  }
-  context->SetFunctionState(scope, dt_ctx);
-}
-
-void TimestampFunctions::UnixAndFromUnixClose(FunctionContext* context,
-    FunctionContext::FunctionStateScope scope) {
-  if (scope == FunctionContext::THREAD_LOCAL) {
-    DateTimeFormatContext* dt_ctx =
-        reinterpret_cast<DateTimeFormatContext*>(context->GetFunctionState(scope));
-    delete dt_ctx;
-  }
-}
-
 StringVal TimestampFunctions::StringValFromTimestamp(FunctionContext* context,
     const TimestampValue& tv, const StringVal& fmt) {
   void* state = context->GetFunctionState(FunctionContext::THREAD_LOCAL);
@@ -370,7 +325,7 @@ StringVal TimestampFunctions::ToDate(FunctionContext* context,
     const TimestampVal& ts_val) {
   if (ts_val.is_null) return StringVal::null();
   const TimestampValue ts_value = TimestampValue::FromTimestampVal(ts_val);
-  string result = to_iso_extended_string(ts_value.date());
+  string result = ToIsoExtendedString(ts_value);
   return AnyValUtil::FromString(context, result);
 }
 
@@ -793,100 +748,6 @@ IntVal TimestampFunctions::DateDiff(FunctionContext* context,
   return IntVal((ts_value1.date() - ts_value2.date()).days());
 }
 
-// This function uses inline asm functions, which we believe to be from the boost library.
-// Inline asm is not currently supported by JIT, so this function should always be run in
-// the interpreted mode. This is handled in ScalarFnCall::GetUdf().
-TimestampVal TimestampFunctions::FromUtc(FunctionContext* context,
-    const TimestampVal& ts_val, const StringVal& tz_string_val) {
-  if (ts_val.is_null || tz_string_val.is_null) return TimestampVal::null();
-  const TimestampValue& ts_value = TimestampValue::FromTimestampVal(ts_val);
-  if (!ts_value.HasDateOrTime()) return TimestampVal::null();
-
-  const StringValue& tz_string_value = StringValue::FromStringVal(tz_string_val);
-  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;
-    ss << "Unknown timezone '" << tz_string_value << "'" << endl;
-    context->AddWarning(ss.str().c_str());
-    return ts_val;
-  }
-
-  ptime temp;
-  ts_value.ToPtime(&temp);
-  local_date_time lt(temp, timezone);
-  TimestampValue return_value = lt.local_time();
-  TimestampVal return_val;
-  return_value.ToTimestampVal(&return_val);
-  return return_val;
-}
-
-// This function uses inline asm functions, which we believe to be from the boost library.
-// Inline asm is not currently supported by JIT, so this function should always be run in
-// the interpreted mode. This is handled in ScalarFnCall::GetUdf().
-TimestampVal TimestampFunctions::ToUtc(FunctionContext* context,
-    const TimestampVal& ts_val, const StringVal& tz_string_val) {
-  if (ts_val.is_null || tz_string_val.is_null) return TimestampVal::null();
-  const TimestampValue& ts_value = TimestampValue::FromTimestampVal(ts_val);
-  if (!ts_value.HasDateOrTime()) return TimestampVal::null();
-
-  const StringValue& tz_string_value = StringValue::FromStringVal(tz_string_val);
-  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;
-    ss << "Unknown timezone '" << tz_string_value << "'" << endl;
-    context->AddWarning(ss.str().c_str());
-    return ts_val;
-  }
-
-  local_date_time lt(ts_value.date(), ts_value.time(),
-      timezone, local_date_time::NOT_DATE_TIME_ON_ERROR);
-  TimestampValue return_value(lt.utc_time());
-  TimestampVal return_val;
-  return_value.ToTimestampVal(&return_val);
-  return return_val;
-}
-
-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)) &&
-      (iequals("Europe/Moscow", tz) || iequals("Moscow", tz) || iequals("MSK", tz))) {
-    // We transition in pre April 2011 from using the tz_database_ to a custom rule
-    // Russia stopped using daylight savings in 2011, the tz_database_ is
-    // set up assuming Russia uses daylight saving every year.
-    // Sun, Mar 27, 2:00AM Moscow clocks moved forward +1 hour (a total of GMT +4)
-    // Specifically,
-    // UTC Time 26 Mar 2011 22:59:59 +0000 ===> Sun Mar 27 01:59:59 MSK 2011
-    // UTC Time 26 Mar 2011 23:00:00 +0000 ===> Sun Mar 27 03:00:00 MSK 2011
-    // This means in 2011, The database rule will apply DST starting March 26 2011.
-    // This will be a correct +4 offset, and the database rule can apply until
-    // Oct 31 when tz_database_ will incorrectly attempt to turn clocks backwards 1 hour.
-    return TIMEZONE_MSK_PRE_2011_DST;
-  }
-
-  // See if they specified a zone id
-  time_zone_ptr tzp = tz_database_.time_zone_from_region(tz);
-  if (tzp != NULL) return tzp;
-
-  for (vector<string>::const_iterator iter = tz_region_list_.begin();
-       iter != tz_region_list_.end(); ++iter) {
-    time_zone_ptr tzp = tz_database_.time_zone_from_region(*iter);
-    DCHECK(tzp != NULL);
-    if (tzp->dst_zone_abbrev() == tz)
-      return tzp;
-    if (tzp->std_zone_abbrev() == tz)
-      return tzp;
-    if (tzp->dst_zone_name() == tz)
-      return tzp;
-    if (tzp->std_zone_name() == tz)
-      return tzp;
-  }
-  return time_zone_ptr();
-}
-
 // Explicit template instantiation is required for proper linking. These functions
 // are only indirectly called via a function pointer provided by the opcode registry
 // which does not trigger implicit template instantiation.

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/27b3b4d4/be/src/exprs/timestamp-functions.cc
----------------------------------------------------------------------
diff --git a/be/src/exprs/timestamp-functions.cc b/be/src/exprs/timestamp-functions.cc
new file mode 100644
index 0000000..de01104
--- /dev/null
+++ b/be/src/exprs/timestamp-functions.cc
@@ -0,0 +1,176 @@
+// 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 "exprs/timestamp-functions.h"
+
+#include <boost/algorithm/string.hpp>
+#include <boost/date_time/posix_time/posix_time.hpp>
+#include <ctime>
+
+#include "exprs/anyval-util.h"
+#include "exprs/timezone_db.h"
+#include "runtime/string-value.inline.h"
+#include "runtime/timestamp-parse-util.h"
+#include "runtime/timestamp-value.h"
+#include "udf/udf.h"
+#include "udf/udf-internal.h"
+
+#include "common/names.h"
+
+using boost::algorithm::iequals;
+using boost::local_time::local_date_time;
+using boost::local_time::time_zone_ptr;
+using boost::posix_time::ptime;
+using boost::posix_time::to_iso_extended_string;
+
+namespace impala {
+
+// This function is not cross-compiled to avoid including unnecessary boost library's
+// header files which bring in a bunch of unused code and global variables and increase
+// the codegen time. boost::posix_time::to_iso_extended_string() is large enough that
+// it won't benefit much from inlining.
+string TimestampFunctions::ToIsoExtendedString(const TimestampValue& ts_value) {
+  return to_iso_extended_string(ts_value.date());
+}
+
+// This function uses inline asm functions, which we believe to be from the boost library.
+// Inline asm is not currently supported by JIT, so this function should always be run in
+// the interpreted mode. This is handled in ScalarFnCall::GetUdf().
+TimestampVal TimestampFunctions::FromUtc(FunctionContext* context,
+    const TimestampVal& ts_val, const StringVal& tz_string_val) {
+  if (ts_val.is_null || tz_string_val.is_null) return TimestampVal::null();
+  const TimestampValue& ts_value = TimestampValue::FromTimestampVal(ts_val);
+  if (!ts_value.HasDateOrTime()) return TimestampVal::null();
+
+  const StringValue& tz_string_value = StringValue::FromStringVal(tz_string_val);
+  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;
+    ss << "Unknown timezone '" << tz_string_value << "'" << endl;
+    context->AddWarning(ss.str().c_str());
+    return ts_val;
+  }
+
+  ptime temp;
+  ts_value.ToPtime(&temp);
+  local_date_time lt(temp, timezone);
+  TimestampValue return_value = lt.local_time();
+  TimestampVal return_val;
+  return_value.ToTimestampVal(&return_val);
+  return return_val;
+}
+
+// This function uses inline asm functions, which we believe to be from the boost library.
+// Inline asm is not currently supported by JIT, so this function should always be run in
+// the interpreted mode. This is handled in ScalarFnCall::GetUdf().
+TimestampVal TimestampFunctions::ToUtc(FunctionContext* context,
+    const TimestampVal& ts_val, const StringVal& tz_string_val) {
+  if (ts_val.is_null || tz_string_val.is_null) return TimestampVal::null();
+  const TimestampValue& ts_value = TimestampValue::FromTimestampVal(ts_val);
+  if (!ts_value.HasDateOrTime()) return TimestampVal::null();
+
+  const StringValue& tz_string_value = StringValue::FromStringVal(tz_string_val);
+  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;
+    ss << "Unknown timezone '" << tz_string_value << "'" << endl;
+    context->AddWarning(ss.str().c_str());
+    return ts_val;
+  }
+
+  local_date_time lt(ts_value.date(), ts_value.time(),
+      timezone, local_date_time::NOT_DATE_TIME_ON_ERROR);
+  TimestampValue return_value(lt.utc_time());
+  TimestampVal return_val;
+  return_value.ToTimestampVal(&return_val);
+  return return_val;
+}
+
+void TimestampFunctions::UnixAndFromUnixPrepare(FunctionContext* context,
+    FunctionContext::FunctionStateScope scope) {
+  if (scope != FunctionContext::THREAD_LOCAL) return;
+  DateTimeFormatContext* dt_ctx = NULL;
+  if (context->IsArgConstant(1)) {
+    StringVal fmt_val = *reinterpret_cast<StringVal*>(context->GetConstantArg(1));
+    const StringValue& fmt_ref = StringValue::FromStringVal(fmt_val);
+    if (fmt_val.is_null || fmt_ref.len == 0) {
+      TimestampFunctions::ReportBadFormat(context, fmt_val, true);
+      return;
+    }
+    dt_ctx = new DateTimeFormatContext(fmt_ref.ptr, fmt_ref.len);
+    bool parse_result = TimestampParser::ParseFormatTokens(dt_ctx);
+    if (!parse_result) {
+      delete dt_ctx;
+      TimestampFunctions::ReportBadFormat(context, fmt_val, true);
+      return;
+    }
+  } else {
+    // If our format string is constant, then we benefit from it only being parsed once in
+    // the code above. If it's not constant, then we can reuse a context by resetting it.
+    // This is much cheaper vs alloc/dealloc'ing a context for each evaluation.
+    dt_ctx = new DateTimeFormatContext();
+  }
+  context->SetFunctionState(scope, dt_ctx);
+}
+
+void TimestampFunctions::UnixAndFromUnixClose(FunctionContext* context,
+    FunctionContext::FunctionStateScope scope) {
+  if (scope == FunctionContext::THREAD_LOCAL) {
+    DateTimeFormatContext* dt_ctx =
+        reinterpret_cast<DateTimeFormatContext*>(context->GetFunctionState(scope));
+    delete dt_ctx;
+  }
+}
+
+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)) &&
+      (iequals("Europe/Moscow", tz) || iequals("Moscow", tz) || iequals("MSK", tz))) {
+    // We transition in pre April 2011 from using the tz_database_ to a custom rule
+    // Russia stopped using daylight savings in 2011, the tz_database_ is
+    // set up assuming Russia uses daylight saving every year.
+    // Sun, Mar 27, 2:00AM Moscow clocks moved forward +1 hour (a total of GMT +4)
+    // Specifically,
+    // UTC Time 26 Mar 2011 22:59:59 +0000 ===> Sun Mar 27 01:59:59 MSK 2011
+    // UTC Time 26 Mar 2011 23:00:00 +0000 ===> Sun Mar 27 03:00:00 MSK 2011
+    // This means in 2011, The database rule will apply DST starting March 26 2011.
+    // This will be a correct +4 offset, and the database rule can apply until
+    // Oct 31 when tz_database_ will incorrectly attempt to turn clocks backwards 1 hour.
+    return TIMEZONE_MSK_PRE_2011_DST;
+  }
+
+  // See if they specified a zone id
+  time_zone_ptr tzp = tz_database_.time_zone_from_region(tz);
+  if (tzp != NULL) return tzp;
+
+  for (vector<string>::const_iterator iter = tz_region_list_.begin();
+       iter != tz_region_list_.end(); ++iter) {
+    time_zone_ptr tzp = tz_database_.time_zone_from_region(*iter);
+    DCHECK(tzp != NULL);
+    if (tzp->dst_zone_abbrev() == tz) return tzp;
+    if (tzp->std_zone_abbrev() == tz) return tzp;
+    if (tzp->dst_zone_name() == tz) return tzp;
+    if (tzp->std_zone_name() == tz) return tzp;
+  }
+  return time_zone_ptr();
+}
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/27b3b4d4/be/src/exprs/timestamp-functions.h
----------------------------------------------------------------------
diff --git a/be/src/exprs/timestamp-functions.h b/be/src/exprs/timestamp-functions.h
index 948a6b4..8abe7d7 100644
--- a/be/src/exprs/timestamp-functions.h
+++ b/be/src/exprs/timestamp-functions.h
@@ -19,8 +19,6 @@
 #ifndef IMPALA_EXPRS_TIMESTAMP_FUNCTIONS_H
 #define IMPALA_EXPRS_TIMESTAMP_FUNCTIONS_H
 
-#include <boost/date_time/local_time/local_time.hpp>
-
 #include "common/status.h"
 #include "udf/udf.h"
 
@@ -187,6 +185,8 @@ class TimestampFunctions {
       const StringVal& format, bool is_error);
 
  private:
+  static std::string ToIsoExtendedString(const TimestampValue& ts_value);
+
   /// Static result values for DayName() function.
   static const char* MONDAY;
   static const char* TUESDAY;
@@ -197,27 +197,6 @@ class TimestampFunctions {
   static const char* SUNDAY;
 };
 
-/// Functions to load and access the timestamp database.
-class TimezoneDatabase {
- public:
-  /// 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
-  /// determine the timezone information.
-  static boost::local_time::time_zone_ptr FindTimezone(const std::string& tz,
-      const TimestampValue& tv);
-
-  /// Moscow Timezone No Daylight Savings Time (GMT+4), for use after March 2011
-  static const boost::local_time::time_zone_ptr TIMEZONE_MSK_PRE_2011_DST;
-
- private:
-  static const char* TIMEZONE_DATABASE_STR;
-  static boost::local_time::tz_database tz_database_;
-  static std::vector<std::string> tz_region_list_;
-};
-
 } // namespace impala
 
 #endif

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/27b3b4d4/be/src/exprs/timezone_db.cc
----------------------------------------------------------------------
diff --git a/be/src/exprs/timezone_db.cc b/be/src/exprs/timezone_db.cc
index 36d25c7..7fed7c2 100644
--- a/be/src/exprs/timezone_db.cc
+++ b/be/src/exprs/timezone_db.cc
@@ -15,8 +15,12 @@
 // specific language governing permissions and limitations
 // under the License.
 
-#include "timestamp-functions.h"
+#include "exprs/timezone_db.h"
 
+#include <boost/date_time/compiler_config.hpp>
+#include <boost/date_time/local_time/posix_time_zone.hpp>
+
+#include "exprs/timestamp-functions.h"
 #include "gutil/strings/substitute.h"
 
 #include "common/names.h"

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/27b3b4d4/be/src/exprs/timezone_db.h
----------------------------------------------------------------------
diff --git a/be/src/exprs/timezone_db.h b/be/src/exprs/timezone_db.h
new file mode 100644
index 0000000..76d08d5
--- /dev/null
+++ b/be/src/exprs/timezone_db.h
@@ -0,0 +1,54 @@
+// 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.
+
+
+#ifndef IMPALA_EXPRS_TIMEZONE_DB_H
+#define IMPALA_EXPRS_TIMEZONE_DB_H
+
+#include <boost/date_time/compiler_config.hpp>
+#include <boost/date_time/local_time/local_time_types.hpp>
+#include <boost/date_time/local_time/tz_database.hpp>
+
+#include "common/status.h"
+#include "runtime/timestamp-value.h"
+
+namespace impala {
+
+/// Functions to load and access the timestamp database.
+class TimezoneDatabase {
+ public:
+  /// 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
+  /// determine the timezone information.
+  static boost::local_time::time_zone_ptr FindTimezone(const std::string& tz,
+      const TimestampValue& tv);
+
+  /// Moscow Timezone No Daylight Savings Time (GMT+4), for use after March 2011
+  static const boost::local_time::time_zone_ptr TIMEZONE_MSK_PRE_2011_DST;
+
+ private:
+  static const char* TIMEZONE_DATABASE_STR;
+  static boost::local_time::tz_database tz_database_;
+  static std::vector<std::string> tz_region_list_;
+};
+
+} // namespace impala
+
+#endif

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/27b3b4d4/be/src/exprs/utility-functions-ir.cc
----------------------------------------------------------------------
diff --git a/be/src/exprs/utility-functions-ir.cc b/be/src/exprs/utility-functions-ir.cc
index d9d361e..ab22bb7 100644
--- a/be/src/exprs/utility-functions-ir.cc
+++ b/be/src/exprs/utility-functions-ir.cc
@@ -16,11 +16,7 @@
 // under the License.
 
 #include "exprs/utility-functions.h"
-
 #include <gutil/strings/substitute.h>
-#include <boost/uuid/uuid_generators.hpp>
-#include <boost/uuid/uuid.hpp>
-#include <boost/uuid/uuid_io.hpp>
 
 #include "exprs/anyval-util.h"
 #include "runtime/runtime-state.h"
@@ -104,6 +100,10 @@ IntVal UtilityFunctions::Pid(FunctionContext* ctx) {
   return IntVal(pid);
 }
 
+StringVal UtilityFunctions::Uuid(FunctionContext* ctx) {
+  return GenUuid(ctx);
+}
+
 BooleanVal UtilityFunctions::Sleep(FunctionContext* ctx, const IntVal& milliseconds ) {
   if (milliseconds.is_null) return BooleanVal::null();
   SleepForMs(milliseconds.val);
@@ -117,39 +117,6 @@ StringVal UtilityFunctions::CurrentDatabase(FunctionContext* ctx) {
   return (database.len > 0) ? database : StringVal::null();
 }
 
-void UtilityFunctions::UuidPrepare(FunctionContext* ctx,
-    FunctionContext::FunctionStateScope scope) {
-  if (scope == FunctionContext::THREAD_LOCAL) {
-    if (ctx->GetFunctionState(FunctionContext::THREAD_LOCAL) == NULL) {
-      boost::uuids::random_generator* uuid_gen =
-          new boost::uuids::random_generator;
-      ctx->SetFunctionState(scope, uuid_gen);
-    }
-  }
-}
-
-StringVal UtilityFunctions::Uuid(FunctionContext* ctx) {
-  void* uuid_gen = ctx->GetFunctionState(FunctionContext::THREAD_LOCAL);
-  DCHECK(uuid_gen != NULL);
-  boost::uuids::uuid uuid_value =
-      (*reinterpret_cast<boost::uuids::random_generator*>(uuid_gen))();
-  const std::string cxx_string = boost::uuids::to_string(uuid_value);
-  return StringVal::CopyFrom(ctx,
-      reinterpret_cast<const uint8_t*>(cxx_string.c_str()),
-      cxx_string.length());
-}
-
-void UtilityFunctions::UuidClose(FunctionContext* ctx,
-    FunctionContext::FunctionStateScope scope){
-  if (scope == FunctionContext::THREAD_LOCAL) {
-    boost::uuids::random_generator* uuid_gen =
-        reinterpret_cast<boost::uuids::random_generator*>(
-            ctx->GetFunctionState(FunctionContext::THREAD_LOCAL));
-    DCHECK(uuid_gen != NULL);
-    delete uuid_gen;
-  }
-}
-
 template<typename T>
 StringVal UtilityFunctions::TypeOf(FunctionContext* ctx, const T& /*input_val*/) {
   FunctionContext::TypeDesc type_desc = *(ctx->GetArgType(0));

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/27b3b4d4/be/src/exprs/utility-functions.cc
----------------------------------------------------------------------
diff --git a/be/src/exprs/utility-functions.cc b/be/src/exprs/utility-functions.cc
new file mode 100644
index 0000000..6159d6d
--- /dev/null
+++ b/be/src/exprs/utility-functions.cc
@@ -0,0 +1,64 @@
+// 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 "exprs/utility-functions.h"
+
+#include <boost/uuid/uuid_generators.hpp>
+#include <boost/uuid/uuid.hpp>
+#include <boost/uuid/uuid_io.hpp>
+
+#include "util/debug-util.h"
+
+namespace impala {
+
+void UtilityFunctions::UuidPrepare(FunctionContext* ctx,
+    FunctionContext::FunctionStateScope scope) {
+  if (scope == FunctionContext::THREAD_LOCAL) {
+    if (ctx->GetFunctionState(FunctionContext::THREAD_LOCAL) == NULL) {
+      boost::uuids::random_generator* uuid_gen =
+          new boost::uuids::random_generator;
+      ctx->SetFunctionState(scope, uuid_gen);
+    }
+  }
+}
+
+// This function is not cross-compiled to avoid including unnecessary boost library's
+// header files which bring in a bunch of unused code and global variables and increase
+// the codegen time. The indirect call in this function is expensive enough that not
+// inlining won't make much of a difference.
+StringVal UtilityFunctions::GenUuid(FunctionContext* ctx) {
+  void* uuid_gen = ctx->GetFunctionState(FunctionContext::THREAD_LOCAL);
+  DCHECK(uuid_gen != NULL);
+  boost::uuids::uuid uuid_value =
+      (*reinterpret_cast<boost::uuids::random_generator*>(uuid_gen))();
+  const std::string cxx_string = boost::uuids::to_string(uuid_value);
+  return StringVal::CopyFrom(ctx, reinterpret_cast<const uint8_t*>(cxx_string.c_str()),
+      cxx_string.length());
+}
+
+void UtilityFunctions::UuidClose(FunctionContext* ctx,
+    FunctionContext::FunctionStateScope scope){
+  if (scope == FunctionContext::THREAD_LOCAL) {
+    boost::uuids::random_generator* uuid_gen =
+        reinterpret_cast<boost::uuids::random_generator*>(
+            ctx->GetFunctionState(FunctionContext::THREAD_LOCAL));
+    DCHECK(uuid_gen != NULL);
+    delete uuid_gen;
+  }
+}
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/27b3b4d4/be/src/exprs/utility-functions.h
----------------------------------------------------------------------
diff --git a/be/src/exprs/utility-functions.h b/be/src/exprs/utility-functions.h
index f8ad31d..55f8dba 100644
--- a/be/src/exprs/utility-functions.h
+++ b/be/src/exprs/utility-functions.h
@@ -72,6 +72,9 @@ class UtilityFunctions {
   /// expression. input_val is not used and it is kept here in order to let
   /// the compiler generate the corresponding fully-qualified function name.
   template <typename T> static StringVal TypeOf(FunctionContext* ctx, const T& input_val);
+
+ private:
+  static StringVal GenUuid(FunctionContext* ctx);
 };
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/27b3b4d4/be/src/runtime/mem-tracker.cc
----------------------------------------------------------------------
diff --git a/be/src/runtime/mem-tracker.cc b/be/src/runtime/mem-tracker.cc
index 0f8c967..463bd5e 100644
--- a/be/src/runtime/mem-tracker.cc
+++ b/be/src/runtime/mem-tracker.cc
@@ -18,6 +18,7 @@
 #include "runtime/mem-tracker.h"
 
 #include <boost/algorithm/string/join.hpp>
+#include <boost/lexical_cast.hpp>
 #include <gperftools/malloc_extension.h>
 #include <gutil/strings/substitute.h>
 

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/27b3b4d4/be/src/runtime/timestamp-value.cc
----------------------------------------------------------------------
diff --git a/be/src/runtime/timestamp-value.cc b/be/src/runtime/timestamp-value.cc
index 25ad97f..7dfa4c6 100644
--- a/be/src/runtime/timestamp-value.cc
+++ b/be/src/runtime/timestamp-value.cc
@@ -17,6 +17,8 @@
 
 #include "runtime/timestamp-value.h"
 
+#include <boost/date_time/posix_time/posix_time.hpp>
+
 #include "runtime/timestamp-parse-util.h"
 
 #include "common/names.h"

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/27b3b4d4/be/src/runtime/timestamp-value.h
----------------------------------------------------------------------
diff --git a/be/src/runtime/timestamp-value.h b/be/src/runtime/timestamp-value.h
index f458002..ab3f081 100644
--- a/be/src/runtime/timestamp-value.h
+++ b/be/src/runtime/timestamp-value.h
@@ -19,12 +19,12 @@
 #ifndef IMPALA_RUNTIME_TIMESTAMP_VALUE_H
 #define IMPALA_RUNTIME_TIMESTAMP_VALUE_H
 
+#include <boost/date_time/compiler_config.hpp>
+#include <boost/date_time/posix_time/conversion.hpp>
+#include <boost/date_time/posix_time/posix_time_types.hpp>
 #include <ctime>
-#include <string>
-
-#include <boost/cstdint.hpp>
-#include <boost/date_time/posix_time/posix_time.hpp>
 #include <gflags/gflags.h>
+#include <string>
 
 #include "udf/udf.h"
 #include "util/hash-util.h"

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/27b3b4d4/be/src/service/impala-http-handler.cc
----------------------------------------------------------------------
diff --git a/be/src/service/impala-http-handler.cc b/be/src/service/impala-http-handler.cc
index 89c1a18..923c864 100644
--- a/be/src/service/impala-http-handler.cc
+++ b/be/src/service/impala-http-handler.cc
@@ -18,6 +18,7 @@
 #include "service/impala-http-handler.h"
 
 #include <sstream>
+#include <boost/lexical_cast.hpp>
 #include <boost/thread/mutex.hpp>
 #include <gutil/strings/substitute.h>
 

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/27b3b4d4/be/src/statestore/statestore.cc
----------------------------------------------------------------------
diff --git a/be/src/statestore/statestore.cc b/be/src/statestore/statestore.cc
index db3a566..e325292 100644
--- a/be/src/statestore/statestore.cc
+++ b/be/src/statestore/statestore.cc
@@ -17,6 +17,7 @@
 
 #include "statestore/statestore.h"
 
+#include <boost/lexical_cast.hpp>
 #include <boost/thread.hpp>
 #include <thrift/Thrift.h>
 #include <gutil/strings/substitute.h>

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/27b3b4d4/be/src/util/debug-util.cc
----------------------------------------------------------------------
diff --git a/be/src/util/debug-util.cc b/be/src/util/debug-util.cc
index f82a6af..596e7d2 100644
--- a/be/src/util/debug-util.cc
+++ b/be/src/util/debug-util.cc
@@ -19,6 +19,7 @@
 
 #include <iomanip>
 #include <sstream>
+#include <boost/tokenizer.hpp>
 
 #include "common/version.h"
 #include "runtime/collection-value.h"