You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@arrow.apache.org by "niyue (via GitHub)" <gi...@apache.org> on 2023/11/08 06:00:45 UTC

[PR] GH-38589: [C++][Gandiva] Support registering external C interface functions [arrow]

niyue opened a new pull request, #38632:
URL: https://github.com/apache/arrow/pull/38632

   ### Rationale for this change
   This PR tries to enhance Gandiva by supporting registering external C interface functions to its function registry, so that developers can author third party functions with complex dependency and expose them as C interface functions to be used in Gandiva expression. See more details in GH-38589.
   
   ### What changes are included in this PR?
   This PR primarily adds a new API to the `FunctionRegistry` so that developers can use it to register external C interface functions:
   ```C++
   arrow::Status Register(
         NativeFunction func, void* c_interface_function_ptr,
         std::optional<FunctionHolderMaker> function_holder_maker = std::nullopt);
   ```
   
   ### Are these changes tested?
   * The changes are tested via unit tests in this PR, and the unit tests include several C interface functions written using C++ and we confirm this kind of functions can be used by Gandiva after registration using the above mentioned new API.
   * Additionally, locally I wrote some Rust based functions, and integrate the Rust based functions into a C++ program by using the new registration API and verified this approach did work, but this piece of work is not included in the PR.
   
   ### Are there any user-facing changes?
   There are several new APIs added to `FunctionRegistry` class:
   ```C++
   /// \brief register a C interface function into the function registry
     /// @param func the registered function's metadata
     /// @param c_interface_function_ptr the function pointer to the
     /// registered function's implementation
     /// @param function_holder_maker this will be used as the function holder if the
     /// function requires a function holder
     arrow::Status Register(
         NativeFunction func, void* c_interface_function_ptr,
         std::optional<FunctionHolderMaker> function_holder_maker = std::nullopt);
   
     /// \brief get a list of C interface functions saved in the registry
     const std::vector<std::pair<NativeFunction, void*>>& GetCInterfaceFunctions() const;
   
     const FunctionHolderMakerRegistry& GetFunctionHolderMakerRegistry() const;
   ```
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] GH-38589: [C++][Gandiva] Support registering external C interface functions [arrow]

Posted by "niyue (via GitHub)" <gi...@apache.org>.
niyue commented on code in PR #38632:
URL: https://github.com/apache/arrow/pull/38632#discussion_r1386047275


##########
cpp/src/gandiva/tests/projector_test.cc:
##########
@@ -3608,4 +3608,80 @@ TEST_F(TestProjector, TestExtendedFunctions) {
   EXPECT_ARROW_ARRAY_EQUALS(out, outs.at(0));
 }
 
+TEST_F(TestProjector, TestExtendedCInterfaceFunctions) {

Review Comment:
   This test case demonstrates registering an external C interface function (but this function doesn't use function holder/context)



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] GH-38589: [C++][Gandiva] Support registering external C functions [arrow]

Posted by "niyue (via GitHub)" <gi...@apache.org>.
niyue commented on code in PR #38632:
URL: https://github.com/apache/arrow/pull/38632#discussion_r1389147440


##########
cpp/src/gandiva/expr_decomposer.cc:
##########
@@ -25,11 +25,12 @@
 
 #include "gandiva/annotator.h"
 #include "gandiva/dex.h"
-#include "gandiva/function_holder_registry.h"
+#include "gandiva/function_holder_maker_registry.h"
 #include "gandiva/function_registry.h"
 #include "gandiva/function_signature.h"
 #include "gandiva/in_holder.h"
 #include "gandiva/node.h"
+#include "gandiva/regex_functions_holder.h"

Review Comment:
   I found `LikeHolder` is used below but its header is not included so I added it (it is likely included indirectly by other header files ). Let me know if this is not recommended in the project (if not included, I ran into some issues in other projects, during refactoring, a indirectly included header file was removed, and causing the other file failed to be compiled)



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] GH-38589: [C++][Gandiva] Support registering external C functions [arrow]

Posted by "niyue (via GitHub)" <gi...@apache.org>.
niyue commented on code in PR #38632:
URL: https://github.com/apache/arrow/pull/38632#discussion_r1387917726


##########
cpp/src/gandiva/tests/test_util.cc:
##########
@@ -42,11 +46,129 @@ NativeFunction GetTestExternalFunction() {
   return multiply_by_two_func;
 }
 
-std::shared_ptr<Configuration> TestConfigurationWithFunctionRegistry(
+static NativeFunction GetTestExternalCInterfaceFunction() {
+  NativeFunction multiply_by_three_func(
+      "multiply_by_three", {}, {arrow::int32()}, arrow::int64(),
+      ResultNullableType::kResultNullIfNull, "multiply_by_three_int32");
+  return multiply_by_three_func;
+}
+
+static NativeFunction GetTestFunctionWithFunctionHolder() {
+  // the 2nd parameter is expected to be an int32 literal
+  NativeFunction multiply_by_n_func("multiply_by_n", {}, {arrow::int32(), arrow::int32()},
+                                    arrow::int64(), ResultNullableType::kResultNullIfNull,
+                                    "multiply_by_n_int32_int32",
+                                    NativeFunction::kNeedsFunctionHolder);
+  return multiply_by_n_func;
+}
+
+static NativeFunction GetTestFunctionWithContext() {
+  NativeFunction multiply_by_two_formula(
+      "multiply_by_two_formula", {}, {arrow::utf8()}, arrow::utf8(),
+      ResultNullableType::kResultNullIfNull, "multiply_by_two_formula_utf8",
+      NativeFunction::kNeedsContext);
+  return multiply_by_two_formula;
+}
+
+static std::shared_ptr<Configuration> BuildConfigurationWithRegistry(
+    std::shared_ptr<FunctionRegistry> registry,
+    const std::function<arrow::Status(std::shared_ptr<FunctionRegistry>)>&
+        register_func) {
+  ARROW_EXPECT_OK(register_func(registry));
+  return ConfigurationBuilder().build(std::move(registry));
+}
+
+std::shared_ptr<Configuration> TestConfigWithFunctionRegistry(
+    std::shared_ptr<FunctionRegistry> registry) {
+  return BuildConfigurationWithRegistry(std::move(registry), [](auto reg) {
+    return reg->Register({GetTestExternalFunction()}, GetTestFunctionLLVMIRPath());
+  });
+}
+
+class MultiplyHolder : public FunctionHolder {
+ public:
+  explicit MultiplyHolder(int32_t num) : num_(num) {}
+
+  static Status Make(const FunctionNode& node, std::shared_ptr<MultiplyHolder>* holder) {
+    ARROW_RETURN_IF(node.children().size() != 2,
+                    Status::Invalid("'multiply_by_n' function requires two parameters"));
+
+    auto literal = dynamic_cast<LiteralNode*>(node.children().at(1).get());
+    ARROW_RETURN_IF(
+        literal == nullptr,
+        Status::Invalid(
+            "'multiply_by_n' function requires a literal as the 2nd parameter"));
+
+    auto literal_type = literal->return_type()->id();
+    ARROW_RETURN_IF(
+        literal_type != arrow::Type::INT32,
+        Status::Invalid(
+            "'multiply_by_n' function requires an int32 literal as the 2nd parameter"));
+
+    *holder = std::make_shared<MultiplyHolder>(
+        literal->is_null() ? 0 : std::get<int32_t>(literal->holder()));
+    return Status::OK();
+  }
+
+  int32_t operator()() const { return num_; }
+
+ private:
+  int32_t num_;
+};
+
+extern "C" {
+// this function is used as an external stub function for testing so it has to be declared

Review Comment:
   Fixed.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] GH-38589: [C++][Gandiva] Support registering external C functions [arrow]

Posted by "niyue (via GitHub)" <gi...@apache.org>.
niyue commented on code in PR #38632:
URL: https://github.com/apache/arrow/pull/38632#discussion_r1387884745


##########
cpp/src/gandiva/gdv_string_function_stubs.cc:
##########
@@ -15,7 +15,7 @@
 // specific language governing permissions and limitations
 // under the License.
 
-//#pragma once
+// #pragma once

Review Comment:
   Sure. Fixed.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] GH-38589: [C++][Gandiva] Support registering external C functions [arrow]

Posted by "niyue (via GitHub)" <gi...@apache.org>.
niyue commented on PR #38632:
URL: https://github.com/apache/arrow/pull/38632#issuecomment-1815668856

   To resolve conflicts, rebased onto the latest main branch.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] GH-38589: [C++][Gandiva] Support registering external C functions [arrow]

Posted by "kou (via GitHub)" <gi...@apache.org>.
kou commented on code in PR #38632:
URL: https://github.com/apache/arrow/pull/38632#discussion_r1390518888


##########
cpp/src/gandiva/external_c_functions.cc:
##########
@@ -0,0 +1,75 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License
+
+#include <llvm/IR/Type.h>
+
+#include "gandiva/engine.h"
+#include "gandiva/exported_funcs.h"
+
+namespace gandiva {
+// calculate the number of arguments for a function signature
+static size_t GetNumArgs(const FunctionSignature& sig, const NativeFunction& func) {

Review Comment:
   Anonymous namespace is better than `static` function.
   
   ```cpp
   namespace {
   size_t GetNumArgs(...) {...}
   Result<std::pair<std::vector<llvm::Type*>, llvm::Type*>> MapToLLVMSignature(...) {...}
   };
   namespace gandiva {
   arrow::Status ExternalCFunctions::AddMappings(Engine* engine) const {...}
   };



##########
cpp/src/gandiva/CMakeLists.txt:
##########
@@ -62,8 +62,10 @@ set(SRC_FILES
     expression_registry.cc
     exported_funcs_registry.cc
     exported_funcs.cc
+    external_c_functions.cc
     filter.cc
     function_ir_builder.cc
+    function_holder_maker_registry.cc

Review Comment:
   ```suggestion
       function_holder_maker_registry.cc
       function_ir_builder.cc
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] GH-38589: [C++][Gandiva] Support registering external C interface functions [arrow]

Posted by "niyue (via GitHub)" <gi...@apache.org>.
niyue commented on code in PR #38632:
URL: https://github.com/apache/arrow/pull/38632#discussion_r1386048070


##########
cpp/src/gandiva/tests/projector_test.cc:
##########
@@ -3608,4 +3608,80 @@ TEST_F(TestProjector, TestExtendedFunctions) {
   EXPECT_ARROW_ARRAY_EQUALS(out, outs.at(0));
 }
 
+TEST_F(TestProjector, TestExtendedCInterfaceFunctions) {
+  auto in_field = field("in", arrow::int32());
+  auto schema = arrow::schema({in_field});
+  auto out_field = field("out", arrow::int64());
+  auto multiply =
+      TreeExprBuilder::MakeExpression("multiply_by_three", {in_field}, out_field);
+
+  std::shared_ptr<Projector> projector;
+  auto external_registry = std::make_shared<FunctionRegistry>();
+  auto config_with_func_registry =
+      TestConfigWithStubFunction(std::move(external_registry));
+  ARROW_EXPECT_OK(
+      Projector::Make(schema, {multiply}, config_with_func_registry, &projector));
+
+  int num_records = 4;
+  auto array = MakeArrowArrayInt32({1, 2, 3, 4}, {true, true, true, true});
+  auto in_batch = arrow::RecordBatch::Make(schema, num_records, {array});
+  auto out = MakeArrowArrayInt64({3, 6, 9, 12}, {true, true, true, true});
+
+  arrow::ArrayVector outs;
+  ARROW_EXPECT_OK(projector->Evaluate(*in_batch, pool_, &outs));
+  EXPECT_ARROW_ARRAY_EQUALS(out, outs.at(0));
+}
+
+TEST_F(TestProjector, TestExtendedCInterfaceFunctionsWithFunctionHolder) {
+  auto multiple = TreeExprBuilder::MakeLiteral(5);
+  auto in_field = field("in", arrow::int32());
+  auto schema = arrow::schema({in_field});
+  auto out_field = field("out", arrow::int64());
+
+  auto in_node = TreeExprBuilder::MakeField(in_field);
+  auto multiply_by_n_func =
+      TreeExprBuilder::MakeFunction("multiply_by_n", {in_node, multiple}, arrow::int64());
+  auto multiply = TreeExprBuilder::MakeExpression(multiply_by_n_func, out_field);
+
+  std::shared_ptr<Projector> projector;
+  auto external_registry = std::make_shared<FunctionRegistry>();
+  auto config_with_func_registry =
+      TestConfigWithHolderFunction(std::move(external_registry));
+  ARROW_EXPECT_OK(
+      Projector::Make(schema, {multiply}, config_with_func_registry, &projector));
+
+  int num_records = 4;
+  auto array = MakeArrowArrayInt32({1, 2, 3, 4}, {true, true, true, true});
+  auto in_batch = arrow::RecordBatch::Make(schema, num_records, {array});
+  auto out = MakeArrowArrayInt64({5, 10, 15, 20}, {true, true, true, true});
+
+  arrow::ArrayVector outs;
+  ARROW_EXPECT_OK(projector->Evaluate(*in_batch, pool_, &outs));
+  EXPECT_ARROW_ARRAY_EQUALS(out, outs.at(0));
+}
+
+TEST_F(TestProjector, TestExtendedCInterfaceFunctionThatNeedsContext) {

Review Comment:
   This test case demonstrates registering an external C interface function that uses context.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] GH-38589: [C++][Gandiva] Support registering external C interface functions [arrow]

Posted by "niyue (via GitHub)" <gi...@apache.org>.
niyue commented on code in PR #38632:
URL: https://github.com/apache/arrow/pull/38632#discussion_r1386757020


##########
cpp/src/gandiva/engine.cc:
##########
@@ -447,7 +447,11 @@ void Engine::AddGlobalMappingForFunc(const std::string& name, llvm::Type* ret_ty
   execution_engine_->addGlobalMapping(fn, function_ptr);
 }
 
-void Engine::AddGlobalMappings() { ExportedFuncsRegistry::AddMappings(this); }
+arrow::Status Engine::AddGlobalMappings() {
+  ARROW_RETURN_NOT_OK(ExportedFuncsRegistry::AddMappings(this));
+  ExternalCInterfaceFunctions c_interface_funcs(function_registry_);
+  return c_interface_funcs.AddMappings(this);

Review Comment:
   The external C interface functions are added and mapped to the JIT engine here.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] GH-38589: [C++][Gandiva] Support registering external C functions [arrow]

Posted by "niyue (via GitHub)" <gi...@apache.org>.
niyue commented on code in PR #38632:
URL: https://github.com/apache/arrow/pull/38632#discussion_r1390556079


##########
cpp/src/gandiva/CMakeLists.txt:
##########
@@ -62,8 +62,10 @@ set(SRC_FILES
     expression_registry.cc
     exported_funcs_registry.cc
     exported_funcs.cc
+    external_c_functions.cc
     filter.cc
     function_ir_builder.cc
+    function_holder_maker_registry.cc

Review Comment:
   Good catch. Fixed.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] GH-38589: [C++][Gandiva] Support registering external C functions [arrow]

Posted by "niyue (via GitHub)" <gi...@apache.org>.
niyue commented on code in PR #38632:
URL: https://github.com/apache/arrow/pull/38632#discussion_r1389140695


##########
cpp/src/gandiva/function_registry.cc:
##########
@@ -109,11 +109,35 @@ arrow::Status FunctionRegistry::Register(const std::vector<NativeFunction>& func
   return Status::OK();
 }
 
+arrow::Status FunctionRegistry::Register(
+    NativeFunction func, void* c_function_ptr,
+    std::optional<FunctionHolderMaker> function_holder_maker) {
+  if (function_holder_maker.has_value()) {
+    // all signatures should have the same base name, use the first signature's base name
+    auto const& func_base_name = func.signatures().begin()->base_name();
+    ARROW_RETURN_NOT_OK(holder_maker_registry_.Register(
+        func_base_name, std::move(function_holder_maker.value())));
+  }
+  c_functions_.emplace_back(func, c_function_ptr);
+  ARROW_RETURN_NOT_OK(FunctionRegistry::Add(std::move(func)));
+  return Status::OK();

Review Comment:
   Good catch. Thanks.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] GH-38589: [C++][Gandiva] Support registering external C interface functions [arrow]

Posted by "niyue (via GitHub)" <gi...@apache.org>.
niyue commented on code in PR #38632:
URL: https://github.com/apache/arrow/pull/38632#discussion_r1386037885


##########
cpp/src/gandiva/engine.cc:
##########
@@ -146,8 +146,13 @@ Engine::Engine(const std::shared_ptr<Configuration>& conf,
 
 Status Engine::Init() {
   std::call_once(register_exported_funcs_flag, gandiva::RegisterExportedFuncs);
+  bool result = ExportedFuncsRegistry::Register(
+      std::make_shared<ExternalStubFunctions>(function_registry_));

Review Comment:
   External stub functions are registered to the engine here



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] GH-38589: [C++][Gandiva] Support registering external C interface functions [arrow]

Posted by "niyue (via GitHub)" <gi...@apache.org>.
niyue commented on code in PR #38632:
URL: https://github.com/apache/arrow/pull/38632#discussion_r1386760417


##########
cpp/src/gandiva/engine.cc:
##########
@@ -147,7 +147,7 @@ Engine::Engine(const std::shared_ptr<Configuration>& conf,
 Status Engine::Init() {
   std::call_once(register_exported_funcs_flag, gandiva::RegisterExportedFuncs);
   // Add mappings for global functions that can be accessed from LLVM/IR module.
-  AddGlobalMappings();
+  ARROW_RETURN_NOT_OK(AddGlobalMappings());

Review Comment:
   For external C interface functions, they may cause some error when adding mapping for them, so I change the `AddGlobalMappings` and `ExportedFuncsBase::AddMappings` functions to return `arrow::Status` to represent the result



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] GH-38589: [C++][Gandiva] Support registering external C functions [arrow]

Posted by "niyue (via GitHub)" <gi...@apache.org>.
niyue commented on code in PR #38632:
URL: https://github.com/apache/arrow/pull/38632#discussion_r1387917349


##########
cpp/src/gandiva/external_c_interface_functions.cc:
##########
@@ -0,0 +1,95 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License
+
+#include "llvm/IR/Type.h"
+
+#include "gandiva/engine.h"
+#include "gandiva/exported_funcs.h"
+
+namespace gandiva {
+static arrow::Result<llvm::Type*> AsLLVMType(const DataTypePtr& from_type,
+                                             LLVMTypes* types) {
+  switch (from_type->id()) {
+    case arrow::Type::BOOL:
+      return types->i1_type();
+    case arrow::Type::INT8:
+    case arrow::Type::UINT8:
+      return types->i8_type();
+    case arrow::Type::INT16:
+    case arrow::Type::UINT16:
+      return types->i16_type();
+    case arrow::Type::INT32:
+    case arrow::Type::UINT32:
+      return types->i32_type();
+    case arrow::Type::INT64:
+    case arrow::Type::UINT64:
+      return types->i64_type();
+    case arrow::Type::FLOAT:
+      return types->float_type();
+    case arrow::Type::DOUBLE:
+      return types->double_type();
+    default:
+      return Status::NotImplemented("Unsupported arrow data type: " +
+                                    from_type->ToString());
+  }
+}
+
+// map from a NativeFunction's signature to the corresponding LLVM signature
+static arrow::Result<std::pair<std::vector<llvm::Type*>, llvm::Type*>> MapToLLVMSignature(
+    const FunctionSignature& sig, const NativeFunction& func, LLVMTypes* types) {
+  std::vector<llvm::Type*> args;
+  args.reserve(sig.param_types().size());
+  if (func.NeedsContext()) {
+    args.emplace_back(types->i64_type());
+  }
+  if (func.NeedsFunctionHolder()) {
+    args.emplace_back(types->i64_type());
+  }
+  for (auto const& arg : sig.param_types()) {
+    if (arg->id() == arrow::Type::STRING) {
+      args.emplace_back(types->i8_ptr_type());
+      args.emplace_back(types->i32_type());
+    } else {
+      ARROW_ASSIGN_OR_RAISE(auto arg_llvm_type, AsLLVMType(arg, types));
+      args.emplace_back(arg_llvm_type);
+    }
+  }
+  llvm::Type* ret_llvm_type;
+  if (sig.ret_type()->id() == arrow::Type::STRING) {
+    // for string output, the last arg is the output length
+    args.emplace_back(types->i32_ptr_type());
+    ret_llvm_type = types->i8_ptr_type();
+  } else {
+    ARROW_ASSIGN_OR_RAISE(ret_llvm_type, AsLLVMType(sig.ret_type(), types));
+  }
+  auto return_type = AsLLVMType(sig.ret_type(), types);
+  return std::make_pair(args, ret_llvm_type);

Review Comment:
   You are right. Updated.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] GH-38589: [C++][Gandiva] Support registering external C interface functions [arrow]

Posted by "niyue (via GitHub)" <gi...@apache.org>.
niyue commented on code in PR #38632:
URL: https://github.com/apache/arrow/pull/38632#discussion_r1386760417


##########
cpp/src/gandiva/engine.cc:
##########
@@ -147,7 +147,7 @@ Engine::Engine(const std::shared_ptr<Configuration>& conf,
 Status Engine::Init() {
   std::call_once(register_exported_funcs_flag, gandiva::RegisterExportedFuncs);
   // Add mappings for global functions that can be accessed from LLVM/IR module.
-  AddGlobalMappings();
+  ARROW_RETURN_NOT_OK(AddGlobalMappings());

Review Comment:
   For external C interface functions, they may cause some error when adding mapping for them, so I change the `AddGlobalMappings` and `ExportedFuncsBase::AddMappings` functions to return `arrow::Status` to represent the result



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] GH-38589: [C++][Gandiva] Support registering external C functions [arrow]

Posted by "niyue (via GitHub)" <gi...@apache.org>.
niyue commented on code in PR #38632:
URL: https://github.com/apache/arrow/pull/38632#discussion_r1389135185


##########
cpp/src/gandiva/external_c_functions.cc:
##########
@@ -0,0 +1,94 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License
+
+#include "llvm/IR/Type.h"
+
+#include "gandiva/engine.h"
+#include "gandiva/exported_funcs.h"
+
+namespace gandiva {
+static arrow::Result<llvm::Type*> AsLLVMType(const DataTypePtr& from_type,
+                                             LLVMTypes* types) {
+  switch (from_type->id()) {
+    case arrow::Type::BOOL:
+      return types->i1_type();
+    case arrow::Type::INT8:
+    case arrow::Type::UINT8:
+      return types->i8_type();
+    case arrow::Type::INT16:
+    case arrow::Type::UINT16:
+      return types->i16_type();
+    case arrow::Type::INT32:
+    case arrow::Type::UINT32:
+      return types->i32_type();
+    case arrow::Type::INT64:
+    case arrow::Type::UINT64:
+      return types->i64_type();
+    case arrow::Type::FLOAT:
+      return types->float_type();
+    case arrow::Type::DOUBLE:
+      return types->double_type();
+    default:
+      return Status::NotImplemented("Unsupported arrow data type: " +
+                                    from_type->ToString());
+  }
+}
+
+// map from a NativeFunction's signature to the corresponding LLVM signature
+static arrow::Result<std::pair<std::vector<llvm::Type*>, llvm::Type*>> MapToLLVMSignature(
+    const FunctionSignature& sig, const NativeFunction& func, LLVMTypes* types) {
+  std::vector<llvm::Type*> arg_llvm_types;
+  arg_llvm_types.reserve(sig.param_types().size());
+  if (func.NeedsContext()) {
+    arg_llvm_types.emplace_back(types->i64_type());
+  }
+  if (func.NeedsFunctionHolder()) {
+    arg_llvm_types.emplace_back(types->i64_type());
+  }
+  for (auto const& arg : sig.param_types()) {
+    if (arg->id() == arrow::Type::STRING) {
+      arg_llvm_types.emplace_back(types->i8_ptr_type());
+      arg_llvm_types.emplace_back(types->i32_type());
+    } else {
+      ARROW_ASSIGN_OR_RAISE(auto arg_llvm_type, AsLLVMType(arg, types));
+      arg_llvm_types.emplace_back(arg_llvm_type);
+    }
+  }
+  llvm::Type* ret_llvm_type;
+  if (sig.ret_type()->id() == arrow::Type::STRING) {
+    // for string output, the last arg is the output length
+    arg_llvm_types.emplace_back(types->i32_ptr_type());

Review Comment:
   Sure. I update them to use `push_back()` now.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] GH-38589: [C++][Gandiva] Support registering external C functions [arrow]

Posted by "niyue (via GitHub)" <gi...@apache.org>.
niyue commented on code in PR #38632:
URL: https://github.com/apache/arrow/pull/38632#discussion_r1389148282


##########
cpp/src/gandiva/external_c_functions.cc:
##########
@@ -0,0 +1,94 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License
+
+#include "llvm/IR/Type.h"
+
+#include "gandiva/engine.h"
+#include "gandiva/exported_funcs.h"
+
+namespace gandiva {
+static arrow::Result<llvm::Type*> AsLLVMType(const DataTypePtr& from_type,

Review Comment:
   That is exactly what is needed here. Thanks so much for the pointer. I updated the code to use this API, and the code here is much more concise now.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] GH-38589: [C++][Gandiva] Support registering external C functions [arrow]

Posted by "niyue (via GitHub)" <gi...@apache.org>.
niyue commented on code in PR #38632:
URL: https://github.com/apache/arrow/pull/38632#discussion_r1387884305


##########
cpp/src/gandiva/function_holder_maker_registry.cc:
##########
@@ -0,0 +1,75 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "gandiva/function_holder_maker_registry.h"
+
+#include <functional>
+
+#include "gandiva/function_holder.h"
+#include "gandiva/interval_holder.h"
+#include "gandiva/random_generator_holder.h"
+#include "gandiva/regex_functions_holder.h"
+#include "gandiva/to_date_holder.h"
+
+namespace gandiva {
+
+FunctionHolderMakerRegistry::FunctionHolderMakerRegistry()
+    : function_holder_makers_(DefaultHolderMakers()) {}
+
+static std::string to_lower(const std::string& str) {
+  std::string data = str;
+  std::transform(data.begin(), data.end(), data.begin(),
+                 [](unsigned char c) { return std::tolower(c); });
+  return data;
+}
+arrow::Status FunctionHolderMakerRegistry::Register(const std::string& name,
+                                                    FunctionHolderMaker holder_maker) {
+  function_holder_makers_.emplace(to_lower(name), std::move(holder_maker));

Review Comment:
   Good to know. Updated. 



##########
cpp/src/gandiva/function_registry.cc:
##########
@@ -109,11 +109,35 @@ arrow::Status FunctionRegistry::Register(const std::vector<NativeFunction>& func
   return Status::OK();
 }
 
+arrow::Status FunctionRegistry::Register(
+    NativeFunction func, void* stub_function_ptr,

Review Comment:
   Fixed.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] GH-38589: [C++][Gandiva] Support registering external C functions [arrow]

Posted by "js8544 (via GitHub)" <gi...@apache.org>.
js8544 commented on code in PR #38632:
URL: https://github.com/apache/arrow/pull/38632#discussion_r1388819476


##########
cpp/src/gandiva/function_registry.cc:
##########
@@ -109,11 +109,35 @@ arrow::Status FunctionRegistry::Register(const std::vector<NativeFunction>& func
   return Status::OK();
 }
 
+arrow::Status FunctionRegistry::Register(
+    NativeFunction func, void* c_function_ptr,
+    std::optional<FunctionHolderMaker> function_holder_maker) {
+  if (function_holder_maker.has_value()) {
+    // all signatures should have the same base name, use the first signature's base name
+    auto const& func_base_name = func.signatures().begin()->base_name();
+    ARROW_RETURN_NOT_OK(holder_maker_registry_.Register(
+        func_base_name, std::move(function_holder_maker.value())));

Review Comment:
   I think it should be `std::move(function_holder_maker).value()` to make it a rvalue



##########
cpp/src/gandiva/expr_decomposer.cc:
##########
@@ -25,11 +25,12 @@
 
 #include "gandiva/annotator.h"
 #include "gandiva/dex.h"
-#include "gandiva/function_holder_registry.h"
+#include "gandiva/function_holder_maker_registry.h"
 #include "gandiva/function_registry.h"
 #include "gandiva/function_signature.h"
 #include "gandiva/in_holder.h"
 #include "gandiva/node.h"
+#include "gandiva/regex_functions_holder.h"

Review Comment:
   Is this include needed?



##########
cpp/src/gandiva/external_c_functions.cc:
##########
@@ -0,0 +1,94 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License
+
+#include "llvm/IR/Type.h"
+
+#include "gandiva/engine.h"
+#include "gandiva/exported_funcs.h"
+
+namespace gandiva {
+static arrow::Result<llvm::Type*> AsLLVMType(const DataTypePtr& from_type,

Review Comment:
   The `LLVMTypes` has a `IRType` function. Can it be reused here?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] GH-38589: [C++][Gandiva] Support registering external C functions [arrow]

Posted by "niyue (via GitHub)" <gi...@apache.org>.
niyue commented on code in PR #38632:
URL: https://github.com/apache/arrow/pull/38632#discussion_r1389140177


##########
cpp/src/gandiva/function_holder_maker_registry.cc:
##########
@@ -0,0 +1,72 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "gandiva/function_holder_maker_registry.h"
+
+#include <functional>
+
+#include "arrow/util/string.h"
+#include "gandiva/function_holder.h"
+#include "gandiva/interval_holder.h"
+#include "gandiva/random_generator_holder.h"
+#include "gandiva/regex_functions_holder.h"
+#include "gandiva/to_date_holder.h"
+
+namespace gandiva {
+
+using arrow::internal::AsciiToLower;
+
+FunctionHolderMakerRegistry::FunctionHolderMakerRegistry()
+    : function_holder_makers_(DefaultHolderMakers()) {}
+
+arrow::Status FunctionHolderMakerRegistry::Register(const std::string& name,
+                                                    FunctionHolderMaker holder_maker) {
+  function_holder_makers_.emplace(AsciiToLower(name), std::move(holder_maker));
+  return arrow::Status::OK();
+}
+
+template <typename HolderType>
+static arrow::Result<FunctionHolderPtr> HolderMaker(const FunctionNode& node) {
+  std::shared_ptr<HolderType> derived_instance;
+  ARROW_RETURN_NOT_OK(HolderType::Make(node, &derived_instance));
+  return derived_instance;
+}

Review Comment:
   Sure. If I make this change, we need to change some existing classes, like `LikeHolder`/`ReplaceHolder`/etc. They are `GANDIVA_EXPORT` classes, but I am not sure if they are really used externally. The code will be simpler after this refactoring. I am glad to submit another PR for this if this is desired.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] GH-38589: [C++][Gandiva] Support registering external C interface functions [arrow]

Posted by "niyue (via GitHub)" <gi...@apache.org>.
niyue commented on code in PR #38632:
URL: https://github.com/apache/arrow/pull/38632#discussion_r1386761436


##########
cpp/src/gandiva/decimal_xlarge.cc:
##########
@@ -38,7 +38,7 @@
 
 namespace gandiva {
 
-void ExportedDecimalFunctions::AddMappings(Engine* engine) const {
+arrow::Status ExportedDecimalFunctions::AddMappings(Engine* engine) const {

Review Comment:
   For external C interface functions, they may cause some error when adding mapping for them, so I change the `AddGlobalMappings` and `ExportedFuncsBase::AddMappings` functions to return arrow::Status to represent the result



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] GH-38589: [C++][Gandiva] Support registering external C interface functions [arrow]

Posted by "niyue (via GitHub)" <gi...@apache.org>.
niyue commented on code in PR #38632:
URL: https://github.com/apache/arrow/pull/38632#discussion_r1386047504


##########
cpp/src/gandiva/tests/projector_test.cc:
##########
@@ -3608,4 +3608,80 @@ TEST_F(TestProjector, TestExtendedFunctions) {
   EXPECT_ARROW_ARRAY_EQUALS(out, outs.at(0));
 }
 
+TEST_F(TestProjector, TestExtendedCInterfaceFunctions) {
+  auto in_field = field("in", arrow::int32());
+  auto schema = arrow::schema({in_field});
+  auto out_field = field("out", arrow::int64());
+  auto multiply =
+      TreeExprBuilder::MakeExpression("multiply_by_three", {in_field}, out_field);
+
+  std::shared_ptr<Projector> projector;
+  auto external_registry = std::make_shared<FunctionRegistry>();
+  auto config_with_func_registry =
+      TestConfigWithStubFunction(std::move(external_registry));
+  ARROW_EXPECT_OK(
+      Projector::Make(schema, {multiply}, config_with_func_registry, &projector));
+
+  int num_records = 4;
+  auto array = MakeArrowArrayInt32({1, 2, 3, 4}, {true, true, true, true});
+  auto in_batch = arrow::RecordBatch::Make(schema, num_records, {array});
+  auto out = MakeArrowArrayInt64({3, 6, 9, 12}, {true, true, true, true});
+
+  arrow::ArrayVector outs;
+  ARROW_EXPECT_OK(projector->Evaluate(*in_batch, pool_, &outs));
+  EXPECT_ARROW_ARRAY_EQUALS(out, outs.at(0));
+}
+
+TEST_F(TestProjector, TestExtendedCInterfaceFunctionsWithFunctionHolder) {

Review Comment:
   This test case demonstrates registering an external C interface function that uses function holder



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] GH-38589: [C++][Gandiva] Support registering external C interface functions [arrow]

Posted by "niyue (via GitHub)" <gi...@apache.org>.
niyue commented on PR #38632:
URL: https://github.com/apache/arrow/pull/38632#issuecomment-1802099367

   @js8544 @kou 
   As discussed in the recent PR https://github.com/apache/arrow/pull/38116, I submitted a follow up PR to further enhance Gandiva's external function registration capability, allowing registering `C interface functions` (`stub function` called by Gandiva internally) in this PR. Could you please help to review? Thanks. 
   
   ### notes
   * The name `C interface function` is different with `stub function` since I think `stub function` may not be easily understood externally, and I am looking for something equivalent to `LLVM IR based function` but I am not sure if there is a better term than `C interface function`, please advice.
   * I am not sure if I need to post a new discussion thread to the mailing list since it was discussed previously for PR 38116, and this PR is a smaller enhancement to the previous one. Please let me know if I need to post a new discussion thread for this. Thanks.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] GH-38589: [C++][Gandiva] Support registering external C functions [arrow]

Posted by "kou (via GitHub)" <gi...@apache.org>.
kou commented on code in PR #38632:
URL: https://github.com/apache/arrow/pull/38632#discussion_r1388534999


##########
cpp/src/gandiva/external_c_functions.cc:
##########
@@ -0,0 +1,94 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License
+
+#include "llvm/IR/Type.h"

Review Comment:
   ```suggestion
   #include <llvm/IR/Type.h>
   ```



##########
cpp/src/gandiva/external_c_functions.cc:
##########
@@ -0,0 +1,94 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License
+
+#include "llvm/IR/Type.h"
+
+#include "gandiva/engine.h"
+#include "gandiva/exported_funcs.h"
+
+namespace gandiva {
+static arrow::Result<llvm::Type*> AsLLVMType(const DataTypePtr& from_type,
+                                             LLVMTypes* types) {
+  switch (from_type->id()) {
+    case arrow::Type::BOOL:
+      return types->i1_type();
+    case arrow::Type::INT8:
+    case arrow::Type::UINT8:
+      return types->i8_type();
+    case arrow::Type::INT16:
+    case arrow::Type::UINT16:
+      return types->i16_type();
+    case arrow::Type::INT32:
+    case arrow::Type::UINT32:
+      return types->i32_type();
+    case arrow::Type::INT64:
+    case arrow::Type::UINT64:
+      return types->i64_type();
+    case arrow::Type::FLOAT:
+      return types->float_type();
+    case arrow::Type::DOUBLE:
+      return types->double_type();
+    default:
+      return Status::NotImplemented("Unsupported arrow data type: " +
+                                    from_type->ToString());
+  }
+}
+
+// map from a NativeFunction's signature to the corresponding LLVM signature
+static arrow::Result<std::pair<std::vector<llvm::Type*>, llvm::Type*>> MapToLLVMSignature(
+    const FunctionSignature& sig, const NativeFunction& func, LLVMTypes* types) {
+  std::vector<llvm::Type*> arg_llvm_types;
+  arg_llvm_types.reserve(sig.param_types().size());
+  if (func.NeedsContext()) {
+    arg_llvm_types.emplace_back(types->i64_type());
+  }
+  if (func.NeedsFunctionHolder()) {
+    arg_llvm_types.emplace_back(types->i64_type());
+  }
+  for (auto const& arg : sig.param_types()) {
+    if (arg->id() == arrow::Type::STRING) {
+      arg_llvm_types.emplace_back(types->i8_ptr_type());
+      arg_llvm_types.emplace_back(types->i32_type());
+    } else {
+      ARROW_ASSIGN_OR_RAISE(auto arg_llvm_type, AsLLVMType(arg, types));
+      arg_llvm_types.emplace_back(arg_llvm_type);
+    }
+  }
+  llvm::Type* ret_llvm_type;
+  if (sig.ret_type()->id() == arrow::Type::STRING) {
+    // for string output, the last arg is the output length
+    arg_llvm_types.emplace_back(types->i32_ptr_type());

Review Comment:
   It seems that `push_back()` is suitable than `emplace_back()` because we already have `llvm::Type*`.



##########
cpp/src/gandiva/function_holder_maker_registry.cc:
##########
@@ -0,0 +1,72 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "gandiva/function_holder_maker_registry.h"
+
+#include <functional>
+
+#include "arrow/util/string.h"
+#include "gandiva/function_holder.h"
+#include "gandiva/interval_holder.h"
+#include "gandiva/random_generator_holder.h"
+#include "gandiva/regex_functions_holder.h"
+#include "gandiva/to_date_holder.h"
+
+namespace gandiva {
+
+using arrow::internal::AsciiToLower;
+
+FunctionHolderMakerRegistry::FunctionHolderMakerRegistry()
+    : function_holder_makers_(DefaultHolderMakers()) {}
+
+arrow::Status FunctionHolderMakerRegistry::Register(const std::string& name,
+                                                    FunctionHolderMaker holder_maker) {
+  function_holder_makers_.emplace(AsciiToLower(name), std::move(holder_maker));
+  return arrow::Status::OK();
+}
+
+template <typename HolderType>
+static arrow::Result<FunctionHolderPtr> HolderMaker(const FunctionNode& node) {
+  std::shared_ptr<HolderType> derived_instance;
+  ARROW_RETURN_NOT_OK(HolderType::Make(node, &derived_instance));
+  return derived_instance;
+}

Review Comment:
   If we change `HolderType::Make()` to use `Result` instead of `Status` (e.g. `Status LikeHolder::Make(...)` -> `Result<std::shared_ptr<LikeHolder>> LikeHolder::Make(...)`), we can remove this helper template function?
   If so, we can do it as a follow-up task.



##########
cpp/src/gandiva/function_registry.cc:
##########
@@ -109,11 +109,35 @@ arrow::Status FunctionRegistry::Register(const std::vector<NativeFunction>& func
   return Status::OK();
 }
 
+arrow::Status FunctionRegistry::Register(
+    NativeFunction func, void* c_function_ptr,
+    std::optional<FunctionHolderMaker> function_holder_maker) {
+  if (function_holder_maker.has_value()) {
+    // all signatures should have the same base name, use the first signature's base name
+    auto const& func_base_name = func.signatures().begin()->base_name();
+    ARROW_RETURN_NOT_OK(holder_maker_registry_.Register(
+        func_base_name, std::move(function_holder_maker.value())));

Review Comment:
   Do we need this `std::move()`?
   I think that a function call result is already rvalue.



##########
cpp/src/gandiva/external_c_functions.cc:
##########
@@ -0,0 +1,94 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License
+
+#include "llvm/IR/Type.h"
+
+#include "gandiva/engine.h"
+#include "gandiva/exported_funcs.h"
+
+namespace gandiva {
+static arrow::Result<llvm::Type*> AsLLVMType(const DataTypePtr& from_type,
+                                             LLVMTypes* types) {
+  switch (from_type->id()) {
+    case arrow::Type::BOOL:
+      return types->i1_type();
+    case arrow::Type::INT8:
+    case arrow::Type::UINT8:
+      return types->i8_type();
+    case arrow::Type::INT16:
+    case arrow::Type::UINT16:
+      return types->i16_type();
+    case arrow::Type::INT32:
+    case arrow::Type::UINT32:
+      return types->i32_type();
+    case arrow::Type::INT64:
+    case arrow::Type::UINT64:
+      return types->i64_type();
+    case arrow::Type::FLOAT:
+      return types->float_type();
+    case arrow::Type::DOUBLE:
+      return types->double_type();
+    default:
+      return Status::NotImplemented("Unsupported arrow data type: " +
+                                    from_type->ToString());
+  }
+}
+
+// map from a NativeFunction's signature to the corresponding LLVM signature
+static arrow::Result<std::pair<std::vector<llvm::Type*>, llvm::Type*>> MapToLLVMSignature(
+    const FunctionSignature& sig, const NativeFunction& func, LLVMTypes* types) {
+  std::vector<llvm::Type*> arg_llvm_types;
+  arg_llvm_types.reserve(sig.param_types().size());

Review Comment:
   It seems that this size may be less than used size. Should we compute correct size here? 



##########
cpp/src/gandiva/function_registry.cc:
##########
@@ -109,11 +109,35 @@ arrow::Status FunctionRegistry::Register(const std::vector<NativeFunction>& func
   return Status::OK();
 }
 
+arrow::Status FunctionRegistry::Register(
+    NativeFunction func, void* c_function_ptr,
+    std::optional<FunctionHolderMaker> function_holder_maker) {
+  if (function_holder_maker.has_value()) {
+    // all signatures should have the same base name, use the first signature's base name
+    auto const& func_base_name = func.signatures().begin()->base_name();
+    ARROW_RETURN_NOT_OK(holder_maker_registry_.Register(
+        func_base_name, std::move(function_holder_maker.value())));
+  }
+  c_functions_.emplace_back(func, c_function_ptr);
+  ARROW_RETURN_NOT_OK(FunctionRegistry::Add(std::move(func)));
+  return Status::OK();

Review Comment:
   ```suggestion
     return FunctionRegistry::Add(std::move(func));
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] GH-38589: [C++][Gandiva] Support registering external C interface functions [arrow]

Posted by "niyue (via GitHub)" <gi...@apache.org>.
niyue commented on code in PR #38632:
URL: https://github.com/apache/arrow/pull/38632#discussion_r1386041257


##########
cpp/src/gandiva/external_c_interface_functions.cc:
##########
@@ -0,0 +1,95 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License
+
+#include "llvm/IR/Type.h"
+
+#include "gandiva/engine.h"
+#include "gandiva/exported_funcs.h"
+
+namespace gandiva {
+static arrow::Result<llvm::Type*> AsLLVMType(const DataTypePtr& from_type,
+                                             LLVMTypes* types) {
+  switch (from_type->id()) {
+    case arrow::Type::BOOL:
+      return types->i1_type();
+    case arrow::Type::INT8:
+    case arrow::Type::UINT8:
+      return types->i8_type();
+    case arrow::Type::INT16:
+    case arrow::Type::UINT16:
+      return types->i16_type();
+    case arrow::Type::INT32:
+    case arrow::Type::UINT32:
+      return types->i32_type();
+    case arrow::Type::INT64:
+    case arrow::Type::UINT64:
+      return types->i64_type();
+    case arrow::Type::FLOAT:
+      return types->float_type();
+    case arrow::Type::DOUBLE:
+      return types->double_type();
+    default:
+      return Status::NotImplemented("Unsupported arrow data type: " +
+                                    from_type->ToString());
+  }
+}
+
+// map from a NativeFunction's signature to the corresponding LLVM signature
+static arrow::Result<std::pair<std::vector<llvm::Type*>, llvm::Type*>> MapToLLVMSignature(
+    const FunctionSignature& sig, const NativeFunction& func, LLVMTypes* types) {
+  std::vector<llvm::Type*> args;
+  args.reserve(sig.param_types().size());
+  if (func.NeedsContext()) {
+    args.emplace_back(types->i64_type());
+  }
+  if (func.NeedsFunctionHolder()) {
+    args.emplace_back(types->i64_type());
+  }
+  for (auto const& arg : sig.param_types()) {
+    if (arg->id() == arrow::Type::STRING) {
+      args.emplace_back(types->i8_ptr_type());
+      args.emplace_back(types->i32_type());
+    } else {
+      ARROW_ASSIGN_OR_RAISE(auto arg_llvm_type, AsLLVMType(arg, types));
+      args.emplace_back(arg_llvm_type);
+    }
+  }
+  llvm::Type* ret_llvm_type;
+  if (sig.ret_type()->id() == arrow::Type::STRING) {
+    // for string output, the last arg is the output length
+    args.emplace_back(types->i32_ptr_type());
+    ret_llvm_type = types->i8_ptr_type();
+  } else {
+    ARROW_ASSIGN_OR_RAISE(ret_llvm_type, AsLLVMType(sig.ret_type(), types));
+  }
+  auto return_type = AsLLVMType(sig.ret_type(), types);
+  return std::make_pair(args, ret_llvm_type);
+}
+
+arrow::Status ExternalCInterfaceFunctions::AddMappings(Engine* engine) const {

Review Comment:
   Previously, the LLVM types of stub functions arguments are hard coded in the code base, now we automatically convert arrow data types of these function's parameter types into their LLVM equivalent types so that users don't have to specify similar info twice.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] GH-38589: [C++][Gandiva] Support registering external C interface functions [arrow]

Posted by "niyue (via GitHub)" <gi...@apache.org>.
niyue commented on code in PR #38632:
URL: https://github.com/apache/arrow/pull/38632#discussion_r1386041943


##########
cpp/src/gandiva/function_holder_registry.h:
##########
@@ -1,80 +0,0 @@
-// Licensed to the Apache Software Foundation (ASF) under one
-// or more contributor license agreements.  See the NOTICE file
-// distributed with this work for additional information
-// regarding copyright ownership.  The ASF licenses this file
-// to you under the Apache License, Version 2.0 (the
-// "License"); you may not use this file except in compliance
-// with the License.  You may obtain a copy of the License at
-//
-//   http://www.apache.org/licenses/LICENSE-2.0
-//
-// Unless required by applicable law or agreed to in writing,
-// software distributed under the License is distributed on an
-// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
-// KIND, either express or implied.  See the License for the
-// specific language governing permissions and limitations
-// under the License.
-
-#pragma once
-
-#include <functional>
-#include <memory>
-#include <string>
-#include <unordered_map>
-
-#include "arrow/status.h"
-#include "gandiva/function_holder.h"
-#include "gandiva/interval_holder.h"
-#include "gandiva/node.h"
-#include "gandiva/random_generator_holder.h"
-#include "gandiva/regex_functions_holder.h"
-#include "gandiva/to_date_holder.h"
-
-namespace gandiva {
-
-#define LAMBDA_MAKER(derived)                               \
-  [](const FunctionNode& node, FunctionHolderPtr* holder) { \
-    std::shared_ptr<derived> derived_instance;              \
-    auto status = derived::Make(node, &derived_instance);   \
-    if (status.ok()) {                                      \
-      *holder = derived_instance;                           \
-    }                                                       \
-    return status;                                          \
-  }
-
-/// Static registry of function holders.
-class FunctionHolderRegistry {
- public:
-  using maker_type = std::function<Status(const FunctionNode&, FunctionHolderPtr*)>;
-  using map_type = std::unordered_map<std::string, maker_type>;
-
-  static Status Make(const std::string& name, const FunctionNode& node,
-                     FunctionHolderPtr* holder) {
-    std::string data = name;
-    std::transform(data.begin(), data.end(), data.begin(),
-                   [](unsigned char c) { return std::tolower(c); });
-
-    auto found = makers().find(data);
-    if (found == makers().end()) {
-      return Status::Invalid("function holder not registered for function " + name);
-    }
-
-    return found->second(node, holder);
-  }
-
- private:
-  static map_type& makers() {

Review Comment:
   Move into `cc` file and make this state non static by copying them into each registry's instance.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] GH-38589: [C++][Gandiva] Support registering external C interface functions [arrow]

Posted by "niyue (via GitHub)" <gi...@apache.org>.
niyue commented on code in PR #38632:
URL: https://github.com/apache/arrow/pull/38632#discussion_r1386042380


##########
cpp/src/gandiva/function_registry.h:
##########
@@ -52,9 +58,24 @@ class GANDIVA_EXPORT FunctionRegistry {
   arrow::Status Register(const std::vector<NativeFunction>& funcs,
                          std::shared_ptr<arrow::Buffer> bitcode_buffer);
 
+  /// \brief register a C interface function into the function registry
+  /// @param func the registered function's metadata
+  /// @param c_interface_function_ptr the function pointer to the
+  /// registered function's implementation
+  /// @param function_holder_maker this will be used as the function holder if the
+  /// function requires a function holder
+  arrow::Status Register(
+      NativeFunction func, void* c_interface_function_ptr,
+      std::optional<FunctionHolderMaker> function_holder_maker = std::nullopt);

Review Comment:
   This is the new API used for registering external C interface functions. If a function requires function holder, it can use the function_holder_maker to specify it



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] GH-38589: [C++][Gandiva] Support registering external C functions [arrow]

Posted by "kou (via GitHub)" <gi...@apache.org>.
kou commented on PR #38632:
URL: https://github.com/apache/arrow/pull/38632#issuecomment-1815898439

   No objection. I'll merge this.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] GH-38589: [C++][Gandiva] Support registering external C interface functions [arrow]

Posted by "github-actions[bot] (via GitHub)" <gi...@apache.org>.
github-actions[bot] commented on PR #38632:
URL: https://github.com/apache/arrow/pull/38632#issuecomment-1801144112

   :warning: GitHub issue #38589 **has been automatically assigned in GitHub** to PR creator.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] GH-38589: [C++][Gandiva] Support registering external C functions [arrow]

Posted by "niyue (via GitHub)" <gi...@apache.org>.
niyue commented on code in PR #38632:
URL: https://github.com/apache/arrow/pull/38632#discussion_r1387883200


##########
cpp/src/gandiva/external_c_interface_functions.cc:
##########
@@ -0,0 +1,95 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License
+
+#include "llvm/IR/Type.h"
+
+#include "gandiva/engine.h"
+#include "gandiva/exported_funcs.h"
+
+namespace gandiva {
+static arrow::Result<llvm::Type*> AsLLVMType(const DataTypePtr& from_type,
+                                             LLVMTypes* types) {
+  switch (from_type->id()) {
+    case arrow::Type::BOOL:
+      return types->i1_type();
+    case arrow::Type::INT8:
+    case arrow::Type::UINT8:
+      return types->i8_type();
+    case arrow::Type::INT16:
+    case arrow::Type::UINT16:
+      return types->i16_type();
+    case arrow::Type::INT32:
+    case arrow::Type::UINT32:
+      return types->i32_type();
+    case arrow::Type::INT64:
+    case arrow::Type::UINT64:
+      return types->i64_type();
+    case arrow::Type::FLOAT:
+      return types->float_type();
+    case arrow::Type::DOUBLE:
+      return types->double_type();
+    default:
+      return Status::NotImplemented("Unsupported arrow data type: " +
+                                    from_type->ToString());
+  }
+}
+
+// map from a NativeFunction's signature to the corresponding LLVM signature
+static arrow::Result<std::pair<std::vector<llvm::Type*>, llvm::Type*>> MapToLLVMSignature(
+    const FunctionSignature& sig, const NativeFunction& func, LLVMTypes* types) {
+  std::vector<llvm::Type*> args;
+  args.reserve(sig.param_types().size());
+  if (func.NeedsContext()) {
+    args.emplace_back(types->i64_type());
+  }
+  if (func.NeedsFunctionHolder()) {
+    args.emplace_back(types->i64_type());
+  }
+  for (auto const& arg : sig.param_types()) {
+    if (arg->id() == arrow::Type::STRING) {
+      args.emplace_back(types->i8_ptr_type());
+      args.emplace_back(types->i32_type());
+    } else {
+      ARROW_ASSIGN_OR_RAISE(auto arg_llvm_type, AsLLVMType(arg, types));
+      args.emplace_back(arg_llvm_type);
+    }
+  }
+  llvm::Type* ret_llvm_type;
+  if (sig.ret_type()->id() == arrow::Type::STRING) {
+    // for string output, the last arg is the output length
+    args.emplace_back(types->i32_ptr_type());
+    ret_llvm_type = types->i8_ptr_type();
+  } else {
+    ARROW_ASSIGN_OR_RAISE(ret_llvm_type, AsLLVMType(sig.ret_type(), types));
+  }
+  auto return_type = AsLLVMType(sig.ret_type(), types);

Review Comment:
   Good catch. I did some refactoring previously and extracted some code as dedicated functions but this line of code was left during refactoring. Fixed now.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] GH-38589: [C++][Gandiva] Support registering external C interface functions [arrow]

Posted by "niyue (via GitHub)" <gi...@apache.org>.
niyue commented on code in PR #38632:
URL: https://github.com/apache/arrow/pull/38632#discussion_r1386042380


##########
cpp/src/gandiva/function_registry.h:
##########
@@ -52,9 +58,24 @@ class GANDIVA_EXPORT FunctionRegistry {
   arrow::Status Register(const std::vector<NativeFunction>& funcs,
                          std::shared_ptr<arrow::Buffer> bitcode_buffer);
 
+  /// \brief register a C interface function into the function registry
+  /// @param func the registered function's metadata
+  /// @param c_interface_function_ptr the function pointer to the
+  /// registered function's implementation
+  /// @param function_holder_maker this will be used as the function holder if the
+  /// function requires a function holder
+  arrow::Status Register(
+      NativeFunction func, void* c_interface_function_ptr,
+      std::optional<FunctionHolderMaker> function_holder_maker = std::nullopt);

Review Comment:
   This is the new API used for registering external C interface functions. If a function requires function holder, it can use the `function_holder_maker` to specify it



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] GH-38589: [C++][Gandiva] Support registering external C functions [arrow]

Posted by "niyue (via GitHub)" <gi...@apache.org>.
niyue commented on code in PR #38632:
URL: https://github.com/apache/arrow/pull/38632#discussion_r1389147440


##########
cpp/src/gandiva/expr_decomposer.cc:
##########
@@ -25,11 +25,12 @@
 
 #include "gandiva/annotator.h"
 #include "gandiva/dex.h"
-#include "gandiva/function_holder_registry.h"
+#include "gandiva/function_holder_maker_registry.h"
 #include "gandiva/function_registry.h"
 #include "gandiva/function_signature.h"
 #include "gandiva/in_holder.h"
 #include "gandiva/node.h"
+#include "gandiva/regex_functions_holder.h"

Review Comment:
   I found `LikeHolder` is used below but its header is not included so I added it (it is likely included by other header files ). Let me know if this is not recommended in the project (if not included, I ran into some issues in other projects, during refactoring, a indirectly included header file was removed, and causing the other file failed to be compiled)



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] GH-38589: [C++][Gandiva] Support registering external C functions [arrow]

Posted by "niyue (via GitHub)" <gi...@apache.org>.
niyue commented on code in PR #38632:
URL: https://github.com/apache/arrow/pull/38632#discussion_r1390555984


##########
cpp/src/gandiva/external_c_functions.cc:
##########
@@ -0,0 +1,75 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License
+
+#include <llvm/IR/Type.h>
+
+#include "gandiva/engine.h"
+#include "gandiva/exported_funcs.h"
+
+namespace gandiva {
+// calculate the number of arguments for a function signature
+static size_t GetNumArgs(const FunctionSignature& sig, const NativeFunction& func) {

Review Comment:
   Done.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] GH-38589: [C++][Gandiva] Support registering external C functions [arrow]

Posted by "kou (via GitHub)" <gi...@apache.org>.
kou merged PR #38632:
URL: https://github.com/apache/arrow/pull/38632


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] GH-38589: [C++][Gandiva] Support registering external C functions [arrow]

Posted by "conbench-apache-arrow[bot] (via GitHub)" <gi...@apache.org>.
conbench-apache-arrow[bot] commented on PR #38632:
URL: https://github.com/apache/arrow/pull/38632#issuecomment-1816183577

   After merging your PR, Conbench analyzed the 5 benchmarking runs that have been run so far on merge-commit c353c81c9d92bfb72b7f033f5870d09e249bfdb7.
   
   There were no benchmark performance regressions. 🎉
   
   The [full Conbench report](https://github.com/apache/arrow/runs/18781589303) has more details. It also includes information about 2 possible false positives for unstable benchmarks that are known to sometimes produce them.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] GH-38589: [C++][Gandiva] Support registering external C functions [arrow]

Posted by "niyue (via GitHub)" <gi...@apache.org>.
niyue commented on code in PR #38632:
URL: https://github.com/apache/arrow/pull/38632#discussion_r1389137076


##########
cpp/src/gandiva/external_c_functions.cc:
##########
@@ -0,0 +1,94 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License
+
+#include "llvm/IR/Type.h"
+
+#include "gandiva/engine.h"
+#include "gandiva/exported_funcs.h"
+
+namespace gandiva {
+static arrow::Result<llvm::Type*> AsLLVMType(const DataTypePtr& from_type,
+                                             LLVMTypes* types) {
+  switch (from_type->id()) {
+    case arrow::Type::BOOL:
+      return types->i1_type();
+    case arrow::Type::INT8:
+    case arrow::Type::UINT8:
+      return types->i8_type();
+    case arrow::Type::INT16:
+    case arrow::Type::UINT16:
+      return types->i16_type();
+    case arrow::Type::INT32:
+    case arrow::Type::UINT32:
+      return types->i32_type();
+    case arrow::Type::INT64:
+    case arrow::Type::UINT64:
+      return types->i64_type();
+    case arrow::Type::FLOAT:
+      return types->float_type();
+    case arrow::Type::DOUBLE:
+      return types->double_type();
+    default:
+      return Status::NotImplemented("Unsupported arrow data type: " +
+                                    from_type->ToString());
+  }
+}
+
+// map from a NativeFunction's signature to the corresponding LLVM signature
+static arrow::Result<std::pair<std::vector<llvm::Type*>, llvm::Type*>> MapToLLVMSignature(
+    const FunctionSignature& sig, const NativeFunction& func, LLVMTypes* types) {
+  std::vector<llvm::Type*> arg_llvm_types;
+  arg_llvm_types.reserve(sig.param_types().size());

Review Comment:
   Initially I don't want to calculate the size since it may duplicate some code but it doesn't seem too bad after I write it, and I add a new function to calculate the number of arguments up front now.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] GH-38589: [C++][Gandiva] Support registering external C functions [arrow]

Posted by "niyue (via GitHub)" <gi...@apache.org>.
niyue commented on code in PR #38632:
URL: https://github.com/apache/arrow/pull/38632#discussion_r1389134581


##########
cpp/src/gandiva/external_c_functions.cc:
##########
@@ -0,0 +1,94 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License
+
+#include "llvm/IR/Type.h"

Review Comment:
   Fixed.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] GH-38589: [C++][Gandiva] Support registering external C functions [arrow]

Posted by "niyue (via GitHub)" <gi...@apache.org>.
niyue commented on code in PR #38632:
URL: https://github.com/apache/arrow/pull/38632#discussion_r1389148282


##########
cpp/src/gandiva/external_c_functions.cc:
##########
@@ -0,0 +1,94 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License
+
+#include "llvm/IR/Type.h"
+
+#include "gandiva/engine.h"
+#include "gandiva/exported_funcs.h"
+
+namespace gandiva {
+static arrow::Result<llvm::Type*> AsLLVMType(const DataTypePtr& from_type,

Review Comment:
   That is exactly what is needed here. Thanks so much for the pointer.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] GH-38589: [C++][Gandiva] Support registering external C functions [arrow]

Posted by "niyue (via GitHub)" <gi...@apache.org>.
niyue commented on code in PR #38632:
URL: https://github.com/apache/arrow/pull/38632#discussion_r1389142043


##########
cpp/src/gandiva/function_registry.cc:
##########
@@ -109,11 +109,35 @@ arrow::Status FunctionRegistry::Register(const std::vector<NativeFunction>& func
   return Status::OK();
 }
 
+arrow::Status FunctionRegistry::Register(
+    NativeFunction func, void* c_function_ptr,
+    std::optional<FunctionHolderMaker> function_holder_maker) {
+  if (function_holder_maker.has_value()) {
+    // all signatures should have the same base name, use the first signature's base name
+    auto const& func_base_name = func.signatures().begin()->base_name();
+    ARROW_RETURN_NOT_OK(holder_maker_registry_.Register(
+        func_base_name, std::move(function_holder_maker.value())));

Review Comment:
   Fat finger. I change it to `std::move(function_holder_maker).value()` now.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] GH-38589: [C++][Gandiva] Support registering external C interface functions [arrow]

Posted by "niyue (via GitHub)" <gi...@apache.org>.
niyue commented on code in PR #38632:
URL: https://github.com/apache/arrow/pull/38632#discussion_r1386037885


##########
cpp/src/gandiva/engine.cc:
##########
@@ -146,8 +146,13 @@ Engine::Engine(const std::shared_ptr<Configuration>& conf,
 
 Status Engine::Init() {
   std::call_once(register_exported_funcs_flag, gandiva::RegisterExportedFuncs);
+  bool result = ExportedFuncsRegistry::Register(
+      std::make_shared<ExternalStubFunctions>(function_registry_));

Review Comment:
   External stub functions are registered to the engine here



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] GH-38589: [C++][Gandiva] Support registering external C functions [arrow]

Posted by "niyue (via GitHub)" <gi...@apache.org>.
niyue commented on code in PR #38632:
URL: https://github.com/apache/arrow/pull/38632#discussion_r1387883990


##########
cpp/src/gandiva/external_c_interface_functions.cc:
##########
@@ -0,0 +1,95 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License
+
+#include "llvm/IR/Type.h"
+
+#include "gandiva/engine.h"
+#include "gandiva/exported_funcs.h"
+
+namespace gandiva {
+static arrow::Result<llvm::Type*> AsLLVMType(const DataTypePtr& from_type,
+                                             LLVMTypes* types) {
+  switch (from_type->id()) {
+    case arrow::Type::BOOL:
+      return types->i1_type();
+    case arrow::Type::INT8:
+    case arrow::Type::UINT8:
+      return types->i8_type();
+    case arrow::Type::INT16:
+    case arrow::Type::UINT16:
+      return types->i16_type();
+    case arrow::Type::INT32:
+    case arrow::Type::UINT32:
+      return types->i32_type();
+    case arrow::Type::INT64:
+    case arrow::Type::UINT64:
+      return types->i64_type();
+    case arrow::Type::FLOAT:
+      return types->float_type();
+    case arrow::Type::DOUBLE:
+      return types->double_type();
+    default:
+      return Status::NotImplemented("Unsupported arrow data type: " +
+                                    from_type->ToString());
+  }
+}
+
+// map from a NativeFunction's signature to the corresponding LLVM signature
+static arrow::Result<std::pair<std::vector<llvm::Type*>, llvm::Type*>> MapToLLVMSignature(
+    const FunctionSignature& sig, const NativeFunction& func, LLVMTypes* types) {
+  std::vector<llvm::Type*> args;

Review Comment:
   Good suggestion. I renamed it to `args_llvm_types` now.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] GH-38589: [C++][Gandiva] Support registering external C functions [arrow]

Posted by "niyue (via GitHub)" <gi...@apache.org>.
niyue commented on code in PR #38632:
URL: https://github.com/apache/arrow/pull/38632#discussion_r1387912117


##########
cpp/src/gandiva/tests/test_util.cc:
##########
@@ -42,11 +46,129 @@ NativeFunction GetTestExternalFunction() {
   return multiply_by_two_func;
 }
 
-std::shared_ptr<Configuration> TestConfigurationWithFunctionRegistry(
+static NativeFunction GetTestExternalCInterfaceFunction() {
+  NativeFunction multiply_by_three_func(
+      "multiply_by_three", {}, {arrow::int32()}, arrow::int64(),
+      ResultNullableType::kResultNullIfNull, "multiply_by_three_int32");
+  return multiply_by_three_func;
+}
+
+static NativeFunction GetTestFunctionWithFunctionHolder() {
+  // the 2nd parameter is expected to be an int32 literal
+  NativeFunction multiply_by_n_func("multiply_by_n", {}, {arrow::int32(), arrow::int32()},
+                                    arrow::int64(), ResultNullableType::kResultNullIfNull,
+                                    "multiply_by_n_int32_int32",
+                                    NativeFunction::kNeedsFunctionHolder);
+  return multiply_by_n_func;
+}
+
+static NativeFunction GetTestFunctionWithContext() {
+  NativeFunction multiply_by_two_formula(
+      "multiply_by_two_formula", {}, {arrow::utf8()}, arrow::utf8(),
+      ResultNullableType::kResultNullIfNull, "multiply_by_two_formula_utf8",
+      NativeFunction::kNeedsContext);
+  return multiply_by_two_formula;
+}
+
+static std::shared_ptr<Configuration> BuildConfigurationWithRegistry(
+    std::shared_ptr<FunctionRegistry> registry,
+    const std::function<arrow::Status(std::shared_ptr<FunctionRegistry>)>&
+        register_func) {
+  ARROW_EXPECT_OK(register_func(registry));
+  return ConfigurationBuilder().build(std::move(registry));
+}
+
+std::shared_ptr<Configuration> TestConfigWithFunctionRegistry(
+    std::shared_ptr<FunctionRegistry> registry) {
+  return BuildConfigurationWithRegistry(std::move(registry), [](auto reg) {
+    return reg->Register({GetTestExternalFunction()}, GetTestFunctionLLVMIRPath());
+  });
+}
+
+class MultiplyHolder : public FunctionHolder {
+ public:
+  explicit MultiplyHolder(int32_t num) : num_(num) {}
+
+  static Status Make(const FunctionNode& node, std::shared_ptr<MultiplyHolder>* holder) {

Review Comment:
   I used this style for consistency with existing holder function makers. But actually I do feel it is simpler to return `Result<std::shared_ptr<MultiplyHolder>>`. Now I change it to return `arrow::Result`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] GH-38589: [C++][Gandiva] Support registering external C functions [arrow]

Posted by "niyue (via GitHub)" <gi...@apache.org>.
niyue commented on PR #38632:
URL: https://github.com/apache/arrow/pull/38632#issuecomment-1803723645

   > How about naming "stub function" as "C function" not "C interface function"?
   
   Thanks for the suggestion. I rename the PR/corresponding issue description/code to use the name `C function` now.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] GH-38589: [C++][Gandiva] Support registering external C interface functions [arrow]

Posted by "kou (via GitHub)" <gi...@apache.org>.
kou commented on code in PR #38632:
URL: https://github.com/apache/arrow/pull/38632#discussion_r1387382042


##########
cpp/src/gandiva/external_c_interface_functions.cc:
##########
@@ -0,0 +1,95 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License
+
+#include "llvm/IR/Type.h"
+
+#include "gandiva/engine.h"
+#include "gandiva/exported_funcs.h"
+
+namespace gandiva {
+static arrow::Result<llvm::Type*> AsLLVMType(const DataTypePtr& from_type,
+                                             LLVMTypes* types) {
+  switch (from_type->id()) {
+    case arrow::Type::BOOL:
+      return types->i1_type();
+    case arrow::Type::INT8:
+    case arrow::Type::UINT8:
+      return types->i8_type();
+    case arrow::Type::INT16:
+    case arrow::Type::UINT16:
+      return types->i16_type();
+    case arrow::Type::INT32:
+    case arrow::Type::UINT32:
+      return types->i32_type();
+    case arrow::Type::INT64:
+    case arrow::Type::UINT64:
+      return types->i64_type();
+    case arrow::Type::FLOAT:
+      return types->float_type();
+    case arrow::Type::DOUBLE:
+      return types->double_type();
+    default:
+      return Status::NotImplemented("Unsupported arrow data type: " +
+                                    from_type->ToString());
+  }
+}
+
+// map from a NativeFunction's signature to the corresponding LLVM signature
+static arrow::Result<std::pair<std::vector<llvm::Type*>, llvm::Type*>> MapToLLVMSignature(
+    const FunctionSignature& sig, const NativeFunction& func, LLVMTypes* types) {
+  std::vector<llvm::Type*> args;
+  args.reserve(sig.param_types().size());
+  if (func.NeedsContext()) {
+    args.emplace_back(types->i64_type());
+  }
+  if (func.NeedsFunctionHolder()) {
+    args.emplace_back(types->i64_type());
+  }
+  for (auto const& arg : sig.param_types()) {
+    if (arg->id() == arrow::Type::STRING) {
+      args.emplace_back(types->i8_ptr_type());
+      args.emplace_back(types->i32_type());
+    } else {
+      ARROW_ASSIGN_OR_RAISE(auto arg_llvm_type, AsLLVMType(arg, types));
+      args.emplace_back(arg_llvm_type);
+    }
+  }
+  llvm::Type* ret_llvm_type;
+  if (sig.ret_type()->id() == arrow::Type::STRING) {
+    // for string output, the last arg is the output length
+    args.emplace_back(types->i32_ptr_type());
+    ret_llvm_type = types->i8_ptr_type();
+  } else {
+    ARROW_ASSIGN_OR_RAISE(ret_llvm_type, AsLLVMType(sig.ret_type(), types));
+  }
+  auto return_type = AsLLVMType(sig.ret_type(), types);

Review Comment:
   It seems that this isn't used.



##########
cpp/src/gandiva/external_c_interface_functions.cc:
##########
@@ -0,0 +1,95 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License
+
+#include "llvm/IR/Type.h"
+
+#include "gandiva/engine.h"
+#include "gandiva/exported_funcs.h"
+
+namespace gandiva {
+static arrow::Result<llvm::Type*> AsLLVMType(const DataTypePtr& from_type,
+                                             LLVMTypes* types) {
+  switch (from_type->id()) {
+    case arrow::Type::BOOL:
+      return types->i1_type();
+    case arrow::Type::INT8:
+    case arrow::Type::UINT8:
+      return types->i8_type();
+    case arrow::Type::INT16:
+    case arrow::Type::UINT16:
+      return types->i16_type();
+    case arrow::Type::INT32:
+    case arrow::Type::UINT32:
+      return types->i32_type();
+    case arrow::Type::INT64:
+    case arrow::Type::UINT64:
+      return types->i64_type();
+    case arrow::Type::FLOAT:
+      return types->float_type();
+    case arrow::Type::DOUBLE:
+      return types->double_type();
+    default:
+      return Status::NotImplemented("Unsupported arrow data type: " +
+                                    from_type->ToString());
+  }
+}
+
+// map from a NativeFunction's signature to the corresponding LLVM signature
+static arrow::Result<std::pair<std::vector<llvm::Type*>, llvm::Type*>> MapToLLVMSignature(
+    const FunctionSignature& sig, const NativeFunction& func, LLVMTypes* types) {
+  std::vector<llvm::Type*> args;
+  args.reserve(sig.param_types().size());
+  if (func.NeedsContext()) {
+    args.emplace_back(types->i64_type());
+  }
+  if (func.NeedsFunctionHolder()) {
+    args.emplace_back(types->i64_type());
+  }
+  for (auto const& arg : sig.param_types()) {
+    if (arg->id() == arrow::Type::STRING) {
+      args.emplace_back(types->i8_ptr_type());
+      args.emplace_back(types->i32_type());
+    } else {
+      ARROW_ASSIGN_OR_RAISE(auto arg_llvm_type, AsLLVMType(arg, types));
+      args.emplace_back(arg_llvm_type);
+    }
+  }
+  llvm::Type* ret_llvm_type;
+  if (sig.ret_type()->id() == arrow::Type::STRING) {
+    // for string output, the last arg is the output length
+    args.emplace_back(types->i32_ptr_type());
+    ret_llvm_type = types->i8_ptr_type();
+  } else {
+    ARROW_ASSIGN_OR_RAISE(ret_llvm_type, AsLLVMType(sig.ret_type(), types));
+  }
+  auto return_type = AsLLVMType(sig.ret_type(), types);
+  return std::make_pair(args, ret_llvm_type);

Review Comment:
   Do we need `std::move()` for `args` here?
   
   ```suggestion
     return std::make_pair(std::move(args), ret_llvm_type);
   ```



##########
cpp/src/gandiva/external_c_interface_functions.cc:
##########
@@ -0,0 +1,95 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License
+
+#include "llvm/IR/Type.h"
+
+#include "gandiva/engine.h"
+#include "gandiva/exported_funcs.h"
+
+namespace gandiva {
+static arrow::Result<llvm::Type*> AsLLVMType(const DataTypePtr& from_type,
+                                             LLVMTypes* types) {
+  switch (from_type->id()) {
+    case arrow::Type::BOOL:
+      return types->i1_type();
+    case arrow::Type::INT8:
+    case arrow::Type::UINT8:
+      return types->i8_type();
+    case arrow::Type::INT16:
+    case arrow::Type::UINT16:
+      return types->i16_type();
+    case arrow::Type::INT32:
+    case arrow::Type::UINT32:
+      return types->i32_type();
+    case arrow::Type::INT64:
+    case arrow::Type::UINT64:
+      return types->i64_type();
+    case arrow::Type::FLOAT:
+      return types->float_type();
+    case arrow::Type::DOUBLE:
+      return types->double_type();
+    default:
+      return Status::NotImplemented("Unsupported arrow data type: " +
+                                    from_type->ToString());
+  }
+}
+
+// map from a NativeFunction's signature to the corresponding LLVM signature
+static arrow::Result<std::pair<std::vector<llvm::Type*>, llvm::Type*>> MapToLLVMSignature(
+    const FunctionSignature& sig, const NativeFunction& func, LLVMTypes* types) {
+  std::vector<llvm::Type*> args;

Review Comment:
   `arg_types` or `arg_llvm_type` may be better because we use `ret_llvm_type` for a return type.



##########
cpp/src/gandiva/tests/test_util.cc:
##########
@@ -42,11 +46,129 @@ NativeFunction GetTestExternalFunction() {
   return multiply_by_two_func;
 }
 
-std::shared_ptr<Configuration> TestConfigurationWithFunctionRegistry(
+static NativeFunction GetTestExternalCInterfaceFunction() {
+  NativeFunction multiply_by_three_func(
+      "multiply_by_three", {}, {arrow::int32()}, arrow::int64(),
+      ResultNullableType::kResultNullIfNull, "multiply_by_three_int32");
+  return multiply_by_three_func;
+}
+
+static NativeFunction GetTestFunctionWithFunctionHolder() {
+  // the 2nd parameter is expected to be an int32 literal
+  NativeFunction multiply_by_n_func("multiply_by_n", {}, {arrow::int32(), arrow::int32()},
+                                    arrow::int64(), ResultNullableType::kResultNullIfNull,
+                                    "multiply_by_n_int32_int32",
+                                    NativeFunction::kNeedsFunctionHolder);
+  return multiply_by_n_func;
+}
+
+static NativeFunction GetTestFunctionWithContext() {
+  NativeFunction multiply_by_two_formula(
+      "multiply_by_two_formula", {}, {arrow::utf8()}, arrow::utf8(),
+      ResultNullableType::kResultNullIfNull, "multiply_by_two_formula_utf8",
+      NativeFunction::kNeedsContext);
+  return multiply_by_two_formula;
+}
+
+static std::shared_ptr<Configuration> BuildConfigurationWithRegistry(
+    std::shared_ptr<FunctionRegistry> registry,
+    const std::function<arrow::Status(std::shared_ptr<FunctionRegistry>)>&
+        register_func) {
+  ARROW_EXPECT_OK(register_func(registry));
+  return ConfigurationBuilder().build(std::move(registry));
+}
+
+std::shared_ptr<Configuration> TestConfigWithFunctionRegistry(
+    std::shared_ptr<FunctionRegistry> registry) {
+  return BuildConfigurationWithRegistry(std::move(registry), [](auto reg) {
+    return reg->Register({GetTestExternalFunction()}, GetTestFunctionLLVMIRPath());
+  });
+}
+
+class MultiplyHolder : public FunctionHolder {
+ public:
+  explicit MultiplyHolder(int32_t num) : num_(num) {}
+
+  static Status Make(const FunctionNode& node, std::shared_ptr<MultiplyHolder>* holder) {

Review Comment:
   How about using `Result` instead of `Status`?
   
   ```suggestion
     static Result<std::shared_ptr<MultiplyHolder>> Make(const FunctionNode& node) {
   ```



##########
cpp/src/gandiva/tests/test_util.cc:
##########
@@ -42,11 +46,129 @@ NativeFunction GetTestExternalFunction() {
   return multiply_by_two_func;
 }
 
-std::shared_ptr<Configuration> TestConfigurationWithFunctionRegistry(
+static NativeFunction GetTestExternalCInterfaceFunction() {
+  NativeFunction multiply_by_three_func(
+      "multiply_by_three", {}, {arrow::int32()}, arrow::int64(),
+      ResultNullableType::kResultNullIfNull, "multiply_by_three_int32");
+  return multiply_by_three_func;
+}
+
+static NativeFunction GetTestFunctionWithFunctionHolder() {
+  // the 2nd parameter is expected to be an int32 literal
+  NativeFunction multiply_by_n_func("multiply_by_n", {}, {arrow::int32(), arrow::int32()},
+                                    arrow::int64(), ResultNullableType::kResultNullIfNull,
+                                    "multiply_by_n_int32_int32",
+                                    NativeFunction::kNeedsFunctionHolder);
+  return multiply_by_n_func;
+}
+
+static NativeFunction GetTestFunctionWithContext() {
+  NativeFunction multiply_by_two_formula(
+      "multiply_by_two_formula", {}, {arrow::utf8()}, arrow::utf8(),
+      ResultNullableType::kResultNullIfNull, "multiply_by_two_formula_utf8",
+      NativeFunction::kNeedsContext);
+  return multiply_by_two_formula;
+}
+
+static std::shared_ptr<Configuration> BuildConfigurationWithRegistry(
+    std::shared_ptr<FunctionRegistry> registry,
+    const std::function<arrow::Status(std::shared_ptr<FunctionRegistry>)>&
+        register_func) {
+  ARROW_EXPECT_OK(register_func(registry));
+  return ConfigurationBuilder().build(std::move(registry));
+}
+
+std::shared_ptr<Configuration> TestConfigWithFunctionRegistry(
+    std::shared_ptr<FunctionRegistry> registry) {
+  return BuildConfigurationWithRegistry(std::move(registry), [](auto reg) {
+    return reg->Register({GetTestExternalFunction()}, GetTestFunctionLLVMIRPath());
+  });
+}
+
+class MultiplyHolder : public FunctionHolder {
+ public:
+  explicit MultiplyHolder(int32_t num) : num_(num) {}
+
+  static Status Make(const FunctionNode& node, std::shared_ptr<MultiplyHolder>* holder) {
+    ARROW_RETURN_IF(node.children().size() != 2,
+                    Status::Invalid("'multiply_by_n' function requires two parameters"));
+
+    auto literal = dynamic_cast<LiteralNode*>(node.children().at(1).get());
+    ARROW_RETURN_IF(
+        literal == nullptr,
+        Status::Invalid(
+            "'multiply_by_n' function requires a literal as the 2nd parameter"));
+
+    auto literal_type = literal->return_type()->id();
+    ARROW_RETURN_IF(
+        literal_type != arrow::Type::INT32,
+        Status::Invalid(
+            "'multiply_by_n' function requires an int32 literal as the 2nd parameter"));
+
+    *holder = std::make_shared<MultiplyHolder>(
+        literal->is_null() ? 0 : std::get<int32_t>(literal->holder()));
+    return Status::OK();
+  }
+
+  int32_t operator()() const { return num_; }
+
+ private:
+  int32_t num_;
+};
+
+extern "C" {
+// this function is used as an external stub function for testing so it has to be declared

Review Comment:
   ```suggestion
   // this function is used as an external C interface function for testing so it has to be declared
   ```



##########
cpp/src/gandiva/function_holder_maker_registry.cc:
##########
@@ -0,0 +1,75 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "gandiva/function_holder_maker_registry.h"
+
+#include <functional>
+
+#include "gandiva/function_holder.h"
+#include "gandiva/interval_holder.h"
+#include "gandiva/random_generator_holder.h"
+#include "gandiva/regex_functions_holder.h"
+#include "gandiva/to_date_holder.h"
+
+namespace gandiva {
+
+FunctionHolderMakerRegistry::FunctionHolderMakerRegistry()
+    : function_holder_makers_(DefaultHolderMakers()) {}
+
+static std::string to_lower(const std::string& str) {
+  std::string data = str;
+  std::transform(data.begin(), data.end(), data.begin(),
+                 [](unsigned char c) { return std::tolower(c); });
+  return data;
+}
+arrow::Status FunctionHolderMakerRegistry::Register(const std::string& name,
+                                                    FunctionHolderMaker holder_maker) {
+  function_holder_makers_.emplace(to_lower(name), std::move(holder_maker));

Review Comment:
   How about using existing `arrow::internal::AsciiToLower()`?
   
   ```suggestion
     function_holder_makers_.emplace(arrow::internal::AsciiToLower(name), std::move(holder_maker));
   ```



##########
cpp/src/gandiva/function_registry.cc:
##########
@@ -109,11 +109,35 @@ arrow::Status FunctionRegistry::Register(const std::vector<NativeFunction>& func
   return Status::OK();
 }
 
+arrow::Status FunctionRegistry::Register(
+    NativeFunction func, void* stub_function_ptr,

Review Comment:
   ```suggestion
       NativeFunction func, void* c_interface_function_ptr,
   ```



##########
cpp/src/gandiva/gdv_string_function_stubs.cc:
##########
@@ -15,7 +15,7 @@
 // specific language governing permissions and limitations
 // under the License.
 
-//#pragma once
+// #pragma once

Review Comment:
   Can we remove this needless comment?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org