You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@arrow.apache.org by ra...@apache.org on 2022/04/29 09:08:13 UTC

[arrow] branch master updated: ARROW-16055: [C++][Gandiva] Skip unnecessary work during cache hit when using object code cache

This is an automated email from the ASF dual-hosted git repository.

ravindra pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/arrow.git


The following commit(s) were added to refs/heads/master by this push:
     new 0e3043d48b ARROW-16055: [C++][Gandiva] Skip unnecessary work during cache hit when using object code cache
0e3043d48b is described below

commit 0e3043d48b70fb5d3fcbc4743ec3fc056e4128da
Author: Projjal Chanda <ia...@pchanda.com>
AuthorDate: Fri Apr 29 14:37:56 2022 +0530

    ARROW-16055: [C++][Gandiva] Skip unnecessary work during cache hit when using object code cache
    
    This change enables object code cache and skips the unnecessary work that was done before loading the compiled object code in memory. The following still needs to be done:
    * expression decomposition in order to create the function and in holders in memory
    * Add mappings for global functions to the llvm module
    
    Closes #12742 from projjal/objectcache
    
    Authored-by: Projjal Chanda <ia...@pchanda.com>
    Signed-off-by: Pindikura Ravindra <ra...@dremio.com>
---
 cpp/src/gandiva/compiled_expr.h         | 10 ++--
 cpp/src/gandiva/engine.cc               | 84 ++++++++++++++++++---------------
 cpp/src/gandiva/engine.h                | 15 ++++--
 cpp/src/gandiva/engine_llvm_test.cc     | 14 +++---
 cpp/src/gandiva/filter.cc               | 37 ++++-----------
 cpp/src/gandiva/gandiva_object_cache.cc |  4 --
 cpp/src/gandiva/gandiva_object_cache.h  |  2 -
 cpp/src/gandiva/llvm_generator.cc       | 51 ++++++++++----------
 cpp/src/gandiva/llvm_generator.h        |  9 ++--
 cpp/src/gandiva/llvm_generator_test.cc  | 12 +++--
 cpp/src/gandiva/projector.cc            | 41 +++++-----------
 11 files changed, 126 insertions(+), 153 deletions(-)

diff --git a/cpp/src/gandiva/compiled_expr.h b/cpp/src/gandiva/compiled_expr.h
index 58d86b8706..b66e7480dc 100644
--- a/cpp/src/gandiva/compiled_expr.h
+++ b/cpp/src/gandiva/compiled_expr.h
@@ -38,11 +38,11 @@ class CompiledExpr {
 
   FieldDescriptorPtr output() const { return output_; }
 
-  void SetIRFunction(SelectionVector::Mode mode, llvm::Function* ir_function) {
-    ir_functions_[static_cast<int>(mode)] = ir_function;
+  void SetFunctionName(SelectionVector::Mode mode, std::string& name) {
+    ir_functions_[static_cast<int>(mode)] = name;
   }
 
-  llvm::Function* GetIRFunction(SelectionVector::Mode mode) const {
+  std::string GetFunctionName(SelectionVector::Mode mode) const {
     return ir_functions_[static_cast<int>(mode)];
   }
 
@@ -61,8 +61,8 @@ class CompiledExpr {
   // output field
   FieldDescriptorPtr output_;
 
-  // IR functions for various modes in the generated code
-  std::array<llvm::Function*, SelectionVector::kNumModes> ir_functions_;
+  // Function names for various modes in the generated code
+  std::array<std::string, SelectionVector::kNumModes> ir_functions_;
 
   // JIT functions in the generated code (set after the module is optimised and finalized)
   std::array<EvalFunc, SelectionVector::kNumModes> jit_functions_;
diff --git a/cpp/src/gandiva/engine.cc b/cpp/src/gandiva/engine.cc
index b369ea3863..3bd5291775 100644
--- a/cpp/src/gandiva/engine.cc
+++ b/cpp/src/gandiva/engine.cc
@@ -117,26 +117,34 @@ void Engine::InitOnce() {
 
 Engine::Engine(const std::shared_ptr<Configuration>& conf,
                std::unique_ptr<llvm::LLVMContext> ctx,
-               std::unique_ptr<llvm::ExecutionEngine> engine, llvm::Module* module)
+               std::unique_ptr<llvm::ExecutionEngine> engine, llvm::Module* module,
+               bool cached)
     : context_(std::move(ctx)),
       execution_engine_(std::move(engine)),
       ir_builder_(arrow::internal::make_unique<llvm::IRBuilder<>>(*context_)),
       module_(module),
       types_(*context_),
-      optimize_(conf->optimize()) {}
+      optimize_(conf->optimize()),
+      cached_(cached) {}
 
 Status Engine::Init() {
-  // Add mappings for functions that can be accessed from LLVM/IR module.
+  // Add mappings for global functions that can be accessed from LLVM/IR module.
   AddGlobalMappings();
 
-  ARROW_RETURN_NOT_OK(LoadPreCompiledIR());
-  ARROW_RETURN_NOT_OK(DecimalIR::AddFunctions(this));
+  return Status::OK();
+}
 
+Status Engine::LoadFunctionIRs() {
+  if (!functions_loaded_) {
+    ARROW_RETURN_NOT_OK(LoadPreCompiledIR());
+    ARROW_RETURN_NOT_OK(DecimalIR::AddFunctions(this));
+    functions_loaded_ = true;
+  }
   return Status::OK();
 }
 
 /// factory method to construct the engine.
-Status Engine::Make(const std::shared_ptr<Configuration>& conf,
+Status Engine::Make(const std::shared_ptr<Configuration>& conf, bool cached,
                     std::unique_ptr<Engine>* out) {
   std::call_once(llvm_init_once_flag, InitOnce);
 
@@ -173,7 +181,7 @@ Status Engine::Make(const std::shared_ptr<Configuration>& conf,
   }
 
   std::unique_ptr<Engine> engine{
-      new Engine(conf, std::move(ctx), std::move(exec_engine), module_ptr)};
+      new Engine(conf, std::move(ctx), std::move(exec_engine), module_ptr, cached)};
   ARROW_RETURN_NOT_OK(engine->Init());
   *out = std::move(engine);
   return Status::OK();
@@ -278,35 +286,37 @@ Status Engine::RemoveUnusedFunctions() {
 
 // Optimise and compile the module.
 Status Engine::FinalizeModule() {
-  ARROW_RETURN_NOT_OK(RemoveUnusedFunctions());
-
-  if (optimize_) {
-    // misc passes to allow for inlining, vectorization, ..
-    std::unique_ptr<llvm::legacy::PassManager> pass_manager(
-        new llvm::legacy::PassManager());
-
-    llvm::TargetIRAnalysis target_analysis =
-        execution_engine_->getTargetMachine()->getTargetIRAnalysis();
-    pass_manager->add(llvm::createTargetTransformInfoWrapperPass(target_analysis));
-    pass_manager->add(llvm::createFunctionInliningPass());
-    pass_manager->add(llvm::createInstructionCombiningPass());
-    pass_manager->add(llvm::createPromoteMemoryToRegisterPass());
-    pass_manager->add(llvm::createGVNPass());
-    pass_manager->add(llvm::createNewGVNPass());
-    pass_manager->add(llvm::createCFGSimplificationPass());
-    pass_manager->add(llvm::createLoopVectorizePass());
-    pass_manager->add(llvm::createSLPVectorizerPass());
-    pass_manager->add(llvm::createGlobalOptimizerPass());
-
-    // run the optimiser
-    llvm::PassManagerBuilder pass_builder;
-    pass_builder.OptLevel = 3;
-    pass_builder.populateModulePassManager(*pass_manager);
-    pass_manager->run(*module_);
-  }
+  if (!cached_) {
+    ARROW_RETURN_NOT_OK(RemoveUnusedFunctions());
+
+    if (optimize_) {
+      // misc passes to allow for inlining, vectorization, ..
+      std::unique_ptr<llvm::legacy::PassManager> pass_manager(
+          new llvm::legacy::PassManager());
+
+      llvm::TargetIRAnalysis target_analysis =
+          execution_engine_->getTargetMachine()->getTargetIRAnalysis();
+      pass_manager->add(llvm::createTargetTransformInfoWrapperPass(target_analysis));
+      pass_manager->add(llvm::createFunctionInliningPass());
+      pass_manager->add(llvm::createInstructionCombiningPass());
+      pass_manager->add(llvm::createPromoteMemoryToRegisterPass());
+      pass_manager->add(llvm::createGVNPass());
+      pass_manager->add(llvm::createNewGVNPass());
+      pass_manager->add(llvm::createCFGSimplificationPass());
+      pass_manager->add(llvm::createLoopVectorizePass());
+      pass_manager->add(llvm::createSLPVectorizerPass());
+      pass_manager->add(llvm::createGlobalOptimizerPass());
+
+      // run the optimiser
+      llvm::PassManagerBuilder pass_builder;
+      pass_builder.OptLevel = 3;
+      pass_builder.populateModulePassManager(*pass_manager);
+      pass_manager->run(*module_);
+    }
 
-  ARROW_RETURN_IF(llvm::verifyModule(*module_, &llvm::errs()),
-                  Status::CodeGenError("Module verification failed after optimizer"));
+    ARROW_RETURN_IF(llvm::verifyModule(*module_, &llvm::errs()),
+                    Status::CodeGenError("Module verification failed after optimizer"));
+  }
 
   // do the compilation
   execution_engine_->finalizeObject();
@@ -315,9 +325,9 @@ Status Engine::FinalizeModule() {
   return Status::OK();
 }
 
-void* Engine::CompiledFunction(llvm::Function* irFunction) {
+void* Engine::CompiledFunction(std::string& function) {
   DCHECK(module_finalized_);
-  return execution_engine_->getPointerToFunction(irFunction);
+  return reinterpret_cast<void*>(execution_engine_->getFunctionAddress(function));
 }
 
 void Engine::AddGlobalMappingForFunc(const std::string& name, llvm::Type* ret_type,
diff --git a/cpp/src/gandiva/engine.h b/cpp/src/gandiva/engine.h
index 2579804ee1..a4d6a5fd1a 100644
--- a/cpp/src/gandiva/engine.h
+++ b/cpp/src/gandiva/engine.h
@@ -43,8 +43,9 @@ class GANDIVA_EXPORT Engine {
   /// Factory method to create and initialize the engine object.
   ///
   /// \param[in] config the engine configuration
+  /// \param[in] cached flag to mark if the module is already compiled and cached
   /// \param[out] engine the created engine
-  static Status Make(const std::shared_ptr<Configuration>& config,
+  static Status Make(const std::shared_ptr<Configuration>& config, bool cached,
                      std::unique_ptr<Engine>* engine);
 
   /// Add the function to the list of IR functions that need to be compiled.
@@ -57,15 +58,13 @@ class GANDIVA_EXPORT Engine {
   /// Optimise and compile the module.
   Status FinalizeModule();
 
-#ifdef GANDIVA_ENABLE_OBJECT_CODE_CACHE
   /// Set LLVM ObjectCache.
   void SetLLVMObjectCache(GandivaObjectCache& object_cache) {
     execution_engine_->setObjectCache(&object_cache);
   }
-#endif
 
   /// Get the compiled function corresponding to the irfunction.
-  void* CompiledFunction(llvm::Function* irFunction);
+  void* CompiledFunction(std::string& function);
 
   // Create and add a mapping for the cpp function to make it accessible from LLVM.
   void AddGlobalMappingForFunc(const std::string& name, llvm::Type* ret_type,
@@ -74,10 +73,14 @@ class GANDIVA_EXPORT Engine {
   /// Return the generated IR for the module.
   std::string DumpIR();
 
+  /// Load the function IRs that can be accessed in the module.
+  Status LoadFunctionIRs();
+
  private:
   Engine(const std::shared_ptr<Configuration>& conf,
          std::unique_ptr<llvm::LLVMContext> ctx,
-         std::unique_ptr<llvm::ExecutionEngine> engine, llvm::Module* module);
+         std::unique_ptr<llvm::ExecutionEngine> engine, llvm::Module* module,
+         bool cached);
 
   // Post construction init. This _must_ be called after the constructor.
   Status Init();
@@ -106,6 +109,8 @@ class GANDIVA_EXPORT Engine {
 
   bool optimize_ = true;
   bool module_finalized_ = false;
+  bool cached_;
+  bool functions_loaded_ = false;
 };
 
 }  // namespace gandiva
diff --git a/cpp/src/gandiva/engine_llvm_test.cc b/cpp/src/gandiva/engine_llvm_test.cc
index ef2275b342..0bf6413cf6 100644
--- a/cpp/src/gandiva/engine_llvm_test.cc
+++ b/cpp/src/gandiva/engine_llvm_test.cc
@@ -28,7 +28,7 @@ typedef int64_t (*add_vector_func_t)(int64_t* data, int n);
 
 class TestEngine : public ::testing::Test {
  protected:
-  llvm::Function* BuildVecAdd(Engine* engine) {
+  std::string BuildVecAdd(Engine* engine) {
     auto types = engine->types();
     llvm::IRBuilder<>* builder = engine->ir_builder();
     llvm::LLVMContext* context = engine->context();
@@ -95,10 +95,10 @@ class TestEngine : public ::testing::Test {
     // Loop exit
     builder->SetInsertPoint(loop_exit);
     builder->CreateRet(sum_update);
-    return fn;
+    return func_name;
   }
 
-  void BuildEngine() { ASSERT_OK(Engine::Make(TestConfiguration(), &engine)); }
+  void BuildEngine() { ASSERT_OK(Engine::Make(TestConfiguration(), false, &engine)); }
 
   std::unique_ptr<Engine> engine;
   std::shared_ptr<Configuration> configuration = TestConfiguration();
@@ -108,9 +108,9 @@ TEST_F(TestEngine, TestAddUnoptimised) {
   configuration->set_optimize(false);
   BuildEngine();
 
-  llvm::Function* ir_func = BuildVecAdd(engine.get());
+  std::string fn_name = BuildVecAdd(engine.get());
   ASSERT_OK(engine->FinalizeModule());
-  auto add_func = reinterpret_cast<add_vector_func_t>(engine->CompiledFunction(ir_func));
+  auto add_func = reinterpret_cast<add_vector_func_t>(engine->CompiledFunction(fn_name));
 
   int64_t my_array[] = {1, 3, -5, 8, 10};
   EXPECT_EQ(add_func(my_array, 5), 17);
@@ -120,9 +120,9 @@ TEST_F(TestEngine, TestAddOptimised) {
   configuration->set_optimize(true);
   BuildEngine();
 
-  llvm::Function* ir_func = BuildVecAdd(engine.get());
+  std::string fn_name = BuildVecAdd(engine.get());
   ASSERT_OK(engine->FinalizeModule());
-  auto add_func = reinterpret_cast<add_vector_func_t>(engine->CompiledFunction(ir_func));
+  auto add_func = reinterpret_cast<add_vector_func_t>(engine->CompiledFunction(fn_name));
 
   int64_t my_array[] = {1, 3, -5, 8, 10};
   EXPECT_EQ(add_func(my_array, 5), 17);
diff --git a/cpp/src/gandiva/filter.cc b/cpp/src/gandiva/filter.cc
index b7b2840714..78917467a0 100644
--- a/cpp/src/gandiva/filter.cc
+++ b/cpp/src/gandiva/filter.cc
@@ -45,12 +45,8 @@ Status Filter::Make(SchemaPtr schema, ConditionPtr condition,
   ARROW_RETURN_IF(configuration == nullptr,
                   Status::Invalid("Configuration cannot be null"));
 
-#ifdef GANDIVA_ENABLE_OBJECT_CODE_CACHE
   std::shared_ptr<Cache<ExpressionCacheKey, std::shared_ptr<llvm::MemoryBuffer>>> cache =
       LLVMGenerator::GetCache();
-#else
-  static Cache<ExpressionCacheKey, std::shared_ptr<Filter>> cache;
-#endif
 
   Condition conditionToKey = *(condition.get());
 
@@ -58,9 +54,8 @@ Status Filter::Make(SchemaPtr schema, ConditionPtr condition,
 
   bool is_cached = false;
 
-#ifdef GANDIVA_ENABLE_OBJECT_CODE_CACHE
-
-  auto prev_cached_obj = cache->GetObjectCode(cache_key);
+  std::shared_ptr<llvm::MemoryBuffer> prev_cached_obj;
+  prev_cached_obj = cache->GetObjectCode(cache_key);
 
   // Verify if previous filter obj code was cached
   if (prev_cached_obj != nullptr) {
@@ -68,29 +63,20 @@ Status Filter::Make(SchemaPtr schema, ConditionPtr condition,
   }
 
   GandivaObjectCache obj_cache(cache, cache_key);
-#else
-  auto prev_cached_obj = cache.GetObjectCode(cache_key);
-  // Verify if previous filter obj code was cached
-  if (prev_cached_obj != nullptr) {
-    *filter = prev_cached_obj;
-    filter->get()->SetBuiltFromCache(true);
-    return Status::OK();
-  }
-#endif
 
   // Build LLVM generator, and generate code for the specified expression
   std::unique_ptr<LLVMGenerator> llvm_gen;
-  ARROW_RETURN_NOT_OK(LLVMGenerator::Make(configuration, &llvm_gen));
+  ARROW_RETURN_NOT_OK(LLVMGenerator::Make(configuration, is_cached, &llvm_gen));
 
-  // Run the validation on the expression.
-  // Return if the expression is invalid since we will not be able to process further.
-  ExprValidator expr_validator(llvm_gen->types(), schema);
-  ARROW_RETURN_NOT_OK(expr_validator.Validate(condition));
+  if (!is_cached) {
+    // Run the validation on the expression.
+    // Return if the expression is invalid since we will not be able to process further.
+    ExprValidator expr_validator(llvm_gen->types(), schema);
+    ARROW_RETURN_NOT_OK(expr_validator.Validate(condition));
+  }
 
-#ifdef GANDIVA_ENABLE_OBJECT_CODE_CACHE
   // Set the object cache for LLVM
   llvm_gen->SetLLVMObjectCache(obj_cache);
-#endif
 
   ARROW_RETURN_NOT_OK(llvm_gen->Build({condition}, SelectionVector::Mode::MODE_NONE));
 
@@ -98,10 +84,6 @@ Status Filter::Make(SchemaPtr schema, ConditionPtr condition,
   *filter = std::make_shared<Filter>(std::move(llvm_gen), schema, configuration);
   filter->get()->SetBuiltFromCache(is_cached);
 
-#ifndef GANDIVA_ENABLE_OBJECT_CODE_CACHE
-  cache.PutObjectCode(cache_key, *filter);
-#endif
-
   return Status::OK();
 }
 
@@ -141,4 +123,5 @@ std::string Filter::DumpIR() { return llvm_generator_->DumpIR(); }
 void Filter::SetBuiltFromCache(bool flag) { built_from_cache_ = flag; }
 
 bool Filter::GetBuiltFromCache() { return built_from_cache_; }
+
 }  // namespace gandiva
diff --git a/cpp/src/gandiva/gandiva_object_cache.cc b/cpp/src/gandiva/gandiva_object_cache.cc
index e181aa3375..baf503d4ab 100644
--- a/cpp/src/gandiva/gandiva_object_cache.cc
+++ b/cpp/src/gandiva/gandiva_object_cache.cc
@@ -15,7 +15,6 @@
 // specific language governing permissions and limitations
 // under the License.
 
-#ifdef GANDIVA_ENABLE_OBJECT_CODE_CACHE
 #include "gandiva/gandiva_object_cache.h"
 
 #include <utility>
@@ -44,12 +43,9 @@ std::unique_ptr<llvm::MemoryBuffer> GandivaObjectCache::getObject(const llvm::Mo
   if (cached_obj != nullptr) {
     std::unique_ptr<llvm::MemoryBuffer> cached_buffer = cached_obj->getMemBufferCopy(
         cached_obj->getBuffer(), cached_obj->getBufferIdentifier());
-    ARROW_LOG(INFO) << "[INFO][CACHE-LOG]: An object code was found on cache.";
     return cached_buffer;
   }
-  ARROW_LOG(INFO) << "[INFO][CACHE-LOG]: No object code was found on cache.";
   return nullptr;
 }
 
 }  // namespace gandiva
-#endif  // GANDIVA_ENABLE_OBJECT_CODE_CACHE
diff --git a/cpp/src/gandiva/gandiva_object_cache.h b/cpp/src/gandiva/gandiva_object_cache.h
index c85d8a9454..62042c7b62 100644
--- a/cpp/src/gandiva/gandiva_object_cache.h
+++ b/cpp/src/gandiva/gandiva_object_cache.h
@@ -17,7 +17,6 @@
 
 #pragma once
 
-#ifdef GANDIVA_ENABLE_OBJECT_CODE_CACHE
 #if defined(_MSC_VER)
 #pragma warning(push)
 #pragma warning(disable : 4244)
@@ -53,4 +52,3 @@ class GandivaObjectCache : public llvm::ObjectCache {
   std::shared_ptr<Cache<ExpressionCacheKey, std::shared_ptr<llvm::MemoryBuffer>>> cache_;
 };
 }  // namespace gandiva
-#endif  // GANDIVA_ENABLE_OBJECT_CODE_CACHE
diff --git a/cpp/src/gandiva/llvm_generator.cc b/cpp/src/gandiva/llvm_generator.cc
index 839e61d157..42a05a7dff 100644
--- a/cpp/src/gandiva/llvm_generator.cc
+++ b/cpp/src/gandiva/llvm_generator.cc
@@ -35,19 +35,18 @@ namespace gandiva {
     AddTrace(__VA_ARGS__); \
   }
 
-LLVMGenerator::LLVMGenerator() : enable_ir_traces_(false) {}
+LLVMGenerator::LLVMGenerator(bool cached) : cached_(cached), enable_ir_traces_(false) {}
 
-Status LLVMGenerator::Make(std::shared_ptr<Configuration> config,
+Status LLVMGenerator::Make(std::shared_ptr<Configuration> config, bool cached,
                            std::unique_ptr<LLVMGenerator>* llvm_generator) {
-  std::unique_ptr<LLVMGenerator> llvmgen_obj(new LLVMGenerator());
+  std::unique_ptr<LLVMGenerator> llvmgen_obj(new LLVMGenerator(cached));
 
-  ARROW_RETURN_NOT_OK(Engine::Make(config, &(llvmgen_obj->engine_)));
+  ARROW_RETURN_NOT_OK(Engine::Make(config, cached, &(llvmgen_obj->engine_)));
   *llvm_generator = std::move(llvmgen_obj);
 
   return Status::OK();
 }
 
-#ifdef GANDIVA_ENABLE_OBJECT_CODE_CACHE
 std::shared_ptr<Cache<ExpressionCacheKey, std::shared_ptr<llvm::MemoryBuffer>>>
 LLVMGenerator::GetCache() {
   static std::shared_ptr<Cache<ExpressionCacheKey, std::shared_ptr<llvm::MemoryBuffer>>>
@@ -60,7 +59,6 @@ LLVMGenerator::GetCache() {
 void LLVMGenerator::SetLLVMObjectCache(GandivaObjectCache& object_cache) {
   engine_->SetLLVMObjectCache(object_cache);
 }
-#endif
 
 Status LLVMGenerator::Add(const ExpressionPtr expr, const FieldDescriptorPtr output) {
   int idx = static_cast<int>(compiled_exprs_.size());
@@ -70,12 +68,15 @@ Status LLVMGenerator::Add(const ExpressionPtr expr, const FieldDescriptorPtr out
   ARROW_RETURN_NOT_OK(decomposer.Decompose(*expr->root(), &value_validity));
   // Generate the IR function for the decomposed expression.
   std::unique_ptr<CompiledExpr> compiled_expr(new CompiledExpr(value_validity, output));
-  llvm::Function* ir_function = nullptr;
-  ARROW_RETURN_NOT_OK(CodeGenExprValue(value_validity->value_expr(),
-                                       annotator_.buffer_count(), output, idx,
-                                       &ir_function, selection_vector_mode_));
-  compiled_expr->SetIRFunction(selection_vector_mode_, ir_function);
-
+  std::string fn_name = "expr_" + std::to_string(idx) + "_" +
+                        std::to_string(static_cast<int>(selection_vector_mode_));
+  if (!cached_) {
+    ARROW_RETURN_NOT_OK(engine_->LoadFunctionIRs());
+    ARROW_RETURN_NOT_OK(CodeGenExprValue(value_validity->value_expr(),
+                                         annotator_.buffer_count(), output, idx, fn_name,
+                                         selection_vector_mode_));
+  }
+  compiled_expr->SetFunctionName(selection_vector_mode_, fn_name);
   compiled_exprs_.push_back(std::move(compiled_expr));
   return Status::OK();
 }
@@ -95,8 +96,8 @@ Status LLVMGenerator::Build(const ExpressionVector& exprs, SelectionVector::Mode
 
   // setup the jit functions for each expression.
   for (auto& compiled_expr : compiled_exprs_) {
-    auto ir_fn = compiled_expr->GetIRFunction(mode);
-    auto jit_fn = reinterpret_cast<EvalFunc>(engine_->CompiledFunction(ir_fn));
+    auto fn_name = compiled_expr->GetFunctionName(mode);
+    auto jit_fn = reinterpret_cast<EvalFunc>(engine_->CompiledFunction(fn_name));
     compiled_expr->SetJITFunction(selection_vector_mode_, jit_fn);
   }
 
@@ -266,7 +267,7 @@ llvm::Value* LLVMGenerator::GetLocalBitMapReference(llvm::Value* arg_bitmaps, in
 // }
 Status LLVMGenerator::CodeGenExprValue(DexPtr value_expr, int buffer_count,
                                        FieldDescriptorPtr output, int suffix_idx,
-                                       llvm::Function** fn,
+                                       std::string& fn_name,
                                        SelectionVector::Mode selection_vector_mode) {
   llvm::IRBuilder<>* builder = ir_builder();
   // Create fn prototype :
@@ -294,15 +295,13 @@ Status LLVMGenerator::CodeGenExprValue(DexPtr value_expr, int buffer_count,
       llvm::FunctionType::get(types()->i32_type(), arguments, false /*isVarArg*/);
 
   // Create fn
-  std::string func_name = "expr_" + std::to_string(suffix_idx) + "_" +
-                          std::to_string(static_cast<int>(selection_vector_mode));
-  engine_->AddFunctionToCompile(func_name);
-  *fn = llvm::Function::Create(prototype, llvm::GlobalValue::ExternalLinkage, func_name,
-                               module());
-  ARROW_RETURN_IF((*fn == nullptr), Status::CodeGenError("Error creating function."));
+  engine_->AddFunctionToCompile(fn_name);
+  llvm::Function* fn = llvm::Function::Create(
+      prototype, llvm::GlobalValue::ExternalLinkage, fn_name, module());
+  ARROW_RETURN_IF((fn == nullptr), Status::CodeGenError("Error creating function."));
 
   // Name the arguments
-  llvm::Function::arg_iterator args = (*fn)->arg_begin();
+  llvm::Function::arg_iterator args = (fn)->arg_begin();
   llvm::Value* arg_addrs = &*args;
   arg_addrs->setName("inputs_addr");
   ++args;
@@ -324,9 +323,9 @@ Status LLVMGenerator::CodeGenExprValue(DexPtr value_expr, int buffer_count,
   llvm::Value* arg_nrecords = &*args;
   arg_nrecords->setName("nrecords");
 
-  llvm::BasicBlock* loop_entry = llvm::BasicBlock::Create(*context(), "entry", *fn);
-  llvm::BasicBlock* loop_body = llvm::BasicBlock::Create(*context(), "loop", *fn);
-  llvm::BasicBlock* loop_exit = llvm::BasicBlock::Create(*context(), "exit", *fn);
+  llvm::BasicBlock* loop_entry = llvm::BasicBlock::Create(*context(), "entry", fn);
+  llvm::BasicBlock* loop_body = llvm::BasicBlock::Create(*context(), "loop", fn);
+  llvm::BasicBlock* loop_exit = llvm::BasicBlock::Create(*context(), "exit", fn);
 
   // Add reference to output vector (in entry block)
   builder->SetInsertPoint(loop_entry);
@@ -359,7 +358,7 @@ Status LLVMGenerator::CodeGenExprValue(DexPtr value_expr, int buffer_count,
   }
 
   // The visitor can add code to both the entry/loop blocks.
-  Visitor visitor(this, *fn, loop_entry, arg_addrs, arg_local_bitmaps, arg_holder_ptrs,
+  Visitor visitor(this, fn, loop_entry, arg_addrs, arg_local_bitmaps, arg_holder_ptrs,
                   slice_offsets, arg_context_ptr, position_var);
   value_expr->Accept(visitor);
   LValuePtr output_value = visitor.result();
diff --git a/cpp/src/gandiva/llvm_generator.h b/cpp/src/gandiva/llvm_generator.h
index 0e820f7d5b..693119128e 100644
--- a/cpp/src/gandiva/llvm_generator.h
+++ b/cpp/src/gandiva/llvm_generator.h
@@ -47,17 +47,15 @@ class FunctionHolder;
 class GANDIVA_EXPORT LLVMGenerator {
  public:
   /// \brief Factory method to initialize the generator.
-  static Status Make(std::shared_ptr<Configuration> config,
+  static Status Make(std::shared_ptr<Configuration> config, bool cached,
                      std::unique_ptr<LLVMGenerator>* llvm_generator);
 
-#ifdef GANDIVA_ENABLE_OBJECT_CODE_CACHE
   /// \brief Get the cache to be used for LLVM ObjectCache.
   static std::shared_ptr<Cache<ExpressionCacheKey, std::shared_ptr<llvm::MemoryBuffer>>>
   GetCache();
 
   /// \brief Set LLVM ObjectCache.
   void SetLLVMObjectCache(GandivaObjectCache& object_cache);
-#endif
 
   /// \brief Build the code for the expression trees for default mode with a LLVM
   /// ObjectCache. Each element in the vector represents an expression tree
@@ -84,7 +82,7 @@ class GANDIVA_EXPORT LLVMGenerator {
   std::string DumpIR() { return engine_->DumpIR(); }
 
  private:
-  LLVMGenerator();
+  explicit LLVMGenerator(bool cached);
 
   FRIEND_TEST(TestLLVMGenerator, VerifyPCFunctions);
   FRIEND_TEST(TestLLVMGenerator, TestAdd);
@@ -202,7 +200,7 @@ class GANDIVA_EXPORT LLVMGenerator {
 
   /// Generate code for the value array of one expression.
   Status CodeGenExprValue(DexPtr value_expr, int num_buffers, FieldDescriptorPtr output,
-                          int suffix_idx, llvm::Function** fn,
+                          int suffix_idx, std::string& fn_name,
                           SelectionVector::Mode selection_vector_mode);
 
   /// Generate code to load the local bitmap specified index and cast it as bitmap.
@@ -251,6 +249,7 @@ class GANDIVA_EXPORT LLVMGenerator {
 
   std::unique_ptr<Engine> engine_;
   std::vector<std::unique_ptr<CompiledExpr>> compiled_exprs_;
+  bool cached_;
   FunctionRegistry function_registry_;
   Annotator annotator_;
   SelectionVector::Mode selection_vector_mode_;
diff --git a/cpp/src/gandiva/llvm_generator_test.cc b/cpp/src/gandiva/llvm_generator_test.cc
index e67e155368..028893b0b4 100644
--- a/cpp/src/gandiva/llvm_generator_test.cc
+++ b/cpp/src/gandiva/llvm_generator_test.cc
@@ -41,9 +41,10 @@ class TestLLVMGenerator : public ::testing::Test {
 // Verify that a valid pc function exists for every function in the registry.
 TEST_F(TestLLVMGenerator, VerifyPCFunctions) {
   std::unique_ptr<LLVMGenerator> generator;
-  ASSERT_OK(LLVMGenerator::Make(TestConfiguration(), &generator));
+  ASSERT_OK(LLVMGenerator::Make(TestConfiguration(), false, &generator));
 
   llvm::Module* module = generator->module();
+  ASSERT_OK(generator->engine_->LoadFunctionIRs());
   for (auto& iter : registry_) {
     EXPECT_NE(module->getFunction(iter.pc_name()), nullptr);
   }
@@ -52,7 +53,7 @@ TEST_F(TestLLVMGenerator, VerifyPCFunctions) {
 TEST_F(TestLLVMGenerator, TestAdd) {
   // Setup LLVM generator to do an arithmetic add of two vectors
   std::unique_ptr<LLVMGenerator> generator;
-  ASSERT_OK(LLVMGenerator::Make(TestConfiguration(), &generator));
+  ASSERT_OK(LLVMGenerator::Make(TestConfiguration(), false, &generator));
   Annotator annotator;
 
   auto field0 = std::make_shared<arrow::Field>("f0", arrow::int32());
@@ -81,16 +82,17 @@ TEST_F(TestLLVMGenerator, TestAdd) {
   auto field_sum = std::make_shared<arrow::Field>("out", arrow::int32());
   auto desc_sum = annotator.CheckAndAddInputFieldDescriptor(field_sum);
 
-  llvm::Function* ir_func = nullptr;
+  std::string fn_name = "codegen";
 
-  ASSERT_OK(generator->CodeGenExprValue(func_dex, 4, desc_sum, 0, &ir_func,
+  ASSERT_OK(generator->engine_->LoadFunctionIRs());
+  ASSERT_OK(generator->CodeGenExprValue(func_dex, 4, desc_sum, 0, fn_name,
                                         SelectionVector::MODE_NONE));
 
   ASSERT_OK(generator->engine_->FinalizeModule());
   auto ir = generator->engine_->DumpIR();
   EXPECT_THAT(ir, testing::HasSubstr("vector.body"));
 
-  EvalFunc eval_func = (EvalFunc)generator->engine_->CompiledFunction(ir_func);
+  EvalFunc eval_func = (EvalFunc)generator->engine_->CompiledFunction(fn_name);
 
   constexpr size_t kNumRecords = 4;
   std::array<uint32_t, kNumRecords> a0{1, 2, 3, 4};
diff --git a/cpp/src/gandiva/projector.cc b/cpp/src/gandiva/projector.cc
index 0192fe3d93..58779571fd 100644
--- a/cpp/src/gandiva/projector.cc
+++ b/cpp/src/gandiva/projector.cc
@@ -61,55 +61,40 @@ Status Projector::Make(SchemaPtr schema, const ExpressionVector& exprs,
   ARROW_RETURN_IF(configuration == nullptr,
                   Status::Invalid("Configuration cannot be null"));
 
-#ifdef GANDIVA_ENABLE_OBJECT_CODE_CACHE
   // see if equivalent projector was already built
   std::shared_ptr<Cache<ExpressionCacheKey, std::shared_ptr<llvm::MemoryBuffer>>> cache =
       LLVMGenerator::GetCache();
-#else
-  static Cache<ExpressionCacheKey, std::shared_ptr<Projector>> cache;
-#endif
 
   ExpressionCacheKey cache_key(schema, configuration, exprs, selection_vector_mode);
 
-#ifdef GANDIVA_ENABLE_OBJECT_CODE_CACHE
-  auto prev_cached_obj = cache->GetObjectCode(cache_key);
-#else
-  auto prev_cached_obj = cache.GetObjectCode(cache_key);
-#endif
+  bool is_cached = false;
 
-  bool was_cached = false;
+  std::shared_ptr<llvm::MemoryBuffer> prev_cached_obj;
+  prev_cached_obj = cache->GetObjectCode(cache_key);
 
-#ifdef GANDIVA_ENABLE_OBJECT_CODE_CACHE
   // Verify if previous projector obj code was cached
   if (prev_cached_obj != nullptr) {
-    was_cached = true;
+    is_cached = true;
   }
 
   GandivaObjectCache obj_cache(cache, cache_key);
-#else
-  if (prev_cached_obj != nullptr) {
-    *projector = prev_cached_obj;
-    projector->get()->SetBuiltFromCache(true);
-    return Status::OK();
-  }
-#endif
 
   // Build LLVM generator, and generate code for the specified expressions
   std::unique_ptr<LLVMGenerator> llvm_gen;
-  ARROW_RETURN_NOT_OK(LLVMGenerator::Make(configuration, &llvm_gen));
+  ARROW_RETURN_NOT_OK(LLVMGenerator::Make(configuration, is_cached, &llvm_gen));
 
   // Run the validation on the expressions.
   // Return if any of the expression is invalid since
   // we will not be able to process further.
-  ExprValidator expr_validator(llvm_gen->types(), schema);
-  for (auto& expr : exprs) {
-    ARROW_RETURN_NOT_OK(expr_validator.Validate(expr));
+  if (!is_cached) {
+    ExprValidator expr_validator(llvm_gen->types(), schema);
+    for (auto& expr : exprs) {
+      ARROW_RETURN_NOT_OK(expr_validator.Validate(expr));
+    }
   }
 
-#ifdef GANDIVA_ENABLE_OBJECT_CODE_CACHE
   // Set the object cache for LLVM
   llvm_gen->SetLLVMObjectCache(obj_cache);
-#endif
 
   ARROW_RETURN_NOT_OK(llvm_gen->Build(exprs, selection_vector_mode));
 
@@ -123,11 +108,7 @@ Status Projector::Make(SchemaPtr schema, const ExpressionVector& exprs,
   // Instantiate the projector with the completely built llvm generator
   *projector = std::shared_ptr<Projector>(
       new Projector(std::move(llvm_gen), schema, output_fields, configuration));
-  projector->get()->SetBuiltFromCache(was_cached);
-
-#ifndef GANDIVA_ENABLE_OBJECT_CODE_CACHE
-  cache.PutObjectCode(cache_key, *projector);
-#endif
+  projector->get()->SetBuiltFromCache(is_cached);
 
   return Status::OK();
 }