You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@impala.apache.org by mi...@apache.org on 2017/12/30 16:40:33 UTC

impala git commit: IMPALA-6177: Cleanup incomplete handcrafted IRs before finalizing module

Repository: impala
Updated Branches:
  refs/heads/master fac2b2cd9 -> 4a45c829b


IMPALA-6177: Cleanup incomplete handcrafted IRs before finalizing module

Currently, if an error is encountered during the creation of a
handcrafted codegen method, then the resulting IR is left in an
incomplete state. This patch ensures that all such IRs are cleaned up
(method is deleted from the module) before the llvm module is finalized.

Testing:
- added a backend test to exercise the added code path.
- tested manually by executing the following query:
select * from charTable A, charTable B
where A.charColumn = B.charColumn and A.charColumn = 'foo';
and looking at the logs to verify that 'InsertRuntimeFilters' and
'FilterContextInsert' methods have been removed.

Change-Id: If975cfb3906482b36dd6ede32ca81de6fcee1d7f
Reviewed-on: http://gerrit.cloudera.org:8080/8541
Reviewed-by: Bikramjeet Vig <bi...@cloudera.com>
Tested-by: Impala Public Jenkins


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

Branch: refs/heads/master
Commit: 4a45c829bd8e5c5bedc434e87bd2a1d19b7d44e6
Parents: fac2b2c
Author: Bikramjeet Vig <bi...@cloudera.com>
Authored: Thu Nov 16 10:26:57 2017 -0800
Committer: Impala Public Jenkins <im...@gerrit.cloudera.org>
Committed: Sat Dec 30 05:31:38 2017 +0000

----------------------------------------------------------------------
 be/src/codegen/llvm-codegen-test.cc | 32 +++++++++++++++++--
 be/src/codegen/llvm-codegen.cc      | 53 +++++++++++++++++++++++---------
 be/src/codegen/llvm-codegen.h       | 51 ++++++++++++++++++++----------
 be/src/exec/hdfs-text-scanner.cc    |  2 +-
 be/src/exprs/scalar-fn-call.cc      |  1 +
 5 files changed, 104 insertions(+), 35 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/impala/blob/4a45c829/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 cdd2b89..9069898 100644
--- a/be/src/codegen/llvm-codegen-test.cc
+++ b/be/src/codegen/llvm-codegen-test.cc
@@ -106,6 +106,11 @@ class LlvmCodeGenTest : public testing:: Test {
   static Status LinkModuleFromHdfs(LlvmCodeGen* codegen, const string& hdfs_file) {
     return codegen->LinkModuleFromHdfs(hdfs_file);
   }
+
+  static bool ContainsHandcraftedFn(LlvmCodeGen* codegen, llvm::Function* function) {
+    const auto& hf = codegen->handcrafted_functions_;
+    return find(hf.begin(), hf.end(), function) != hf.end();
+  }
 };
 
 // Simple test to just make and destroy llvmcodegen objects.  LLVM
@@ -163,7 +168,7 @@ llvm::Function* CodegenInnerLoop(
   builder.CreateStore(incremented_value, counter_ptr);
   builder.CreateRetVoid();
 
-  return jitted_loop_call;
+  return codegen->FinalizeFunction(jitted_loop_call);
 }
 
 // This test loads a precompiled IR file (compiled from testdata/llvm/test-loop.cc).
@@ -303,7 +308,7 @@ llvm::Function* CodegenStringTest(LlvmCodeGen* codegen) {
   builder.CreateStore(codegen->GetIntConstant(TYPE_INT, 1), len_ptr);
   builder.CreateRet(len);
 
-  return interop_fn;
+  return codegen->FinalizeFunction(interop_fn);
 }
 
 // This test validates that the llvm StringValue struct matches the c++ stringvalue
@@ -324,7 +329,6 @@ TEST_F(LlvmCodeGenTest, StringValue) {
 
   llvm::Function* string_test_fn = CodegenStringTest(codegen.get());
   EXPECT_TRUE(string_test_fn != NULL);
-  EXPECT_TRUE(VerifyFunction(codegen.get(), string_test_fn));
 
   // Jit compile function
   void* jitted_fn = NULL;
@@ -505,6 +509,28 @@ TEST_F(LlvmCodeGenTest, CpuAttrWhitelist) {
          "+avx512cd", "+avx512vbmi", "+avx512pf"}));
 }
 
+// Test that exercises the code path that deletes non-finalized methods before it
+// finalizes the llvm module.
+TEST_F(LlvmCodeGenTest, CleanupNonFinalizedMethodsTest) {
+  scoped_ptr<LlvmCodeGen> codegen;
+  ASSERT_OK(LlvmCodeGen::CreateImpalaCodegen(runtime_state_, nullptr, "test", &codegen));
+  ASSERT_TRUE(codegen.get() != nullptr);
+  const auto close_codegen = MakeScopeExitTrigger([&codegen]() { codegen->Close(); });
+  LlvmBuilder builder(codegen->context());
+  LlvmCodeGen::FnPrototype incomplete_prototype(
+      codegen.get(), "IncompleteFn", codegen->void_type());
+  LlvmCodeGen::FnPrototype complete_prototype(
+      codegen.get(), "CompleteFn", codegen->void_type());
+  llvm::Function* incomplete_fn =
+      incomplete_prototype.GeneratePrototype(&builder, nullptr);
+  llvm::Function* complete_fn = complete_prototype.GeneratePrototype(&builder, nullptr);
+  builder.CreateRetVoid();
+  complete_fn = codegen->FinalizeFunction(complete_fn);
+  EXPECT_TRUE(complete_fn != nullptr);
+  ASSERT_TRUE(ContainsHandcraftedFn(codegen.get(), incomplete_fn));
+  ASSERT_TRUE(ContainsHandcraftedFn(codegen.get(), complete_fn));
+  ASSERT_OK(FinalizeModule(codegen.get()));
+}
 }
 
 int main(int argc, char **argv) {

http://git-wip-us.apache.org/repos/asf/impala/blob/4a45c829/be/src/codegen/llvm-codegen.cc
----------------------------------------------------------------------
diff --git a/be/src/codegen/llvm-codegen.cc b/be/src/codegen/llvm-codegen.cc
index ca78bbb..600d709 100644
--- a/be/src/codegen/llvm-codegen.cc
+++ b/be/src/codegen/llvm-codegen.cc
@@ -192,9 +192,9 @@ LlvmCodeGen::LlvmCodeGen(RuntimeState* state, ObjectPool* pool,
     is_corrupt_(false),
     is_compiled_(false),
     context_(new llvm::LLVMContext()),
-    module_(NULL),
-    memory_manager_(NULL),
-    loaded_functions_(IRFunction::FN_END, NULL) {
+    module_(nullptr),
+    memory_manager_(nullptr),
+    cross_compiled_functions_(IRFunction::FN_END, nullptr) {
   DCHECK(llvm_initialized_) << "Must call LlvmCodeGen::InitializeLlvm first.";
 
   context_->setDiagnosticHandler(&DiagnosticHandler::DiagnosticHandlerFn, this);
@@ -472,8 +472,8 @@ string LlvmCodeGen::GetIR(bool full_module) const {
   if (full_module) {
     module_->print(stream, NULL);
   } else {
-    for (int i = 0; i < codegend_functions_.size(); ++i) {
-      codegend_functions_[i]->print(stream, nullptr, false, true);
+    for (int i = 0; i < handcrafted_functions_.size(); ++i) {
+      handcrafted_functions_[i]->print(stream, nullptr, false, true);
     }
   }
   return str;
@@ -668,16 +668,16 @@ llvm::Function* LlvmCodeGen::GetFunction(const string& symbol, bool clone) {
 }
 
 llvm::Function* LlvmCodeGen::GetFunction(IRFunction::Type ir_type, bool clone) {
-  llvm::Function* fn = loaded_functions_[ir_type];
+  llvm::Function* fn = cross_compiled_functions_[ir_type];
   if (fn == NULL) {
-    DCHECK(FN_MAPPINGS[ir_type].fn == ir_type);
+    DCHECK_EQ(FN_MAPPINGS[ir_type].fn, ir_type);
     const string& fn_name = FN_MAPPINGS[ir_type].fn_name;
     fn = module_->getFunction(fn_name);
     if (fn == NULL) {
       LOG(ERROR) << "Unable to locate function " << fn_name;
       return NULL;
     }
-    loaded_functions_[ir_type] = fn;
+    cross_compiled_functions_[ir_type] = fn;
   }
   Status status = MaterializeFunction(fn);
   if (UNLIKELY(!status.ok())) return NULL;
@@ -760,7 +760,7 @@ LlvmCodeGen::FnPrototype::FnPrototype(
 }
 
 llvm::Function* LlvmCodeGen::FnPrototype::GeneratePrototype(
-    LlvmBuilder* builder, llvm::Value** params, bool print_ir) {
+    LlvmBuilder* builder, llvm::Value** params) {
   vector<llvm::Type*> arguments;
   for (int i = 0; i < args_.size(); ++i) {
     arguments.push_back(args_[i].type);
@@ -782,10 +782,11 @@ llvm::Function* LlvmCodeGen::FnPrototype::GeneratePrototype(
   if (builder != NULL) {
     llvm::BasicBlock* entry_block =
         llvm::BasicBlock::Create(codegen_->context(), "entry", fn);
+    // Add it to the llvm module via the builder and to the list of handcrafted functions
+    // that are a part of the module.
     builder->SetInsertPoint(entry_block);
+    codegen_->handcrafted_functions_.push_back(fn);
   }
-
-  if (print_ir) codegen_->codegend_functions_.push_back(fn);
   return fn;
 }
 
@@ -855,7 +856,7 @@ Status LlvmCodeGen::LoadFunction(const TFunction& fn, const std::string& symbol,
     // Create a Function* with the generated type. This is only a function
     // declaration, not a definition, since we do not create any basic blocks or
     // instructions in it.
-    *llvm_fn = prototype.GeneratePrototype(NULL, NULL, false);
+    *llvm_fn = prototype.GeneratePrototype(nullptr, nullptr);
 
     // Associate the dynamically loaded function pointer with the Function* we defined.
     // This tells LLVM where the compiled function definition is located in memory.
@@ -900,6 +901,9 @@ int LlvmCodeGen::ReplaceCallSites(
   DCHECK(caller->getParent() == module_);
   DCHECK(caller != NULL);
   DCHECK(new_fn != NULL);
+  DCHECK(find(handcrafted_functions_.begin(), handcrafted_functions_.end(), new_fn)
+          == handcrafted_functions_.end()
+      || finalized_functions_.find(new_fn) != finalized_functions_.end());
 
   vector<llvm::CallInst*> call_sites;
   FindCallSites(caller, target_name, &call_sites);
@@ -1014,6 +1018,7 @@ llvm::Function* LlvmCodeGen::FinalizeFunction(llvm::Function* function) {
     function->eraseFromParent(); // deletes function
     return NULL;
   }
+  finalized_functions_.insert(function);
   if (FLAGS_dump_ir) function->dump();
   return function;
 }
@@ -1070,6 +1075,24 @@ Status LlvmCodeGen::FinalizeModule() {
   if (is_corrupt_) return Status("Module is corrupt.");
   SCOPED_TIMER(profile_->total_time_counter());
 
+  // Clean up handcrafted functions that have not been finalized. Clean up is done by
+  // deleting the function from the module. Any reference to deleted functions in the
+  // module will crash LLVM and thus Impala during finalization of the module.
+  stringstream ss;
+  for (llvm::Function* fn : handcrafted_functions_) {
+    if (finalized_functions_.find(fn) == finalized_functions_.end()) {
+      ss << fn->getName().str() << "\n";
+      fn->eraseFromParent();
+    }
+  }
+  string non_finalized_fns_str = ss.str();
+  if (!non_finalized_fns_str.empty()) {
+    LOG(INFO) << "For query " << state_->query_id()
+              << " the following functions were not finalized and have been removed from "
+                 "the module:\n"
+              << non_finalized_fns_str;
+  }
+
   // Don't waste time optimizing module if there are no functions to JIT. This can happen
   // if the codegen object is created but no functions are successfully codegen'd.
   if (fns_to_jit_compile_.empty()) {
@@ -1210,8 +1233,8 @@ Status LlvmCodeGen::OptimizeModule() {
 
 void LlvmCodeGen::DestroyModule() {
   // Clear all references to LLVM objects owned by the module.
-  loaded_functions_.clear();
-  codegend_functions_.clear();
+  cross_compiled_functions_.clear();
+  handcrafted_functions_.clear();
   registered_exprs_map_.clear();
   registered_exprs_.clear();
   llvm_intrinsics_.clear();
@@ -1222,6 +1245,8 @@ void LlvmCodeGen::DestroyModule() {
 }
 
 void LlvmCodeGen::AddFunctionToJit(llvm::Function* fn, void** fn_ptr) {
+  DCHECK(finalized_functions_.find(fn) != finalized_functions_.end())
+      << "Attempted to add a non-finalized function to Jit: " << fn->getName().str();
   llvm::Type* decimal_val_type = GetType(CodegenAnyVal::LLVM_DECIMALVAL_NAME);
   if (fn->getReturnType() == decimal_val_type) {
     // Per the x86 calling convention ABI, DecimalVals should be returned via an extra

http://git-wip-us.apache.org/repos/asf/impala/blob/4a45c829/be/src/codegen/llvm-codegen.h
----------------------------------------------------------------------
diff --git a/be/src/codegen/llvm-codegen.h b/be/src/codegen/llvm-codegen.h
index 672ed27..da5d965 100644
--- a/be/src/codegen/llvm-codegen.h
+++ b/be/src/codegen/llvm-codegen.h
@@ -25,6 +25,7 @@
 #include <memory>
 #include <string>
 #include <vector>
+#include <unordered_set>
 #include <boost/scoped_ptr.hpp>
 
 #include <boost/unordered_set.hpp>
@@ -102,6 +103,13 @@ class LlvmBuilder : public llvm::IRBuilder<> {
 /// be called directly.  The test interface can be used to load any precompiled
 /// module or none at all (but this class will not validate the module).
 //
+/// There are two classes of functions defined based on how they are generated:
+/// 1. Handcrafted functions - These functions are built from scratch using the IRbuilder
+/// interface.
+/// 2. Cross-compiled functions - These functions are loaded directly from a
+/// cross-compiled IR module and are either directly used or are cloned and modified
+/// before use.
+//
 /// This class is not threadsafe.  During the Prepare() phase of the fragment execution,
 /// nodes should codegen functions, and register those functions with AddFunctionToJit().
 /// Afterward, FinalizeModule() should be called at which point all codegened functions
@@ -168,7 +176,7 @@ class LlvmCodeGen {
 
   /// For debugging. Returns the IR that was generated.  If full_module, the
   /// entire module is dumped, including what was loaded from precompiled IR.
-  /// If false, only output IR for functions which were generated.
+  /// If false, only output IR for functions which were handcrafted.
   std::string GetIR(bool full_module) const;
 
   /// Utility struct that wraps a variable name and llvm type.
@@ -206,18 +214,18 @@ class LlvmCodeGen {
     }
 
     /// Generate LLVM function prototype.
-    /// If a non-null 'builder' is passed, this function will also create the entry block
-    /// and set the builder's insert point to there.
+    /// This is the canonical way to start generating a handcrafted codegen'd function.
+    /// If a non-null 'builder' is passed, this function will also create the entry
+    /// block, add it to the llvm module via the builder by setting the builder's insert
+    /// point to the entry block, and add it to the list of functions handcrafted by
+    /// impala. FinalizeFunction() must be called for any function generated this way
+    /// otherwise it will be deleted during FinalizeModule().
     ///
     /// If 'params' is non-null, this function will also return the arguments values
     /// (params[0] is the first arg, etc). In that case, 'params' should be preallocated
     /// to be number of arguments
-    ///
-    /// If 'print_ir' is true, the generated llvm::Function's IR will be printed when
-    /// GetIR() is called. Avoid doing so for IR function prototypes generated for
-    /// externally defined native function.
     llvm::Function* GeneratePrototype(
-        LlvmBuilder* builder = NULL, llvm::Value** params = NULL, bool print_ir = true);
+        LlvmBuilder* builder = nullptr, llvm::Value** params = nullptr);
 
    private:
     friend class LlvmCodeGen;
@@ -363,14 +371,20 @@ class LlvmCodeGen {
   /// Returns the i-th argument of fn.
   llvm::Argument* GetArgument(llvm::Function* fn, int i);
 
-  /// Verify function.  This should be called at the end for each codegen'd function.  If
-  /// the function does not verify, it will delete the function and return NULL,
+  /// Verify function. All handcrafted functions need to be finalized before being
+  /// passed to AddFunctionToJit() otherwise the functions will be deleted from the
+  /// module when the module is finalized. Also, all loaded functions that need to be JIT
+  /// compiled after modification also need to be finalized.
+  /// If the function does not verify, it will delete the function and return NULL,
   /// otherwise, it returns the function object.
   llvm::Function* FinalizeFunction(llvm::Function* function);
 
   /// Adds the function to be automatically jit compiled when the codegen object is
   /// finalized. FinalizeModule() will set fn_ptr to point to the jitted function.
   ///
+  /// Pre-condition: FinalizeFunction() must have been called on the function passed to
+  /// this method.
+  ///
   /// Only functions registered with AddFunctionToJit() and their dependencies are
   /// compiled by FinalizeModule(): other functions are considered dead code and will
   /// be removed during optimization.
@@ -420,7 +434,7 @@ class LlvmCodeGen {
   ///   int32_t Hash(int8_t* data, int len, int32_t seed);
   /// If num_bytes is non-zero, the returned function will be codegen'd to only
   /// work for that number of bytes.  It is invalid to call that function with a
-  /// different 'len'.
+  /// different 'len'. Functions returned by these methods have already been finalized.
   llvm::Function* GetHashFunction(int num_bytes = -1);
   llvm::Function* GetFnvHashFunction(int num_bytes = -1);
   llvm::Function* GetMurmurHashFunction(int num_bytes = -1);
@@ -752,13 +766,16 @@ class LlvmCodeGen {
   /// The memory manager used by 'execution_engine_'. Owned by 'execution_engine_'.
   ImpalaMCJITMemoryManager* memory_manager_;
 
-  /// Functions parsed from pre-compiled module.  Indexed by ImpalaIR::Function enum
-  std::vector<llvm::Function*> loaded_functions_;
+  /// Functions parsed from pre-compiled module. Indexed by ImpalaIR::Function enum.
+  std::vector<llvm::Function*> cross_compiled_functions_;
+
+  /// Stores functions handcrafted by impala.  This does not contain cross compiled
+  /// functions, only function that were generated from scratch at runtime. Does not
+  /// overlap with loaded_functions_.
+  std::vector<llvm::Function*> handcrafted_functions_;
 
-  /// Stores functions codegen'd by impala.  This does not contain cross compiled
-  /// functions, only function that were generated at runtime.  Does not overlap
-  /// with loaded_functions_.
-  std::vector<llvm::Function*> codegend_functions_;
+  /// Stores the functions that have been finalized.
+  std::unordered_set<llvm::Function*> finalized_functions_;
 
   /// A mapping of unique id to registered expr functions
   std::map<int64_t, llvm::Function*> registered_exprs_map_;

http://git-wip-us.apache.org/repos/asf/impala/blob/4a45c829/be/src/exec/hdfs-text-scanner.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/hdfs-text-scanner.cc b/be/src/exec/hdfs-text-scanner.cc
index 487c6fc..9ef7ce2 100644
--- a/be/src/exec/hdfs-text-scanner.cc
+++ b/be/src/exec/hdfs-text-scanner.cc
@@ -736,7 +736,7 @@ Status HdfsTextScanner::CheckForSplitDelimiter(bool* split_delimiter) {
 }
 
 // Codegen for materializing parsed data into tuples.  The function WriteCompleteTuple is
-// codegen'd using the IRBuilder for the specific tuple description.  This function
+// handcrafted using the IRBuilder for the specific tuple description.  This function
 // is then injected into the cross-compiled driving function, WriteAlignedTuples().
 Status HdfsTextScanner::Codegen(HdfsScanNodeBase* node,
     const vector<ScalarExpr*>& conjuncts, llvm::Function** write_aligned_tuples_fn) {

http://git-wip-us.apache.org/repos/asf/impala/blob/4a45c829/be/src/exprs/scalar-fn-call.cc
----------------------------------------------------------------------
diff --git a/be/src/exprs/scalar-fn-call.cc b/be/src/exprs/scalar-fn-call.cc
index 9e25175..ffe8d27 100644
--- a/be/src/exprs/scalar-fn-call.cc
+++ b/be/src/exprs/scalar-fn-call.cc
@@ -438,6 +438,7 @@ Status ScalarFnCall::GetFunction(LlvmCodeGen* codegen, const string& symbol, voi
          << fn_.hdfs_location;
       return Status(ss.str());
     }
+    ir_fn = codegen->FinalizeFunction(ir_fn);
     codegen->AddFunctionToJit(ir_fn, fn);
     return Status::OK();
   }