You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@impala.apache.org by kw...@apache.org on 2017/02/03 23:57:44 UTC

[7/7] incubator-impala git commit: IMPALA-4705, IMPALA-4779, IMPALA-4780: Fix some Expr bugs with codegen

IMPALA-4705, IMPALA-4779, IMPALA-4780: Fix some Expr bugs with codegen

This change fixes expr-test.cc to work with codegen as it's
originally intended. Fixing it uncovers a couple of bugs fixed
in this patch:

IMPALA-4705: When an IR function is materialized, its
function body is parsed to find all its callee functions
to be materialized too. However, the old code doesn't
detect callee fnctions referenced indirectly (e.g. a
callee function passed as argument to another function).

This change fixes the problem above inspecting the use
lists of llvm::Function objects. When parsing the bitcode
module into memory, LLVM already establishes a use list
for each llvm::Value object which llvm::Function is a
subclass of. A use list contains all the locations in
the module in which the Value is referenced. For a
llvm::Function object, that would be its call sites and
constant expressions referencing the functions. By using
the use lists of llvm::Function in the module, a global
map is established at Impala initialization time to map
functions to their corresponding callee functions. This
map is then used when materializing a function to ensure
all its callee functions are also materialized recursively.

IMPALA-4779: conditional function isfalse(), istrue(),
isnotfalse(), isnotrue() aren't cross-compiled so they
will lead to unexpected query failure when codegen is enabled.
This change will cross-compile these functions.

IMPALA-4780: next_day() always returns NULL when codegen
is enabled. The bound checks for next_day() use some class
static variables initialized in the global constructors
(@llvm.global_ctors). However, we never execute the global
constructors before calling the JIT compiled functions.
This causes these variables to remain as zero, causing all
executions of next_day() to fail the bound checks. The reason
why these class static variables aren't compiled as global
constants in LLVM IR is that TimestampFunctions::MIN_YEAR is
not a compile time constant. This change fixes the problem
above by setting TimestampFunctions::MIN_YEAR to a known constant
value. A DCHECK is added to verify that it matches the value
defined in the boost library.

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

Branch: refs/heads/master
Commit: 1d933919ee964d8766ba028623d66ec20cd123ac
Parents: 933f2ce
Author: Michael Ho <kw...@cloudera.com>
Authored: Mon Jan 16 15:51:34 2017 -0800
Committer: Impala Public Jenkins <im...@gerrit.cloudera.org>
Committed: Fri Feb 3 23:35:25 2017 +0000

----------------------------------------------------------------------
 be/src/codegen/llvm-codegen-test.cc             |   3 +-
 be/src/codegen/llvm-codegen.cc                  | 188 ++++++++-----------
 be/src/codegen/llvm-codegen.h                   |  66 +++----
 be/src/exprs/conditional-functions-ir.cc        |  20 ++
 be/src/exprs/conditional-functions.cc           |  19 --
 be/src/exprs/expr-codegen-test.cc               |   9 +-
 be/src/exprs/expr-test.cc                       |  13 +-
 be/src/exprs/timestamp-functions-ir.cc          |  45 +----
 be/src/exprs/timestamp-functions.cc             |  16 ++
 be/src/exprs/timestamp-functions.h              |  35 ++--
 be/src/service/fe-support.cc                    |  10 +-
 be/src/service/fe-support.h                     |   2 +-
 be/src/service/impalad-main.cc                  |   2 +-
 be/src/testutil/test-udfs.cc                    |  20 ++
 .../functional-query/queries/QueryTest/udf.test |   7 +
 tests/query_test/test_udfs.py                   |   4 +
 16 files changed, 225 insertions(+), 234 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/1d933919/be/src/codegen/llvm-codegen-test.cc
----------------------------------------------------------------------
diff --git a/be/src/codegen/llvm-codegen-test.cc b/be/src/codegen/llvm-codegen-test.cc
index 1238dde..412426f 100644
--- a/be/src/codegen/llvm-codegen-test.cc
+++ b/be/src/codegen/llvm-codegen-test.cc
@@ -55,7 +55,8 @@ class LlvmCodeGenTest : public testing:: Test {
   // Wrapper to call private test-only methods on LlvmCodeGen object
   static Status CreateFromFile(
       ObjectPool* pool, const string& filename, scoped_ptr<LlvmCodeGen>* codegen) {
-    return LlvmCodeGen::CreateFromFile(pool, NULL, filename, "test", codegen);
+    RETURN_IF_ERROR(LlvmCodeGen::CreateFromFile(pool, NULL, filename, "test", codegen));
+    return (*codegen)->MaterializeModule();
   }
 
   static LlvmCodeGen* CreateCodegen(ObjectPool* pool) {

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/1d933919/be/src/codegen/llvm-codegen.cc
----------------------------------------------------------------------
diff --git a/be/src/codegen/llvm-codegen.cc b/be/src/codegen/llvm-codegen.cc
index b856fb2..3d730d5 100644
--- a/be/src/codegen/llvm-codegen.cc
+++ b/be/src/codegen/llvm-codegen.cc
@@ -101,7 +101,8 @@ bool LlvmCodeGen::llvm_initialized_ = false;
 
 string LlvmCodeGen::cpu_name_;
 vector<string> LlvmCodeGen::cpu_attrs_;
-unordered_set<string> LlvmCodeGen::gv_ref_ir_fns_;
+unordered_set<string> LlvmCodeGen::fns_to_always_materialize_;
+FnRefsMap LlvmCodeGen::fn_refs_map_;
 
 [[noreturn]] static void LlvmCodegenHandleError(
     void* user_data, const std::string& reason, bool gen_crash_diag) {
@@ -115,62 +116,29 @@ bool LlvmCodeGen::IsDefinedInImpalad(const string& fn_name) {
   return status.ok();
 }
 
-void LlvmCodeGen::ParseGlobalConstant(Value* val, unordered_set<string>* ref_fns) {
-  // Parse constants to find any referenced functions.
-  vector<string> fn_names;
-  if (isa<Function>(val)) {
-    fn_names.push_back(cast<Function>(val)->getName().str());
-  } else if (isa<BlockAddress>(val)) {
-    const BlockAddress *ba = cast<BlockAddress>(val);
-    fn_names.push_back(ba->getFunction()->getName().str());
-  } else if (isa<GlobalAlias>(val)) {
-    GlobalAlias* alias = cast<GlobalAlias>(val);
-    ParseGlobalConstant(alias->getAliasee(), ref_fns);
-  } else if (isa<ConstantExpr>(val)) {
-    const ConstantExpr* ce = cast<ConstantExpr>(val);
-    if (ce->isCast()) {
-      for (User::const_op_iterator oi=ce->op_begin(); oi != ce->op_end(); ++oi) {
-        Function* fn = dyn_cast<Function>(*oi);
-        if (fn != NULL) fn_names.push_back(fn->getName().str());
+void LlvmCodeGen::FindGlobalUsers(User* val, vector<GlobalObject*>* users) {
+  for (Use& u: val->uses()) {
+    User* user = u.getUser();
+    if (isa<Instruction>(user)) {
+      Instruction* inst = dyn_cast<Instruction>(u.getUser());
+      users->push_back(inst->getFunction());
+    } else if (isa<GlobalVariable>(user)) {
+      GlobalVariable* gv = cast<GlobalVariable>(user);
+      string val_name = gv->getName();
+      // We strip global ctors and dtors out of the modules as they are not run.
+      if (val_name.find("llvm.global_ctors") == string::npos &&
+          val_name.find("llvm.global_dtors") == string::npos) {
+        users->push_back(gv);;
       }
-    }
-  } else if (isa<ConstantStruct>(val) || isa<ConstantArray>(val) ||
-      isa<ConstantDataArray>(val)) {
-    const Constant* val_constant = cast<Constant>(val);
-    for (int i = 0; i < val_constant->getNumOperands(); ++i) {
-      ParseGlobalConstant(val_constant->getOperand(i), ref_fns);
-    }
-  } else if (isa<ConstantVector>(val) || isa<ConstantDataVector>(val)) {
-    const Constant* val_const = cast<Constant>(val);
-    for (int i = 0; i < val->getType()->getVectorNumElements(); ++i) {
-      ParseGlobalConstant(val_const->getAggregateElement(i), ref_fns);
-    }
-  } else {
-    // Ignore constants which cannot contain function pointers. Ignore other global
-    // variables referenced by this global variable as InitializeLlvm() will parse
-    // all global variables.
-    DCHECK(isa<UndefValue>(val) || isa<ConstantFP>(val) || isa<ConstantInt>(val) ||
-        isa<GlobalVariable>(val) || isa<ConstantTokenNone>(val) ||
-        isa<ConstantPointerNull>(val) || isa<ConstantAggregateZero>(val) ||
-        isa<ConstantDataSequential>(val));
-  }
-
-  // Adds all functions not defined in Impalad native binary.
-  for (const string& fn_name: fn_names) {
-    if (!IsDefinedInImpalad(fn_name)) ref_fns->insert(fn_name);
-  }
-}
-
-void LlvmCodeGen::ParseGVForFunctions(Module* module, unordered_set<string>* ref_fns) {
-  for (GlobalVariable& gv: module->globals()) {
-    if (gv.hasInitializer() && gv.isConstant()) {
-      Constant* val = gv.getInitializer();
-      if (val->getNumOperands() > 0) ParseGlobalConstant(val, ref_fns);
+    } else if (isa<Constant>(user)) {
+      FindGlobalUsers(user, users);
+    } else {
+      DCHECK(false) << "Unknown user's types for " << val->getName().str();
     }
   }
 }
 
-void LlvmCodeGen::InitializeLlvm(bool load_backend) {
+Status LlvmCodeGen::InitializeLlvm(bool load_backend) {
   DCHECK(!llvm_initialized_);
   llvm::remove_fatal_error_handler();
   llvm::install_fatal_error_handler(LlvmCodegenHandleError);
@@ -201,17 +169,43 @@ void LlvmCodeGen::InitializeLlvm(bool load_backend) {
 
   ObjectPool init_pool;
   scoped_ptr<LlvmCodeGen> init_codegen;
-  Status status = LlvmCodeGen::CreateFromMemory(&init_pool, NULL, "init", &init_codegen);
-  ParseGVForFunctions(init_codegen->module_, &gv_ref_ir_fns_);
+  RETURN_IF_ERROR(LlvmCodeGen::CreateFromMemory(&init_pool, NULL, "init", &init_codegen));
+  // LLVM will construct "use" lists only when the entire module is materialized.
+  RETURN_IF_ERROR(init_codegen->MaterializeModule());
 
   // Validate the module by verifying that functions for all IRFunction::Type
   // can be found.
   for (int i = IRFunction::FN_START; i < IRFunction::FN_END; ++i) {
     DCHECK(FN_MAPPINGS[i].fn == i);
     const string& fn_name = FN_MAPPINGS[i].fn_name;
-    DCHECK(init_codegen->module_->getFunction(fn_name) != NULL)
-        << "Failed to find function " << fn_name;
+    if (init_codegen->module_->getFunction(fn_name) == nullptr) {
+      return Status(Substitute("Failed to find function $0", fn_name));
+    }
+  }
+
+  // Create a mapping of functions to their referenced functions.
+  for (Function& fn: init_codegen->module_->functions()) {
+    if (fn.isIntrinsic() || fn.isDeclaration()) continue;
+    string fn_name = fn.getName();
+    vector<GlobalObject*> users;
+    FindGlobalUsers(&fn, &users);
+    for (GlobalValue* val: users) {
+      string key = val->getName();
+      DCHECK(isa<GlobalVariable>(val) || isa<Function>(val));
+      // 'fn_refs_map_' contains functions which need to be materialized when a certain
+      // IR Function is materialized. We choose to include functions referenced by
+      // another IR function in the map even if it's defined in Impalad binary so it
+      // can be inlined for further optimization. This is not applicable for functions
+      // referenced by global variables only.
+      if (isa<GlobalVariable>(val)) {
+        if (IsDefinedInImpalad(fn_name)) continue;
+        fns_to_always_materialize_.insert(fn_name);
+      } else {
+        fn_refs_map_[key].insert(fn_name);
+      }
+    }
   }
+  return Status::OK();
 }
 
 LlvmCodeGen::LlvmCodeGen(
@@ -329,19 +323,17 @@ Status LlvmCodeGen::LinkModule(const string& file) {
   // The module data layout must match the one selected by the execution engine.
   new_module->setDataLayout(execution_engine_->getDataLayout());
 
-  // Record all IR functions in 'new_module' referenced by the module's global variables
-  // if they are not defined in the Impalad native code. They must be materialized to
-  // avoid linking error.
-  unordered_set<string> ref_fns;
-  ParseGVForFunctions(new_module.get(), &ref_fns);
-
-  // Record all the materializable functions in the new module before linking.
-  // Linking the new module to the main module (i.e. 'module_') may materialize
-  // functions in the new module. These materialized functions need to be parsed
-  // to materialize any functions they call in 'module_'.
-  unordered_set<string> materializable_fns;
+  // Parse all functions' names from the new module and find those which also exist in
+  // the main module. They are declarations in the new module or duplicated definitions
+  // of the same function in both modules. For the latter case, it's unclear which one
+  // the linker will choose. Materialize these functions in the main module in case they
+  // are chosen by the linker or referenced by functions in the new module. Note that
+  // linkModules() will materialize functions defined only in the new module.
   for (Function& fn: new_module->functions()) {
-    if (fn.isMaterializable()) materializable_fns.insert(fn.getName().str());
+    if (fn_refs_map_.find(fn.getName()) != fn_refs_map_.end()) {
+      Function* local_fn = module_->getFunction(fn.getName());
+      RETURN_IF_ERROR(MaterializeFunction(local_fn));
+    }
   }
 
   bool error = Linker::linkModules(*module_, std::move(new_module));
@@ -352,23 +344,6 @@ Status LlvmCodeGen::LinkModule(const string& file) {
   }
   linked_modules_.insert(file);
 
-  for (const string& fn_name: ref_fns) {
-    Function* fn = module_->getFunction(fn_name);
-    // The global variable from source module which references 'fn' can have private
-    // linkage and it may not be linked into 'module_'.
-    if (fn != NULL && fn->isMaterializable()) {
-      RETURN_IF_ERROR(MaterializeFunction(fn));
-      materializable_fns.erase(fn->getName().str());
-    }
-  }
-  // Parse functions in the source module materialized during linking and materialize
-  // their callees. Do it after linking so LLVM has "merged" functions defined in both
-  // modules. LLVM may not link in functions (and their callees) from source module if
-  // they're defined in destination module already.
-  for (const string& fn_name: materializable_fns) {
-    Function* fn = module_->getFunction(fn_name);
-    if (fn != NULL && !fn->isMaterializable()) RETURN_IF_ERROR(MaterializeCallees(fn));
-  }
   return Status::OK();
 }
 
@@ -403,11 +378,11 @@ Status LlvmCodeGen::CreateImpalaCodegen(ObjectPool* pool, MemTracker* parent_mem
     return Status("Could not create llvm struct type for StringVal");
   }
 
-  // Materialize functions implicitly referenced by the global variables.
-  for (const string& fn_name : gv_ref_ir_fns_) {
+  // Materialize functions referenced by the global variables.
+  for (const string& fn_name : fns_to_always_materialize_) {
     Function* fn = codegen->module_->getFunction(fn_name);
-    DCHECK(fn != NULL);
-    codegen->MaterializeFunction(fn);
+    DCHECK(fn != nullptr);
+    RETURN_IF_ERROR(codegen->MaterializeFunction(fn));
   }
   return Status::OK();
 }
@@ -642,22 +617,6 @@ void LlvmCodeGen::CreateIfElseBlocks(Function* fn, const string& if_name,
   *else_block = BasicBlock::Create(context(), else_name, fn, insert_before);
 }
 
-Status LlvmCodeGen::MaterializeCallees(Function* fn) {
-  for (inst_iterator iter = inst_begin(fn); iter != inst_end(fn); ++iter) {
-    Instruction* instr = &*iter;
-    Function* called_fn = NULL;
-    if (isa<CallInst>(instr)) {
-      CallInst* call_instr = reinterpret_cast<CallInst*>(instr);
-      called_fn = call_instr->getCalledFunction();
-    } else if (isa<InvokeInst>(instr)) {
-      InvokeInst* invoke_instr = reinterpret_cast<InvokeInst*>(instr);
-      called_fn = invoke_instr->getCalledFunction();
-    }
-    if (called_fn != NULL) RETURN_IF_ERROR(MaterializeFunctionHelper(called_fn));
-  }
-  return Status::OK();
-}
-
 Status LlvmCodeGen::MaterializeFunctionHelper(Function *fn) {
   DCHECK(!is_compiled_);
   if (fn->isIntrinsic() || !fn->isMaterializable()) return Status::OK();
@@ -670,7 +629,12 @@ Status LlvmCodeGen::MaterializeFunctionHelper(Function *fn) {
 
   // Materialized functions are marked as not materializable by LLVM.
   DCHECK(!fn->isMaterializable());
-  RETURN_IF_ERROR(MaterializeCallees(fn));
+  const unordered_set<string>& callees = fn_refs_map_[fn->getName().str()];
+  for (const string& callee: callees) {
+    Function* callee_fn = module_->getFunction(callee);
+    DCHECK(callee_fn != nullptr);
+    RETURN_IF_ERROR(MaterializeFunctionHelper(callee_fn));
+  }
   return Status::OK();
 }
 
@@ -886,13 +850,11 @@ Function* LlvmCodeGen::FinalizeFunction(Function* function) {
   return function;
 }
 
-Status LlvmCodeGen::MaterializeModule(Module* module) {
-  std::error_code err = module->materializeAll();
+Status LlvmCodeGen::MaterializeModule() {
+  std::error_code err = module_->materializeAll();
   if (UNLIKELY(err)) {
-    stringstream err_msg;
-    err_msg << "Failed to complete materialization of module " << module->getName().str()
-        << ": " << err.message();
-    return Status(err_msg.str());
+    return Status(Substitute("Failed to materialize module $0: $1",
+        module_->getName().str(), err.message()));
   }
   return Status::OK();
 }
@@ -919,7 +881,7 @@ Status LlvmCodeGen::FinalizeLazyMaterialization() {
   // All unused functions are now not materializable so it should be quick to call
   // materializeAll(). We need to call this function in order to destroy the
   // materializer so that DCE will not assert fail.
-  return MaterializeModule(module_);
+  return MaterializeModule();
 }
 
 Status LlvmCodeGen::FinalizeModule() {

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/1d933919/be/src/codegen/llvm-codegen.h
----------------------------------------------------------------------
diff --git a/be/src/codegen/llvm-codegen.h b/be/src/codegen/llvm-codegen.h
index 03656c8..9615664 100644
--- a/be/src/codegen/llvm-codegen.h
+++ b/be/src/codegen/llvm-codegen.h
@@ -27,6 +27,7 @@
 #include <vector>
 #include <boost/scoped_ptr.hpp>
 
+#include <boost/unordered_map.hpp>
 #include <boost/unordered_set.hpp>
 
 #include <llvm/IR/DerivedTypes.h>
@@ -81,6 +82,9 @@ class LlvmBuilder : public llvm::IRBuilder<> {
   using llvm::IRBuilder<>::IRBuilder;
 };
 
+/// Map of functions' names to their callee functions' names.
+typedef boost::unordered_map<std::string, boost::unordered_set<std::string>> FnRefsMap;
+
 /// LLVM code generator.  This is the top level object to generate jitted code.
 //
 /// LLVM provides a c++ IR builder interface so IR does not need to be written
@@ -138,12 +142,12 @@ class LlvmCodeGen {
   /// This function must be called once per process before any llvm API calls are
   /// made.  It is not valid to call it multiple times. LLVM needs to allocate data
   /// structures for multi-threading support and to enable dynamic linking of jitted code.
-  /// if 'load_backend', load the backend static object for llvm.  This is needed
-  /// when libbackend.so is loaded from java.  llvm will be default only look in
+  /// if 'load_backend', load the backend static object for llvm. This is needed
+  /// when libfesupport.so is loaded from java. llvm will by default only look in
   /// the current object and not be able to find the backend symbols
   /// TODO: this can probably be removed after impalad refactor where the java
   /// side is not loading the be explicitly anymore.
-  static void InitializeLlvm(bool load_backend = false);
+  static Status InitializeLlvm(bool load_backend = false);
 
   /// Creates a codegen instance for Impala initialized with the cross-compiled Impala IR.
   /// 'codegen' will contain the created object on success.
@@ -153,7 +157,9 @@ class LlvmCodeGen {
       const std::string& id, boost::scoped_ptr<LlvmCodeGen>* codegen);
 
   /// Creates a LlvmCodeGen instance initialized with the module bitcode from 'file'.
-  /// 'codegen' will contain the created object on success.
+  /// 'codegen' will contain the created object on success. The functions in the module
+  /// are materialized lazily. Getting a reference to a function via GetFunction() will
+  /// materialize the function and its callees recursively.
   static Status CreateFromFile(ObjectPool*, MemTracker* parent_mem_tracker,
       const std::string& file, const std::string& id,
       boost::scoped_ptr<LlvmCodeGen>* codegen);
@@ -500,19 +506,11 @@ class LlvmCodeGen {
   /// Returns true if the function 'fn' is defined in the Impalad native code.
   static bool IsDefinedInImpalad(const std::string& fn);
 
-  /// Parses the given global constant recursively and adds functions referenced in it
-  /// to the set 'ref_fns' if they are not defined in the Impalad native code. These
-  /// functions need to be materialized to avoid linking error.
-  static void ParseGlobalConstant(llvm::Value* global_const,
-      boost::unordered_set<string>* ref_fns);
-
-  /// Parses all the global variables in 'module' and adds any functions referenced by
-  /// them to the set 'ref_fns' if they are not defined in the Impalad native code.
-  /// These functions need to be materialized to avoid linking error.
-  static void ParseGVForFunctions(
-      llvm::Module* module, boost::unordered_set<string>* ref_fns);
+  /// Find all global variables and functions which reference the llvm::Value 'val'
+  /// and return them in 'users'.
+  static void FindGlobalUsers(llvm::User* val, std::vector<llvm::GlobalObject*>* users);
 
-  /// Top level codegen object.  'module_id' is used for debugging when outputting the IR.
+  /// Top level codegen object. 'module_id' is used for debugging when outputting the IR.
   LlvmCodeGen(
       ObjectPool* pool, MemTracker* parent_mem_tracker, const std::string& module_id);
 
@@ -520,23 +518,23 @@ class LlvmCodeGen {
   Status Init(std::unique_ptr<llvm::Module> module);
 
   /// Creates a LlvmCodeGen instance initialized with the module bitcode in memory.
-  /// 'codegen' will contain the created object on success. Note that the functions
-  /// are not materialized. Getting a reference to the function via GetFunction()
-  /// will materialize the function and its callees recursively.
+  /// 'codegen' will contain the created object on success. The functions in the module
+  /// are materialized lazily. Getting a reference to a function via GetFunction() will
+  /// materialize the function and its callees recursively.
   static Status CreateFromMemory(ObjectPool* pool, MemTracker* parent_mem_tracker,
       const std::string& id, boost::scoped_ptr<LlvmCodeGen>* codegen);
 
   /// Loads an LLVM module from 'file' which is the local path to the LLVM bitcode file.
-  /// The functions in the module are not materialized. Getting a reference to the
+  /// The functions in the module are materialized lazily. Getting a reference to the
   /// function via GetFunction() will materialize the function and its callees
   /// recursively. The caller is responsible for cleaning up the module.
   Status LoadModuleFromFile(const string& file, std::unique_ptr<llvm::Module>* module);
 
   /// Loads an LLVM module. 'module_ir_buf' is the memory buffer containing LLVM bitcode.
-  /// 'module_name' is the name of the module to use when reporting errors.
-  /// The caller is responsible for cleaning up 'module'. The functions in the module
-  /// aren't materialized. Getting a reference to the function via GetFunction() will
-  /// materialize the function and its callees recursively.
+  /// 'module_name' is the name of the module to use when reporting errors. The caller is
+  /// responsible for cleaning up 'module'. The functions in the module aren't
+  /// materialized. Getting a reference to the functiom via GetFunction() will materialize
+  /// the function and its callees recursively.
   Status LoadModuleFromMemory(std::unique_ptr<llvm::MemoryBuffer> module_ir_buf,
       std::string module_name, std::unique_ptr<llvm::Module>* module);
 
@@ -589,9 +587,9 @@ class LlvmCodeGen {
   /// to do the actual work. Return error status for any error.
   Status MaterializeFunction(llvm::Function* fn);
 
-  /// Materialize the given module by materializing all its unmaterialized functions
-  /// and deleting the module's materializer. Returns error status for any error.
-  Status MaterializeModule(llvm::Module* module);
+  /// Materialize the module owned by this codegen object. This will materialize all
+  /// functions and delete the module's materializer. Returns error status for any error.
+  Status MaterializeModule();
 
   /// With lazy materialization, functions which haven't been materialized when the module
   /// is finalized must be dead code or referenced only by global variables (e.g. boost
@@ -614,11 +612,15 @@ class LlvmCodeGen {
   static std::string cpu_name_;
   static std::vector<std::string> cpu_attrs_;
 
-  /// This set contains names of functions referenced by global variables which aren't
-  /// defined in the Impalad native code (they may have been inlined by gcc). These
-  /// functions are always materialized each time the module is loaded to ensure that
-  /// LLVM can resolve references to them.
-  static boost::unordered_set<std::string> gv_ref_ir_fns_;
+  /// A call graph for all IR functions in the main module. Used for determining
+  /// dependencies when materializing IR functions.
+  static FnRefsMap fn_refs_map_;
+
+  /// This set contains names of all functions which always need to be materialized.
+  /// They are referenced by global variables but NOT defined in the Impalad native
+  /// code (they may have been inlined by gcc). These functions are always materialized
+  /// when a module is loaded to ensure that LLVM can resolve references to them.
+  static boost::unordered_set<std::string> fns_to_always_materialize_;
 
   /// ID used for debugging (can be e.g. the fragment instance ID)
   std::string id_;

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/1d933919/be/src/exprs/conditional-functions-ir.cc
----------------------------------------------------------------------
diff --git a/be/src/exprs/conditional-functions-ir.cc b/be/src/exprs/conditional-functions-ir.cc
index 4c77d5c..ed98a87 100644
--- a/be/src/exprs/conditional-functions-ir.cc
+++ b/be/src/exprs/conditional-functions-ir.cc
@@ -155,3 +155,23 @@ COALESCE_COMPUTE_FUNCTION(DoubleVal);
 COALESCE_COMPUTE_FUNCTION(StringVal);
 COALESCE_COMPUTE_FUNCTION(TimestampVal);
 COALESCE_COMPUTE_FUNCTION(DecimalVal);
+
+BooleanVal ConditionalFunctions::IsFalse(FunctionContext* ctx, const BooleanVal& val) {
+  if (val.is_null) return BooleanVal(false);
+  return BooleanVal(!val.val);
+}
+
+BooleanVal ConditionalFunctions::IsNotFalse(FunctionContext* ctx, const BooleanVal& val) {
+  if (val.is_null) return BooleanVal(true);
+  return BooleanVal(val.val);
+}
+
+BooleanVal ConditionalFunctions::IsTrue(FunctionContext* ctx, const BooleanVal& val) {
+  if (val.is_null) return BooleanVal(false);
+  return BooleanVal(val.val);
+}
+
+BooleanVal ConditionalFunctions::IsNotTrue(FunctionContext* ctx, const BooleanVal& val) {
+  if (val.is_null) return BooleanVal(true);
+  return BooleanVal(!val.val);
+}

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/1d933919/be/src/exprs/conditional-functions.cc
----------------------------------------------------------------------
diff --git a/be/src/exprs/conditional-functions.cc b/be/src/exprs/conditional-functions.cc
index 6f8f362..e815b8a 100644
--- a/be/src/exprs/conditional-functions.cc
+++ b/be/src/exprs/conditional-functions.cc
@@ -34,22 +34,3 @@ CONDITIONAL_CODEGEN_FN(NullIfExpr);
 CONDITIONAL_CODEGEN_FN(IfExpr);
 CONDITIONAL_CODEGEN_FN(CoalesceExpr);
 
-BooleanVal ConditionalFunctions::IsFalse(FunctionContext* ctx, const BooleanVal& val) {
-  if (val.is_null) return BooleanVal(false);
-  return BooleanVal(!val.val);
-}
-
-BooleanVal ConditionalFunctions::IsNotFalse(FunctionContext* ctx, const BooleanVal& val) {
-  if (val.is_null) return BooleanVal(true);
-  return BooleanVal(val.val);
-}
-
-BooleanVal ConditionalFunctions::IsTrue(FunctionContext* ctx, const BooleanVal& val) {
-  if (val.is_null) return BooleanVal(false);
-  return BooleanVal(val.val);
-}
-
-BooleanVal ConditionalFunctions::IsNotTrue(FunctionContext* ctx, const BooleanVal& val) {
-  if (val.is_null) return BooleanVal(true);
-  return BooleanVal(!val.val);
-}

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/1d933919/be/src/exprs/expr-codegen-test.cc
----------------------------------------------------------------------
diff --git a/be/src/exprs/expr-codegen-test.cc b/be/src/exprs/expr-codegen-test.cc
index cbb5617..0c484f4 100644
--- a/be/src/exprs/expr-codegen-test.cc
+++ b/be/src/exprs/expr-codegen-test.cc
@@ -77,6 +77,12 @@ class ExprCodegenTest : public ::testing::Test {
     return expr->InlineConstants(codegen, fn);
   }
 
+  static Status CreateFromFile(
+      ObjectPool* pool, const string& filename, scoped_ptr<LlvmCodeGen>* codegen) {
+    RETURN_IF_ERROR(LlvmCodeGen::CreateFromFile(pool, NULL, filename, "test", codegen));
+    return (*codegen)->MaterializeModule();
+  }
+
   virtual void SetUp() {
     FunctionContext::TypeDesc return_type;
     return_type.type = FunctionContext::TYPE_INT;
@@ -251,8 +257,7 @@ TEST_F(ExprCodegenTest, TestInlineConstants) {
   stringstream test_udf_file;
   test_udf_file << getenv("IMPALA_HOME") << "/be/build/latest/exprs/expr-codegen-test.ll";
   scoped_ptr<LlvmCodeGen> codegen;
-  ASSERT_OK(
-      LlvmCodeGen::CreateFromFile(&pool, NULL, test_udf_file.str(), "test", &codegen));
+  ASSERT_OK(ExprCodegenTest::CreateFromFile(&pool, test_udf_file.str(), &codegen));
   Function* fn = codegen->GetFunction(TEST_GET_CONSTANT_SYMBOL, false);
   ASSERT_TRUE(fn != NULL);
 

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/1d933919/be/src/exprs/expr-test.cc
----------------------------------------------------------------------
diff --git a/be/src/exprs/expr-test.cc b/be/src/exprs/expr-test.cc
index 8210a4d..8624a85 100644
--- a/be/src/exprs/expr-test.cc
+++ b/be/src/exprs/expr-test.cc
@@ -3966,7 +3966,7 @@ TEST_F(ExprTest, TimestampFunctions) {
       "to_date(cast('2011-12-22 09:10:11.12345678' as timestamp))", "2011-12-22");
 
   // Check that timeofday() does not crash or return incorrect results
-  GetValue("timeofday()", TYPE_STRING);
+  TestIsNotNull("timeofday()", TYPE_STRING);
 
   TestValue("timestamp_cmp('1964-05-04 15:33:45','1966-05-04 15:33:45')", TYPE_INT, -1);
   TestValue("timestamp_cmp('1966-09-04 15:33:45','1966-05-04 15:33:45')", TYPE_INT, 1);
@@ -4105,6 +4105,7 @@ TEST_F(ExprTest, TimestampFunctions) {
         TYPE_DOUBLE, 1.2938724610001E9);
     TestStringValue("cast(cast(1.3041352164485E9 as timestamp) as string)",
         "2011-04-29 20:46:56.448500000");
+
     // NULL arguments.
     TestIsNull("from_utc_timestamp(NULL, 'PST')", TYPE_TIMESTAMP);
     TestIsNull("from_utc_timestamp(cast('2011-01-01 01:01:01.1' as timestamp), NULL)",
@@ -4255,7 +4256,6 @@ TEST_F(ExprTest, TimestampFunctions) {
   TestIsNull("unix_timestamp('1970-01', 'yyyy-MM-dd')", TYPE_BIGINT);
   TestIsNull("unix_timestamp('1970-20-01', 'yyyy-MM-dd')", TYPE_BIGINT);
 
-
   // regression test for IMPALA-1105
   TestIsNull("cast(trunc('2014-07-22 01:34:55 +0100', 'year') as STRING)", TYPE_STRING);
   TestStringValue("cast(trunc(cast('2014-04-01' as timestamp), 'SYYYY') as string)",
@@ -4665,8 +4665,6 @@ TEST_F(ExprTest, TimestampFunctions) {
   TestNextDayFunction();
 }
 
-
-
 TEST_F(ExprTest, ConditionalFunctions) {
   // If first param evaluates to true, should return second parameter,
   // false or NULL should return the third.
@@ -5036,7 +5034,6 @@ TEST_F(ExprTest, ConditionalFunctionIsNotFalse) {
 //     exprs that have the same byte size can end up in a number of locations
 void ValidateLayout(const vector<Expr*>& exprs, int expected_byte_size,
     int expected_var_begin, const map<int, set<int>>& expected_offsets) {
-
   vector<int> offsets;
   set<int> offsets_found;
 
@@ -6160,7 +6157,7 @@ TEST_F(ExprTest, UuidTest) {
 int main(int argc, char **argv) {
   ::testing::InitGoogleTest(&argc, argv);
   InitCommonRuntime(argc, argv, true, TestInfo::BE_TEST);
-  InitFeSupport();
+  InitFeSupport(false);
   impala::LlvmCodeGen::InitializeLlvm();
 
   // Disable llvm optimization passes if the env var is no set to true. Running without
@@ -6186,6 +6183,7 @@ int main(int argc, char **argv) {
 
   // Disable FE expr rewrites to make sure the Exprs get executed exactly as specified
   // in the tests here.
+  int ret;
   vector<string> options;
   options.push_back("ENABLE_EXPR_REWRITES=0");
   options.push_back("DISABLE_CODEGEN=1");
@@ -6193,12 +6191,13 @@ int main(int argc, char **argv) {
   enable_expr_rewrites_ = false;
   executor_->setExecOptions(options);
   cout << "Running without codegen" << endl;
-  int ret = RUN_ALL_TESTS();
+  ret = RUN_ALL_TESTS();
   if (ret != 0) return ret;
 
   options.clear();
   options.push_back("ENABLE_EXPR_REWRITES=0");
   options.push_back("DISABLE_CODEGEN=0");
+  options.push_back("EXEC_SINGLE_NODE_ROWS_THRESHOLD=0");
   disable_codegen_ = false;
   enable_expr_rewrites_ = false;
   executor_->setExecOptions(options);

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/1d933919/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 c70f286..3a6ea4f 100644
--- a/be/src/exprs/timestamp-functions-ir.cc
+++ b/be/src/exprs/timestamp-functions-ir.cc
@@ -33,6 +33,7 @@
 #include "common/names.h"
 
 using boost::gregorian::greg_month;
+using boost::gregorian::max_date_time;
 using boost::gregorian::min_date_time;
 using boost::posix_time::not_a_date_time;
 using boost::posix_time::ptime;
@@ -53,44 +54,6 @@ typedef boost::posix_time::seconds Seconds;
 
 namespace impala {
 
-// Constant strings used for DayName function.
-const char* TimestampFunctions::SUNDAY = "Sunday";
-const char* TimestampFunctions::MONDAY = "Monday";
-const char* TimestampFunctions::TUESDAY = "Tuesday";
-const char* TimestampFunctions::WEDNESDAY = "Wednesday";
-const char* TimestampFunctions::THURSDAY = "Thursday";
-const char* TimestampFunctions::FRIDAY = "Friday";
-const char* TimestampFunctions::SATURDAY = "Saturday";
-
-// To workaround a boost bug (where adding very large intervals to ptimes causes the
-// value to wrap around instead or throwing an exception -- the root cause of
-// IMPALA-1675), max interval value are defined below. Some values below are less than
-// the minimum interval needed to trigger IMPALA-1675 but the values are greater or
-// equal to the interval that would definitely result in an out of bounds value. The
-// min and max year are also defined for manual error checking. Boost is inconsistent
-// with its defined max year. date(max_date_time).year() will give 9999 but testing shows
-// the actual max date is 1 year later.
-const int64_t TimestampFunctions::MAX_YEAR = 10000;
-const int64_t TimestampFunctions::MIN_YEAR = Date(min_date_time).year();
-const int64_t TimestampFunctions::MAX_YEAR_INTERVAL =
-    TimestampFunctions::MAX_YEAR - TimestampFunctions::MIN_YEAR;
-const int64_t TimestampFunctions::MAX_MONTH_INTERVAL =
-    TimestampFunctions::MAX_YEAR_INTERVAL * 12;
-const int64_t TimestampFunctions::MAX_WEEK_INTERVAL =
-    TimestampFunctions::MAX_YEAR_INTERVAL * 53;
-const int64_t TimestampFunctions::MAX_DAY_INTERVAL =
-    TimestampFunctions::MAX_YEAR_INTERVAL * 366;
-const int64_t TimestampFunctions::MAX_HOUR_INTERVAL =
-    TimestampFunctions::MAX_DAY_INTERVAL * 24;
-const int64_t TimestampFunctions::MAX_MINUTE_INTERVAL =
-    TimestampFunctions::MAX_DAY_INTERVAL * 60;
-const int64_t TimestampFunctions::MAX_SEC_INTERVAL =
-    TimestampFunctions::MAX_MINUTE_INTERVAL * 60;
-const int64_t TimestampFunctions::MAX_MILLI_INTERVAL =
-    TimestampFunctions::MAX_SEC_INTERVAL * 1000;
-const int64_t TimestampFunctions::MAX_MICRO_INTERVAL =
-    TimestampFunctions::MAX_MILLI_INTERVAL * 1000;
-
 StringVal TimestampFunctions::StringValFromTimestamp(FunctionContext* context,
     const TimestampValue& tv, const StringVal& fmt) {
   void* state = context->GetFunctionState(FunctionContext::THREAD_LOCAL);
@@ -477,8 +440,6 @@ inline void AddInterval<Years>(FunctionContext* context, int64_t interval,
 string TimestampFunctions::ShortDayName(FunctionContext* context,
     const TimestampVal& ts) {
   if (ts.is_null) return NULL;
-  static const string DAY_ARRAY[7] = {"Sun", "Mon", "Tue", "Wed", "Thu", "Fri",
-      "Sat"};
   IntVal dow = DayOfWeek(context, ts);
   DCHECK_GT(dow.val, 0);
   DCHECK_LT(dow.val, 8);
@@ -488,8 +449,6 @@ string TimestampFunctions::ShortDayName(FunctionContext* context,
 string TimestampFunctions::ShortMonthName(FunctionContext* context,
     const TimestampVal& ts) {
   if (ts.is_null) return NULL;
-  static const string MONTH_ARRAY[12] = {"Jan", "Feb", "Mar", "Apr", "May",
-      "Jun", "Jul", "Aug", "Sep", "Oct", "Nov", "Dec"};
   IntVal mth = Month(context, ts);
   DCHECK_GT(mth.val, 0);
   DCHECK_LT(mth.val, 13);
@@ -723,6 +682,8 @@ template <bool is_add, typename AnyIntVal, typename Interval,
     bool is_add_months_keep_last_day>
 TimestampVal TimestampFunctions::AddSub(FunctionContext* context,
     const TimestampVal& timestamp, const AnyIntVal& num_interval_units) {
+  DCHECK_EQ(Date(max_date_time).year(), MAX_YEAR);
+  DCHECK_EQ(Date(min_date_time).year(), MIN_YEAR);
   if (timestamp.is_null || num_interval_units.is_null) return TimestampVal::null();
   const TimestampValue& value = TimestampValue::FromTimestampVal(timestamp);
   if (!value.HasDate()) return TimestampVal::null();

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/1d933919/be/src/exprs/timestamp-functions.cc
----------------------------------------------------------------------
diff --git a/be/src/exprs/timestamp-functions.cc b/be/src/exprs/timestamp-functions.cc
index 365021f..a519696 100644
--- a/be/src/exprs/timestamp-functions.cc
+++ b/be/src/exprs/timestamp-functions.cc
@@ -38,8 +38,24 @@ using boost::local_time::time_zone_ptr;
 using boost::posix_time::ptime;
 using boost::posix_time::to_iso_extended_string;
 
+typedef boost::gregorian::date Date;
+
 namespace impala {
 
+// Constant strings used for DayName function.
+const char* TimestampFunctions::SUNDAY = "Sunday";
+const char* TimestampFunctions::MONDAY = "Monday";
+const char* TimestampFunctions::TUESDAY = "Tuesday";
+const char* TimestampFunctions::WEDNESDAY = "Wednesday";
+const char* TimestampFunctions::THURSDAY = "Thursday";
+const char* TimestampFunctions::FRIDAY = "Friday";
+const char* TimestampFunctions::SATURDAY = "Saturday";
+
+const string TimestampFunctions::DAY_ARRAY[7] = {"Sun", "Mon", "Tue", "Wed", "Thu",
+    "Fri", "Sat"};
+const string TimestampFunctions::MONTH_ARRAY[12] = {"Jan", "Feb", "Mar", "Apr", "May",
+    "Jun", "Jul", "Aug", "Sep", "Oct", "Nov", "Dec"};
+
 namespace {
 /// Uses Boost's internal checking to throw an exception if 'date' is out of the
 /// supported range of boost::gregorian.

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/1d933919/be/src/exprs/timestamp-functions.h
----------------------------------------------------------------------
diff --git a/be/src/exprs/timestamp-functions.h b/be/src/exprs/timestamp-functions.h
index 433054b..d703da6 100644
--- a/be/src/exprs/timestamp-functions.h
+++ b/be/src/exprs/timestamp-functions.h
@@ -35,19 +35,24 @@ class TupleRow;
 /// TODO: Reconsider whether this class needs to exist.
 class TimestampFunctions {
  public:
-  // To workaround IMPALA-1675 (boost doesn't throw an error for very large intervals),
-  // define the max intervals.
-  static const int64_t MAX_YEAR;
-  static const int64_t MIN_YEAR;
-  static const int64_t MAX_YEAR_INTERVAL;
-  static const int64_t MAX_MONTH_INTERVAL;
-  static const int64_t MAX_WEEK_INTERVAL;
-  static const int64_t MAX_DAY_INTERVAL;
-  static const int64_t MAX_HOUR_INTERVAL;
-  static const int64_t MAX_MINUTE_INTERVAL;
-  static const int64_t MAX_SEC_INTERVAL;
-  static const int64_t MAX_MILLI_INTERVAL;
-  static const int64_t MAX_MICRO_INTERVAL;
+  /// To workaround a boost bug (where adding very large intervals to ptimes causes the
+  /// value to wrap around instead or throwing an exception -- the root cause of
+  /// IMPALA-1675), max interval value are defined below. Some values below are less than
+  /// the minimum interval needed to trigger IMPALA-1675 but the values are greater or
+  /// equal to the interval that would definitely result in an out of bounds value. The
+  /// min and max year are also defined for manual error checking. The min / max years
+  /// are derived from date(min_date_time).year() / date(max_date_time).year().
+  static const int64_t MAX_YEAR = 9999;
+  static const int64_t MIN_YEAR = 1400;
+  static const int64_t MAX_YEAR_INTERVAL = MAX_YEAR - MIN_YEAR;
+  static const int64_t MAX_MONTH_INTERVAL = MAX_YEAR_INTERVAL * 12;
+  static const int64_t MAX_WEEK_INTERVAL = MAX_YEAR_INTERVAL * 53;
+  static const int64_t MAX_DAY_INTERVAL = MAX_YEAR_INTERVAL * 366;
+  static const int64_t MAX_HOUR_INTERVAL = MAX_DAY_INTERVAL * 24;
+  static const int64_t MAX_MINUTE_INTERVAL = MAX_HOUR_INTERVAL * 60;
+  static const int64_t MAX_SEC_INTERVAL = MAX_MINUTE_INTERVAL * 60;
+  static const int64_t MAX_MILLI_INTERVAL = MAX_SEC_INTERVAL * 1000;
+  static const int64_t MAX_MICRO_INTERVAL = MAX_MILLI_INTERVAL * 1000;
 
   /// Parse and initialize format string if it is a constant. Raise error if invalid.
   static void UnixAndFromUnixPrepare(FunctionContext* context,
@@ -193,6 +198,10 @@ class TimestampFunctions {
   static const char* FRIDAY;
   static const char* SATURDAY;
   static const char* SUNDAY;
+
+  /// Static result values for ShortDayName() and ShortMonthName() functions.
+  static const std::string DAY_ARRAY[7];
+  static const std::string MONTH_ARRAY[12];
 };
 
 } // namespace impala

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/1d933919/be/src/service/fe-support.cc
----------------------------------------------------------------------
diff --git a/be/src/service/fe-support.cc b/be/src/service/fe-support.cc
index f02483b..925a59c 100644
--- a/be/src/service/fe-support.cc
+++ b/be/src/service/fe-support.cc
@@ -52,6 +52,8 @@
 using namespace impala;
 using namespace apache::thrift::server;
 
+static bool fe_support_disable_codegen = true;
+
 // Called from the FE when it explicitly loads libfesupport.so for tests.
 // This creates the minimal state necessary to service the other JNI calls.
 // This is not called when we first start up the BE.
@@ -91,8 +93,9 @@ Java_org_apache_impala_service_FeSupport_NativeEvalExprsWithoutRow(
   DeserializeThriftMsg(env, thrift_expr_batch, &expr_batch);
   DeserializeThriftMsg(env, thrift_query_ctx_bytes, &query_ctx);
   vector<TExpr>& texprs = expr_batch.exprs;
-  // Disable codegen advisory to avoid unnecessary latency.
-  query_ctx.disable_codegen_hint = true;
+  // Disable codegen advisorily to avoid unnecessary latency. For testing purposes
+  // (expr-test.cc), fe_support_disable_codegen may be set to false.
+  query_ctx.disable_codegen_hint = fe_support_disable_codegen;
   // Allow logging of at least one error, so we can detect and convert it into a
   // Java exception.
   query_ctx.client_request.query_options.max_errors = 1;
@@ -368,7 +371,8 @@ static JNINativeMethod native_methods[] = {
   },
 };
 
-void InitFeSupport() {
+void InitFeSupport(bool disable_codegen) {
+  fe_support_disable_codegen = disable_codegen;
   JNIEnv* env = getJNIEnv();
   jclass native_backend_cl = env->FindClass("org/apache/impala/service/FeSupport");
   env->RegisterNatives(native_backend_cl, native_methods,

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/1d933919/be/src/service/fe-support.h
----------------------------------------------------------------------
diff --git a/be/src/service/fe-support.h b/be/src/service/fe-support.h
index af48b62..11ad18b 100644
--- a/be/src/service/fe-support.h
+++ b/be/src/service/fe-support.h
@@ -29,7 +29,7 @@ namespace impala {
 /// native functions". See this link:
 ///     http://java.sun.com/docs/books/jni/html/other.html#29535
 /// for details.
-void InitFeSupport();
+void InitFeSupport(bool disable_codegen = true);
 
 }
 #endif

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/1d933919/be/src/service/impalad-main.cc
----------------------------------------------------------------------
diff --git a/be/src/service/impalad-main.cc b/be/src/service/impalad-main.cc
index 55d9ad6..2a5f3fd 100644
--- a/be/src/service/impalad-main.cc
+++ b/be/src/service/impalad-main.cc
@@ -59,7 +59,7 @@ DECLARE_bool(enable_rm);
 int ImpaladMain(int argc, char** argv) {
   InitCommonRuntime(argc, argv, true);
 
-  LlvmCodeGen::InitializeLlvm();
+  ABORT_IF_ERROR(LlvmCodeGen::InitializeLlvm());
   JniUtil::InitLibhdfs();
   ABORT_IF_ERROR(HBaseTableScanner::Init());
   ABORT_IF_ERROR(HBaseTable::InitJNI());

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/1d933919/be/src/testutil/test-udfs.cc
----------------------------------------------------------------------
diff --git a/be/src/testutil/test-udfs.cc b/be/src/testutil/test-udfs.cc
index d9425d2..0c85c6e 100644
--- a/be/src/testutil/test-udfs.cc
+++ b/be/src/testutil/test-udfs.cc
@@ -161,6 +161,26 @@ StringVal ToLower(FunctionContext* context, const StringVal& str) {
           context, str);
 }
 
+// Call a function defined in Impalad proper to make sure linking works correctly.
+extern "C" StringVal
+    _ZN6impala15StringFunctions5UpperEPN10impala_udf15FunctionContextERKNS1_9StringValE(
+        FunctionContext* context, const StringVal& str);
+
+typedef StringVal (*ToUpperFn)(FunctionContext* context, const StringVal& str);
+
+StringVal ToUpperWork(FunctionContext* context, const StringVal& str, ToUpperFn fn) {
+  return fn(context, str);
+}
+
+StringVal ToUpper(FunctionContext* context, const StringVal& str) {
+  // StringVal::null() doesn't inline its callee when compiled without optimization.
+  // Useful for testing cases such as IMPALA-4595.
+  if (str.is_null) return StringVal::null();
+  // Test for IMPALA-4705: pass a function as argument and make sure it's materialized.
+  return ToUpperWork(context, str,
+      _ZN6impala15StringFunctions5UpperEPN10impala_udf15FunctionContextERKNS1_9StringValE);
+}
+
 typedef DoubleVal (*TestFn)(const DoubleVal& base, const DoubleVal& exp);
 
 // This function is dropped upon linking when tested as IR UDF as it has internal linkage

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/1d933919/testdata/workloads/functional-query/queries/QueryTest/udf.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-query/queries/QueryTest/udf.test b/testdata/workloads/functional-query/queries/QueryTest/udf.test
index d605d76..6260304 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/udf.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/udf.test
@@ -402,6 +402,13 @@ string
 'hello'
 ====
 ---- QUERY
+select to_upper("foobar")
+---- TYPES
+string
+---- RESULTS
+'FOOBAR'
+====
+---- QUERY
 select tinyint_col, int_col, var_sum_multiply(2, tinyint_col, int_col)
 from functional.alltypestiny
 ---- TYPES

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/1d933919/tests/query_test/test_udfs.py
----------------------------------------------------------------------
diff --git a/tests/query_test/test_udfs.py b/tests/query_test/test_udfs.py
index 1b2a51c..746cf88 100644
--- a/tests/query_test/test_udfs.py
+++ b/tests/query_test/test_udfs.py
@@ -178,6 +178,10 @@ create function {database}.to_lower(string) returns string
 location '{location}'
 symbol='_Z7ToLowerPN10impala_udf15FunctionContextERKNS_9StringValE';
 
+create function {database}.to_upper(string) returns string
+location '{location}'
+symbol='_Z7ToUpperPN10impala_udf15FunctionContextERKNS_9StringValE';
+
 create function {database}.constant_timestamp() returns timestamp
 location '{location}' symbol='ConstantTimestamp';