You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@arrow.apache.org by ap...@apache.org on 2022/05/03 07:46:23 UTC

[arrow] branch master updated: ARROW-15639 [C++][Python] UDF Scalar Function Implementation

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

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


The following commit(s) were added to refs/heads/master by this push:
     new 7a0f00c16e ARROW-15639 [C++][Python] UDF Scalar Function Implementation
7a0f00c16e is described below

commit 7a0f00c16e084d194ae53d209b33b809cfc8f2d5
Author: Vibhatha Abeykoon <vi...@gmail.com>
AuthorDate: Tue May 3 09:46:11 2022 +0200

    ARROW-15639 [C++][Python] UDF Scalar Function Implementation
    
    PR for Scalar UDF integration
    
    This is the first phase of UDF integration to Arrow. This version only includes ScalarFunctions.
    In future of PRs, Vector UDF (using Arrow VectorFunction), UDTF (user-defined table function)
    and Aggregation UDFs will be integrated. This PR includes the following;
    
    - [x] UDF Python Scalar Function registration and usage
    - [x] UDF Python Scalar Function Examples
    - [x] UDF Python Scalar Function test cases
    - [x] UDF C++ Example extended from Compute Function Example
    - [x] Added aggregation example (optional to this PR: if required can remove and push in a different PR)
    
    Closes #12590 from vibhatha/arrow-15639
    
    Lead-authored-by: Vibhatha Abeykoon <vi...@gmail.com>
    Co-authored-by: Vibhatha Lakmal Abeykoon <vi...@users.noreply.github.com>
    Co-authored-by: Antoine Pitrou <an...@python.org>
    Co-authored-by: Weston Pace <we...@gmail.com>
    Co-authored-by: David Li <li...@gmail.com>
    Signed-off-by: Antoine Pitrou <an...@python.org>
---
 cpp/examples/arrow/CMakeLists.txt                  |   3 +
 cpp/examples/arrow/compute_register_example.cc     |   2 +-
 cpp/examples/arrow/udf_example.cc                  | 103 +++++
 cpp/src/arrow/compute/cast.cc                      |   4 +-
 cpp/src/arrow/compute/exec_test.cc                 |  18 +-
 cpp/src/arrow/compute/function.cc                  |   8 +-
 cpp/src/arrow/compute/function.h                   |  47 +-
 cpp/src/arrow/compute/function_test.cc             |  20 +-
 cpp/src/arrow/compute/kernels/aggregate_basic.cc   |  26 +-
 cpp/src/arrow/compute/kernels/aggregate_mode.cc    |   2 +-
 .../arrow/compute/kernels/aggregate_quantile.cc    |   2 +-
 cpp/src/arrow/compute/kernels/aggregate_tdigest.cc |   4 +-
 cpp/src/arrow/compute/kernels/aggregate_var_std.cc |   6 +-
 cpp/src/arrow/compute/kernels/hash_aggregate.cc    |  34 +-
 cpp/src/arrow/compute/kernels/scalar_arithmetic.cc | 171 ++++---
 cpp/src/arrow/compute/kernels/scalar_boolean.cc    |  20 +-
 cpp/src/arrow/compute/kernels/scalar_compare.cc    |  31 +-
 cpp/src/arrow/compute/kernels/scalar_if_else.cc    |   8 +-
 cpp/src/arrow/compute/kernels/scalar_nested.cc     |  12 +-
 cpp/src/arrow/compute/kernels/scalar_random.cc     |   2 +-
 cpp/src/arrow/compute/kernels/scalar_set_lookup.cc |   9 +-
 .../arrow/compute/kernels/scalar_string_ascii.cc   |  92 ++--
 .../arrow/compute/kernels/scalar_string_internal.h |  12 +-
 .../arrow/compute/kernels/scalar_string_utf8.cc    |  64 +--
 .../compute/kernels/scalar_temporal_binary.cc      |  31 +-
 .../arrow/compute/kernels/scalar_temporal_unary.cc |  69 +--
 cpp/src/arrow/compute/kernels/scalar_validity.cc   |  37 +-
 cpp/src/arrow/compute/kernels/vector_array_sort.cc |   4 +-
 cpp/src/arrow/compute/kernels/vector_hash.cc       |   6 +-
 cpp/src/arrow/compute/kernels/vector_nested.cc     |   4 +-
 cpp/src/arrow/compute/kernels/vector_replace.cc    |   6 +-
 cpp/src/arrow/compute/kernels/vector_selection.cc  |  22 +-
 cpp/src/arrow/compute/kernels/vector_sort.cc       |   4 +-
 cpp/src/arrow/compute/registry_test.cc             |  10 +-
 cpp/src/arrow/datum.cc                             |  20 +
 cpp/src/arrow/datum.h                              |   2 +
 cpp/src/arrow/python/CMakeLists.txt                |   3 +-
 cpp/src/arrow/python/common.h                      |   5 +
 cpp/src/arrow/python/udf.cc                        | 133 ++++++
 cpp/src/arrow/python/udf.h                         |  59 +++
 python/pyarrow/_compute.pxd                        |   5 +
 python/pyarrow/_compute.pyx                        | 212 +++++++++
 python/pyarrow/compute.py                          |   3 +
 python/pyarrow/includes/libarrow.pxd               |  22 +
 python/pyarrow/lib.pxd                             |   3 +
 python/pyarrow/memory.pxi                          |   6 +
 python/pyarrow/tests/test_udf.py                   | 503 +++++++++++++++++++++
 47 files changed, 1476 insertions(+), 393 deletions(-)

diff --git a/cpp/examples/arrow/CMakeLists.txt b/cpp/examples/arrow/CMakeLists.txt
index e11b3bd0ab..229373665d 100644
--- a/cpp/examples/arrow/CMakeLists.txt
+++ b/cpp/examples/arrow/CMakeLists.txt
@@ -133,4 +133,7 @@ if(ARROW_PARQUET AND ARROW_DATASET)
 
   add_arrow_example(join_example EXTRA_LINK_LIBS ${DATASET_EXAMPLES_LINK_LIBS})
   add_dependencies(join-example parquet)
+
+  add_arrow_example(udf_example)
+
 endif()
diff --git a/cpp/examples/arrow/compute_register_example.cc b/cpp/examples/arrow/compute_register_example.cc
index 0f6165a064..f089b910ec 100644
--- a/cpp/examples/arrow/compute_register_example.cc
+++ b/cpp/examples/arrow/compute_register_example.cc
@@ -126,7 +126,7 @@ const cp::FunctionDoc func_doc{
 
 int main(int argc, char** argv) {
   const std::string name = "compute_register_example";
-  auto func = std::make_shared<cp::ScalarFunction>(name, cp::Arity::Unary(), &func_doc);
+  auto func = std::make_shared<cp::ScalarFunction>(name, cp::Arity::Unary(), func_doc);
   cp::ScalarKernel kernel({cp::InputType::Array(arrow::int64())}, arrow::int64(),
                           ExampleFunctionImpl);
   kernel.mem_allocation = cp::MemAllocation::NO_PREALLOCATE;
diff --git a/cpp/examples/arrow/udf_example.cc b/cpp/examples/arrow/udf_example.cc
new file mode 100644
index 0000000000..f45e2c644d
--- /dev/null
+++ b/cpp/examples/arrow/udf_example.cc
@@ -0,0 +1,103 @@
+// 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 <arrow/api.h>
+#include <arrow/compute/api.h>
+
+#include <cstdlib>
+#include <iostream>
+#include <memory>
+#include <string>
+#include <type_traits>
+#include <utility>
+#include <vector>
+
+// Demonstrate registering a user-defined Arrow compute function outside of the Arrow
+// source tree
+
+namespace cp = ::arrow::compute;
+
+#define ABORT_ON_FAILURE(expr)                     \
+  do {                                             \
+    arrow::Status status_ = (expr);                \
+    if (!status_.ok()) {                           \
+      std::cerr << status_.message() << std::endl; \
+      abort();                                     \
+    }                                              \
+  } while (0);
+
+template <typename TYPE,
+          typename = typename std::enable_if<arrow::is_number_type<TYPE>::value |
+                                             arrow::is_boolean_type<TYPE>::value |
+                                             arrow::is_temporal_type<TYPE>::value>::type>
+arrow::Result<std::shared_ptr<arrow::Array>> GetArrayDataSample(
+    const std::vector<typename TYPE::c_type>& values) {
+  using ArrowBuilderType = typename arrow::TypeTraits<TYPE>::BuilderType;
+  ArrowBuilderType builder;
+  ARROW_RETURN_NOT_OK(builder.Reserve(values.size()));
+  ARROW_RETURN_NOT_OK(builder.AppendValues(values));
+  return builder.Finish();
+}
+
+const cp::FunctionDoc func_doc{
+    "User-defined-function usage to demonstrate registering an out-of-tree function",
+    "returns x + y + z",
+    {"x", "y", "z"},
+    "UDFOptions"};
+
+arrow::Status SampleFunction(cp::KernelContext* ctx, const cp::ExecBatch& batch,
+                             arrow::Datum* out) {
+  // temp = x + y; return temp + z
+  ARROW_ASSIGN_OR_RAISE(auto temp, cp::CallFunction("add", {batch[0], batch[1]}));
+  return cp::CallFunction("add", {temp, batch[2]}).Value(out);
+}
+
+arrow::Status Execute() {
+  const std::string name = "add_three";
+  auto func = std::make_shared<cp::ScalarFunction>(name, cp::Arity::Ternary(), func_doc);
+  cp::ScalarKernel kernel(
+      {cp::InputType::Array(arrow::int64()), cp::InputType::Array(arrow::int64()),
+       cp::InputType::Array(arrow::int64())},
+      arrow::int64(), SampleFunction);
+
+  kernel.mem_allocation = cp::MemAllocation::NO_PREALLOCATE;
+  kernel.null_handling = cp::NullHandling::COMPUTED_NO_PREALLOCATE;
+
+  ARROW_RETURN_NOT_OK(func->AddKernel(std::move(kernel)));
+
+  auto registry = cp::GetFunctionRegistry();
+  ARROW_RETURN_NOT_OK(registry->AddFunction(std::move(func)));
+
+  ARROW_ASSIGN_OR_RAISE(auto x, GetArrayDataSample<arrow::Int64Type>({1, 2, 3}));
+  ARROW_ASSIGN_OR_RAISE(auto y, GetArrayDataSample<arrow::Int64Type>({4, 5, 6}));
+  ARROW_ASSIGN_OR_RAISE(auto z, GetArrayDataSample<arrow::Int64Type>({7, 8, 9}));
+
+  ARROW_ASSIGN_OR_RAISE(auto res, cp::CallFunction(name, {x, y, z}));
+  auto res_array = res.make_array();
+  std::cout << "Result" << std::endl;
+  std::cout << res_array->ToString() << std::endl;
+  return arrow::Status::OK();
+}
+
+int main(int argc, char** argv) {
+  auto status = Execute();
+  if (!status.ok()) {
+    std::cerr << "Error occurred : " << status.message() << std::endl;
+    return EXIT_FAILURE;
+  }
+  return EXIT_SUCCESS;
+}
diff --git a/cpp/src/arrow/compute/cast.cc b/cpp/src/arrow/compute/cast.cc
index 4de68ba8d9..bd49041b4f 100644
--- a/cpp/src/arrow/compute/cast.cc
+++ b/cpp/src/arrow/compute/cast.cc
@@ -95,7 +95,7 @@ const FunctionDoc cast_doc{"Cast values to another data type",
 // to the standard SQL CAST(expr AS target_type)
 class CastMetaFunction : public MetaFunction {
  public:
-  CastMetaFunction() : MetaFunction("cast", Arity::Unary(), &cast_doc) {}
+  CastMetaFunction() : MetaFunction("cast", Arity::Unary(), cast_doc) {}
 
   Result<const CastOptions*> ValidateOptions(const FunctionOptions* options) const {
     auto cast_options = static_cast<const CastOptions*>(options);
@@ -153,7 +153,7 @@ CastOptions::CastOptions(bool safe)
 constexpr char CastOptions::kTypeName[];
 
 CastFunction::CastFunction(std::string name, Type::type out_type_id)
-    : ScalarFunction(std::move(name), Arity::Unary(), /*doc=*/nullptr),
+    : ScalarFunction(std::move(name), Arity::Unary(), FunctionDoc::Empty()),
       out_type_id_(out_type_id) {}
 
 Status CastFunction::AddKernel(Type::type in_type_id, ScalarKernel kernel) {
diff --git a/cpp/src/arrow/compute/exec_test.cc b/cpp/src/arrow/compute/exec_test.cc
index 198cb84ff5..b00bcb319c 100644
--- a/cpp/src/arrow/compute/exec_test.cc
+++ b/cpp/src/arrow/compute/exec_test.cc
@@ -681,8 +681,8 @@ class TestCallScalarFunction : public TestComputeInternals {
 
     // This function simply copies memory from the input argument into the
     // (preallocated) output
-    auto func =
-        std::make_shared<ScalarFunction>("test_copy", Arity::Unary(), /*doc=*/nullptr);
+    auto func = std::make_shared<ScalarFunction>("test_copy", Arity::Unary(),
+                                                 /*doc=*/FunctionDoc::Empty());
 
     // Add a few kernels. Our implementation only accepts arrays
     ASSERT_OK(func->AddKernel({InputType::Array(uint8())}, uint8(), ExecCopy));
@@ -691,8 +691,8 @@ class TestCallScalarFunction : public TestComputeInternals {
     ASSERT_OK(registry->AddFunction(func));
 
     // A version which doesn't want the executor to call PropagateNulls
-    auto func2 = std::make_shared<ScalarFunction>("test_copy_computed_bitmap",
-                                                  Arity::Unary(), /*doc=*/nullptr);
+    auto func2 = std::make_shared<ScalarFunction>(
+        "test_copy_computed_bitmap", Arity::Unary(), /*doc=*/FunctionDoc::Empty());
     ScalarKernel kernel({InputType::Array(uint8())}, uint8(), ExecComputedBitmap);
     kernel.null_handling = NullHandling::COMPUTED_PREALLOCATE;
     ASSERT_OK(func2->AddKernel(kernel));
@@ -705,9 +705,9 @@ class TestCallScalarFunction : public TestComputeInternals {
     // A function that allocates its own output memory. We have cases for both
     // non-preallocated data and non-preallocated validity bitmap
     auto f1 = std::make_shared<ScalarFunction>("test_nopre_data", Arity::Unary(),
-                                               /*doc=*/nullptr);
-    auto f2 = std::make_shared<ScalarFunction>("test_nopre_validity_or_data",
-                                               Arity::Unary(), /*doc=*/nullptr);
+                                               /*doc=*/FunctionDoc::Empty());
+    auto f2 = std::make_shared<ScalarFunction>(
+        "test_nopre_validity_or_data", Arity::Unary(), /*doc=*/FunctionDoc::Empty());
 
     ScalarKernel kernel({InputType::Array(uint8())}, uint8(), ExecNoPreallocatedData);
     kernel.mem_allocation = MemAllocation::NO_PREALLOCATE;
@@ -727,7 +727,7 @@ class TestCallScalarFunction : public TestComputeInternals {
     // This function's behavior depends on a static parameter that is made
     // available to the kernel's execution function through its Options object
     auto func = std::make_shared<ScalarFunction>("test_stateful", Arity::Unary(),
-                                                 /*doc=*/nullptr);
+                                                 /*doc=*/FunctionDoc::Empty());
 
     ScalarKernel kernel({InputType::Array(int32())}, int32(), ExecStateful, InitStateful);
     ASSERT_OK(func->AddKernel(kernel));
@@ -738,7 +738,7 @@ class TestCallScalarFunction : public TestComputeInternals {
     auto registry = GetFunctionRegistry();
 
     auto func = std::make_shared<ScalarFunction>("test_scalar_add_int32", Arity::Binary(),
-                                                 /*doc=*/nullptr);
+                                                 /*doc=*/FunctionDoc::Empty());
     ASSERT_OK(func->AddKernel({InputType::Scalar(int32()), InputType::Scalar(int32())},
                               int32(), ExecAddInt32));
     ASSERT_OK(registry->AddFunction(func));
diff --git a/cpp/src/arrow/compute/function.cc b/cpp/src/arrow/compute/function.cc
index 1c18243f5f..2b3d4e6feb 100644
--- a/cpp/src/arrow/compute/function.cc
+++ b/cpp/src/arrow/compute/function.cc
@@ -290,9 +290,9 @@ Status ValidateFunctionDescription(const std::string& s) {
 }  // namespace
 
 Status Function::Validate() const {
-  if (!doc_->summary.empty()) {
+  if (!doc_.summary.empty()) {
     // Documentation given, check its contents
-    int arg_count = static_cast<int>(doc_->arg_names.size());
+    int arg_count = static_cast<int>(doc_.arg_names.size());
     // Some varargs functions allow 0 vararg, others expect at least 1,
     // hence the two possible values below.
     bool arg_count_match = (arg_count == arity_.num_args) ||
@@ -302,9 +302,9 @@ Status Function::Validate() const {
           "In function '", name_,
           "': ", "number of argument names for function documentation != function arity");
     }
-    Status st = ValidateFunctionSummary(doc_->summary);
+    Status st = ValidateFunctionSummary(doc_.summary);
     if (st.ok()) {
-      st &= ValidateFunctionDescription(doc_->description);
+      st &= ValidateFunctionDescription(doc_.description);
     }
     if (!st.ok()) {
       return st.WithMessage("In function '", name_, "': ", st.message());
diff --git a/cpp/src/arrow/compute/function.h b/cpp/src/arrow/compute/function.h
index 1273ab09c4..face491690 100644
--- a/cpp/src/arrow/compute/function.h
+++ b/cpp/src/arrow/compute/function.h
@@ -205,7 +205,7 @@ class ARROW_EXPORT Function {
   const Arity& arity() const { return arity_; }
 
   /// \brief Return the function documentation
-  const FunctionDoc& doc() const { return *doc_; }
+  const FunctionDoc& doc() const { return doc_; }
 
   /// \brief Returns the number of registered kernels for this function.
   virtual int num_kernels() const = 0;
@@ -244,12 +244,12 @@ class ARROW_EXPORT Function {
   virtual Status Validate() const;
 
  protected:
-  Function(std::string name, Function::Kind kind, const Arity& arity,
-           const FunctionDoc* doc, const FunctionOptions* default_options)
+  Function(std::string name, Function::Kind kind, const Arity& arity, FunctionDoc doc,
+           const FunctionOptions* default_options)
       : name_(std::move(name)),
         kind_(kind),
         arity_(arity),
-        doc_(doc ? doc : &FunctionDoc::Empty()),
+        doc_(std::move(doc)),
         default_options_(default_options) {}
 
   Status CheckArity(const std::vector<InputType>&) const;
@@ -258,7 +258,7 @@ class ARROW_EXPORT Function {
   std::string name_;
   Function::Kind kind_;
   Arity arity_;
-  const FunctionDoc* doc_;
+  const FunctionDoc doc_;
   const FunctionOptions* default_options_ = NULLPTR;
 };
 
@@ -279,9 +279,9 @@ class FunctionImpl : public Function {
   int num_kernels() const override { return static_cast<int>(kernels_.size()); }
 
  protected:
-  FunctionImpl(std::string name, Function::Kind kind, const Arity& arity,
-               const FunctionDoc* doc, const FunctionOptions* default_options)
-      : Function(std::move(name), kind, arity, doc, default_options) {}
+  FunctionImpl(std::string name, Function::Kind kind, const Arity& arity, FunctionDoc doc,
+               const FunctionOptions* default_options)
+      : Function(std::move(name), kind, arity, std::move(doc), default_options) {}
 
   std::vector<KernelType> kernels_;
 };
@@ -305,10 +305,10 @@ class ARROW_EXPORT ScalarFunction : public detail::FunctionImpl<ScalarKernel> {
  public:
   using KernelType = ScalarKernel;
 
-  ScalarFunction(std::string name, const Arity& arity, const FunctionDoc* doc,
+  ScalarFunction(std::string name, const Arity& arity, FunctionDoc doc,
                  const FunctionOptions* default_options = NULLPTR)
-      : detail::FunctionImpl<ScalarKernel>(std::move(name), Function::SCALAR, arity, doc,
-                                           default_options) {}
+      : detail::FunctionImpl<ScalarKernel>(std::move(name), Function::SCALAR, arity,
+                                           std::move(doc), default_options) {}
 
   /// \brief Add a kernel with given input/output types, no required state
   /// initialization, preallocation for fixed-width types, and default null
@@ -329,10 +329,10 @@ class ARROW_EXPORT VectorFunction : public detail::FunctionImpl<VectorKernel> {
  public:
   using KernelType = VectorKernel;
 
-  VectorFunction(std::string name, const Arity& arity, const FunctionDoc* doc,
+  VectorFunction(std::string name, const Arity& arity, FunctionDoc doc,
                  const FunctionOptions* default_options = NULLPTR)
-      : detail::FunctionImpl<VectorKernel>(std::move(name), Function::VECTOR, arity, doc,
-                                           default_options) {}
+      : detail::FunctionImpl<VectorKernel>(std::move(name), Function::VECTOR, arity,
+                                           std::move(doc), default_options) {}
 
   /// \brief Add a simple kernel with given input/output types, no required
   /// state initialization, no data preallocation, and no preallocation of the
@@ -350,10 +350,11 @@ class ARROW_EXPORT ScalarAggregateFunction
  public:
   using KernelType = ScalarAggregateKernel;
 
-  ScalarAggregateFunction(std::string name, const Arity& arity, const FunctionDoc* doc,
+  ScalarAggregateFunction(std::string name, const Arity& arity, FunctionDoc doc,
                           const FunctionOptions* default_options = NULLPTR)
-      : detail::FunctionImpl<ScalarAggregateKernel>(
-            std::move(name), Function::SCALAR_AGGREGATE, arity, doc, default_options) {}
+      : detail::FunctionImpl<ScalarAggregateKernel>(std::move(name),
+                                                    Function::SCALAR_AGGREGATE, arity,
+                                                    std::move(doc), default_options) {}
 
   /// \brief Add a kernel (function implementation). Returns error if the
   /// kernel's signature does not match the function's arity.
@@ -365,10 +366,11 @@ class ARROW_EXPORT HashAggregateFunction
  public:
   using KernelType = HashAggregateKernel;
 
-  HashAggregateFunction(std::string name, const Arity& arity, const FunctionDoc* doc,
+  HashAggregateFunction(std::string name, const Arity& arity, FunctionDoc doc,
                         const FunctionOptions* default_options = NULLPTR)
-      : detail::FunctionImpl<HashAggregateKernel>(
-            std::move(name), Function::HASH_AGGREGATE, arity, doc, default_options) {}
+      : detail::FunctionImpl<HashAggregateKernel>(std::move(name),
+                                                  Function::HASH_AGGREGATE, arity,
+                                                  std::move(doc), default_options) {}
 
   /// \brief Add a kernel (function implementation). Returns error if the
   /// kernel's signature does not match the function's arity.
@@ -392,9 +394,10 @@ class ARROW_EXPORT MetaFunction : public Function {
                                     const FunctionOptions* options,
                                     ExecContext* ctx) const = 0;
 
-  MetaFunction(std::string name, const Arity& arity, const FunctionDoc* doc,
+  MetaFunction(std::string name, const Arity& arity, FunctionDoc doc,
                const FunctionOptions* default_options = NULLPTR)
-      : Function(std::move(name), Function::META, arity, doc, default_options) {}
+      : Function(std::move(name), Function::META, arity, std::move(doc),
+                 default_options) {}
 };
 
 /// @}
diff --git a/cpp/src/arrow/compute/function_test.cc b/cpp/src/arrow/compute/function_test.cc
index 13de2a29ab..94e86c7bd5 100644
--- a/cpp/src/arrow/compute/function_test.cc
+++ b/cpp/src/arrow/compute/function_test.cc
@@ -179,8 +179,9 @@ TEST(Arity, Basics) {
 }
 
 TEST(ScalarFunction, Basics) {
-  ScalarFunction func("scalar_test", Arity::Binary(), /*doc=*/nullptr);
-  ScalarFunction varargs_func("varargs_test", Arity::VarArgs(1), /*doc=*/nullptr);
+  ScalarFunction func("scalar_test", Arity::Binary(), /*doc=*/FunctionDoc::Empty());
+  ScalarFunction varargs_func("varargs_test", Arity::VarArgs(1),
+                              /*doc=*/FunctionDoc::Empty());
 
   ASSERT_EQ("scalar_test", func.name());
   ASSERT_EQ(2, func.arity().num_args);
@@ -194,8 +195,9 @@ TEST(ScalarFunction, Basics) {
 }
 
 TEST(VectorFunction, Basics) {
-  VectorFunction func("vector_test", Arity::Binary(), /*doc=*/nullptr);
-  VectorFunction varargs_func("varargs_test", Arity::VarArgs(1), /*doc=*/nullptr);
+  VectorFunction func("vector_test", Arity::Binary(), /*doc=*/FunctionDoc::Empty());
+  VectorFunction varargs_func("varargs_test", Arity::VarArgs(1),
+                              /*doc=*/FunctionDoc::Empty());
 
   ASSERT_EQ("vector_test", func.name());
   ASSERT_EQ(2, func.arity().num_args);
@@ -260,15 +262,15 @@ void CheckAddDispatch(FunctionType* func) {
 }
 
 TEST(ScalarVectorFunction, DispatchExact) {
-  ScalarFunction func1("scalar_test", Arity::Binary(), /*doc=*/nullptr);
-  VectorFunction func2("vector_test", Arity::Binary(), /*doc=*/nullptr);
+  ScalarFunction func1("scalar_test", Arity::Binary(), /*doc=*/FunctionDoc::Empty());
+  VectorFunction func2("vector_test", Arity::Binary(), /*doc=*/FunctionDoc::Empty());
 
   CheckAddDispatch(&func1);
   CheckAddDispatch(&func2);
 }
 
 TEST(ArrayFunction, VarArgs) {
-  ScalarFunction va_func("va_test", Arity::VarArgs(1), /*doc=*/nullptr);
+  ScalarFunction va_func("va_test", Arity::VarArgs(1), /*doc=*/FunctionDoc::Empty());
 
   std::vector<InputType> va_args = {int8()};
 
@@ -294,7 +296,7 @@ TEST(ArrayFunction, VarArgs) {
 }
 
 TEST(ScalarAggregateFunction, Basics) {
-  ScalarAggregateFunction func("agg_test", Arity::Unary(), /*doc=*/nullptr);
+  ScalarAggregateFunction func("agg_test", Arity::Unary(), /*doc=*/FunctionDoc::Empty());
 
   ASSERT_EQ("agg_test", func.name());
   ASSERT_EQ(1, func.arity().num_args);
@@ -313,7 +315,7 @@ Status NoopMerge(KernelContext*, const KernelState&, KernelState*) {
 Status NoopFinalize(KernelContext*, Datum*) { return Status::OK(); }
 
 TEST(ScalarAggregateFunction, DispatchExact) {
-  ScalarAggregateFunction func("agg_test", Arity::Unary(), /*doc=*/nullptr);
+  ScalarAggregateFunction func("agg_test", Arity::Unary(), FunctionDoc::Empty());
 
   std::vector<InputType> in_args = {ValueDescr::Array(int8())};
   ScalarAggregateKernel kernel(std::move(in_args), int64(), NoopInit, NoopConsume,
diff --git a/cpp/src/arrow/compute/kernels/aggregate_basic.cc b/cpp/src/arrow/compute/kernels/aggregate_basic.cc
index c9e2d85a26..16495bc803 100644
--- a/cpp/src/arrow/compute/kernels/aggregate_basic.cc
+++ b/cpp/src/arrow/compute/kernels/aggregate_basic.cc
@@ -918,7 +918,7 @@ void RegisterScalarAggregateBasic(FunctionRegistry* registry) {
   static auto default_count_options = CountOptions::Defaults();
 
   auto func = std::make_shared<ScalarAggregateFunction>(
-      "count", Arity::Unary(), &count_doc, &default_count_options);
+      "count", Arity::Unary(), count_doc, &default_count_options);
 
   // Takes any input, outputs int64 scalar
   InputType any_input;
@@ -927,12 +927,12 @@ void RegisterScalarAggregateBasic(FunctionRegistry* registry) {
   DCHECK_OK(registry->AddFunction(std::move(func)));
 
   func = std::make_shared<ScalarAggregateFunction>(
-      "count_distinct", Arity::Unary(), &count_distinct_doc, &default_count_options);
+      "count_distinct", Arity::Unary(), count_distinct_doc, &default_count_options);
   // Takes any input, outputs int64 scalar
   AddCountDistinctKernels(func.get());
   DCHECK_OK(registry->AddFunction(std::move(func)));
 
-  func = std::make_shared<ScalarAggregateFunction>("sum", Arity::Unary(), &sum_doc,
+  func = std::make_shared<ScalarAggregateFunction>("sum", Arity::Unary(), sum_doc,
                                                    &default_scalar_aggregate_options);
   AddArrayScalarAggKernels(SumInit, {boolean()}, uint64(), func.get());
   AddAggKernel(
@@ -961,7 +961,7 @@ void RegisterScalarAggregateBasic(FunctionRegistry* registry) {
 #endif
   DCHECK_OK(registry->AddFunction(std::move(func)));
 
-  func = std::make_shared<ScalarAggregateFunction>("mean", Arity::Unary(), &mean_doc,
+  func = std::make_shared<ScalarAggregateFunction>("mean", Arity::Unary(), mean_doc,
                                                    &default_scalar_aggregate_options);
   AddArrayScalarAggKernels(MeanInit, {boolean()}, float64(), func.get());
   AddArrayScalarAggKernels(MeanInit, NumericTypes(), float64(), func.get());
@@ -985,8 +985,8 @@ void RegisterScalarAggregateBasic(FunctionRegistry* registry) {
 #endif
   DCHECK_OK(registry->AddFunction(std::move(func)));
 
-  func = std::make_shared<ScalarAggregateFunction>(
-      "min_max", Arity::Unary(), &min_max_doc, &default_scalar_aggregate_options);
+  func = std::make_shared<ScalarAggregateFunction>("min_max", Arity::Unary(), min_max_doc,
+                                                   &default_scalar_aggregate_options);
   AddMinMaxKernels(MinMaxInit, {null(), boolean()}, func.get());
   AddMinMaxKernels(MinMaxInit, NumericTypes(), func.get());
   AddMinMaxKernels(MinMaxInit, TemporalTypes(), func.get());
@@ -1011,18 +1011,18 @@ void RegisterScalarAggregateBasic(FunctionRegistry* registry) {
   DCHECK_OK(registry->AddFunction(std::move(func)));
 
   // Add min/max as convenience functions
-  func = std::make_shared<ScalarAggregateFunction>("min", Arity::Unary(), &min_or_max_doc,
+  func = std::make_shared<ScalarAggregateFunction>("min", Arity::Unary(), min_or_max_doc,
                                                    &default_scalar_aggregate_options);
   AddMinOrMaxAggKernel<MinOrMax::Min>(func.get(), min_max_func);
   DCHECK_OK(registry->AddFunction(std::move(func)));
 
-  func = std::make_shared<ScalarAggregateFunction>("max", Arity::Unary(), &min_or_max_doc,
+  func = std::make_shared<ScalarAggregateFunction>("max", Arity::Unary(), min_or_max_doc,
                                                    &default_scalar_aggregate_options);
   AddMinOrMaxAggKernel<MinOrMax::Max>(func.get(), min_max_func);
   DCHECK_OK(registry->AddFunction(std::move(func)));
 
-  func = std::make_shared<ScalarAggregateFunction>(
-      "product", Arity::Unary(), &product_doc, &default_scalar_aggregate_options);
+  func = std::make_shared<ScalarAggregateFunction>("product", Arity::Unary(), product_doc,
+                                                   &default_scalar_aggregate_options);
   AddArrayScalarAggKernels(ProductInit::Init, {boolean()}, uint64(), func.get());
   AddArrayScalarAggKernels(ProductInit::Init, SignedIntTypes(), int64(), func.get());
   AddArrayScalarAggKernels(ProductInit::Init, UnsignedIntTypes(), uint64(), func.get());
@@ -1038,19 +1038,19 @@ void RegisterScalarAggregateBasic(FunctionRegistry* registry) {
   DCHECK_OK(registry->AddFunction(std::move(func)));
 
   // any
-  func = std::make_shared<ScalarAggregateFunction>("any", Arity::Unary(), &any_doc,
+  func = std::make_shared<ScalarAggregateFunction>("any", Arity::Unary(), any_doc,
                                                    &default_scalar_aggregate_options);
   AddArrayScalarAggKernels(AnyInit, {boolean()}, boolean(), func.get());
   DCHECK_OK(registry->AddFunction(std::move(func)));
 
   // all
-  func = std::make_shared<ScalarAggregateFunction>("all", Arity::Unary(), &all_doc,
+  func = std::make_shared<ScalarAggregateFunction>("all", Arity::Unary(), all_doc,
                                                    &default_scalar_aggregate_options);
   AddArrayScalarAggKernels(AllInit, {boolean()}, boolean(), func.get());
   DCHECK_OK(registry->AddFunction(std::move(func)));
 
   // index
-  func = std::make_shared<ScalarAggregateFunction>("index", Arity::Unary(), &index_doc);
+  func = std::make_shared<ScalarAggregateFunction>("index", Arity::Unary(), index_doc);
   AddBasicAggKernels(IndexInit::Init, BaseBinaryTypes(), int64(), func.get());
   AddBasicAggKernels(IndexInit::Init, PrimitiveTypes(), int64(), func.get());
   AddBasicAggKernels(IndexInit::Init, TemporalTypes(), int64(), func.get());
diff --git a/cpp/src/arrow/compute/kernels/aggregate_mode.cc b/cpp/src/arrow/compute/kernels/aggregate_mode.cc
index 287c2c5d36..7d3440cbef 100644
--- a/cpp/src/arrow/compute/kernels/aggregate_mode.cc
+++ b/cpp/src/arrow/compute/kernels/aggregate_mode.cc
@@ -428,7 +428,7 @@ const FunctionDoc mode_doc{
 
 void RegisterScalarAggregateMode(FunctionRegistry* registry) {
   static auto default_options = ModeOptions::Defaults();
-  auto func = std::make_shared<VectorFunction>("mode", Arity::Unary(), &mode_doc,
+  auto func = std::make_shared<VectorFunction>("mode", Arity::Unary(), mode_doc,
                                                &default_options);
   DCHECK_OK(func->AddKernel(
       NewModeKernel(boolean(), ModeExecutor<StructType, BooleanType>::Exec)));
diff --git a/cpp/src/arrow/compute/kernels/aggregate_quantile.cc b/cpp/src/arrow/compute/kernels/aggregate_quantile.cc
index 1ca030130b..810fb53991 100644
--- a/cpp/src/arrow/compute/kernels/aggregate_quantile.cc
+++ b/cpp/src/arrow/compute/kernels/aggregate_quantile.cc
@@ -531,7 +531,7 @@ const FunctionDoc quantile_doc{
 
 void RegisterScalarAggregateQuantile(FunctionRegistry* registry) {
   static QuantileOptions default_options;
-  auto func = std::make_shared<VectorFunction>("quantile", Arity::Unary(), &quantile_doc,
+  auto func = std::make_shared<VectorFunction>("quantile", Arity::Unary(), quantile_doc,
                                                &default_options);
   AddQuantileKernels(func.get());
   DCHECK_OK(registry->AddFunction(std::move(func)));
diff --git a/cpp/src/arrow/compute/kernels/aggregate_tdigest.cc b/cpp/src/arrow/compute/kernels/aggregate_tdigest.cc
index 7c86267d94..037bba42f1 100644
--- a/cpp/src/arrow/compute/kernels/aggregate_tdigest.cc
+++ b/cpp/src/arrow/compute/kernels/aggregate_tdigest.cc
@@ -196,7 +196,7 @@ const FunctionDoc approximate_median_doc{
 std::shared_ptr<ScalarAggregateFunction> AddTDigestAggKernels() {
   static auto default_tdigest_options = TDigestOptions::Defaults();
   auto func = std::make_shared<ScalarAggregateFunction>(
-      "tdigest", Arity::Unary(), &tdigest_doc, &default_tdigest_options);
+      "tdigest", Arity::Unary(), tdigest_doc, &default_tdigest_options);
   AddTDigestKernels(TDigestInit, NumericTypes(), func.get());
   AddTDigestKernels(TDigestInit, {decimal128(1, 1), decimal256(1, 1)}, func.get());
   return func;
@@ -207,7 +207,7 @@ std::shared_ptr<ScalarAggregateFunction> AddApproximateMedianAggKernels(
   static ScalarAggregateOptions default_scalar_aggregate_options;
 
   auto median = std::make_shared<ScalarAggregateFunction>(
-      "approximate_median", Arity::Unary(), &approximate_median_doc,
+      "approximate_median", Arity::Unary(), approximate_median_doc,
       &default_scalar_aggregate_options);
 
   auto sig =
diff --git a/cpp/src/arrow/compute/kernels/aggregate_var_std.cc b/cpp/src/arrow/compute/kernels/aggregate_var_std.cc
index feb98718ae..1f9a26960b 100644
--- a/cpp/src/arrow/compute/kernels/aggregate_var_std.cc
+++ b/cpp/src/arrow/compute/kernels/aggregate_var_std.cc
@@ -289,8 +289,8 @@ const FunctionDoc variance_doc{
 
 std::shared_ptr<ScalarAggregateFunction> AddStddevAggKernels() {
   static auto default_std_options = VarianceOptions::Defaults();
-  auto func = std::make_shared<ScalarAggregateFunction>(
-      "stddev", Arity::Unary(), &stddev_doc, &default_std_options);
+  auto func = std::make_shared<ScalarAggregateFunction>("stddev", Arity::Unary(),
+                                                        stddev_doc, &default_std_options);
   AddVarStdKernels(StddevInit, NumericTypes(), func.get());
   AddVarStdKernels(StddevInit, {decimal128(1, 1), decimal256(1, 1)}, func.get());
   return func;
@@ -299,7 +299,7 @@ std::shared_ptr<ScalarAggregateFunction> AddStddevAggKernels() {
 std::shared_ptr<ScalarAggregateFunction> AddVarianceAggKernels() {
   static auto default_var_options = VarianceOptions::Defaults();
   auto func = std::make_shared<ScalarAggregateFunction>(
-      "variance", Arity::Unary(), &variance_doc, &default_var_options);
+      "variance", Arity::Unary(), variance_doc, &default_var_options);
   AddVarStdKernels(VarianceInit, NumericTypes(), func.get());
   AddVarStdKernels(VarianceInit, {decimal128(1, 1), decimal256(1, 1)}, func.get());
   return func;
diff --git a/cpp/src/arrow/compute/kernels/hash_aggregate.cc b/cpp/src/arrow/compute/kernels/hash_aggregate.cc
index db34ee6c59..d9ffcda596 100644
--- a/cpp/src/arrow/compute/kernels/hash_aggregate.cc
+++ b/cpp/src/arrow/compute/kernels/hash_aggregate.cc
@@ -3554,7 +3554,7 @@ void RegisterHashAggregateBasic(FunctionRegistry* registry) {
 
   {
     auto func = std::make_shared<HashAggregateFunction>(
-        "hash_count", Arity::Binary(), &hash_count_doc, &default_count_options);
+        "hash_count", Arity::Binary(), hash_count_doc, &default_count_options);
 
     DCHECK_OK(func->AddKernel(
         MakeKernel(ValueDescr::ARRAY, HashAggregateInit<GroupedCountImpl>)));
@@ -3563,7 +3563,7 @@ void RegisterHashAggregateBasic(FunctionRegistry* registry) {
 
   {
     auto func = std::make_shared<HashAggregateFunction>(
-        "hash_sum", Arity::Binary(), &hash_sum_doc, &default_scalar_aggregate_options);
+        "hash_sum", Arity::Binary(), hash_sum_doc, &default_scalar_aggregate_options);
     DCHECK_OK(AddHashAggKernels({boolean()}, GroupedSumFactory::Make, func.get()));
     DCHECK_OK(AddHashAggKernels(SignedIntTypes(), GroupedSumFactory::Make, func.get()));
     DCHECK_OK(AddHashAggKernels(UnsignedIntTypes(), GroupedSumFactory::Make, func.get()));
@@ -3578,7 +3578,7 @@ void RegisterHashAggregateBasic(FunctionRegistry* registry) {
 
   {
     auto func = std::make_shared<HashAggregateFunction>(
-        "hash_product", Arity::Binary(), &hash_product_doc,
+        "hash_product", Arity::Binary(), hash_product_doc,
         &default_scalar_aggregate_options);
     DCHECK_OK(AddHashAggKernels({boolean()}, GroupedProductFactory::Make, func.get()));
     DCHECK_OK(
@@ -3596,7 +3596,7 @@ void RegisterHashAggregateBasic(FunctionRegistry* registry) {
 
   {
     auto func = std::make_shared<HashAggregateFunction>(
-        "hash_mean", Arity::Binary(), &hash_mean_doc, &default_scalar_aggregate_options);
+        "hash_mean", Arity::Binary(), hash_mean_doc, &default_scalar_aggregate_options);
     DCHECK_OK(AddHashAggKernels({boolean()}, GroupedMeanFactory::Make, func.get()));
     DCHECK_OK(AddHashAggKernels(SignedIntTypes(), GroupedMeanFactory::Make, func.get()));
     DCHECK_OK(
@@ -3612,7 +3612,7 @@ void RegisterHashAggregateBasic(FunctionRegistry* registry) {
 
   {
     auto func = std::make_shared<HashAggregateFunction>(
-        "hash_stddev", Arity::Binary(), &hash_stddev_doc, &default_variance_options);
+        "hash_stddev", Arity::Binary(), hash_stddev_doc, &default_variance_options);
     DCHECK_OK(AddHashAggKernels(SignedIntTypes(),
                                 GroupedVarStdFactory<VarOrStd::Std>::Make, func.get()));
     DCHECK_OK(AddHashAggKernels(UnsignedIntTypes(),
@@ -3626,7 +3626,7 @@ void RegisterHashAggregateBasic(FunctionRegistry* registry) {
 
   {
     auto func = std::make_shared<HashAggregateFunction>(
-        "hash_variance", Arity::Binary(), &hash_variance_doc, &default_variance_options);
+        "hash_variance", Arity::Binary(), hash_variance_doc, &default_variance_options);
     DCHECK_OK(AddHashAggKernels(SignedIntTypes(),
                                 GroupedVarStdFactory<VarOrStd::Var>::Make, func.get()));
     DCHECK_OK(AddHashAggKernels(UnsignedIntTypes(),
@@ -3641,7 +3641,7 @@ void RegisterHashAggregateBasic(FunctionRegistry* registry) {
   HashAggregateFunction* tdigest_func = nullptr;
   {
     auto func = std::make_shared<HashAggregateFunction>(
-        "hash_tdigest", Arity::Binary(), &hash_tdigest_doc, &default_tdigest_options);
+        "hash_tdigest", Arity::Binary(), hash_tdigest_doc, &default_tdigest_options);
     DCHECK_OK(
         AddHashAggKernels(SignedIntTypes(), GroupedTDigestFactory::Make, func.get()));
     DCHECK_OK(
@@ -3657,7 +3657,7 @@ void RegisterHashAggregateBasic(FunctionRegistry* registry) {
 
   {
     auto func = std::make_shared<HashAggregateFunction>(
-        "hash_approximate_median", Arity::Binary(), &hash_approximate_median_doc,
+        "hash_approximate_median", Arity::Binary(), hash_approximate_median_doc,
         &default_scalar_aggregate_options);
     DCHECK_OK(func->AddKernel(MakeApproximateMedianKernel(tdigest_func)));
     DCHECK_OK(registry->AddFunction(std::move(func)));
@@ -3666,7 +3666,7 @@ void RegisterHashAggregateBasic(FunctionRegistry* registry) {
   HashAggregateFunction* min_max_func = nullptr;
   {
     auto func = std::make_shared<HashAggregateFunction>(
-        "hash_min_max", Arity::Binary(), &hash_min_max_doc,
+        "hash_min_max", Arity::Binary(), hash_min_max_doc,
         &default_scalar_aggregate_options);
     DCHECK_OK(AddHashAggKernels(NumericTypes(), GroupedMinMaxFactory::Make, func.get()));
     DCHECK_OK(AddHashAggKernels(TemporalTypes(), GroupedMinMaxFactory::Make, func.get()));
@@ -3682,7 +3682,7 @@ void RegisterHashAggregateBasic(FunctionRegistry* registry) {
 
   {
     auto func = std::make_shared<HashAggregateFunction>(
-        "hash_min", Arity::Binary(), &hash_min_or_max_doc,
+        "hash_min", Arity::Binary(), hash_min_or_max_doc,
         &default_scalar_aggregate_options);
     DCHECK_OK(func->AddKernel(MakeMinOrMaxKernel<MinOrMax::Min>(min_max_func)));
     DCHECK_OK(registry->AddFunction(std::move(func)));
@@ -3690,7 +3690,7 @@ void RegisterHashAggregateBasic(FunctionRegistry* registry) {
 
   {
     auto func = std::make_shared<HashAggregateFunction>(
-        "hash_max", Arity::Binary(), &hash_min_or_max_doc,
+        "hash_max", Arity::Binary(), hash_min_or_max_doc,
         &default_scalar_aggregate_options);
     DCHECK_OK(func->AddKernel(MakeMinOrMaxKernel<MinOrMax::Max>(min_max_func)));
     DCHECK_OK(registry->AddFunction(std::move(func)));
@@ -3698,21 +3698,21 @@ void RegisterHashAggregateBasic(FunctionRegistry* registry) {
 
   {
     auto func = std::make_shared<HashAggregateFunction>(
-        "hash_any", Arity::Binary(), &hash_any_doc, &default_scalar_aggregate_options);
+        "hash_any", Arity::Binary(), hash_any_doc, &default_scalar_aggregate_options);
     DCHECK_OK(func->AddKernel(MakeKernel(boolean(), HashAggregateInit<GroupedAnyImpl>)));
     DCHECK_OK(registry->AddFunction(std::move(func)));
   }
 
   {
     auto func = std::make_shared<HashAggregateFunction>(
-        "hash_all", Arity::Binary(), &hash_all_doc, &default_scalar_aggregate_options);
+        "hash_all", Arity::Binary(), hash_all_doc, &default_scalar_aggregate_options);
     DCHECK_OK(func->AddKernel(MakeKernel(boolean(), HashAggregateInit<GroupedAllImpl>)));
     DCHECK_OK(registry->AddFunction(std::move(func)));
   }
 
   {
     auto func = std::make_shared<HashAggregateFunction>(
-        "hash_count_distinct", Arity::Binary(), &hash_count_distinct_doc,
+        "hash_count_distinct", Arity::Binary(), hash_count_distinct_doc,
         &default_count_options);
     DCHECK_OK(func->AddKernel(
         MakeKernel(ValueDescr::ARRAY, GroupedDistinctInit<GroupedCountDistinctImpl>)));
@@ -3721,7 +3721,7 @@ void RegisterHashAggregateBasic(FunctionRegistry* registry) {
 
   {
     auto func = std::make_shared<HashAggregateFunction>(
-        "hash_distinct", Arity::Binary(), &hash_distinct_doc, &default_count_options);
+        "hash_distinct", Arity::Binary(), hash_distinct_doc, &default_count_options);
     DCHECK_OK(func->AddKernel(
         MakeKernel(ValueDescr::ARRAY, GroupedDistinctInit<GroupedDistinctImpl>)));
     DCHECK_OK(registry->AddFunction(std::move(func)));
@@ -3729,7 +3729,7 @@ void RegisterHashAggregateBasic(FunctionRegistry* registry) {
 
   {
     auto func = std::make_shared<HashAggregateFunction>("hash_one", Arity::Binary(),
-                                                        &hash_one_doc);
+                                                        hash_one_doc);
     DCHECK_OK(AddHashAggKernels(NumericTypes(), GroupedOneFactory::Make, func.get()));
     DCHECK_OK(AddHashAggKernels(TemporalTypes(), GroupedOneFactory::Make, func.get()));
     DCHECK_OK(AddHashAggKernels(BaseBinaryTypes(), GroupedOneFactory::Make, func.get()));
@@ -3741,7 +3741,7 @@ void RegisterHashAggregateBasic(FunctionRegistry* registry) {
 
   {
     auto func = std::make_shared<HashAggregateFunction>("hash_list", Arity::Binary(),
-                                                        &hash_list_doc);
+                                                        hash_list_doc);
     DCHECK_OK(AddHashAggKernels(NumericTypes(), GroupedListFactory::Make, func.get()));
     DCHECK_OK(AddHashAggKernels(TemporalTypes(), GroupedListFactory::Make, func.get()));
     DCHECK_OK(AddHashAggKernels(BaseBinaryTypes(), GroupedListFactory::Make, func.get()));
diff --git a/cpp/src/arrow/compute/kernels/scalar_arithmetic.cc b/cpp/src/arrow/compute/kernels/scalar_arithmetic.cc
index f53d9f0c7f..4365ad4e76 100644
--- a/cpp/src/arrow/compute/kernels/scalar_arithmetic.cc
+++ b/cpp/src/arrow/compute/kernels/scalar_arithmetic.cc
@@ -1935,8 +1935,8 @@ void AddNullExec(ScalarFunction* func) {
 
 template <typename Op, typename FunctionImpl = ArithmeticFunction>
 std::shared_ptr<ScalarFunction> MakeArithmeticFunction(std::string name,
-                                                       const FunctionDoc* doc) {
-  auto func = std::make_shared<FunctionImpl>(name, Arity::Binary(), doc);
+                                                       FunctionDoc doc) {
+  auto func = std::make_shared<FunctionImpl>(name, Arity::Binary(), std::move(doc));
   for (const auto& ty : NumericTypes()) {
     auto exec = ArithmeticExecFromOp<ScalarBinaryEqualTypes, Op>(ty);
     DCHECK_OK(func->AddKernel({ty, ty}, ty, exec));
@@ -1949,8 +1949,8 @@ std::shared_ptr<ScalarFunction> MakeArithmeticFunction(std::string name,
 // only on non-null output.
 template <typename Op, typename FunctionImpl = ArithmeticFunction>
 std::shared_ptr<ScalarFunction> MakeArithmeticFunctionNotNull(std::string name,
-                                                              const FunctionDoc* doc) {
-  auto func = std::make_shared<FunctionImpl>(name, Arity::Binary(), doc);
+                                                              FunctionDoc doc) {
+  auto func = std::make_shared<FunctionImpl>(name, Arity::Binary(), std::move(doc));
   for (const auto& ty : NumericTypes()) {
     auto exec = ArithmeticExecFromOp<ScalarBinaryNotNullEqualTypes, Op>(ty);
     DCHECK_OK(func->AddKernel({ty, ty}, ty, exec));
@@ -1961,8 +1961,8 @@ std::shared_ptr<ScalarFunction> MakeArithmeticFunctionNotNull(std::string name,
 
 template <typename Op>
 std::shared_ptr<ScalarFunction> MakeUnaryArithmeticFunction(std::string name,
-                                                            const FunctionDoc* doc) {
-  auto func = std::make_shared<ArithmeticFunction>(name, Arity::Unary(), doc);
+                                                            FunctionDoc doc) {
+  auto func = std::make_shared<ArithmeticFunction>(name, Arity::Unary(), std::move(doc));
   for (const auto& ty : NumericTypes()) {
     auto exec = ArithmeticExecFromOp<ScalarUnary, Op>(ty);
     DCHECK_OK(func->AddKernel({ty}, ty, exec));
@@ -1975,9 +1975,9 @@ std::shared_ptr<ScalarFunction> MakeUnaryArithmeticFunction(std::string name,
 // output type for integral inputs.
 template <typename Op, typename IntOutType>
 std::shared_ptr<ScalarFunction> MakeUnaryArithmeticFunctionWithFixedIntOutType(
-    std::string name, const FunctionDoc* doc) {
+    std::string name, FunctionDoc doc) {
   auto int_out_ty = TypeTraits<IntOutType>::type_singleton();
-  auto func = std::make_shared<ArithmeticFunction>(name, Arity::Unary(), doc);
+  auto func = std::make_shared<ArithmeticFunction>(name, Arity::Unary(), std::move(doc));
   for (const auto& ty : NumericTypes()) {
     auto out_ty = arrow::is_floating(ty->id()) ? ty : int_out_ty;
     auto exec = GenerateArithmeticWithFixedIntOutType<ScalarUnary, IntOutType, Op>(ty);
@@ -1996,9 +1996,9 @@ std::shared_ptr<ScalarFunction> MakeUnaryArithmeticFunctionWithFixedIntOutType(
 // Like MakeUnaryArithmeticFunction, but for arithmetic ops that need to run
 // only on non-null output.
 template <typename Op>
-std::shared_ptr<ScalarFunction> MakeUnaryArithmeticFunctionNotNull(
-    std::string name, const FunctionDoc* doc) {
-  auto func = std::make_shared<ArithmeticFunction>(name, Arity::Unary(), doc);
+std::shared_ptr<ScalarFunction> MakeUnaryArithmeticFunctionNotNull(std::string name,
+                                                                   FunctionDoc doc) {
+  auto func = std::make_shared<ArithmeticFunction>(name, Arity::Unary(), std::move(doc));
   for (const auto& ty : NumericTypes()) {
     auto exec = ArithmeticExecFromOp<ScalarUnaryNotNull, Op>(ty);
     DCHECK_OK(func->AddKernel({ty}, ty, exec));
@@ -2075,11 +2075,11 @@ Status ExecRound(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
 // kernel dispatch based on RoundMode, only on non-null output.
 template <template <typename, RoundMode, typename...> class Op, typename OptionsType>
 std::shared_ptr<ScalarFunction> MakeUnaryRoundFunction(std::string name,
-                                                       const FunctionDoc* doc) {
+                                                       FunctionDoc doc) {
   using State = RoundOptionsWrapper<OptionsType>;
   static const OptionsType kDefaultOptions = OptionsType::Defaults();
   auto func = std::make_shared<ArithmeticIntegerToFloatingPointFunction>(
-      name, Arity::Unary(), doc, &kDefaultOptions);
+      name, Arity::Unary(), std::move(doc), &kDefaultOptions);
   for (const auto& ty : {float32(), float64(), decimal128(1, 0), decimal256(1, 0)}) {
     auto type_id = ty->id();
     auto exec = [type_id](KernelContext* ctx, const ExecBatch& batch, Datum* out) {
@@ -2110,8 +2110,8 @@ std::shared_ptr<ScalarFunction> MakeUnaryRoundFunction(std::string name,
 // only on non-null output.
 template <typename Op>
 std::shared_ptr<ScalarFunction> MakeUnarySignedArithmeticFunctionNotNull(
-    std::string name, const FunctionDoc* doc) {
-  auto func = std::make_shared<ArithmeticFunction>(name, Arity::Unary(), doc);
+    std::string name, FunctionDoc doc) {
+  auto func = std::make_shared<ArithmeticFunction>(name, Arity::Unary(), std::move(doc));
   for (const auto& ty : NumericTypes()) {
     if (!arrow::is_unsigned_integer(ty->id())) {
       auto exec = ArithmeticExecFromOp<ScalarUnaryNotNull, Op>(ty);
@@ -2124,8 +2124,8 @@ std::shared_ptr<ScalarFunction> MakeUnarySignedArithmeticFunctionNotNull(
 
 template <typename Op>
 std::shared_ptr<ScalarFunction> MakeBitWiseFunctionNotNull(std::string name,
-                                                           const FunctionDoc* doc) {
-  auto func = std::make_shared<ArithmeticFunction>(name, Arity::Binary(), doc);
+                                                           FunctionDoc doc) {
+  auto func = std::make_shared<ArithmeticFunction>(name, Arity::Binary(), std::move(doc));
   for (const auto& ty : IntTypes()) {
     auto exec = TypeAgnosticBitWiseExecFromOp<ScalarBinaryNotNullEqualTypes, Op>(ty);
     DCHECK_OK(func->AddKernel({ty, ty}, ty, exec));
@@ -2136,8 +2136,8 @@ std::shared_ptr<ScalarFunction> MakeBitWiseFunctionNotNull(std::string name,
 
 template <typename Op>
 std::shared_ptr<ScalarFunction> MakeShiftFunctionNotNull(std::string name,
-                                                         const FunctionDoc* doc) {
-  auto func = std::make_shared<ArithmeticFunction>(name, Arity::Binary(), doc);
+                                                         FunctionDoc doc) {
+  auto func = std::make_shared<ArithmeticFunction>(name, Arity::Binary(), std::move(doc));
   for (const auto& ty : IntTypes()) {
     auto exec = ShiftExecFromOp<ScalarBinaryNotNullEqualTypes, Op>(ty);
     DCHECK_OK(func->AddKernel({ty, ty}, ty, exec));
@@ -2148,8 +2148,8 @@ std::shared_ptr<ScalarFunction> MakeShiftFunctionNotNull(std::string name,
 
 template <typename Op, typename FunctionImpl = ArithmeticFloatingPointFunction>
 std::shared_ptr<ScalarFunction> MakeUnaryArithmeticFunctionFloatingPoint(
-    std::string name, const FunctionDoc* doc) {
-  auto func = std::make_shared<FunctionImpl>(name, Arity::Unary(), doc);
+    std::string name, FunctionDoc doc) {
+  auto func = std::make_shared<FunctionImpl>(name, Arity::Unary(), std::move(doc));
   for (const auto& ty : FloatingPointTypes()) {
     auto exec = GenerateArithmeticFloatingPoint<ScalarUnary, Op>(ty);
     DCHECK_OK(func->AddKernel({ty}, ty, exec));
@@ -2160,9 +2160,9 @@ std::shared_ptr<ScalarFunction> MakeUnaryArithmeticFunctionFloatingPoint(
 
 template <typename Op>
 std::shared_ptr<ScalarFunction> MakeUnaryArithmeticFunctionFloatingPointNotNull(
-    std::string name, const FunctionDoc* doc) {
-  auto func =
-      std::make_shared<ArithmeticFloatingPointFunction>(name, Arity::Unary(), doc);
+    std::string name, FunctionDoc doc) {
+  auto func = std::make_shared<ArithmeticFloatingPointFunction>(name, Arity::Unary(),
+                                                                std::move(doc));
   for (const auto& ty : FloatingPointTypes()) {
     auto exec = GenerateArithmeticFloatingPoint<ScalarUnaryNotNull, Op>(ty);
     DCHECK_OK(func->AddKernel({ty}, ty, exec));
@@ -2172,10 +2172,10 @@ std::shared_ptr<ScalarFunction> MakeUnaryArithmeticFunctionFloatingPointNotNull(
 }
 
 template <typename Op>
-std::shared_ptr<ScalarFunction> MakeArithmeticFunctionFloatingPoint(
-    std::string name, const FunctionDoc* doc) {
-  auto func =
-      std::make_shared<ArithmeticFloatingPointFunction>(name, Arity::Binary(), doc);
+std::shared_ptr<ScalarFunction> MakeArithmeticFunctionFloatingPoint(std::string name,
+                                                                    FunctionDoc doc) {
+  auto func = std::make_shared<ArithmeticFloatingPointFunction>(name, Arity::Binary(),
+                                                                std::move(doc));
   for (const auto& ty : FloatingPointTypes()) {
     auto exec = GenerateArithmeticFloatingPoint<ScalarBinaryEqualTypes, Op>(ty);
     DCHECK_OK(func->AddKernel({ty, ty}, ty, exec));
@@ -2186,9 +2186,9 @@ std::shared_ptr<ScalarFunction> MakeArithmeticFunctionFloatingPoint(
 
 template <typename Op>
 std::shared_ptr<ScalarFunction> MakeArithmeticFunctionFloatingPointNotNull(
-    std::string name, const FunctionDoc* doc) {
-  auto func =
-      std::make_shared<ArithmeticFloatingPointFunction>(name, Arity::Binary(), doc);
+    std::string name, FunctionDoc doc) {
+  auto func = std::make_shared<ArithmeticFloatingPointFunction>(name, Arity::Binary(),
+                                                                std::move(doc));
   for (const auto& ty : FloatingPointTypes()) {
     auto output = is_integer(ty->id()) ? float64() : ty;
     auto exec = GenerateArithmeticFloatingPoint<ScalarBinaryNotNullEqualTypes, Op>(ty);
@@ -2573,18 +2573,18 @@ void RegisterScalarArithmetic(FunctionRegistry* registry) {
 
   // ----------------------------------------------------------------------
   auto absolute_value =
-      MakeUnaryArithmeticFunction<AbsoluteValue>("abs", &absolute_value_doc);
+      MakeUnaryArithmeticFunction<AbsoluteValue>("abs", absolute_value_doc);
   AddDecimalUnaryKernels<AbsoluteValue>(absolute_value.get());
   DCHECK_OK(registry->AddFunction(std::move(absolute_value)));
 
   // ----------------------------------------------------------------------
   auto absolute_value_checked = MakeUnaryArithmeticFunctionNotNull<AbsoluteValueChecked>(
-      "abs_checked", &absolute_value_checked_doc);
+      "abs_checked", absolute_value_checked_doc);
   AddDecimalUnaryKernels<AbsoluteValueChecked>(absolute_value_checked.get());
   DCHECK_OK(registry->AddFunction(std::move(absolute_value_checked)));
 
   // ----------------------------------------------------------------------
-  auto add = MakeArithmeticFunction<Add>("add", &add_doc);
+  auto add = MakeArithmeticFunction<Add>("add", add_doc);
   AddDecimalBinaryKernels<Add>("add", add.get());
 
   // Add add(timestamp, duration) -> timestamp
@@ -2609,7 +2609,7 @@ void RegisterScalarArithmetic(FunctionRegistry* registry) {
 
   // ----------------------------------------------------------------------
   auto add_checked =
-      MakeArithmeticFunctionNotNull<AddChecked>("add_checked", &add_checked_doc);
+      MakeArithmeticFunctionNotNull<AddChecked>("add_checked", add_checked_doc);
   AddDecimalBinaryKernels<AddChecked>("add_checked", add_checked.get());
 
   // Add add_checked(timestamp, duration) -> timestamp
@@ -2636,7 +2636,7 @@ void RegisterScalarArithmetic(FunctionRegistry* registry) {
   DCHECK_OK(registry->AddFunction(std::move(add_checked)));
 
   // ----------------------------------------------------------------------
-  auto subtract = MakeArithmeticFunction<Subtract>("subtract", &sub_doc);
+  auto subtract = MakeArithmeticFunction<Subtract>("subtract", sub_doc);
   AddDecimalBinaryKernels<Subtract>("subtract", subtract.get());
 
   // Add subtract(timestamp, timestamp) -> duration
@@ -2694,8 +2694,8 @@ void RegisterScalarArithmetic(FunctionRegistry* registry) {
   DCHECK_OK(registry->AddFunction(std::move(subtract)));
 
   // ----------------------------------------------------------------------
-  auto subtract_checked = MakeArithmeticFunctionNotNull<SubtractChecked>(
-      "subtract_checked", &sub_checked_doc);
+  auto subtract_checked =
+      MakeArithmeticFunctionNotNull<SubtractChecked>("subtract_checked", sub_checked_doc);
   AddDecimalBinaryKernels<SubtractChecked>("subtract_checked", subtract_checked.get());
 
   // Add subtract_checked(timestamp, timestamp) -> duration
@@ -2760,7 +2760,7 @@ void RegisterScalarArithmetic(FunctionRegistry* registry) {
   DCHECK_OK(registry->AddFunction(std::move(subtract_checked)));
 
   // ----------------------------------------------------------------------
-  auto multiply = MakeArithmeticFunction<Multiply>("multiply", &mul_doc);
+  auto multiply = MakeArithmeticFunction<Multiply>("multiply", mul_doc);
   AddDecimalBinaryKernels<Multiply>("multiply", multiply.get());
 
   // Add multiply(duration, int64) -> duration
@@ -2773,8 +2773,8 @@ void RegisterScalarArithmetic(FunctionRegistry* registry) {
   DCHECK_OK(registry->AddFunction(std::move(multiply)));
 
   // ----------------------------------------------------------------------
-  auto multiply_checked = MakeArithmeticFunctionNotNull<MultiplyChecked>(
-      "multiply_checked", &mul_checked_doc);
+  auto multiply_checked =
+      MakeArithmeticFunctionNotNull<MultiplyChecked>("multiply_checked", mul_checked_doc);
   AddDecimalBinaryKernels<MultiplyChecked>("multiply_checked", multiply_checked.get());
 
   // Add multiply_checked(duration, int64) -> duration
@@ -2790,7 +2790,7 @@ void RegisterScalarArithmetic(FunctionRegistry* registry) {
   DCHECK_OK(registry->AddFunction(std::move(multiply_checked)));
 
   // ----------------------------------------------------------------------
-  auto divide = MakeArithmeticFunctionNotNull<Divide>("divide", &div_doc);
+  auto divide = MakeArithmeticFunctionNotNull<Divide>("divide", div_doc);
   AddDecimalBinaryKernels<Divide>("divide", divide.get());
 
   // Add divide(duration, int64) -> duration
@@ -2803,7 +2803,7 @@ void RegisterScalarArithmetic(FunctionRegistry* registry) {
 
   // ----------------------------------------------------------------------
   auto divide_checked =
-      MakeArithmeticFunctionNotNull<DivideChecked>("divide_checked", &div_checked_doc);
+      MakeArithmeticFunctionNotNull<DivideChecked>("divide_checked", div_checked_doc);
   AddDecimalBinaryKernels<DivideChecked>("divide_checked", divide_checked.get());
 
   // Add divide_checked(duration, int64) -> duration
@@ -2816,47 +2816,47 @@ void RegisterScalarArithmetic(FunctionRegistry* registry) {
   DCHECK_OK(registry->AddFunction(std::move(divide_checked)));
 
   // ----------------------------------------------------------------------
-  auto negate = MakeUnaryArithmeticFunction<Negate>("negate", &negate_doc);
+  auto negate = MakeUnaryArithmeticFunction<Negate>("negate", negate_doc);
   AddDecimalUnaryKernels<Negate>(negate.get());
   DCHECK_OK(registry->AddFunction(std::move(negate)));
 
   // ----------------------------------------------------------------------
   auto negate_checked = MakeUnarySignedArithmeticFunctionNotNull<NegateChecked>(
-      "negate_checked", &negate_checked_doc);
+      "negate_checked", negate_checked_doc);
   AddDecimalUnaryKernels<NegateChecked>(negate_checked.get());
   DCHECK_OK(registry->AddFunction(std::move(negate_checked)));
 
   // ----------------------------------------------------------------------
   auto power = MakeArithmeticFunction<Power, ArithmeticDecimalToFloatingPointFunction>(
-      "power", &pow_doc);
+      "power", pow_doc);
   DCHECK_OK(registry->AddFunction(std::move(power)));
 
   // ----------------------------------------------------------------------
   auto power_checked =
       MakeArithmeticFunctionNotNull<PowerChecked,
                                     ArithmeticDecimalToFloatingPointFunction>(
-          "power_checked", &pow_checked_doc);
+          "power_checked", pow_checked_doc);
   DCHECK_OK(registry->AddFunction(std::move(power_checked)));
 
   // ----------------------------------------------------------------------
-  auto sqrt = MakeUnaryArithmeticFunctionFloatingPoint<SquareRoot>("sqrt", &sqrt_doc);
+  auto sqrt = MakeUnaryArithmeticFunctionFloatingPoint<SquareRoot>("sqrt", sqrt_doc);
   DCHECK_OK(registry->AddFunction(std::move(sqrt)));
 
   // ----------------------------------------------------------------------
   auto sqrt_checked = MakeUnaryArithmeticFunctionFloatingPointNotNull<SquareRootChecked>(
-      "sqrt_checked", &sqrt_checked_doc);
+      "sqrt_checked", sqrt_checked_doc);
   DCHECK_OK(registry->AddFunction(std::move(sqrt_checked)));
 
   // ----------------------------------------------------------------------
   auto sign =
-      MakeUnaryArithmeticFunctionWithFixedIntOutType<Sign, Int8Type>("sign", &sign_doc);
+      MakeUnaryArithmeticFunctionWithFixedIntOutType<Sign, Int8Type>("sign", sign_doc);
   DCHECK_OK(registry->AddFunction(std::move(sign)));
 
   // ----------------------------------------------------------------------
   // Bitwise functions
   {
     auto bit_wise_not = std::make_shared<ArithmeticFunction>(
-        "bit_wise_not", Arity::Unary(), &bit_wise_not_doc);
+        "bit_wise_not", Arity::Unary(), bit_wise_not_doc);
     for (const auto& ty : IntTypes()) {
       auto exec = TypeAgnosticBitWiseExecFromOp<ScalarUnaryNotNull, BitWiseNot>(ty);
       DCHECK_OK(bit_wise_not->AddKernel({ty}, ty, exec));
@@ -2866,110 +2866,109 @@ void RegisterScalarArithmetic(FunctionRegistry* registry) {
   }
 
   auto bit_wise_and =
-      MakeBitWiseFunctionNotNull<BitWiseAnd>("bit_wise_and", &bit_wise_and_doc);
+      MakeBitWiseFunctionNotNull<BitWiseAnd>("bit_wise_and", bit_wise_and_doc);
   DCHECK_OK(registry->AddFunction(std::move(bit_wise_and)));
 
   auto bit_wise_or =
-      MakeBitWiseFunctionNotNull<BitWiseOr>("bit_wise_or", &bit_wise_or_doc);
+      MakeBitWiseFunctionNotNull<BitWiseOr>("bit_wise_or", bit_wise_or_doc);
   DCHECK_OK(registry->AddFunction(std::move(bit_wise_or)));
 
   auto bit_wise_xor =
-      MakeBitWiseFunctionNotNull<BitWiseXor>("bit_wise_xor", &bit_wise_xor_doc);
+      MakeBitWiseFunctionNotNull<BitWiseXor>("bit_wise_xor", bit_wise_xor_doc);
   DCHECK_OK(registry->AddFunction(std::move(bit_wise_xor)));
 
-  auto shift_left = MakeShiftFunctionNotNull<ShiftLeft>("shift_left", &shift_left_doc);
+  auto shift_left = MakeShiftFunctionNotNull<ShiftLeft>("shift_left", shift_left_doc);
   DCHECK_OK(registry->AddFunction(std::move(shift_left)));
 
   auto shift_left_checked = MakeShiftFunctionNotNull<ShiftLeftChecked>(
-      "shift_left_checked", &shift_left_checked_doc);
+      "shift_left_checked", shift_left_checked_doc);
   DCHECK_OK(registry->AddFunction(std::move(shift_left_checked)));
 
-  auto shift_right =
-      MakeShiftFunctionNotNull<ShiftRight>("shift_right", &shift_right_doc);
+  auto shift_right = MakeShiftFunctionNotNull<ShiftRight>("shift_right", shift_right_doc);
   DCHECK_OK(registry->AddFunction(std::move(shift_right)));
 
   auto shift_right_checked = MakeShiftFunctionNotNull<ShiftRightChecked>(
-      "shift_right_checked", &shift_right_checked_doc);
+      "shift_right_checked", shift_right_checked_doc);
   DCHECK_OK(registry->AddFunction(std::move(shift_right_checked)));
 
   // ----------------------------------------------------------------------
   // Trig functions
-  auto sin = MakeUnaryArithmeticFunctionFloatingPoint<Sin>("sin", &sin_doc);
+  auto sin = MakeUnaryArithmeticFunctionFloatingPoint<Sin>("sin", sin_doc);
   DCHECK_OK(registry->AddFunction(std::move(sin)));
 
   auto sin_checked = MakeUnaryArithmeticFunctionFloatingPointNotNull<SinChecked>(
-      "sin_checked", &sin_checked_doc);
+      "sin_checked", sin_checked_doc);
   DCHECK_OK(registry->AddFunction(std::move(sin_checked)));
 
-  auto cos = MakeUnaryArithmeticFunctionFloatingPoint<Cos>("cos", &cos_doc);
+  auto cos = MakeUnaryArithmeticFunctionFloatingPoint<Cos>("cos", cos_doc);
   DCHECK_OK(registry->AddFunction(std::move(cos)));
 
   auto cos_checked = MakeUnaryArithmeticFunctionFloatingPointNotNull<CosChecked>(
-      "cos_checked", &cos_checked_doc);
+      "cos_checked", cos_checked_doc);
   DCHECK_OK(registry->AddFunction(std::move(cos_checked)));
 
-  auto tan = MakeUnaryArithmeticFunctionFloatingPoint<Tan>("tan", &tan_doc);
+  auto tan = MakeUnaryArithmeticFunctionFloatingPoint<Tan>("tan", tan_doc);
   DCHECK_OK(registry->AddFunction(std::move(tan)));
 
   auto tan_checked = MakeUnaryArithmeticFunctionFloatingPointNotNull<TanChecked>(
-      "tan_checked", &tan_checked_doc);
+      "tan_checked", tan_checked_doc);
   DCHECK_OK(registry->AddFunction(std::move(tan_checked)));
 
-  auto asin = MakeUnaryArithmeticFunctionFloatingPoint<Asin>("asin", &asin_doc);
+  auto asin = MakeUnaryArithmeticFunctionFloatingPoint<Asin>("asin", asin_doc);
   DCHECK_OK(registry->AddFunction(std::move(asin)));
 
   auto asin_checked = MakeUnaryArithmeticFunctionFloatingPointNotNull<AsinChecked>(
-      "asin_checked", &asin_checked_doc);
+      "asin_checked", asin_checked_doc);
   DCHECK_OK(registry->AddFunction(std::move(asin_checked)));
 
-  auto acos = MakeUnaryArithmeticFunctionFloatingPoint<Acos>("acos", &acos_doc);
+  auto acos = MakeUnaryArithmeticFunctionFloatingPoint<Acos>("acos", acos_doc);
   DCHECK_OK(registry->AddFunction(std::move(acos)));
 
   auto acos_checked = MakeUnaryArithmeticFunctionFloatingPointNotNull<AcosChecked>(
-      "acos_checked", &acos_checked_doc);
+      "acos_checked", acos_checked_doc);
   DCHECK_OK(registry->AddFunction(std::move(acos_checked)));
 
-  auto atan = MakeUnaryArithmeticFunctionFloatingPoint<Atan>("atan", &atan_doc);
+  auto atan = MakeUnaryArithmeticFunctionFloatingPoint<Atan>("atan", atan_doc);
   DCHECK_OK(registry->AddFunction(std::move(atan)));
 
-  auto atan2 = MakeArithmeticFunctionFloatingPoint<Atan2>("atan2", &atan2_doc);
+  auto atan2 = MakeArithmeticFunctionFloatingPoint<Atan2>("atan2", atan2_doc);
   DCHECK_OK(registry->AddFunction(std::move(atan2)));
 
   // ----------------------------------------------------------------------
   // Logarithms
-  auto ln = MakeUnaryArithmeticFunctionFloatingPoint<LogNatural>("ln", &ln_doc);
+  auto ln = MakeUnaryArithmeticFunctionFloatingPoint<LogNatural>("ln", ln_doc);
   DCHECK_OK(registry->AddFunction(std::move(ln)));
 
   auto ln_checked = MakeUnaryArithmeticFunctionFloatingPointNotNull<LogNaturalChecked>(
-      "ln_checked", &ln_checked_doc);
+      "ln_checked", ln_checked_doc);
   DCHECK_OK(registry->AddFunction(std::move(ln_checked)));
 
-  auto log10 = MakeUnaryArithmeticFunctionFloatingPoint<Log10>("log10", &log10_doc);
+  auto log10 = MakeUnaryArithmeticFunctionFloatingPoint<Log10>("log10", log10_doc);
   DCHECK_OK(registry->AddFunction(std::move(log10)));
 
   auto log10_checked = MakeUnaryArithmeticFunctionFloatingPointNotNull<Log10Checked>(
-      "log10_checked", &log10_checked_doc);
+      "log10_checked", log10_checked_doc);
   DCHECK_OK(registry->AddFunction(std::move(log10_checked)));
 
-  auto log2 = MakeUnaryArithmeticFunctionFloatingPoint<Log2>("log2", &log2_doc);
+  auto log2 = MakeUnaryArithmeticFunctionFloatingPoint<Log2>("log2", log2_doc);
   DCHECK_OK(registry->AddFunction(std::move(log2)));
 
   auto log2_checked = MakeUnaryArithmeticFunctionFloatingPointNotNull<Log2Checked>(
-      "log2_checked", &log2_checked_doc);
+      "log2_checked", log2_checked_doc);
   DCHECK_OK(registry->AddFunction(std::move(log2_checked)));
 
-  auto log1p = MakeUnaryArithmeticFunctionFloatingPoint<Log1p>("log1p", &log1p_doc);
+  auto log1p = MakeUnaryArithmeticFunctionFloatingPoint<Log1p>("log1p", log1p_doc);
   DCHECK_OK(registry->AddFunction(std::move(log1p)));
 
   auto log1p_checked = MakeUnaryArithmeticFunctionFloatingPointNotNull<Log1pChecked>(
-      "log1p_checked", &log1p_checked_doc);
+      "log1p_checked", log1p_checked_doc);
   DCHECK_OK(registry->AddFunction(std::move(log1p_checked)));
 
-  auto logb = MakeArithmeticFunctionFloatingPoint<Logb>("logb", &logb_doc);
+  auto logb = MakeArithmeticFunctionFloatingPoint<Logb>("logb", logb_doc);
   DCHECK_OK(registry->AddFunction(std::move(logb)));
 
   auto logb_checked = MakeArithmeticFunctionFloatingPointNotNull<LogbChecked>(
-      "logb_checked", &logb_checked_doc);
+      "logb_checked", logb_checked_doc);
   DCHECK_OK(registry->AddFunction(std::move(logb_checked)));
 
   // ----------------------------------------------------------------------
@@ -2977,7 +2976,7 @@ void RegisterScalarArithmetic(FunctionRegistry* registry) {
   auto floor =
       MakeUnaryArithmeticFunctionFloatingPoint<Floor,
                                                ArithmeticIntegerToFloatingPointFunction>(
-          "floor", &floor_doc);
+          "floor", floor_doc);
   DCHECK_OK(floor->AddKernel(
       {InputType(Type::DECIMAL128)}, OutputType(FirstType),
       FixedRoundDecimalExec<Decimal128Type, RoundMode::DOWN, /*ndigits=*/0>));
@@ -2989,7 +2988,7 @@ void RegisterScalarArithmetic(FunctionRegistry* registry) {
   auto ceil =
       MakeUnaryArithmeticFunctionFloatingPoint<Ceil,
                                                ArithmeticIntegerToFloatingPointFunction>(
-          "ceil", &ceil_doc);
+          "ceil", ceil_doc);
   DCHECK_OK(ceil->AddKernel(
       {InputType(Type::DECIMAL128)}, OutputType(FirstType),
       FixedRoundDecimalExec<Decimal128Type, RoundMode::UP, /*ndigits=*/0>));
@@ -3001,7 +3000,7 @@ void RegisterScalarArithmetic(FunctionRegistry* registry) {
   auto trunc =
       MakeUnaryArithmeticFunctionFloatingPoint<Trunc,
                                                ArithmeticIntegerToFloatingPointFunction>(
-          "trunc", &trunc_doc);
+          "trunc", trunc_doc);
   DCHECK_OK(trunc->AddKernel(
       {InputType(Type::DECIMAL128)}, OutputType(FirstType),
       FixedRoundDecimalExec<Decimal128Type, RoundMode::TOWARDS_ZERO, /*ndigits=*/0>));
@@ -3010,12 +3009,12 @@ void RegisterScalarArithmetic(FunctionRegistry* registry) {
       FixedRoundDecimalExec<Decimal256Type, RoundMode::TOWARDS_ZERO, /*ndigits=*/0>));
   DCHECK_OK(registry->AddFunction(std::move(trunc)));
 
-  auto round = MakeUnaryRoundFunction<Round, RoundOptions>("round", &round_doc);
+  auto round = MakeUnaryRoundFunction<Round, RoundOptions>("round", round_doc);
   DCHECK_OK(registry->AddFunction(std::move(round)));
 
   auto round_to_multiple =
       MakeUnaryRoundFunction<RoundToMultiple, RoundToMultipleOptions>(
-          "round_to_multiple", &round_to_multiple_doc);
+          "round_to_multiple", round_to_multiple_doc);
   DCHECK_OK(registry->AddFunction(std::move(round_to_multiple)));
 }
 
diff --git a/cpp/src/arrow/compute/kernels/scalar_boolean.cc b/cpp/src/arrow/compute/kernels/scalar_boolean.cc
index 9c625b5599..1aea247ddf 100644
--- a/cpp/src/arrow/compute/kernels/scalar_boolean.cc
+++ b/cpp/src/arrow/compute/kernels/scalar_boolean.cc
@@ -455,9 +455,9 @@ struct KleeneAndNotOp {
 };
 
 void MakeFunction(const std::string& name, int arity, ArrayKernelExec exec,
-                  const FunctionDoc* doc, FunctionRegistry* registry,
+                  FunctionDoc doc, FunctionRegistry* registry,
                   NullHandling::type null_handling = NullHandling::INTERSECTION) {
-  auto func = std::make_shared<ScalarFunction>(name, Arity(arity), doc);
+  auto func = std::make_shared<ScalarFunction>(name, Arity(arity), std::move(doc));
 
   std::vector<InputType> in_types(arity, InputType(boolean()));
   ScalarKernel kernel(std::move(in_types), boolean(), exec);
@@ -541,19 +541,19 @@ namespace internal {
 
 void RegisterScalarBoolean(FunctionRegistry* registry) {
   // These functions can write into sliced output bitmaps
-  MakeFunction("invert", 1, applicator::SimpleUnary<InvertOp>, &invert_doc, registry);
-  MakeFunction("and", 2, applicator::SimpleBinary<AndOp>, &and_doc, registry);
-  MakeFunction("and_not", 2, applicator::SimpleBinary<AndNotOp>, &and_not_doc, registry);
-  MakeFunction("or", 2, applicator::SimpleBinary<OrOp>, &or_doc, registry);
-  MakeFunction("xor", 2, applicator::SimpleBinary<XorOp>, &xor_doc, registry);
+  MakeFunction("invert", 1, applicator::SimpleUnary<InvertOp>, invert_doc, registry);
+  MakeFunction("and", 2, applicator::SimpleBinary<AndOp>, and_doc, registry);
+  MakeFunction("and_not", 2, applicator::SimpleBinary<AndNotOp>, and_not_doc, registry);
+  MakeFunction("or", 2, applicator::SimpleBinary<OrOp>, or_doc, registry);
+  MakeFunction("xor", 2, applicator::SimpleBinary<XorOp>, xor_doc, registry);
 
   // The null bitmap is not preallocated for Kleene kernels, as sometimes
   // all outputs are valid even though some inputs may be null.
-  MakeFunction("and_kleene", 2, applicator::SimpleBinary<KleeneAndOp>, &and_kleene_doc,
+  MakeFunction("and_kleene", 2, applicator::SimpleBinary<KleeneAndOp>, and_kleene_doc,
                registry, NullHandling::COMPUTED_NO_PREALLOCATE);
   MakeFunction("and_not_kleene", 2, applicator::SimpleBinary<KleeneAndNotOp>,
-               &and_not_kleene_doc, registry, NullHandling::COMPUTED_NO_PREALLOCATE);
-  MakeFunction("or_kleene", 2, applicator::SimpleBinary<KleeneOrOp>, &or_kleene_doc,
+               and_not_kleene_doc, registry, NullHandling::COMPUTED_NO_PREALLOCATE);
+  MakeFunction("or_kleene", 2, applicator::SimpleBinary<KleeneOrOp>, or_kleene_doc,
                registry, NullHandling::COMPUTED_NO_PREALLOCATE);
 }
 
diff --git a/cpp/src/arrow/compute/kernels/scalar_compare.cc b/cpp/src/arrow/compute/kernels/scalar_compare.cc
index fc85278102..f5ccb991d9 100644
--- a/cpp/src/arrow/compute/kernels/scalar_compare.cc
+++ b/cpp/src/arrow/compute/kernels/scalar_compare.cc
@@ -240,9 +240,8 @@ struct VarArgsCompareFunction : ScalarFunction {
 };
 
 template <typename Op>
-std::shared_ptr<ScalarFunction> MakeCompareFunction(std::string name,
-                                                    const FunctionDoc* doc) {
-  auto func = std::make_shared<CompareFunction>(name, Arity::Binary(), doc);
+std::shared_ptr<ScalarFunction> MakeCompareFunction(std::string name, FunctionDoc doc) {
+  auto func = std::make_shared<CompareFunction>(name, Arity::Binary(), std::move(doc));
 
   DCHECK_OK(func->AddKernel(
       {boolean(), boolean()}, boolean(),
@@ -313,8 +312,9 @@ std::shared_ptr<ScalarFunction> MakeCompareFunction(std::string name,
 
 std::shared_ptr<ScalarFunction> MakeFlippedFunction(std::string name,
                                                     const ScalarFunction& func,
-                                                    const FunctionDoc* doc) {
-  auto flipped_func = std::make_shared<CompareFunction>(name, Arity::Binary(), doc);
+                                                    FunctionDoc doc) {
+  auto flipped_func =
+      std::make_shared<CompareFunction>(name, Arity::Binary(), std::move(doc));
   for (const ScalarKernel* kernel : func.kernels()) {
     ScalarKernel flipped_kernel = *kernel;
     flipped_kernel.exec = MakeFlippedBinaryExec(kernel->exec);
@@ -691,13 +691,12 @@ Result<ValueDescr> ResolveMinOrMaxOutputType(KernelContext*,
 }
 
 template <typename Op>
-std::shared_ptr<ScalarFunction> MakeScalarMinMax(std::string name,
-                                                 const FunctionDoc* doc) {
+std::shared_ptr<ScalarFunction> MakeScalarMinMax(std::string name, FunctionDoc doc) {
   static auto default_element_wise_aggregate_options =
       ElementWiseAggregateOptions::Defaults();
 
   auto func = std::make_shared<VarArgsCompareFunction>(
-      name, Arity::VarArgs(), doc, &default_element_wise_aggregate_options);
+      name, Arity::VarArgs(), std::move(doc), &default_element_wise_aggregate_options);
   for (const auto& ty : NumericTypes()) {
     auto exec = GeneratePhysicalNumeric<ScalarMinMax, Op>(ty);
     ScalarKernel kernel{KernelSignature::Make({ty}, ty, /*is_varargs=*/true), exec,
@@ -789,16 +788,16 @@ const FunctionDoc max_element_wise_doc{
 }  // namespace
 
 void RegisterScalarComparison(FunctionRegistry* registry) {
-  DCHECK_OK(registry->AddFunction(MakeCompareFunction<Equal>("equal", &equal_doc)));
+  DCHECK_OK(registry->AddFunction(MakeCompareFunction<Equal>("equal", equal_doc)));
   DCHECK_OK(
-      registry->AddFunction(MakeCompareFunction<NotEqual>("not_equal", &not_equal_doc)));
+      registry->AddFunction(MakeCompareFunction<NotEqual>("not_equal", not_equal_doc)));
 
-  auto greater = MakeCompareFunction<Greater>("greater", &greater_doc);
+  auto greater = MakeCompareFunction<Greater>("greater", greater_doc);
   auto greater_equal =
-      MakeCompareFunction<GreaterEqual>("greater_equal", &greater_equal_doc);
+      MakeCompareFunction<GreaterEqual>("greater_equal", greater_equal_doc);
 
-  auto less = MakeFlippedFunction("less", *greater, &less_doc);
-  auto less_equal = MakeFlippedFunction("less_equal", *greater_equal, &less_equal_doc);
+  auto less = MakeFlippedFunction("less", *greater, less_doc);
+  auto less_equal = MakeFlippedFunction("less_equal", *greater_equal, less_equal_doc);
   DCHECK_OK(registry->AddFunction(std::move(less)));
   DCHECK_OK(registry->AddFunction(std::move(less_equal)));
   DCHECK_OK(registry->AddFunction(std::move(greater)));
@@ -808,11 +807,11 @@ void RegisterScalarComparison(FunctionRegistry* registry) {
   // Variadic element-wise functions
 
   auto min_element_wise =
-      MakeScalarMinMax<Minimum>("min_element_wise", &min_element_wise_doc);
+      MakeScalarMinMax<Minimum>("min_element_wise", min_element_wise_doc);
   DCHECK_OK(registry->AddFunction(std::move(min_element_wise)));
 
   auto max_element_wise =
-      MakeScalarMinMax<Maximum>("max_element_wise", &max_element_wise_doc);
+      MakeScalarMinMax<Maximum>("max_element_wise", max_element_wise_doc);
   DCHECK_OK(registry->AddFunction(std::move(max_element_wise)));
 }
 
diff --git a/cpp/src/arrow/compute/kernels/scalar_if_else.cc b/cpp/src/arrow/compute/kernels/scalar_if_else.cc
index 5e1edbe4ab..00dcc55a8a 100644
--- a/cpp/src/arrow/compute/kernels/scalar_if_else.cc
+++ b/cpp/src/arrow/compute/kernels/scalar_if_else.cc
@@ -2773,7 +2773,7 @@ const FunctionDoc choose_doc{
 void RegisterScalarIfElse(FunctionRegistry* registry) {
   {
     auto func =
-        std::make_shared<IfElseFunction>("if_else", Arity::Ternary(), &if_else_doc);
+        std::make_shared<IfElseFunction>("if_else", Arity::Ternary(), if_else_doc);
 
     AddPrimitiveIfElseKernels(func, NumericTypes());
     AddPrimitiveIfElseKernels(func, TemporalTypes());
@@ -2789,7 +2789,7 @@ void RegisterScalarIfElse(FunctionRegistry* registry) {
   }
   {
     auto func = std::make_shared<CaseWhenFunction>(
-        "case_when", Arity::VarArgs(/*min_args=*/2), &case_when_doc);
+        "case_when", Arity::VarArgs(/*min_args=*/2), case_when_doc);
     AddPrimitiveCaseWhenKernels(func, NumericTypes());
     AddPrimitiveCaseWhenKernels(func, TemporalTypes());
     AddPrimitiveCaseWhenKernels(func, IntervalTypes());
@@ -2812,7 +2812,7 @@ void RegisterScalarIfElse(FunctionRegistry* registry) {
   }
   {
     auto func = std::make_shared<CoalesceFunction>(
-        "coalesce", Arity::VarArgs(/*min_args=*/1), &coalesce_doc);
+        "coalesce", Arity::VarArgs(/*min_args=*/1), coalesce_doc);
     AddPrimitiveCoalesceKernels(func, NumericTypes());
     AddPrimitiveCoalesceKernels(func, TemporalTypes());
     AddPrimitiveCoalesceKernels(func, IntervalTypes());
@@ -2837,7 +2837,7 @@ void RegisterScalarIfElse(FunctionRegistry* registry) {
   }
   {
     auto func = std::make_shared<ChooseFunction>("choose", Arity::VarArgs(/*min_args=*/2),
-                                                 &choose_doc);
+                                                 choose_doc);
     AddPrimitiveChooseKernels(func, NumericTypes());
     AddPrimitiveChooseKernels(func, TemporalTypes());
     AddPrimitiveChooseKernels(func, IntervalTypes());
diff --git a/cpp/src/arrow/compute/kernels/scalar_nested.cc b/cpp/src/arrow/compute/kernels/scalar_nested.cc
index cf8a7e08b0..1e40d7aaba 100644
--- a/cpp/src/arrow/compute/kernels/scalar_nested.cc
+++ b/cpp/src/arrow/compute/kernels/scalar_nested.cc
@@ -689,7 +689,7 @@ const FunctionDoc map_lookup_doc{
 
 void RegisterScalarNested(FunctionRegistry* registry) {
   auto list_value_length = std::make_shared<ScalarFunction>(
-      "list_value_length", Arity::Unary(), &list_value_length_doc);
+      "list_value_length", Arity::Unary(), list_value_length_doc);
   DCHECK_OK(list_value_length->AddKernel({InputType(Type::LIST)}, int32(),
                                          ListValueLength<ListType>));
   DCHECK_OK(list_value_length->AddKernel({InputType(Type::FIXED_SIZE_LIST)}, int32(),
@@ -698,25 +698,25 @@ void RegisterScalarNested(FunctionRegistry* registry) {
                                          ListValueLength<LargeListType>));
   DCHECK_OK(registry->AddFunction(std::move(list_value_length)));
 
-  auto list_element = std::make_shared<ScalarFunction>("list_element", Arity::Binary(),
-                                                       &list_element_doc);
+  auto list_element =
+      std::make_shared<ScalarFunction>("list_element", Arity::Binary(), list_element_doc);
   AddListElementArrayKernels(list_element.get());
   AddListElementScalarKernels(list_element.get());
   DCHECK_OK(registry->AddFunction(std::move(list_element)));
 
   auto struct_field =
-      std::make_shared<ScalarFunction>("struct_field", Arity::Unary(), &struct_field_doc);
+      std::make_shared<ScalarFunction>("struct_field", Arity::Unary(), struct_field_doc);
   AddStructFieldKernels(struct_field.get());
   DCHECK_OK(registry->AddFunction(std::move(struct_field)));
 
   auto map_lookup =
-      std::make_shared<ScalarFunction>("map_lookup", Arity::Unary(), &map_lookup_doc);
+      std::make_shared<ScalarFunction>("map_lookup", Arity::Unary(), map_lookup_doc);
   AddMapLookupKernels(map_lookup.get());
   DCHECK_OK(registry->AddFunction(std::move(map_lookup)));
 
   static MakeStructOptions kDefaultMakeStructOptions;
   auto make_struct_function = std::make_shared<ScalarFunction>(
-      "make_struct", Arity::VarArgs(), &make_struct_doc, &kDefaultMakeStructOptions);
+      "make_struct", Arity::VarArgs(), make_struct_doc, &kDefaultMakeStructOptions);
 
   ScalarKernel kernel{KernelSignature::Make({InputType{}}, OutputType{MakeStructResolve},
                                             /*is_varargs=*/true),
diff --git a/cpp/src/arrow/compute/kernels/scalar_random.cc b/cpp/src/arrow/compute/kernels/scalar_random.cc
index 3d0e71eb1c..2204dbf020 100644
--- a/cpp/src/arrow/compute/kernels/scalar_random.cc
+++ b/cpp/src/arrow/compute/kernels/scalar_random.cc
@@ -99,7 +99,7 @@ void RegisterScalarRandom(FunctionRegistry* registry) {
   static auto random_options = RandomOptions::Defaults();
 
   auto random_func = std::make_shared<ScalarFunction>("random", Arity::Nullary(),
-                                                      &random_doc, &random_options);
+                                                      random_doc, &random_options);
   ScalarKernel kernel{
       {}, ValueDescr(float64(), ValueDescr::Shape::ARRAY), ExecRandom, RandomState::Init};
   kernel.null_handling = NullHandling::OUTPUT_NOT_NULL;
diff --git a/cpp/src/arrow/compute/kernels/scalar_set_lookup.cc b/cpp/src/arrow/compute/kernels/scalar_set_lookup.cc
index 97eedd53ee..7d8d2edc4b 100644
--- a/cpp/src/arrow/compute/kernels/scalar_set_lookup.cc
+++ b/cpp/src/arrow/compute/kernels/scalar_set_lookup.cc
@@ -481,8 +481,7 @@ const FunctionDoc index_in_meta_doc{
 // Enables calling is_in with CallFunction as though it were binary.
 class IsInMetaBinary : public MetaFunction {
  public:
-  IsInMetaBinary()
-      : MetaFunction("is_in_meta_binary", Arity::Binary(), &is_in_meta_doc) {}
+  IsInMetaBinary() : MetaFunction("is_in_meta_binary", Arity::Binary(), is_in_meta_doc) {}
 
   Result<Datum> ExecuteImpl(const std::vector<Datum>& args,
                             const FunctionOptions* options,
@@ -498,7 +497,7 @@ class IsInMetaBinary : public MetaFunction {
 class IndexInMetaBinary : public MetaFunction {
  public:
   IndexInMetaBinary()
-      : MetaFunction("index_in_meta_binary", Arity::Binary(), &index_in_meta_doc) {}
+      : MetaFunction("index_in_meta_binary", Arity::Binary(), index_in_meta_doc) {}
 
   Result<Datum> ExecuteImpl(const std::vector<Datum>& args,
                             const FunctionOptions* options,
@@ -529,7 +528,7 @@ void RegisterScalarSetLookup(FunctionRegistry* registry) {
     isin_base.exec =
         TrivialScalarUnaryAsArraysExec(ExecIsIn, NullHandling::OUTPUT_NOT_NULL);
     isin_base.null_handling = NullHandling::OUTPUT_NOT_NULL;
-    auto is_in = std::make_shared<SetLookupFunction>("is_in", Arity::Unary(), &is_in_doc);
+    auto is_in = std::make_shared<SetLookupFunction>("is_in", Arity::Unary(), is_in_doc);
 
     AddBasicSetLookupKernels(isin_base, /*output_type=*/boolean(), is_in.get());
 
@@ -549,7 +548,7 @@ void RegisterScalarSetLookup(FunctionRegistry* registry) {
     index_in_base.null_handling = NullHandling::COMPUTED_NO_PREALLOCATE;
     index_in_base.mem_allocation = MemAllocation::NO_PREALLOCATE;
     auto index_in =
-        std::make_shared<SetLookupFunction>("index_in", Arity::Unary(), &index_in_doc);
+        std::make_shared<SetLookupFunction>("index_in", Arity::Unary(), index_in_doc);
 
     AddBasicSetLookupKernels(index_in_base, /*output_type=*/int32(), index_in.get());
 
diff --git a/cpp/src/arrow/compute/kernels/scalar_string_ascii.cc b/cpp/src/arrow/compute/kernels/scalar_string_ascii.cc
index a332364bd8..611601cab8 100644
--- a/cpp/src/arrow/compute/kernels/scalar_string_ascii.cc
+++ b/cpp/src/arrow/compute/kernels/scalar_string_ascii.cc
@@ -776,21 +776,21 @@ const auto ascii_is_title_doc = StringPredicateDoc(
      "an uppercase character."));
 
 void AddAsciiStringPredicates(FunctionRegistry* registry) {
-  AddUnaryStringPredicate<IsAscii>("string_is_ascii", registry, &string_is_ascii_doc);
+  AddUnaryStringPredicate<IsAscii>("string_is_ascii", registry, string_is_ascii_doc);
 
   AddUnaryStringPredicate<IsAlphaNumericAscii>("ascii_is_alnum", registry,
-                                               &ascii_is_alnum_doc);
-  AddUnaryStringPredicate<IsAlphaAscii>("ascii_is_alpha", registry, &ascii_is_alpha_doc);
+                                               ascii_is_alnum_doc);
+  AddUnaryStringPredicate<IsAlphaAscii>("ascii_is_alpha", registry, ascii_is_alpha_doc);
   AddUnaryStringPredicate<IsDecimalAscii>("ascii_is_decimal", registry,
-                                          &ascii_is_decimal_doc);
+                                          ascii_is_decimal_doc);
   // no is_digit for ascii, since it is the same as is_decimal
-  AddUnaryStringPredicate<IsLowerAscii>("ascii_is_lower", registry, &ascii_is_lower_doc);
+  AddUnaryStringPredicate<IsLowerAscii>("ascii_is_lower", registry, ascii_is_lower_doc);
   // no is_numeric for ascii, since it is the same as is_decimal
   AddUnaryStringPredicate<IsPrintableAscii>("ascii_is_printable", registry,
-                                            &ascii_is_printable_doc);
-  AddUnaryStringPredicate<IsSpaceAscii>("ascii_is_space", registry, &ascii_is_space_doc);
-  AddUnaryStringPredicate<IsUpperAscii>("ascii_is_upper", registry, &ascii_is_upper_doc);
-  AddUnaryStringPredicate<IsTitleAscii>("ascii_is_title", registry, &ascii_is_title_doc);
+                                            ascii_is_printable_doc);
+  AddUnaryStringPredicate<IsSpaceAscii>("ascii_is_space", registry, ascii_is_space_doc);
+  AddUnaryStringPredicate<IsUpperAscii>("ascii_is_upper", registry, ascii_is_upper_doc);
+  AddUnaryStringPredicate<IsTitleAscii>("ascii_is_title", registry, ascii_is_title_doc);
 }
 
 // ----------------------------------------------------------------------
@@ -929,15 +929,15 @@ void AddAsciiStringCaseConversion(FunctionRegistry* registry) {
   // Some kernels are able to reuse the original offsets buffer, so don't
   // preallocate them in the output. Only kernels that invoke
   // "StringDataTransform" support no preallocation.
-  MakeUnaryStringBatchKernel<AsciiUpper>("ascii_upper", registry, &ascii_upper_doc,
+  MakeUnaryStringBatchKernel<AsciiUpper>("ascii_upper", registry, ascii_upper_doc,
                                          MemAllocation::NO_PREALLOCATE);
-  MakeUnaryStringBatchKernel<AsciiLower>("ascii_lower", registry, &ascii_lower_doc,
+  MakeUnaryStringBatchKernel<AsciiLower>("ascii_lower", registry, ascii_lower_doc,
                                          MemAllocation::NO_PREALLOCATE);
   MakeUnaryStringBatchKernel<AsciiSwapCase>(
-      "ascii_swapcase", registry, &ascii_swapcase_doc, MemAllocation::NO_PREALLOCATE);
+      "ascii_swapcase", registry, ascii_swapcase_doc, MemAllocation::NO_PREALLOCATE);
   MakeUnaryStringBatchKernel<AsciiCapitalize>("ascii_capitalize", registry,
-                                              &ascii_capitalize_doc);
-  MakeUnaryStringBatchKernel<AsciiTitle>("ascii_title", registry, &ascii_title_doc);
+                                              ascii_capitalize_doc);
+  MakeUnaryStringBatchKernel<AsciiTitle>("ascii_title", registry, ascii_title_doc);
 }
 
 // ----------------------------------------------------------------------
@@ -971,7 +971,7 @@ const FunctionDoc binary_length_doc(
 
 void AddAsciiStringLength(FunctionRegistry* registry) {
   auto func = std::make_shared<ScalarFunction>("binary_length", Arity::Unary(),
-                                               &binary_length_doc);
+                                               binary_length_doc);
   for (const auto& ty : {binary(), utf8()}) {
     auto exec =
         GenerateVarBinaryBase<applicator::ScalarUnaryNotNull, Int32Type, BinaryLength>(
@@ -1041,14 +1041,14 @@ const FunctionDoc ascii_reverse_doc(
 void AddAsciiStringReverse(FunctionRegistry* registry) {
   {
     auto func = std::make_shared<ScalarFunction>("binary_reverse", Arity::Unary(),
-                                                 &binary_reverse_doc);
+                                                 binary_reverse_doc);
     for (const auto& ty : BinaryTypes()) {
       DCHECK_OK(
           func->AddKernel({ty}, ty, GenerateVarBinaryToVarBinary<BinaryReverse>(ty)));
     }
     DCHECK_OK(registry->AddFunction(std::move(func)));
   }
-  MakeUnaryStringBatchKernel<AsciiReverse>("ascii_reverse", registry, &ascii_reverse_doc);
+  MakeUnaryStringBatchKernel<AsciiReverse>("ascii_reverse", registry, ascii_reverse_doc);
 }
 
 // ----------------------------------------------------------------------
@@ -1189,17 +1189,17 @@ const FunctionDoc ascii_rtrim_whitespace_doc(
     {"strings"});
 
 void AddAsciiStringTrim(FunctionRegistry* registry) {
-  MakeUnaryStringBatchKernelWithState<AsciiTrim>("ascii_trim", registry, &ascii_trim_doc);
+  MakeUnaryStringBatchKernelWithState<AsciiTrim>("ascii_trim", registry, ascii_trim_doc);
   MakeUnaryStringBatchKernelWithState<AsciiLTrim>("ascii_ltrim", registry,
-                                                  &ascii_ltrim_doc);
+                                                  ascii_ltrim_doc);
   MakeUnaryStringBatchKernelWithState<AsciiRTrim>("ascii_rtrim", registry,
-                                                  &ascii_rtrim_doc);
+                                                  ascii_rtrim_doc);
   MakeUnaryStringBatchKernel<AsciiTrimWhitespace>("ascii_trim_whitespace", registry,
-                                                  &ascii_trim_whitespace_doc);
+                                                  ascii_trim_whitespace_doc);
   MakeUnaryStringBatchKernel<AsciiLTrimWhitespace>("ascii_ltrim_whitespace", registry,
-                                                   &ascii_ltrim_whitespace_doc);
+                                                   ascii_ltrim_whitespace_doc);
   MakeUnaryStringBatchKernel<AsciiRTrimWhitespace>("ascii_rtrim_whitespace", registry,
-                                                   &ascii_rtrim_whitespace_doc);
+                                                   ascii_rtrim_whitespace_doc);
 }
 
 // ----------------------------------------------------------------------
@@ -1284,10 +1284,10 @@ const FunctionDoc ascii_center_doc(
     {"strings"}, "PadOptions", /*options_required=*/true);
 
 void AddAsciiStringPad(FunctionRegistry* registry) {
-  MakeUnaryStringBatchKernelWithState<AsciiLPad>("ascii_lpad", registry, &ascii_lpad_doc);
-  MakeUnaryStringBatchKernelWithState<AsciiRPad>("ascii_rpad", registry, &ascii_rpad_doc);
+  MakeUnaryStringBatchKernelWithState<AsciiLPad>("ascii_lpad", registry, ascii_lpad_doc);
+  MakeUnaryStringBatchKernelWithState<AsciiRPad>("ascii_rpad", registry, ascii_rpad_doc);
   MakeUnaryStringBatchKernelWithState<AsciiCenter>("ascii_center", registry,
-                                                   &ascii_center_doc);
+                                                   ascii_center_doc);
 }
 
 // ----------------------------------------------------------------------
@@ -1717,7 +1717,7 @@ const FunctionDoc match_like_doc(
 void AddAsciiStringMatchSubstring(FunctionRegistry* registry) {
   {
     auto func = std::make_shared<ScalarFunction>("match_substring", Arity::Unary(),
-                                                 &match_substring_doc);
+                                                 match_substring_doc);
     for (const auto& ty : BaseBinaryTypes()) {
       auto exec = GenerateVarBinaryToVarBinary<MatchSubstring, PlainSubstringMatcher>(ty);
       DCHECK_OK(
@@ -1727,7 +1727,7 @@ void AddAsciiStringMatchSubstring(FunctionRegistry* registry) {
   }
   {
     auto func =
-        std::make_shared<ScalarFunction>("starts_with", Arity::Unary(), &starts_with_doc);
+        std::make_shared<ScalarFunction>("starts_with", Arity::Unary(), starts_with_doc);
     for (const auto& ty : BaseBinaryTypes()) {
       auto exec =
           GenerateVarBinaryToVarBinary<MatchSubstring, PlainStartsWithMatcher>(ty);
@@ -1738,7 +1738,7 @@ void AddAsciiStringMatchSubstring(FunctionRegistry* registry) {
   }
   {
     auto func =
-        std::make_shared<ScalarFunction>("ends_with", Arity::Unary(), &ends_with_doc);
+        std::make_shared<ScalarFunction>("ends_with", Arity::Unary(), ends_with_doc);
     for (const auto& ty : BaseBinaryTypes()) {
       auto exec = GenerateVarBinaryToVarBinary<MatchSubstring, PlainEndsWithMatcher>(ty);
       DCHECK_OK(
@@ -1749,7 +1749,7 @@ void AddAsciiStringMatchSubstring(FunctionRegistry* registry) {
 #ifdef ARROW_WITH_RE2
   {
     auto func = std::make_shared<ScalarFunction>("match_substring_regex", Arity::Unary(),
-                                                 &match_substring_regex_doc);
+                                                 match_substring_regex_doc);
     for (const auto& ty : BaseBinaryTypes()) {
       auto exec = GenerateVarBinaryToVarBinary<MatchSubstring, RegexSubstringMatcher>(ty);
       DCHECK_OK(
@@ -1759,7 +1759,7 @@ void AddAsciiStringMatchSubstring(FunctionRegistry* registry) {
   }
   {
     auto func =
-        std::make_shared<ScalarFunction>("match_like", Arity::Unary(), &match_like_doc);
+        std::make_shared<ScalarFunction>("match_like", Arity::Unary(), match_like_doc);
     for (const auto& ty : BaseBinaryTypes()) {
       auto exec = GenerateVarBinaryToVarBinary<MatchLike>(ty);
       DCHECK_OK(
@@ -1861,7 +1861,7 @@ const FunctionDoc find_substring_regex_doc(
 void AddAsciiStringFindSubstring(FunctionRegistry* registry) {
   {
     auto func = std::make_shared<ScalarFunction>("find_substring", Arity::Unary(),
-                                                 &find_substring_doc);
+                                                 find_substring_doc);
     for (const auto& ty : BaseBinaryTypes()) {
       auto offset_type = offset_bit_width(ty->id()) == 64 ? int64() : int32();
       DCHECK_OK(func->AddKernel({ty}, offset_type,
@@ -1876,7 +1876,7 @@ void AddAsciiStringFindSubstring(FunctionRegistry* registry) {
 #ifdef ARROW_WITH_RE2
   {
     auto func = std::make_shared<ScalarFunction>("find_substring_regex", Arity::Unary(),
-                                                 &find_substring_regex_doc);
+                                                 find_substring_regex_doc);
     for (const auto& ty : BaseBinaryTypes()) {
       auto offset_type = offset_bit_width(ty->id()) == 64 ? int64() : int32();
       DCHECK_OK(func->AddKernel({ty}, offset_type,
@@ -2010,7 +2010,7 @@ const FunctionDoc count_substring_regex_doc(
 void AddAsciiStringCountSubstring(FunctionRegistry* registry) {
   {
     auto func = std::make_shared<ScalarFunction>("count_substring", Arity::Unary(),
-                                                 &count_substring_doc);
+                                                 count_substring_doc);
     for (const auto& ty : BaseBinaryTypes()) {
       auto offset_type = offset_bit_width(ty->id()) == 64 ? int64() : int32();
       DCHECK_OK(func->AddKernel({ty}, offset_type,
@@ -2025,7 +2025,7 @@ void AddAsciiStringCountSubstring(FunctionRegistry* registry) {
 #ifdef ARROW_WITH_RE2
   {
     auto func = std::make_shared<ScalarFunction>("count_substring_regex", Arity::Unary(),
-                                                 &count_substring_regex_doc);
+                                                 count_substring_regex_doc);
     for (const auto& ty : BaseBinaryTypes()) {
       auto offset_type = offset_bit_width(ty->id()) == 64 ? int64() : int32();
       DCHECK_OK(func->AddKernel({ty}, offset_type,
@@ -2251,7 +2251,7 @@ const FunctionDoc replace_substring_regex_doc(
 void AddAsciiStringReplaceSubstring(FunctionRegistry* registry) {
   {
     auto func = std::make_shared<ScalarFunction>("replace_substring", Arity::Unary(),
-                                                 &replace_substring_doc);
+                                                 replace_substring_doc);
     for (const auto& ty : BaseBinaryTypes()) {
       auto exec = GenerateVarBinaryToVarBinary<ReplaceSubstringPlain>(ty);
       ScalarKernel kernel{{ty}, ty, std::move(exec), ReplaceState::Init};
@@ -2263,7 +2263,7 @@ void AddAsciiStringReplaceSubstring(FunctionRegistry* registry) {
 #ifdef ARROW_WITH_RE2
   {
     auto func = std::make_shared<ScalarFunction>(
-        "replace_substring_regex", Arity::Unary(), &replace_substring_regex_doc);
+        "replace_substring_regex", Arity::Unary(), replace_substring_regex_doc);
     for (const auto& ty : BaseBinaryTypes()) {
       auto exec = GenerateVarBinaryToVarBinary<ReplaceSubstringRegex>(ty);
       ScalarKernel kernel{{ty}, ty, std::move(exec), ReplaceState::Init};
@@ -2446,7 +2446,7 @@ const FunctionDoc extract_regex_doc(
 
 void AddAsciiStringExtractRegex(FunctionRegistry* registry) {
   auto func = std::make_shared<ScalarFunction>("extract_regex", Arity::Unary(),
-                                               &extract_regex_doc);
+                                               extract_regex_doc);
   OutputType out_ty(ResolveExtractRegexOutput);
   for (const auto& ty : BaseBinaryTypes()) {
     ScalarKernel kernel{{ty},
@@ -2534,7 +2534,7 @@ const FunctionDoc binary_replace_slice_doc(
 
 void AddAsciiStringReplaceSlice(FunctionRegistry* registry) {
   auto func = std::make_shared<ScalarFunction>("binary_replace_slice", Arity::Unary(),
-                                               &binary_replace_slice_doc);
+                                               binary_replace_slice_doc);
   for (const auto& ty : BaseBinaryTypes()) {
     DCHECK_OK(func->AddKernel({ty}, ty,
                               GenerateTypeAgnosticVarBinaryBase<BinaryReplaceSlice>(ty),
@@ -2620,7 +2620,7 @@ const FunctionDoc split_pattern_doc(
 
 void AddAsciiStringSplitPattern(FunctionRegistry* registry) {
   auto func = std::make_shared<ScalarFunction>("split_pattern", Arity::Unary(),
-                                               &split_pattern_doc);
+                                               split_pattern_doc);
   for (const auto& ty : BaseBinaryTypes()) {
     auto exec = GenerateVarBinaryToVarBinary<SplitPatternExec, ListType>(ty);
     DCHECK_OK(
@@ -2690,7 +2690,7 @@ void AddAsciiStringSplitWhitespace(FunctionRegistry* registry) {
   static const SplitOptions default_options{};
   auto func =
       std::make_shared<ScalarFunction>("ascii_split_whitespace", Arity::Unary(),
-                                       &ascii_split_whitespace_doc, &default_options);
+                                       ascii_split_whitespace_doc, &default_options);
 
   for (const auto& ty : StringTypes()) {
     auto exec = GenerateVarBinaryToVarBinary<SplitWhitespaceAsciiExec, ListType>(ty);
@@ -2760,7 +2760,7 @@ const FunctionDoc split_pattern_regex_doc(
 
 void AddAsciiStringSplitRegex(FunctionRegistry* registry) {
   auto func = std::make_shared<ScalarFunction>("split_pattern_regex", Arity::Unary(),
-                                               &split_pattern_regex_doc);
+                                               split_pattern_regex_doc);
   for (const auto& ty : BaseBinaryTypes()) {
     auto exec = GenerateVarBinaryToVarBinary<SplitRegexExec, ListType>(ty);
     DCHECK_OK(
@@ -3265,8 +3265,8 @@ void AddBinaryJoinForListType(ScalarFunction* func) {
 
 void AddAsciiStringJoin(FunctionRegistry* registry) {
   {
-    auto func = std::make_shared<ScalarFunction>("binary_join", Arity::Binary(),
-                                                 &binary_join_doc);
+    auto func =
+        std::make_shared<ScalarFunction>("binary_join", Arity::Binary(), binary_join_doc);
     AddBinaryJoinForListType<ListType>(func.get());
     AddBinaryJoinForListType<LargeListType>(func.get());
     DCHECK_OK(registry->AddFunction(std::move(func)));
@@ -3274,7 +3274,7 @@ void AddAsciiStringJoin(FunctionRegistry* registry) {
   {
     auto func = std::make_shared<ScalarFunction>(
         "binary_join_element_wise", Arity::VarArgs(/*min_args=*/1),
-        &binary_join_element_wise_doc, GetDefaultJoinOptions());
+        binary_join_element_wise_doc, GetDefaultJoinOptions());
     for (const auto& ty : BaseBinaryTypes()) {
       ScalarKernel kernel{KernelSignature::Make({InputType(ty)}, ty, /*is_varargs=*/true),
                           GenerateTypeAgnosticVarBinaryBase<BinaryJoinElementWise>(ty),
@@ -3412,7 +3412,7 @@ const FunctionDoc binary_repeat_doc(
 
 void AddAsciiStringRepeat(FunctionRegistry* registry) {
   auto func = std::make_shared<ScalarCTypeToInt64Function>(
-      "binary_repeat", Arity::Binary(), &binary_repeat_doc);
+      "binary_repeat", Arity::Binary(), binary_repeat_doc);
   for (const auto& ty : BaseBinaryTypes()) {
     auto exec = GenerateVarBinaryToVarBinary<BinaryRepeat, Int64Type>(ty);
     ScalarKernel kernel{{ty, int64()}, ty, exec};
diff --git a/cpp/src/arrow/compute/kernels/scalar_string_internal.h b/cpp/src/arrow/compute/kernels/scalar_string_internal.h
index a60c3d1ae2..3d3f030f3b 100644
--- a/cpp/src/arrow/compute/kernels/scalar_string_internal.h
+++ b/cpp/src/arrow/compute/kernels/scalar_string_internal.h
@@ -171,9 +171,9 @@ struct StringTransformExecWithState
 
 template <template <typename> class ExecFunctor>
 void MakeUnaryStringBatchKernel(
-    std::string name, FunctionRegistry* registry, const FunctionDoc* doc,
+    std::string name, FunctionRegistry* registry, FunctionDoc doc,
     MemAllocation::type mem_allocation = MemAllocation::PREALLOCATE) {
-  auto func = std::make_shared<ScalarFunction>(name, Arity::Unary(), doc);
+  auto func = std::make_shared<ScalarFunction>(name, Arity::Unary(), std::move(doc));
   for (const auto& ty : StringTypes()) {
     auto exec = GenerateVarBinaryToVarBinary<ExecFunctor>(ty);
     ScalarKernel kernel{{ty}, ty, std::move(exec)};
@@ -185,9 +185,9 @@ void MakeUnaryStringBatchKernel(
 
 template <template <typename> class ExecFunctor>
 void MakeUnaryStringBatchKernelWithState(
-    std::string name, FunctionRegistry* registry, const FunctionDoc* doc,
+    std::string name, FunctionRegistry* registry, FunctionDoc doc,
     MemAllocation::type mem_allocation = MemAllocation::PREALLOCATE) {
-  auto func = std::make_shared<ScalarFunction>(name, Arity::Unary(), doc);
+  auto func = std::make_shared<ScalarFunction>(name, Arity::Unary(), std::move(doc));
   {
     using t32 = ExecFunctor<StringType>;
     ScalarKernel kernel{{utf8()}, utf8(), t32::Exec, t32::State::Init};
@@ -271,8 +271,8 @@ struct StringPredicateFunctor {
 
 template <typename Predicate>
 void AddUnaryStringPredicate(std::string name, FunctionRegistry* registry,
-                             const FunctionDoc* doc) {
-  auto func = std::make_shared<ScalarFunction>(name, Arity::Unary(), doc);
+                             FunctionDoc doc) {
+  auto func = std::make_shared<ScalarFunction>(name, Arity::Unary(), std::move(doc));
   for (const auto& ty : StringTypes()) {
     auto exec = GenerateVarBinaryToVarBinary<StringPredicateFunctor, Predicate>(ty);
     DCHECK_OK(func->AddKernel({ty}, boolean(), std::move(exec)));
diff --git a/cpp/src/arrow/compute/kernels/scalar_string_utf8.cc b/cpp/src/arrow/compute/kernels/scalar_string_utf8.cc
index 5f1178bbb1..9e06012728 100644
--- a/cpp/src/arrow/compute/kernels/scalar_string_utf8.cc
+++ b/cpp/src/arrow/compute/kernels/scalar_string_utf8.cc
@@ -39,8 +39,8 @@ namespace {
 
 template <template <typename> class Transformer>
 void MakeUnaryStringUTF8TransformKernel(std::string name, FunctionRegistry* registry,
-                                        const FunctionDoc* doc) {
-  auto func = std::make_shared<ScalarFunction>(name, Arity::Unary(), doc);
+                                        FunctionDoc doc) {
+  auto func = std::make_shared<ScalarFunction>(name, Arity::Unary(), std::move(doc));
   for (const auto& ty : StringTypes()) {
     auto exec = GenerateVarBinaryToVarBinary<Transformer>(ty);
     DCHECK_OK(func->AddKernel({ty}, ty, std::move(exec)));
@@ -319,19 +319,19 @@ const auto utf8_is_title_doc = StringPredicateDoc(
 
 void AddUtf8StringPredicates(FunctionRegistry* registry) {
   AddUnaryStringPredicate<IsAlphaNumericUnicode>("utf8_is_alnum", registry,
-                                                 &utf8_is_alnum_doc);
-  AddUnaryStringPredicate<IsAlphaUnicode>("utf8_is_alpha", registry, &utf8_is_alpha_doc);
+                                                 utf8_is_alnum_doc);
+  AddUnaryStringPredicate<IsAlphaUnicode>("utf8_is_alpha", registry, utf8_is_alpha_doc);
   AddUnaryStringPredicate<IsDecimalUnicode>("utf8_is_decimal", registry,
-                                            &utf8_is_decimal_doc);
-  AddUnaryStringPredicate<IsDigitUnicode>("utf8_is_digit", registry, &utf8_is_digit_doc);
+                                            utf8_is_decimal_doc);
+  AddUnaryStringPredicate<IsDigitUnicode>("utf8_is_digit", registry, utf8_is_digit_doc);
   AddUnaryStringPredicate<IsNumericUnicode>("utf8_is_numeric", registry,
-                                            &utf8_is_numeric_doc);
-  AddUnaryStringPredicate<IsLowerUnicode>("utf8_is_lower", registry, &utf8_is_lower_doc);
+                                            utf8_is_numeric_doc);
+  AddUnaryStringPredicate<IsLowerUnicode>("utf8_is_lower", registry, utf8_is_lower_doc);
   AddUnaryStringPredicate<IsPrintableUnicode>("utf8_is_printable", registry,
-                                              &utf8_is_printable_doc);
-  AddUnaryStringPredicate<IsSpaceUnicode>("utf8_is_space", registry, &utf8_is_space_doc);
-  AddUnaryStringPredicate<IsTitleUnicode>("utf8_is_title", registry, &utf8_is_title_doc);
-  AddUnaryStringPredicate<IsUpperUnicode>("utf8_is_upper", registry, &utf8_is_upper_doc);
+                                              utf8_is_printable_doc);
+  AddUnaryStringPredicate<IsSpaceUnicode>("utf8_is_space", registry, utf8_is_space_doc);
+  AddUnaryStringPredicate<IsTitleUnicode>("utf8_is_title", registry, utf8_is_title_doc);
+  AddUnaryStringPredicate<IsUpperUnicode>("utf8_is_upper", registry, utf8_is_upper_doc);
 }
 
 #endif  // ARROW_WITH_UTF8PROC
@@ -501,13 +501,13 @@ const FunctionDoc utf8_title_doc(
     {"strings"});
 
 void AddUtf8StringCaseConversion(FunctionRegistry* registry) {
-  MakeUnaryStringUTF8TransformKernel<UTF8Upper>("utf8_upper", registry, &utf8_upper_doc);
-  MakeUnaryStringUTF8TransformKernel<UTF8Lower>("utf8_lower", registry, &utf8_lower_doc);
+  MakeUnaryStringUTF8TransformKernel<UTF8Upper>("utf8_upper", registry, utf8_upper_doc);
+  MakeUnaryStringUTF8TransformKernel<UTF8Lower>("utf8_lower", registry, utf8_lower_doc);
   MakeUnaryStringUTF8TransformKernel<UTF8SwapCase>("utf8_swapcase", registry,
-                                                   &utf8_swapcase_doc);
+                                                   utf8_swapcase_doc);
   MakeUnaryStringBatchKernel<Utf8Capitalize>("utf8_capitalize", registry,
-                                             &utf8_capitalize_doc);
-  MakeUnaryStringBatchKernel<Utf8Title>("utf8_title", registry, &utf8_title_doc);
+                                             utf8_capitalize_doc);
+  MakeUnaryStringBatchKernel<Utf8Title>("utf8_title", registry, utf8_title_doc);
 }
 
 #endif  // ARROW_WITH_UTF8PROC
@@ -667,7 +667,7 @@ const FunctionDoc utf8_normalize_doc(
 
 void AddUtf8StringNormalize(FunctionRegistry* registry) {
   MakeUnaryStringBatchKernelWithState<Utf8NormalizeExec>("utf8_normalize", registry,
-                                                         &utf8_normalize_doc);
+                                                         utf8_normalize_doc);
 }
 
 #endif  // ARROW_WITH_UTF8PROC
@@ -692,7 +692,7 @@ const FunctionDoc utf8_length_doc(
 
 void AddUtf8StringLength(FunctionRegistry* registry) {
   auto func =
-      std::make_shared<ScalarFunction>("utf8_length", Arity::Unary(), &utf8_length_doc);
+      std::make_shared<ScalarFunction>("utf8_length", Arity::Unary(), utf8_length_doc);
   {
     auto exec = applicator::ScalarUnaryNotNull<Int32Type, StringType, Utf8Length>::Exec;
     DCHECK_OK(func->AddKernel({utf8()}, int32(), std::move(exec)));
@@ -734,7 +734,7 @@ const FunctionDoc utf8_reverse_doc(
     {"strings"});
 
 void AddUtf8StringReverse(FunctionRegistry* registry) {
-  MakeUnaryStringBatchKernel<Utf8Reverse>("utf8_reverse", registry, &utf8_reverse_doc);
+  MakeUnaryStringBatchKernel<Utf8Reverse>("utf8_reverse", registry, utf8_reverse_doc);
 }
 
 // ----------------------------------------------------------------------
@@ -896,16 +896,16 @@ const FunctionDoc utf8_rtrim_whitespace_doc(
 #endif  // ARROW_WITH_UTF8PROC
 
 void AddUtf8StringTrim(FunctionRegistry* registry) {
-  MakeUnaryStringBatchKernelWithState<UTF8Trim>("utf8_trim", registry, &utf8_trim_doc);
-  MakeUnaryStringBatchKernelWithState<UTF8LTrim>("utf8_ltrim", registry, &utf8_ltrim_doc);
-  MakeUnaryStringBatchKernelWithState<UTF8RTrim>("utf8_rtrim", registry, &utf8_rtrim_doc);
+  MakeUnaryStringBatchKernelWithState<UTF8Trim>("utf8_trim", registry, utf8_trim_doc);
+  MakeUnaryStringBatchKernelWithState<UTF8LTrim>("utf8_ltrim", registry, utf8_ltrim_doc);
+  MakeUnaryStringBatchKernelWithState<UTF8RTrim>("utf8_rtrim", registry, utf8_rtrim_doc);
 #ifdef ARROW_WITH_UTF8PROC
   MakeUnaryStringBatchKernel<UTF8TrimWhitespace>("utf8_trim_whitespace", registry,
-                                                 &utf8_trim_whitespace_doc);
+                                                 utf8_trim_whitespace_doc);
   MakeUnaryStringBatchKernel<UTF8LTrimWhitespace>("utf8_ltrim_whitespace", registry,
-                                                  &utf8_ltrim_whitespace_doc);
+                                                  utf8_ltrim_whitespace_doc);
   MakeUnaryStringBatchKernel<UTF8RTrimWhitespace>("utf8_rtrim_whitespace", registry,
-                                                  &utf8_rtrim_whitespace_doc);
+                                                  utf8_rtrim_whitespace_doc);
 #endif  // ARROW_WITH_UTF8PROC
 }
 
@@ -1002,10 +1002,10 @@ const FunctionDoc utf8_rpad_doc(
     {"strings"}, "PadOptions", /*options_required=*/true);
 
 void AddUtf8StringPad(FunctionRegistry* registry) {
-  MakeUnaryStringBatchKernelWithState<Utf8LPad>("utf8_lpad", registry, &utf8_lpad_doc);
-  MakeUnaryStringBatchKernelWithState<Utf8RPad>("utf8_rpad", registry, &utf8_rpad_doc);
+  MakeUnaryStringBatchKernelWithState<Utf8LPad>("utf8_lpad", registry, utf8_lpad_doc);
+  MakeUnaryStringBatchKernelWithState<Utf8RPad>("utf8_rpad", registry, utf8_rpad_doc);
   MakeUnaryStringBatchKernelWithState<Utf8Center>("utf8_center", registry,
-                                                  &utf8_center_doc);
+                                                  utf8_center_doc);
 }
 
 // ----------------------------------------------------------------------
@@ -1089,7 +1089,7 @@ const FunctionDoc utf8_replace_slice_doc(
 
 void AddUtf8StringReplaceSlice(FunctionRegistry* registry) {
   auto func = std::make_shared<ScalarFunction>("utf8_replace_slice", Arity::Unary(),
-                                               &utf8_replace_slice_doc);
+                                               utf8_replace_slice_doc);
 
   for (const auto& ty : StringTypes()) {
     auto exec = GenerateVarBinaryToVarBinary<Utf8ReplaceSlice>(ty);
@@ -1278,7 +1278,7 @@ const FunctionDoc utf8_slice_codeunits_doc(
 
 void AddUtf8StringSlice(FunctionRegistry* registry) {
   auto func = std::make_shared<ScalarFunction>("utf8_slice_codeunits", Arity::Unary(),
-                                               &utf8_slice_codeunits_doc);
+                                               utf8_slice_codeunits_doc);
   for (const auto& ty : StringTypes()) {
     auto exec = GenerateVarBinaryToVarBinary<SliceCodeunits>(ty);
     DCHECK_OK(
@@ -1364,7 +1364,7 @@ void AddUtf8StringSplitWhitespace(FunctionRegistry* registry) {
   static const SplitOptions default_options;
   auto func =
       std::make_shared<ScalarFunction>("utf8_split_whitespace", Arity::Unary(),
-                                       &utf8_split_whitespace_doc, &default_options);
+                                       utf8_split_whitespace_doc, &default_options);
   for (const auto& ty : StringTypes()) {
     auto exec = GenerateVarBinaryToVarBinary<SplitWhitespaceUtf8Exec, ListType>(ty);
     DCHECK_OK(func->AddKernel({ty}, {list(ty)}, std::move(exec), StringSplitState::Init));
diff --git a/cpp/src/arrow/compute/kernels/scalar_temporal_binary.cc b/cpp/src/arrow/compute/kernels/scalar_temporal_binary.cc
index e73c89857b..90d21273e7 100644
--- a/cpp/src/arrow/compute/kernels/scalar_temporal_binary.cc
+++ b/cpp/src/arrow/compute/kernels/scalar_temporal_binary.cc
@@ -326,12 +326,13 @@ struct BinaryTemporalFactory {
 
   template <typename... WithTypes>
   static std::shared_ptr<ScalarFunction> Make(
-      std::string name, OutputType out_type, const FunctionDoc* doc,
+      std::string name, OutputType out_type, FunctionDoc doc,
       const FunctionOptions* default_options = NULLPTR, KernelInit init = NULLPTR) {
     DCHECK_NE(sizeof...(WithTypes), 0);
     BinaryTemporalFactory self{
         out_type, init,
-        std::make_shared<ScalarFunction>(name, Arity::Binary(), doc, default_options)};
+        std::make_shared<ScalarFunction>(name, Arity::Binary(), std::move(doc),
+                                         default_options)};
     AddTemporalKernels(&self, WithTypes{}...);
     return self.func;
   }
@@ -458,18 +459,18 @@ void RegisterScalarTemporalBinary(FunctionRegistry* registry) {
   // Temporal difference functions
   auto years_between =
       BinaryTemporalFactory<YearsBetween, TemporalBinary, Int64Type>::Make<
-          WithDates, WithTimestamps>("years_between", int64(), &years_between_doc);
+          WithDates, WithTimestamps>("years_between", int64(), years_between_doc);
   DCHECK_OK(registry->AddFunction(std::move(years_between)));
 
   auto quarters_between =
       BinaryTemporalFactory<QuartersBetween, TemporalBinary, Int64Type>::Make<
-          WithDates, WithTimestamps>("quarters_between", int64(), &quarters_between_doc);
+          WithDates, WithTimestamps>("quarters_between", int64(), quarters_between_doc);
   DCHECK_OK(registry->AddFunction(std::move(quarters_between)));
 
   auto month_interval_between =
       BinaryTemporalFactory<MonthsBetween, TemporalBinary, MonthIntervalType>::Make<
           WithDates, WithTimestamps>("month_interval_between", month_interval(),
-                                     &months_between_doc);
+                                     months_between_doc);
   DCHECK_OK(registry->AddFunction(std::move(month_interval_between)));
 
   auto month_day_nano_interval_between =
@@ -477,13 +478,13 @@ void RegisterScalarTemporalBinary(FunctionRegistry* registry) {
                             MonthDayNanoIntervalType>::Make<WithDates, WithTimes,
                                                             WithTimestamps>(
           "month_day_nano_interval_between", month_day_nano_interval(),
-          &month_day_nano_interval_between_doc);
+          month_day_nano_interval_between_doc);
   DCHECK_OK(registry->AddFunction(std::move(month_day_nano_interval_between)));
 
   static const auto default_day_of_week_options = DayOfWeekOptions::Defaults();
   auto weeks_between =
       BinaryTemporalFactory<WeeksBetween, TemporalDayOfWeekBinary, Int64Type>::Make<
-          WithDates, WithTimestamps>("weeks_between", int64(), &weeks_between_doc,
+          WithDates, WithTimestamps>("weeks_between", int64(), weeks_between_doc,
                                      &default_day_of_week_options, DayOfWeekState::Init);
   DCHECK_OK(registry->AddFunction(std::move(weeks_between)));
 
@@ -491,49 +492,49 @@ void RegisterScalarTemporalBinary(FunctionRegistry* registry) {
       BinaryTemporalFactory<DayTimeBetween, TemporalBinary, DayTimeIntervalType>::Make<
           WithDates, WithTimes, WithTimestamps>("day_time_interval_between",
                                                 day_time_interval(),
-                                                &day_time_interval_between_doc);
+                                                day_time_interval_between_doc);
   DCHECK_OK(registry->AddFunction(std::move(day_time_interval_between)));
 
   auto days_between =
       BinaryTemporalFactory<DaysBetween, TemporalBinary, Int64Type>::Make<WithDates,
                                                                           WithTimestamps>(
-          "days_between", int64(), &days_between_doc);
+          "days_between", int64(), days_between_doc);
   DCHECK_OK(registry->AddFunction(std::move(days_between)));
 
   auto hours_between =
       BinaryTemporalFactory<HoursBetween, TemporalBinary, Int64Type>::Make<
           WithDates, WithTimes, WithTimestamps>("hours_between", int64(),
-                                                &hours_between_doc);
+                                                hours_between_doc);
   DCHECK_OK(registry->AddFunction(std::move(hours_between)));
 
   auto minutes_between =
       BinaryTemporalFactory<MinutesBetween, TemporalBinary, Int64Type>::Make<
           WithDates, WithTimes, WithTimestamps>("minutes_between", int64(),
-                                                &minutes_between_doc);
+                                                minutes_between_doc);
   DCHECK_OK(registry->AddFunction(std::move(minutes_between)));
 
   auto seconds_between =
       BinaryTemporalFactory<SecondsBetween, TemporalBinary, Int64Type>::Make<
           WithDates, WithTimes, WithTimestamps>("seconds_between", int64(),
-                                                &seconds_between_doc);
+                                                seconds_between_doc);
   DCHECK_OK(registry->AddFunction(std::move(seconds_between)));
 
   auto milliseconds_between =
       BinaryTemporalFactory<MillisecondsBetween, TemporalBinary, Int64Type>::Make<
           WithDates, WithTimes, WithTimestamps>("milliseconds_between", int64(),
-                                                &milliseconds_between_doc);
+                                                milliseconds_between_doc);
   DCHECK_OK(registry->AddFunction(std::move(milliseconds_between)));
 
   auto microseconds_between =
       BinaryTemporalFactory<MicrosecondsBetween, TemporalBinary, Int64Type>::Make<
           WithDates, WithTimes, WithTimestamps>("microseconds_between", int64(),
-                                                &microseconds_between_doc);
+                                                microseconds_between_doc);
   DCHECK_OK(registry->AddFunction(std::move(microseconds_between)));
 
   auto nanoseconds_between =
       BinaryTemporalFactory<NanosecondsBetween, TemporalBinary, Int64Type>::Make<
           WithDates, WithTimes, WithTimestamps>("nanoseconds_between", int64(),
-                                                &nanoseconds_between_doc);
+                                                nanoseconds_between_doc);
   DCHECK_OK(registry->AddFunction(std::move(nanoseconds_between)));
 }
 
diff --git a/cpp/src/arrow/compute/kernels/scalar_temporal_unary.cc b/cpp/src/arrow/compute/kernels/scalar_temporal_unary.cc
index 8df24f733f..7484de2a00 100644
--- a/cpp/src/arrow/compute/kernels/scalar_temporal_unary.cc
+++ b/cpp/src/arrow/compute/kernels/scalar_temporal_unary.cc
@@ -1474,12 +1474,12 @@ struct UnaryTemporalFactory {
 
   template <typename... WithTypes>
   static std::shared_ptr<ScalarFunction> Make(
-      std::string name, OutputType out_type, const FunctionDoc* doc,
+      std::string name, OutputType out_type, FunctionDoc doc,
       const FunctionOptions* default_options = NULLPTR, KernelInit init = NULLPTR) {
     DCHECK_NE(sizeof...(WithTypes), 0);
-    UnaryTemporalFactory self{
-        out_type, init,
-        std::make_shared<ScalarFunction>(name, Arity::Unary(), doc, default_options)};
+    UnaryTemporalFactory self{out_type, init,
+                              std::make_shared<ScalarFunction>(
+                                  name, Arity::Unary(), std::move(doc), default_options)};
     AddTemporalKernels(&self, WithTypes{}...);
     return self.func;
   }
@@ -1499,12 +1499,13 @@ struct SimpleUnaryTemporalFactory {
 
   template <typename... WithTypes>
   static std::shared_ptr<ScalarFunction> Make(
-      std::string name, OutputType out_type, const FunctionDoc* doc,
+      std::string name, OutputType out_type, FunctionDoc doc,
       const FunctionOptions* default_options = NULLPTR, KernelInit init = NULLPTR) {
     DCHECK_NE(sizeof...(WithTypes), 0);
     SimpleUnaryTemporalFactory self{
         out_type, init,
-        std::make_shared<ScalarFunction>(name, Arity::Unary(), doc, default_options)};
+        std::make_shared<ScalarFunction>(name, Arity::Unary(), std::move(doc),
+                                         default_options)};
     AddTemporalKernels(&self, WithTypes{}...);
     return self.func;
   }
@@ -1763,35 +1764,35 @@ void RegisterScalarTemporalUnary(FunctionRegistry* registry) {
   auto year =
       UnaryTemporalFactory<Year, TemporalComponentExtract,
                            Int64Type>::Make<WithDates, WithTimestamps>("year", int64(),
-                                                                       &year_doc);
+                                                                       year_doc);
   DCHECK_OK(registry->AddFunction(std::move(year)));
 
   auto is_leap_year =
       UnaryTemporalFactory<IsLeapYear, TemporalComponentExtract, BooleanType>::Make<
-          WithDates, WithTimestamps>("is_leap_year", boolean(), &is_leap_year_doc);
+          WithDates, WithTimestamps>("is_leap_year", boolean(), is_leap_year_doc);
   DCHECK_OK(registry->AddFunction(std::move(is_leap_year)));
 
   auto month =
       UnaryTemporalFactory<Month, TemporalComponentExtract,
                            Int64Type>::Make<WithDates, WithTimestamps>("month", int64(),
-                                                                       &month_doc);
+                                                                       month_doc);
   DCHECK_OK(registry->AddFunction(std::move(month)));
 
   auto day =
       UnaryTemporalFactory<Day, TemporalComponentExtract,
                            Int64Type>::Make<WithDates, WithTimestamps>("day", int64(),
-                                                                       &day_doc);
+                                                                       day_doc);
   DCHECK_OK(registry->AddFunction(std::move(day)));
 
   auto year_month_day =
       SimpleUnaryTemporalFactory<YearMonthDay>::Make<WithDates, WithTimestamps>(
-          "year_month_day", YearMonthDayType(), &year_month_day_doc);
+          "year_month_day", YearMonthDayType(), year_month_day_doc);
   DCHECK_OK(registry->AddFunction(std::move(year_month_day)));
 
   static const auto default_day_of_week_options = DayOfWeekOptions::Defaults();
   auto day_of_week =
       UnaryTemporalFactory<DayOfWeek, TemporalComponentExtractDayOfWeek, Int64Type>::Make<
-          WithDates, WithTimestamps>("day_of_week", int64(), &day_of_week_doc,
+          WithDates, WithTimestamps>("day_of_week", int64(), day_of_week_doc,
                                      &default_day_of_week_options, DayOfWeekState::Init);
   DCHECK_OK(registry->AddFunction(std::move(day_of_week)));
 
@@ -1799,110 +1800,110 @@ void RegisterScalarTemporalUnary(FunctionRegistry* registry) {
       UnaryTemporalFactory<DayOfYear, TemporalComponentExtract,
                            Int64Type>::Make<WithDates, WithTimestamps>("day_of_year",
                                                                        int64(),
-                                                                       &day_of_year_doc);
+                                                                       day_of_year_doc);
   DCHECK_OK(registry->AddFunction(std::move(day_of_year)));
 
   auto iso_year =
       UnaryTemporalFactory<ISOYear, TemporalComponentExtract,
                            Int64Type>::Make<WithDates, WithTimestamps>("iso_year",
                                                                        int64(),
-                                                                       &iso_year_doc);
+                                                                       iso_year_doc);
   DCHECK_OK(registry->AddFunction(std::move(iso_year)));
 
   auto us_year =
       UnaryTemporalFactory<USYear, TemporalComponentExtract,
                            Int64Type>::Make<WithDates, WithTimestamps>("us_year", int64(),
-                                                                       &us_year_doc);
+                                                                       us_year_doc);
   DCHECK_OK(registry->AddFunction(std::move(us_year)));
 
   static const auto default_iso_week_options = WeekOptions::ISODefaults();
   auto iso_week =
       UnaryTemporalFactory<Week, TemporalComponentExtractWeek, Int64Type>::Make<
-          WithDates, WithTimestamps>("iso_week", int64(), &iso_week_doc,
+          WithDates, WithTimestamps>("iso_week", int64(), iso_week_doc,
                                      &default_iso_week_options, WeekState::Init);
   DCHECK_OK(registry->AddFunction(std::move(iso_week)));
 
   static const auto default_us_week_options = WeekOptions::USDefaults();
   auto us_week =
       UnaryTemporalFactory<Week, TemporalComponentExtractWeek, Int64Type>::Make<
-          WithDates, WithTimestamps>("us_week", int64(), &us_week_doc,
+          WithDates, WithTimestamps>("us_week", int64(), us_week_doc,
                                      &default_us_week_options, WeekState::Init);
   DCHECK_OK(registry->AddFunction(std::move(us_week)));
 
   static const auto default_week_options = WeekOptions();
   auto week = UnaryTemporalFactory<Week, TemporalComponentExtractWeek, Int64Type>::Make<
-      WithDates, WithTimestamps>("week", int64(), &week_doc, &default_week_options,
+      WithDates, WithTimestamps>("week", int64(), week_doc, &default_week_options,
                                  WeekState::Init);
   DCHECK_OK(registry->AddFunction(std::move(week)));
 
   auto iso_calendar =
       SimpleUnaryTemporalFactory<ISOCalendar>::Make<WithDates, WithTimestamps>(
-          "iso_calendar", IsoCalendarType(), &iso_calendar_doc);
+          "iso_calendar", IsoCalendarType(), iso_calendar_doc);
   DCHECK_OK(registry->AddFunction(std::move(iso_calendar)));
 
   auto quarter =
       UnaryTemporalFactory<Quarter, TemporalComponentExtract,
                            Int64Type>::Make<WithDates, WithTimestamps>("quarter", int64(),
-                                                                       &quarter_doc);
+                                                                       quarter_doc);
   DCHECK_OK(registry->AddFunction(std::move(quarter)));
 
   // Date / time extractors
   auto hour =
       UnaryTemporalFactory<Hour, TemporalComponentExtract,
                            Int64Type>::Make<WithTimes, WithTimestamps>("hour", int64(),
-                                                                       &hour_doc);
+                                                                       hour_doc);
   DCHECK_OK(registry->AddFunction(std::move(hour)));
 
   auto minute =
       UnaryTemporalFactory<Minute, TemporalComponentExtract,
                            Int64Type>::Make<WithTimes, WithTimestamps>("minute", int64(),
-                                                                       &minute_doc);
+                                                                       minute_doc);
   DCHECK_OK(registry->AddFunction(std::move(minute)));
 
   auto second =
       UnaryTemporalFactory<Second, TemporalComponentExtract,
                            Int64Type>::Make<WithTimes, WithTimestamps>("second", int64(),
-                                                                       &second_doc);
+                                                                       second_doc);
   DCHECK_OK(registry->AddFunction(std::move(second)));
 
   auto millisecond =
       UnaryTemporalFactory<Millisecond, TemporalComponentExtract,
                            Int64Type>::Make<WithTimes, WithTimestamps>("millisecond",
                                                                        int64(),
-                                                                       &millisecond_doc);
+                                                                       millisecond_doc);
   DCHECK_OK(registry->AddFunction(std::move(millisecond)));
 
   auto microsecond =
       UnaryTemporalFactory<Microsecond, TemporalComponentExtract,
                            Int64Type>::Make<WithTimes, WithTimestamps>("microsecond",
                                                                        int64(),
-                                                                       &microsecond_doc);
+                                                                       microsecond_doc);
   DCHECK_OK(registry->AddFunction(std::move(microsecond)));
 
   auto nanosecond =
       UnaryTemporalFactory<Nanosecond, TemporalComponentExtract,
                            Int64Type>::Make<WithTimes, WithTimestamps>("nanosecond",
                                                                        int64(),
-                                                                       &nanosecond_doc);
+                                                                       nanosecond_doc);
   DCHECK_OK(registry->AddFunction(std::move(nanosecond)));
 
   auto subsecond =
       UnaryTemporalFactory<Subsecond, TemporalComponentExtract,
                            DoubleType>::Make<WithTimes, WithTimestamps>("subsecond",
                                                                         float64(),
-                                                                        &subsecond_doc);
+                                                                        subsecond_doc);
   DCHECK_OK(registry->AddFunction(std::move(subsecond)));
 
   // Timezone-related functions
   static const auto default_strftime_options = StrftimeOptions();
   auto strftime =
       SimpleUnaryTemporalFactory<Strftime>::Make<WithTimes, WithDates, WithTimestamps>(
-          "strftime", utf8(), &strftime_doc, &default_strftime_options,
+          "strftime", utf8(), strftime_doc, &default_strftime_options,
           StrftimeState::Init);
   DCHECK_OK(registry->AddFunction(std::move(strftime)));
 
   auto strptime = SimpleUnaryTemporalFactory<Strptime>::Make<WithStringTypes>(
-      "strptime", OutputType::Resolver(ResolveStrptimeOutput), &strptime_doc, nullptr,
+      "strptime", OutputType::Resolver(ResolveStrptimeOutput), strptime_doc, nullptr,
       StrptimeState::Init);
   DCHECK_OK(registry->AddFunction(std::move(strptime)));
 
@@ -1910,13 +1911,13 @@ void RegisterScalarTemporalUnary(FunctionRegistry* registry) {
       UnaryTemporalFactory<AssumeTimezone, AssumeTimezoneExtractor, TimestampType>::Make<
           WithTimestamps>("assume_timezone",
                           OutputType::Resolver(ResolveAssumeTimezoneOutput),
-                          &assume_timezone_doc, nullptr, AssumeTimezoneState::Init);
+                          assume_timezone_doc, nullptr, AssumeTimezoneState::Init);
   DCHECK_OK(registry->AddFunction(std::move(assume_timezone)));
 
   auto is_dst =
       UnaryTemporalFactory<IsDaylightSavings, DaylightSavingsExtractor,
                            BooleanType>::Make<WithTimestamps>("is_dst", boolean(),
-                                                              &is_dst_doc);
+                                                              is_dst_doc);
   DCHECK_OK(registry->AddFunction(std::move(is_dst)));
 
   // Temporal rounding functions
@@ -1924,19 +1925,19 @@ void RegisterScalarTemporalUnary(FunctionRegistry* registry) {
   auto floor_temporal = UnaryTemporalFactory<FloorTemporal, TemporalComponentExtractRound,
                                              TimestampType>::Make<WithDates, WithTimes,
                                                                   WithTimestamps>(
-      "floor_temporal", OutputType(FirstType), &floor_temporal_doc,
+      "floor_temporal", OutputType(FirstType), floor_temporal_doc,
       &default_round_temporal_options, RoundTemporalState::Init);
   DCHECK_OK(registry->AddFunction(std::move(floor_temporal)));
   auto ceil_temporal = UnaryTemporalFactory<CeilTemporal, TemporalComponentExtractRound,
                                             TimestampType>::Make<WithDates, WithTimes,
                                                                  WithTimestamps>(
-      "ceil_temporal", OutputType(FirstType), &ceil_temporal_doc,
+      "ceil_temporal", OutputType(FirstType), ceil_temporal_doc,
       &default_round_temporal_options, RoundTemporalState::Init);
   DCHECK_OK(registry->AddFunction(std::move(ceil_temporal)));
   auto round_temporal = UnaryTemporalFactory<RoundTemporal, TemporalComponentExtractRound,
                                              TimestampType>::Make<WithDates, WithTimes,
                                                                   WithTimestamps>(
-      "round_temporal", OutputType(FirstType), &round_temporal_doc,
+      "round_temporal", OutputType(FirstType), round_temporal_doc,
       &default_round_temporal_options, RoundTemporalState::Init);
   DCHECK_OK(registry->AddFunction(std::move(round_temporal)));
 }
diff --git a/cpp/src/arrow/compute/kernels/scalar_validity.cc b/cpp/src/arrow/compute/kernels/scalar_validity.cc
index ff16e9d935..1b6eb3d648 100644
--- a/cpp/src/arrow/compute/kernels/scalar_validity.cc
+++ b/cpp/src/arrow/compute/kernels/scalar_validity.cc
@@ -185,15 +185,15 @@ struct IsNanOperator {
   }
 };
 
-void MakeFunction(std::string name, const FunctionDoc* doc,
-                  std::vector<InputType> in_types, OutputType out_type,
-                  ArrayKernelExec exec, FunctionRegistry* registry,
+void MakeFunction(std::string name, FunctionDoc doc, std::vector<InputType> in_types,
+                  OutputType out_type, ArrayKernelExec exec, FunctionRegistry* registry,
                   MemAllocation::type mem_allocation, NullHandling::type null_handling,
                   bool can_write_into_slices,
                   const FunctionOptions* default_options = NULLPTR,
                   KernelInit init = NULLPTR) {
   Arity arity{static_cast<int>(in_types.size())};
-  auto func = std::make_shared<ScalarFunction>(name, arity, doc, default_options);
+  auto func =
+      std::make_shared<ScalarFunction>(name, arity, std::move(doc), default_options);
 
   ScalarKernel kernel(std::move(in_types), out_type, exec, init);
   kernel.null_handling = null_handling;
@@ -222,9 +222,8 @@ Status ConstBoolExec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
   return Status::OK();
 }
 
-std::shared_ptr<ScalarFunction> MakeIsFiniteFunction(std::string name,
-                                                     const FunctionDoc* doc) {
-  auto func = std::make_shared<ScalarFunction>(name, Arity::Unary(), doc);
+std::shared_ptr<ScalarFunction> MakeIsFiniteFunction(std::string name, FunctionDoc doc) {
+  auto func = std::make_shared<ScalarFunction>(name, Arity::Unary(), std::move(doc));
 
   AddFloatValidityKernel<FloatType, IsFiniteOperator>(float32(), func.get());
   AddFloatValidityKernel<DoubleType, IsFiniteOperator>(float64(), func.get());
@@ -241,9 +240,8 @@ std::shared_ptr<ScalarFunction> MakeIsFiniteFunction(std::string name,
   return func;
 }
 
-std::shared_ptr<ScalarFunction> MakeIsInfFunction(std::string name,
-                                                  const FunctionDoc* doc) {
-  auto func = std::make_shared<ScalarFunction>(name, Arity::Unary(), doc);
+std::shared_ptr<ScalarFunction> MakeIsInfFunction(std::string name, FunctionDoc doc) {
+  auto func = std::make_shared<ScalarFunction>(name, Arity::Unary(), std::move(doc));
 
   AddFloatValidityKernel<FloatType, IsInfOperator>(float32(), func.get());
   AddFloatValidityKernel<DoubleType, IsInfOperator>(float64(), func.get());
@@ -260,9 +258,8 @@ std::shared_ptr<ScalarFunction> MakeIsInfFunction(std::string name,
   return func;
 }
 
-std::shared_ptr<ScalarFunction> MakeIsNanFunction(std::string name,
-                                                  const FunctionDoc* doc) {
-  auto func = std::make_shared<ScalarFunction>(name, Arity::Unary(), doc);
+std::shared_ptr<ScalarFunction> MakeIsNanFunction(std::string name, FunctionDoc doc) {
+  auto func = std::make_shared<ScalarFunction>(name, Arity::Unary(), std::move(doc));
 
   AddFloatValidityKernel<FloatType, IsNanOperator>(float32(), func.get());
   AddFloatValidityKernel<DoubleType, IsNanOperator>(float64(), func.get());
@@ -339,22 +336,22 @@ const FunctionDoc is_nan_doc("Return true if NaN",
 
 void RegisterScalarValidity(FunctionRegistry* registry) {
   static auto kNullOptions = NullOptions::Defaults();
-  MakeFunction("is_valid", &is_valid_doc, {ValueDescr::ANY}, boolean(), IsValidExec,
+  MakeFunction("is_valid", is_valid_doc, {ValueDescr::ANY}, boolean(), IsValidExec,
                registry, MemAllocation::NO_PREALLOCATE, NullHandling::OUTPUT_NOT_NULL,
                /*can_write_into_slices=*/false);
 
-  MakeFunction("is_null", &is_null_doc, {ValueDescr::ANY}, boolean(), IsNullExec,
-               registry, MemAllocation::PREALLOCATE, NullHandling::OUTPUT_NOT_NULL,
+  MakeFunction("is_null", is_null_doc, {ValueDescr::ANY}, boolean(), IsNullExec, registry,
+               MemAllocation::PREALLOCATE, NullHandling::OUTPUT_NOT_NULL,
                /*can_write_into_slices=*/true, &kNullOptions, NanOptionsState::Init);
 
-  MakeFunction("true_unless_null", &true_unless_null_doc, {ValueDescr::ANY}, boolean(),
+  MakeFunction("true_unless_null", true_unless_null_doc, {ValueDescr::ANY}, boolean(),
                TrueUnlessNullExec, registry, MemAllocation::NO_PREALLOCATE,
                NullHandling::INTERSECTION,
                /*can_write_into_slices=*/false);
 
-  DCHECK_OK(registry->AddFunction(MakeIsFiniteFunction("is_finite", &is_finite_doc)));
-  DCHECK_OK(registry->AddFunction(MakeIsInfFunction("is_inf", &is_inf_doc)));
-  DCHECK_OK(registry->AddFunction(MakeIsNanFunction("is_nan", &is_nan_doc)));
+  DCHECK_OK(registry->AddFunction(MakeIsFiniteFunction("is_finite", is_finite_doc)));
+  DCHECK_OK(registry->AddFunction(MakeIsInfFunction("is_inf", is_inf_doc)));
+  DCHECK_OK(registry->AddFunction(MakeIsNanFunction("is_nan", is_nan_doc)));
 }
 
 }  // namespace internal
diff --git a/cpp/src/arrow/compute/kernels/vector_array_sort.cc b/cpp/src/arrow/compute/kernels/vector_array_sort.cc
index 1335882a25..ac4fe443d3 100644
--- a/cpp/src/arrow/compute/kernels/vector_array_sort.cc
+++ b/cpp/src/arrow/compute/kernels/vector_array_sort.cc
@@ -555,7 +555,7 @@ void RegisterVectorArraySort(FunctionRegistry* registry) {
   base.can_execute_chunkwise = false;
 
   auto array_sort_indices = std::make_shared<VectorFunction>(
-      "array_sort_indices", Arity::Unary(), &array_sort_indices_doc,
+      "array_sort_indices", Arity::Unary(), array_sort_indices_doc,
       GetDefaultArraySortOptions());
   base.init = ArraySortIndicesState::Init;
   AddArraySortingKernels<ArraySortIndices>(base, array_sort_indices.get());
@@ -563,7 +563,7 @@ void RegisterVectorArraySort(FunctionRegistry* registry) {
 
   // partition_nth_indices has a parameter so needs its init function
   auto part_indices = std::make_shared<VectorFunction>(
-      "partition_nth_indices", Arity::Unary(), &partition_nth_indices_doc);
+      "partition_nth_indices", Arity::Unary(), partition_nth_indices_doc);
   base.init = PartitionNthToIndicesState::Init;
   AddArraySortingKernels<PartitionNthToIndices>(base, part_indices.get());
   DCHECK_OK(registry->AddFunction(std::move(part_indices)));
diff --git a/cpp/src/arrow/compute/kernels/vector_hash.cc b/cpp/src/arrow/compute/kernels/vector_hash.cc
index 80fc29c94a..fccd73e399 100644
--- a/cpp/src/arrow/compute/kernels/vector_hash.cc
+++ b/cpp/src/arrow/compute/kernels/vector_hash.cc
@@ -758,7 +758,7 @@ void RegisterVectorHash(FunctionRegistry* registry) {
 
   base.finalize = UniqueFinalize;
   base.output_chunked = false;
-  auto unique = std::make_shared<VectorFunction>("unique", Arity::Unary(), &unique_doc);
+  auto unique = std::make_shared<VectorFunction>("unique", Arity::Unary(), unique_doc);
   AddHashKernels<UniqueAction>(unique.get(), base, OutputType(FirstType));
 
   // Dictionary unique
@@ -775,7 +775,7 @@ void RegisterVectorHash(FunctionRegistry* registry) {
 
   base.finalize = ValueCountsFinalize;
   auto value_counts =
-      std::make_shared<VectorFunction>("value_counts", Arity::Unary(), &value_counts_doc);
+      std::make_shared<VectorFunction>("value_counts", Arity::Unary(), value_counts_doc);
   AddHashKernels<ValueCountsAction>(value_counts.get(), base,
                                     OutputType(ValueCountsOutput));
 
@@ -796,7 +796,7 @@ void RegisterVectorHash(FunctionRegistry* registry) {
   base.output_chunked = true;
 
   auto dict_encode = std::make_shared<VectorFunction>(
-      "dictionary_encode", Arity::Unary(), &dictionary_encode_doc,
+      "dictionary_encode", Arity::Unary(), dictionary_encode_doc,
       GetDefaultDictionaryEncodeOptions());
   AddHashKernels<DictEncodeAction>(dict_encode.get(), base, OutputType(DictEncodeOutput));
 
diff --git a/cpp/src/arrow/compute/kernels/vector_nested.cc b/cpp/src/arrow/compute/kernels/vector_nested.cc
index b5bba71793..9cc1e72bdb 100644
--- a/cpp/src/arrow/compute/kernels/vector_nested.cc
+++ b/cpp/src/arrow/compute/kernels/vector_nested.cc
@@ -122,7 +122,7 @@ const FunctionDoc list_parent_indices_doc(
 class ListParentIndicesFunction : public MetaFunction {
  public:
   ListParentIndicesFunction()
-      : MetaFunction("list_parent_indices", Arity::Unary(), &list_parent_indices_doc) {}
+      : MetaFunction("list_parent_indices", Arity::Unary(), list_parent_indices_doc) {}
 
   Result<Datum> ExecuteImpl(const std::vector<Datum>& args,
                             const FunctionOptions* options,
@@ -157,7 +157,7 @@ class ListParentIndicesFunction : public MetaFunction {
 
 void RegisterVectorNested(FunctionRegistry* registry) {
   auto flatten =
-      std::make_shared<VectorFunction>("list_flatten", Arity::Unary(), &list_flatten_doc);
+      std::make_shared<VectorFunction>("list_flatten", Arity::Unary(), list_flatten_doc);
   DCHECK_OK(flatten->AddKernel({InputType::Array(Type::LIST)}, OutputType(ListValuesType),
                                ListFlatten<ListType>));
   DCHECK_OK(flatten->AddKernel({InputType::Array(Type::FIXED_SIZE_LIST)},
diff --git a/cpp/src/arrow/compute/kernels/vector_replace.cc b/cpp/src/arrow/compute/kernels/vector_replace.cc
index 612e9e7974..294f6fe040 100644
--- a/cpp/src/arrow/compute/kernels/vector_replace.cc
+++ b/cpp/src/arrow/compute/kernels/vector_replace.cc
@@ -899,17 +899,17 @@ const FunctionDoc fill_null_backward_doc(
 void RegisterVectorReplace(FunctionRegistry* registry) {
   {
     auto func = std::make_shared<VectorFunction>("replace_with_mask", Arity::Ternary(),
-                                                 &replace_with_mask_doc);
+                                                 replace_with_mask_doc);
     RegisterVectorFunction<ReplaceWithMaskFunctor>(registry, func);
   }
   {
     auto func = std::make_shared<VectorFunction>("fill_null_forward", Arity::Unary(),
-                                                 &fill_null_forward_doc);
+                                                 fill_null_forward_doc);
     RegisterVectorFunction<FillNullForwardFunctor>(registry, func);
   }
   {
     auto func = std::make_shared<VectorFunction>("fill_null_backward", Arity::Unary(),
-                                                 &fill_null_backward_doc);
+                                                 fill_null_backward_doc);
     RegisterVectorFunction<FillNullBackwardFunctor>(registry, func);
   }
 }
diff --git a/cpp/src/arrow/compute/kernels/vector_selection.cc b/cpp/src/arrow/compute/kernels/vector_selection.cc
index 32d639ec8f..00c702f223 100644
--- a/cpp/src/arrow/compute/kernels/vector_selection.cc
+++ b/cpp/src/arrow/compute/kernels/vector_selection.cc
@@ -1958,7 +1958,7 @@ const FunctionDoc filter_doc(
 class FilterMetaFunction : public MetaFunction {
  public:
   FilterMetaFunction()
-      : MetaFunction("filter", Arity::Binary(), &filter_doc, GetDefaultFilterOptions()) {}
+      : MetaFunction("filter", Arity::Binary(), filter_doc, GetDefaultFilterOptions()) {}
 
   Result<Datum> ExecuteImpl(const std::vector<Datum>& args,
                             const FunctionOptions* options,
@@ -2115,7 +2115,7 @@ const FunctionDoc take_doc(
 class TakeMetaFunction : public MetaFunction {
  public:
   TakeMetaFunction()
-      : MetaFunction("take", Arity::Binary(), &take_doc, GetDefaultTakeOptions()) {}
+      : MetaFunction("take", Arity::Binary(), take_doc, GetDefaultTakeOptions()) {}
 
   Result<Datum> ExecuteImpl(const std::vector<Datum>& args,
                             const FunctionOptions* options,
@@ -2276,7 +2276,7 @@ const FunctionDoc drop_null_doc(
 
 class DropNullMetaFunction : public MetaFunction {
  public:
-  DropNullMetaFunction() : MetaFunction("drop_null", Arity::Unary(), &drop_null_doc) {}
+  DropNullMetaFunction() : MetaFunction("drop_null", Arity::Unary(), drop_null_doc) {}
 
   Result<Datum> ExecuteImpl(const std::vector<Datum>& args,
                             const FunctionOptions* options,
@@ -2329,13 +2329,13 @@ struct SelectionKernelDescr {
   ArrayKernelExec exec;
 };
 
-void RegisterSelectionFunction(const std::string& name, const FunctionDoc* doc,
+void RegisterSelectionFunction(const std::string& name, FunctionDoc doc,
                                VectorKernel base_kernel, InputType selection_type,
                                const std::vector<SelectionKernelDescr>& descrs,
                                const FunctionOptions* default_options,
                                FunctionRegistry* registry) {
-  auto func =
-      std::make_shared<VectorFunction>(name, Arity::Binary(), doc, default_options);
+  auto func = std::make_shared<VectorFunction>(name, Arity::Binary(), std::move(doc),
+                                               default_options);
   for (auto& descr : descrs) {
     base_kernel.signature = KernelSignature::Make(
         {std::move(descr.input), selection_type}, OutputType(FirstType));
@@ -2428,8 +2428,8 @@ Status IndicesNonZeroExec(KernelContext* ctx, const ExecBatch& batch, Datum* out
 }
 
 std::shared_ptr<VectorFunction> MakeIndicesNonZeroFunction(std::string name,
-                                                           const FunctionDoc* doc) {
-  auto func = std::make_shared<VectorFunction>(name, Arity::Unary(), doc);
+                                                           FunctionDoc doc) {
+  auto func = std::make_shared<VectorFunction>(name, Arity::Unary(), std::move(doc));
 
   VectorKernel kernel;
   kernel.null_handling = NullHandling::OUTPUT_NOT_NULL;
@@ -2481,7 +2481,7 @@ void RegisterVectorSelection(FunctionRegistry* registry) {
 
   VectorKernel filter_base;
   filter_base.init = FilterState::Init;
-  RegisterSelectionFunction("array_filter", &array_filter_doc, filter_base,
+  RegisterSelectionFunction("array_filter", array_filter_doc, filter_base,
                             /*selection_type=*/InputType::Array(boolean()),
                             filter_kernel_descrs, GetDefaultFilterOptions(), registry);
 
@@ -2513,7 +2513,7 @@ void RegisterVectorSelection(FunctionRegistry* registry) {
   take_base.init = TakeState::Init;
   take_base.can_execute_chunkwise = false;
   RegisterSelectionFunction(
-      "array_take", &array_take_doc, take_base,
+      "array_take", array_take_doc, take_base,
       /*selection_type=*/InputType(match::Integer(), ValueDescr::ARRAY),
       take_kernel_descrs, GetDefaultTakeOptions(), registry);
 
@@ -2523,7 +2523,7 @@ void RegisterVectorSelection(FunctionRegistry* registry) {
   DCHECK_OK(registry->AddFunction(std::make_shared<DropNullMetaFunction>()));
 
   DCHECK_OK(registry->AddFunction(
-      MakeIndicesNonZeroFunction("indices_nonzero", &indices_nonzero_doc)));
+      MakeIndicesNonZeroFunction("indices_nonzero", indices_nonzero_doc)));
 }
 
 }  // namespace internal
diff --git a/cpp/src/arrow/compute/kernels/vector_sort.cc b/cpp/src/arrow/compute/kernels/vector_sort.cc
index ad3323199a..88c8a193fd 100644
--- a/cpp/src/arrow/compute/kernels/vector_sort.cc
+++ b/cpp/src/arrow/compute/kernels/vector_sort.cc
@@ -1168,7 +1168,7 @@ const FunctionDoc sort_indices_doc(
 class SortIndicesMetaFunction : public MetaFunction {
  public:
   SortIndicesMetaFunction()
-      : MetaFunction("sort_indices", Arity::Unary(), &sort_indices_doc,
+      : MetaFunction("sort_indices", Arity::Unary(), sort_indices_doc,
                      GetDefaultSortOptions()) {}
 
   Result<Datum> ExecuteImpl(const std::vector<Datum>& args,
@@ -1840,7 +1840,7 @@ static Status CheckConsistency(const Schema& schema,
 class SelectKUnstableMetaFunction : public MetaFunction {
  public:
   SelectKUnstableMetaFunction()
-      : MetaFunction("select_k_unstable", Arity::Unary(), &select_k_unstable_doc,
+      : MetaFunction("select_k_unstable", Arity::Unary(), select_k_unstable_doc,
                      GetDefaultSelectKOptions()) {}
 
   Result<Datum> ExecuteImpl(const std::vector<Datum>& args,
diff --git a/cpp/src/arrow/compute/registry_test.cc b/cpp/src/arrow/compute/registry_test.cc
index e1e0d52318..faf47a46f6 100644
--- a/cpp/src/arrow/compute/registry_test.cc
+++ b/cpp/src/arrow/compute/registry_test.cc
@@ -49,12 +49,13 @@ TEST_F(TestRegistry, CreateBuiltInRegistry) {
 TEST_F(TestRegistry, Basics) {
   ASSERT_EQ(0, registry_->num_functions());
 
-  std::shared_ptr<Function> func =
-      std::make_shared<ScalarFunction>("f1", Arity::Unary(), /*doc=*/nullptr);
+  std::shared_ptr<Function> func = std::make_shared<ScalarFunction>(
+      "f1", Arity::Unary(), /*doc=*/FunctionDoc::Empty());
   ASSERT_OK(registry_->AddFunction(func));
   ASSERT_EQ(1, registry_->num_functions());
 
-  func = std::make_shared<VectorFunction>("f0", Arity::Binary(), /*doc=*/nullptr);
+  func = std::make_shared<VectorFunction>("f0", Arity::Binary(),
+                                          /*doc=*/FunctionDoc::Empty());
   ASSERT_OK(registry_->AddFunction(func));
   ASSERT_EQ(2, registry_->num_functions());
 
@@ -65,7 +66,8 @@ TEST_F(TestRegistry, Basics) {
   ASSERT_RAISES(KeyError, registry_->GetFunction("f2"));
 
   // Try adding a function with name collision
-  func = std::make_shared<ScalarAggregateFunction>("f1", Arity::Unary(), /*doc=*/nullptr);
+  func = std::make_shared<ScalarAggregateFunction>("f1", Arity::Unary(),
+                                                   /*doc=*/FunctionDoc::Empty());
   ASSERT_RAISES(KeyError, registry_->AddFunction(func));
 
   // Allow overwriting by flag
diff --git a/cpp/src/arrow/datum.cc b/cpp/src/arrow/datum.cc
index e51c64d8e7..ac2e9a7e69 100644
--- a/cpp/src/arrow/datum.cc
+++ b/cpp/src/arrow/datum.cc
@@ -279,4 +279,24 @@ void PrintTo(const Datum& datum, std::ostream* os) {
   }
 }
 
+std::string ToString(Datum::Kind kind) {
+  switch (kind) {
+    case Datum::NONE:
+      return "None";
+    case Datum::SCALAR:
+      return "Scalar";
+    case Datum::ARRAY:
+      return "Array";
+    case Datum::CHUNKED_ARRAY:
+      return "ChunkedArray";
+    case Datum::RECORD_BATCH:
+      return "RecordBatch";
+    case Datum::TABLE:
+      return "Table";
+    default:
+      DCHECK(false);
+      return "";
+  }
+}
+
 }  // namespace arrow
diff --git a/cpp/src/arrow/datum.h b/cpp/src/arrow/datum.h
index bce53decd4..a732dc429d 100644
--- a/cpp/src/arrow/datum.h
+++ b/cpp/src/arrow/datum.h
@@ -285,4 +285,6 @@ struct ARROW_EXPORT Datum {
   ARROW_EXPORT friend void PrintTo(const Datum&, std::ostream*);
 };
 
+ARROW_EXPORT std::string ToString(Datum::Kind kind);
+
 }  // namespace arrow
diff --git a/cpp/src/arrow/python/CMakeLists.txt b/cpp/src/arrow/python/CMakeLists.txt
index 23c97cc320..7235e2d0fe 100644
--- a/cpp/src/arrow/python/CMakeLists.txt
+++ b/cpp/src/arrow/python/CMakeLists.txt
@@ -44,7 +44,8 @@ set(ARROW_PYTHON_SRCS
     numpy_to_arrow.cc
     python_to_arrow.cc
     pyarrow.cc
-    serialize.cc)
+    serialize.cc
+    udf.cc)
 
 set_source_files_properties(init.cc PROPERTIES SKIP_PRECOMPILE_HEADERS ON
                                                SKIP_UNITY_BUILD_INCLUSION ON)
diff --git a/cpp/src/arrow/python/common.h b/cpp/src/arrow/python/common.h
index 24dcb130a2..5c16106730 100644
--- a/cpp/src/arrow/python/common.h
+++ b/cpp/src/arrow/python/common.h
@@ -180,6 +180,11 @@ class ARROW_PYTHON_EXPORT OwnedRefNoGIL : public OwnedRef {
   explicit OwnedRefNoGIL(PyObject* obj) : OwnedRef(obj) {}
 
   ~OwnedRefNoGIL() {
+    // This destructor may be called after the Python interpreter is finalized.
+    // At least avoid spurious attempts to take the GIL when not necessary.
+    if (obj() == NULLPTR) {
+      return;
+    }
     PyAcquireGIL lock;
     reset();
   }
diff --git a/cpp/src/arrow/python/udf.cc b/cpp/src/arrow/python/udf.cc
new file mode 100644
index 0000000000..41309d27bb
--- /dev/null
+++ b/cpp/src/arrow/python/udf.cc
@@ -0,0 +1,133 @@
+// 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 "arrow/python/udf.h"
+#include "arrow/compute/function.h"
+#include "arrow/python/common.h"
+
+namespace arrow {
+
+namespace py {
+
+namespace {
+Status CheckOutputType(const DataType& expected, const DataType& actual) {
+  if (!expected.Equals(actual)) {
+    return Status::TypeError("Expected output datatype ", expected.ToString(),
+                             ", but function returned datatype ", actual.ToString());
+  }
+  return Status::OK();
+}
+
+struct PythonUdf {
+  ScalarUdfWrapperCallback cb;
+  std::shared_ptr<OwnedRefNoGIL> function;
+  compute::OutputType output_type;
+
+  // function needs to be destroyed at process exit
+  // and Python may no longer be initialized.
+  ~PythonUdf() {
+    if (_Py_IsFinalizing()) {
+      function->detach();
+    }
+  }
+
+  Status operator()(compute::KernelContext* ctx, const compute::ExecBatch& batch,
+                    Datum* out) {
+    return SafeCallIntoPython([&]() -> Status { return Execute(ctx, batch, out); });
+  }
+
+  Status Execute(compute::KernelContext* ctx, const compute::ExecBatch& batch,
+                 Datum* out) {
+    const auto num_args = batch.values.size();
+    ScalarUdfContext udf_context{ctx->memory_pool(), static_cast<int64_t>(batch.length)};
+
+    OwnedRef arg_tuple(PyTuple_New(num_args));
+    RETURN_NOT_OK(CheckPyError());
+    for (size_t arg_id = 0; arg_id < num_args; arg_id++) {
+      switch (batch[arg_id].kind()) {
+        case Datum::SCALAR: {
+          auto c_data = batch[arg_id].scalar();
+          PyObject* data = wrap_scalar(c_data);
+          PyTuple_SetItem(arg_tuple.obj(), arg_id, data);
+          break;
+        }
+        case Datum::ARRAY: {
+          auto c_data = batch[arg_id].make_array();
+          PyObject* data = wrap_array(c_data);
+          PyTuple_SetItem(arg_tuple.obj(), arg_id, data);
+          break;
+        }
+        default:
+          auto datum = batch[arg_id];
+          return Status::NotImplemented(
+              "User-defined-functions are not supported for the datum kind ",
+              ToString(batch[arg_id].kind()));
+      }
+    }
+
+    OwnedRef result(cb(function->obj(), udf_context, arg_tuple.obj()));
+    RETURN_NOT_OK(CheckPyError());
+    // unwrapping the output for expected output type
+    if (is_scalar(result.obj())) {
+      ARROW_ASSIGN_OR_RAISE(auto val, unwrap_scalar(result.obj()));
+      RETURN_NOT_OK(CheckOutputType(*output_type.type(), *val->type));
+      *out = Datum(val);
+      return Status::OK();
+    } else if (is_array(result.obj())) {
+      ARROW_ASSIGN_OR_RAISE(auto val, unwrap_array(result.obj()));
+      RETURN_NOT_OK(CheckOutputType(*output_type.type(), *val->type()));
+      *out = Datum(val);
+      return Status::OK();
+    } else {
+      return Status::TypeError("Unexpected output type: ", Py_TYPE(result.obj())->tp_name,
+                               " (expected Scalar or Array)");
+    }
+    return Status::OK();
+  }
+};
+
+}  // namespace
+
+Status RegisterScalarFunction(PyObject* user_function, ScalarUdfWrapperCallback wrapper,
+                              const ScalarUdfOptions& options) {
+  if (!PyCallable_Check(user_function)) {
+    return Status::TypeError("Expected a callable Python object.");
+  }
+  auto scalar_func = std::make_shared<compute::ScalarFunction>(
+      options.func_name, options.arity, options.func_doc);
+  Py_INCREF(user_function);
+  std::vector<compute::InputType> input_types;
+  for (const auto& in_dtype : options.input_types) {
+    input_types.emplace_back(in_dtype);
+  }
+  compute::OutputType output_type(options.output_type);
+  PythonUdf exec{wrapper, std::make_shared<OwnedRefNoGIL>(user_function), output_type};
+  compute::ScalarKernel kernel(
+      compute::KernelSignature::Make(std::move(input_types), std::move(output_type),
+                                     options.arity.is_varargs),
+      std::move(exec));
+  kernel.mem_allocation = compute::MemAllocation::NO_PREALLOCATE;
+  kernel.null_handling = compute::NullHandling::COMPUTED_NO_PREALLOCATE;
+  RETURN_NOT_OK(scalar_func->AddKernel(std::move(kernel)));
+  auto registry = compute::GetFunctionRegistry();
+  RETURN_NOT_OK(registry->AddFunction(std::move(scalar_func)));
+  return Status::OK();
+}
+
+}  // namespace py
+
+}  // namespace arrow
diff --git a/cpp/src/arrow/python/udf.h b/cpp/src/arrow/python/udf.h
new file mode 100644
index 0000000000..4ab3e7cc72
--- /dev/null
+++ b/cpp/src/arrow/python/udf.h
@@ -0,0 +1,59 @@
+// 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 "arrow/compute/exec.h"
+#include "arrow/compute/function.h"
+#include "arrow/compute/registry.h"
+#include "arrow/python/platform.h"
+
+#include "arrow/python/common.h"
+#include "arrow/python/pyarrow.h"
+#include "arrow/python/visibility.h"
+
+namespace arrow {
+
+namespace py {
+
+// TODO: TODO(ARROW-16041): UDF Options are not exposed to the Python
+// users. This feature will be included when extending to provide advanced
+// options for the users.
+struct ARROW_PYTHON_EXPORT ScalarUdfOptions {
+  std::string func_name;
+  compute::Arity arity;
+  compute::FunctionDoc func_doc;
+  std::vector<std::shared_ptr<DataType>> input_types;
+  std::shared_ptr<DataType> output_type;
+};
+
+struct ARROW_PYTHON_EXPORT ScalarUdfContext {
+  MemoryPool* pool;
+  int64_t batch_length;
+};
+
+using ScalarUdfWrapperCallback = std::function<PyObject*(
+    PyObject* user_function, const ScalarUdfContext& context, PyObject* inputs)>;
+
+/// \brief register a Scalar user-defined-function from Python
+Status ARROW_PYTHON_EXPORT RegisterScalarFunction(PyObject* user_function,
+                                                  ScalarUdfWrapperCallback wrapper,
+                                                  const ScalarUdfOptions& options);
+
+}  // namespace py
+
+}  // namespace arrow
diff --git a/python/pyarrow/_compute.pxd b/python/pyarrow/_compute.pxd
index ebf2066d04..8b09cbd445 100644
--- a/python/pyarrow/_compute.pxd
+++ b/python/pyarrow/_compute.pxd
@@ -21,6 +21,11 @@ from pyarrow.lib cimport *
 from pyarrow.includes.common cimport *
 from pyarrow.includes.libarrow cimport *
 
+cdef class ScalarUdfContext(_Weakrefable):
+    cdef:
+        CScalarUdfContext c_context
+
+    cdef void init(self, const CScalarUdfContext& c_context)
 
 cdef class FunctionOptions(_Weakrefable):
     cdef:
diff --git a/python/pyarrow/_compute.pyx b/python/pyarrow/_compute.pyx
index 2ba6ae2462..d17bce47cb 100644
--- a/python/pyarrow/_compute.pyx
+++ b/python/pyarrow/_compute.pyx
@@ -26,9 +26,13 @@ from collections import namedtuple
 
 from pyarrow.lib import frombytes, tobytes, ordered_dict
 from pyarrow.lib cimport *
+from pyarrow.includes.common cimport *
 from pyarrow.includes.libarrow cimport *
 import pyarrow.lib as lib
 
+from libcpp cimport bool as c_bool
+
+import inspect
 import numpy as np
 
 
@@ -2275,3 +2279,211 @@ cdef CExpression _bind(Expression filter, Schema schema) except *:
 
     return GetResultValue(filter.unwrap().Bind(
         deref(pyarrow_unwrap_schema(schema).get())))
+
+
+cdef class ScalarUdfContext:
+    """
+    Per-invocation function context/state.
+
+    This object will always be the first argument to a user-defined
+    function. It should not be used outside of a call to the function.
+    """
+
+    def __init__(self):
+        raise TypeError("Do not call {}'s constructor directly"
+                        .format(self.__class__.__name__))
+
+    cdef void init(self, const CScalarUdfContext &c_context):
+        self.c_context = c_context
+
+    @property
+    def batch_length(self):
+        """
+        The common length of all input arguments (int).
+
+        In the case that all arguments are scalars, this value
+        is used to pass the "actual length" of the arguments,
+        e.g. because the scalar values are encoding a column
+        with a constant value.
+        """
+        return self.c_context.batch_length
+
+    @property
+    def memory_pool(self):
+        """
+        A memory pool for allocations (:class:`MemoryPool`).
+
+        This is the memory pool supplied by the user when they invoked
+        the function and it should be used in any calls to arrow that the
+        UDF makes if that call accepts a memory_pool.
+        """
+        return box_memory_pool(self.c_context.pool)
+
+
+cdef inline CFunctionDoc _make_function_doc(dict func_doc) except *:
+    """
+    Helper function to generate the FunctionDoc
+    This function accepts a dictionary and expects the 
+    summary(str), description(str) and arg_names(List[str]) keys. 
+    """
+    cdef:
+        CFunctionDoc f_doc
+        vector[c_string] c_arg_names
+
+    f_doc.summary = tobytes(func_doc["summary"])
+    f_doc.description = tobytes(func_doc["description"])
+    for arg_name in func_doc["arg_names"]:
+        c_arg_names.push_back(tobytes(arg_name))
+    f_doc.arg_names = c_arg_names
+    # UDFOptions integration:
+    # TODO: https://issues.apache.org/jira/browse/ARROW-16041
+    f_doc.options_class = b""
+    f_doc.options_required = False
+    return f_doc
+
+
+cdef object box_scalar_udf_context(const CScalarUdfContext& c_context):
+    cdef ScalarUdfContext context = ScalarUdfContext.__new__(ScalarUdfContext)
+    context.init(c_context)
+    return context
+
+
+cdef _scalar_udf_callback(user_function, const CScalarUdfContext& c_context, inputs):
+    """
+    Helper callback function used to wrap the ScalarUdfContext from Python to C++
+    execution.
+    """
+    context = box_scalar_udf_context(c_context)
+    return user_function(context, *inputs)
+
+
+def _get_scalar_udf_context(memory_pool, batch_length):
+    cdef CScalarUdfContext c_context
+    c_context.pool = maybe_unbox_memory_pool(memory_pool)
+    c_context.batch_length = batch_length
+    context = box_scalar_udf_context(c_context)
+    return context
+
+
+def register_scalar_function(func, function_name, function_doc, in_types,
+                             out_type):
+    """
+    Register a user-defined scalar function. 
+
+    A scalar function is a function that executes elementwise
+    operations on arrays or scalars, i.e. a scalar function must
+    be computed row-by-row with no state where each output row 
+    is computed only from its corresponding input row.
+    In other words, all argument arrays have the same length,
+    and the output array is of the same length as the arguments.
+    Scalar functions are the only functions allowed in query engine
+    expressions.
+
+    Parameters
+    ----------
+    func : callable
+        A callable implementing the user-defined function.
+        The first argument is the context argument of type
+        ScalarUdfContext.
+        Then, it must take arguments equal to the number of
+        in_types defined. It must return an Array or Scalar
+        matching the out_type. It must return a Scalar if
+        all arguments are scalar, else it must return an Array.
+
+        To define a varargs function, pass a callable that takes
+        varargs. The last in_type will be the type of all varargs
+        arguments.
+    function_name : str
+        Name of the function. This name must be globally unique. 
+    function_doc : dict
+        A dictionary object with keys "summary" (str),
+        and "description" (str).
+    in_types : Dict[str, DataType]
+        A dictionary mapping function argument names to
+        their respective DataType.
+        The argument names will be used to generate
+        documentation for the function. The number of
+        arguments specified here determines the function
+        arity.
+    out_type : DataType
+        Output type of the function.
+
+    Examples
+    --------
+
+    >>> import pyarrow.compute as pc
+    >>> 
+    >>> func_doc = {}
+    >>> func_doc["summary"] = "simple udf"
+    >>> func_doc["description"] = "add a constant to a scalar"
+    >>> 
+    >>> def add_constant(ctx, array):
+    ...     return pc.add(array, 1, memory_pool=ctx.memory_pool)
+    >>> 
+    >>> func_name = "py_add_func"
+    >>> in_types = {"array": pa.int64()}
+    >>> out_type = pa.int64()
+    >>> pc.register_scalar_function(add_constant, func_name, func_doc,
+    ...                   in_types, out_type)
+    >>> 
+    >>> func = pc.get_function(func_name)
+    >>> func.name
+    'py_add_func'
+    >>> answer = pc.call_function(func_name, [pa.array([20])])
+    >>> answer
+    <pyarrow.lib.Int64Array object at 0x10c22e700>
+    [
+    21
+    ]
+    """
+    cdef:
+        c_string c_func_name
+        CArity c_arity
+        CFunctionDoc c_func_doc
+        vector[shared_ptr[CDataType]] c_in_types
+        PyObject* c_function
+        shared_ptr[CDataType] c_out_type
+        CScalarUdfOptions c_options
+
+    if callable(func):
+        c_function = <PyObject*>func
+    else:
+        raise TypeError("func must be a callable")
+
+    c_func_name = tobytes(function_name)
+
+    func_spec = inspect.getfullargspec(func)
+    num_args = -1
+    if isinstance(in_types, dict):
+        for in_type in in_types.values():
+            c_in_types.push_back(
+                pyarrow_unwrap_data_type(ensure_type(in_type)))
+        function_doc["arg_names"] = in_types.keys()
+        num_args = len(in_types)
+    else:
+        raise TypeError(
+            "in_types must be a dictionary of DataType")
+
+    c_arity = CArity(num_args, func_spec.varargs)
+
+    if "summary" not in function_doc:
+        raise ValueError("Function doc must contain a summary")
+
+    if "description" not in function_doc:
+        raise ValueError("Function doc must contain a description")
+
+    if "arg_names" not in function_doc:
+        raise ValueError("Function doc must contain arg_names")
+
+    c_func_doc = _make_function_doc(function_doc)
+
+    c_out_type = pyarrow_unwrap_data_type(ensure_type(out_type))
+
+    c_options.func_name = c_func_name
+    c_options.arity = c_arity
+    c_options.func_doc = c_func_doc
+    c_options.input_types = c_in_types
+    c_options.output_type = c_out_type
+
+    check_status(RegisterScalarFunction(c_function,
+                                        <function[CallbackUdf]> &_scalar_udf_callback, c_options))
diff --git a/python/pyarrow/compute.py b/python/pyarrow/compute.py
index 40751eab26..99e0da677a 100644
--- a/python/pyarrow/compute.py
+++ b/python/pyarrow/compute.py
@@ -76,6 +76,9 @@ from pyarrow._compute import (  # noqa
     get_function,
     list_functions,
     _group_by,
+    # Udf
+    register_scalar_function,
+    ScalarUdfContext,
     # Expressions
     Expression,
 )
diff --git a/python/pyarrow/includes/libarrow.pxd b/python/pyarrow/includes/libarrow.pxd
index cc52102ef8..819ba567df 100644
--- a/python/pyarrow/includes/libarrow.pxd
+++ b/python/pyarrow/includes/libarrow.pxd
@@ -1838,6 +1838,10 @@ cdef extern from "arrow/compute/api.h" namespace "arrow::compute" nogil:
         int num_args
         c_bool is_varargs
 
+        CArity()
+
+        CArity(int num_args, c_bool is_varargs)
+
     cdef enum FunctionKind" arrow::compute::Function::Kind":
         FunctionKind_SCALAR" arrow::compute::Function::SCALAR"
         FunctionKind_VECTOR" arrow::compute::Function::VECTOR"
@@ -2346,6 +2350,7 @@ cdef extern from "arrow/compute/api.h" namespace "arrow::compute" nogil:
         const shared_ptr[CTable]& table() const
         const shared_ptr[CScalar]& scalar() const
 
+    cdef c_string ToString(DatumType kind)
 
 cdef extern from * namespace "arrow::compute":
     # inlined from compute/function_internal.h to avoid exposing
@@ -2671,3 +2676,20 @@ cdef extern from "arrow/util/byte_size.h" namespace "arrow::util" nogil:
     int64_t TotalBufferSize(const CChunkedArray& array)
     int64_t TotalBufferSize(const CRecordBatch& record_batch)
     int64_t TotalBufferSize(const CTable& table)
+
+ctypedef PyObject* CallbackUdf(object user_function, const CScalarUdfContext& context, object inputs)
+
+cdef extern from "arrow/python/udf.h" namespace "arrow::py":
+    cdef cppclass CScalarUdfContext" arrow::py::ScalarUdfContext":
+        CMemoryPool *pool
+        int64_t batch_length
+
+    cdef cppclass CScalarUdfOptions" arrow::py::ScalarUdfOptions":
+        c_string func_name
+        CArity arity
+        CFunctionDoc func_doc
+        vector[shared_ptr[CDataType]] input_types
+        shared_ptr[CDataType] output_type
+
+    CStatus RegisterScalarFunction(PyObject* function,
+                                   function[CallbackUdf] wrapper, const CScalarUdfOptions& options)
diff --git a/python/pyarrow/lib.pxd b/python/pyarrow/lib.pxd
index 28889badeb..953b0e7b51 100644
--- a/python/pyarrow/lib.pxd
+++ b/python/pyarrow/lib.pxd
@@ -61,6 +61,9 @@ cdef class MemoryPool(_Weakrefable):
 cdef CMemoryPool* maybe_unbox_memory_pool(MemoryPool memory_pool)
 
 
+cdef object box_memory_pool(CMemoryPool* pool)
+
+
 cdef class DataType(_Weakrefable):
     cdef:
         shared_ptr[CDataType] sp_type
diff --git a/python/pyarrow/memory.pxi b/python/pyarrow/memory.pxi
index 4847141f36..2258be78d5 100644
--- a/python/pyarrow/memory.pxi
+++ b/python/pyarrow/memory.pxi
@@ -84,6 +84,12 @@ cdef CMemoryPool* maybe_unbox_memory_pool(MemoryPool memory_pool):
         return memory_pool.pool
 
 
+cdef api object box_memory_pool(CMemoryPool *c_pool):
+    cdef MemoryPool pool = MemoryPool.__new__(MemoryPool)
+    pool.init(c_pool)
+    return pool
+
+
 cdef class LoggingMemoryPool(MemoryPool):
     cdef:
         unique_ptr[CLoggingMemoryPool] logging_pool
diff --git a/python/pyarrow/tests/test_udf.py b/python/pyarrow/tests/test_udf.py
new file mode 100644
index 0000000000..33315fc12d
--- /dev/null
+++ b/python/pyarrow/tests/test_udf.py
@@ -0,0 +1,503 @@
+# 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.
+
+
+import pytest
+
+import pyarrow as pa
+from pyarrow import compute as pc
+
+# UDFs are all tested with a dataset scan
+pytestmark = pytest.mark.dataset
+
+
+try:
+    import pyarrow.dataset as ds
+except ImportError:
+    ds = None
+
+
+def mock_udf_context(batch_length=10):
+    from pyarrow._compute import _get_scalar_udf_context
+    return _get_scalar_udf_context(pa.default_memory_pool(), batch_length)
+
+
+class MyError(RuntimeError):
+    pass
+
+
+@pytest.fixture(scope="session")
+def unary_func_fixture():
+    """
+    Register a unary scalar function.
+    """
+    def unary_function(ctx, x):
+        return pc.call_function("add", [x, 1],
+                                memory_pool=ctx.memory_pool)
+    func_name = "y=x+1"
+    unary_doc = {"summary": "add function",
+                 "description": "test add function"}
+    pc.register_scalar_function(unary_function,
+                                func_name,
+                                unary_doc,
+                                {"array": pa.int64()},
+                                pa.int64())
+    return unary_function, func_name
+
+
+@pytest.fixture(scope="session")
+def binary_func_fixture():
+    """
+    Register a binary scalar function.
+    """
+    def binary_function(ctx, m, x):
+        return pc.call_function("multiply", [m, x],
+                                memory_pool=ctx.memory_pool)
+    func_name = "y=mx"
+    binary_doc = {"summary": "y=mx",
+                  "description": "find y from y = mx"}
+    pc.register_scalar_function(binary_function,
+                                func_name,
+                                binary_doc,
+                                {"m": pa.int64(),
+                                 "x": pa.int64(),
+                                 },
+                                pa.int64())
+    return binary_function, func_name
+
+
+@pytest.fixture(scope="session")
+def ternary_func_fixture():
+    """
+    Register a ternary scalar function.
+    """
+    def ternary_function(ctx, m, x, c):
+        mx = pc.call_function("multiply", [m, x],
+                              memory_pool=ctx.memory_pool)
+        return pc.call_function("add", [mx, c],
+                                memory_pool=ctx.memory_pool)
+    ternary_doc = {"summary": "y=mx+c",
+                   "description": "find y from y = mx + c"}
+    func_name = "y=mx+c"
+    pc.register_scalar_function(ternary_function,
+                                func_name,
+                                ternary_doc,
+                                {
+                                    "array1": pa.int64(),
+                                    "array2": pa.int64(),
+                                    "array3": pa.int64(),
+                                },
+                                pa.int64())
+    return ternary_function, func_name
+
+
+@pytest.fixture(scope="session")
+def varargs_func_fixture():
+    """
+    Register a varargs scalar function with at least two arguments.
+    """
+    def varargs_function(ctx, first, *values):
+        acc = first
+        for val in values:
+            acc = pc.call_function("add", [acc, val],
+                                   memory_pool=ctx.memory_pool)
+        return acc
+    func_name = "z=ax+by+c"
+    varargs_doc = {"summary": "z=ax+by+c",
+                   "description": "find z from z = ax + by + c"
+                   }
+    pc.register_scalar_function(varargs_function,
+                                func_name,
+                                varargs_doc,
+                                {
+                                    "array1": pa.int64(),
+                                    "array2": pa.int64(),
+                                },
+                                pa.int64())
+    return varargs_function, func_name
+
+
+@pytest.fixture(scope="session")
+def nullary_func_fixture():
+    """
+    Register a nullary scalar function.
+    """
+    def nullary_func(context):
+        return pa.array([42] * context.batch_length, type=pa.int64(),
+                        memory_pool=context.memory_pool)
+
+    func_doc = {
+        "summary": "random function",
+        "description": "generates a random value"
+    }
+    func_name = "test_nullary_func"
+    pc.register_scalar_function(nullary_func,
+                                func_name,
+                                func_doc,
+                                {},
+                                pa.int64())
+
+    return nullary_func, func_name
+
+
+@pytest.fixture(scope="session")
+def wrong_output_type_func_fixture():
+    """
+    Register a scalar function which returns something that is neither
+    a Arrow scalar or array.
+    """
+    def wrong_output_type(ctx):
+        return 42
+
+    func_name = "test_wrong_output_type"
+    in_types = {}
+    out_type = pa.int64()
+    doc = {
+        "summary": "return wrong output type",
+        "description": ""
+    }
+    pc.register_scalar_function(wrong_output_type, func_name, doc,
+                                in_types, out_type)
+    return wrong_output_type, func_name
+
+
+@pytest.fixture(scope="session")
+def wrong_output_datatype_func_fixture():
+    """
+    Register a scalar function whose actual output DataType doesn't
+    match the declared output DataType.
+    """
+    def wrong_output_datatype(ctx, array):
+        return pc.call_function("add", [array, 1])
+    func_name = "test_wrong_output_datatype"
+    in_types = {"array": pa.int64()}
+    # The actual output DataType will be int64.
+    out_type = pa.int16()
+    doc = {
+        "summary": "return wrong output datatype",
+        "description": ""
+    }
+    pc.register_scalar_function(wrong_output_datatype, func_name, doc,
+                                in_types, out_type)
+    return wrong_output_datatype, func_name
+
+
+@pytest.fixture(scope="session")
+def wrong_signature_func_fixture():
+    """
+    Register a scalar function with the wrong signature.
+    """
+    # Missing the context argument
+    def wrong_signature():
+        return pa.scalar(1, type=pa.int64())
+
+    func_name = "test_wrong_signature"
+    in_types = {}
+    out_type = pa.int64()
+    doc = {
+        "summary": "UDF with wrong signature",
+        "description": ""
+    }
+    pc.register_scalar_function(wrong_signature, func_name, doc,
+                                in_types, out_type)
+    return wrong_signature, func_name
+
+
+@pytest.fixture(scope="session")
+def raising_func_fixture():
+    """
+    Register a scalar function which raises a custom exception.
+    """
+    def raising_func(ctx):
+        raise MyError("error raised by scalar UDF")
+    func_name = "test_raise"
+    doc = {
+        "summary": "raising function",
+        "description": ""
+    }
+    pc.register_scalar_function(raising_func, func_name, doc,
+                                {}, pa.int64())
+    return raising_func, func_name
+
+
+def check_scalar_function(func_fixture,
+                          inputs, *,
+                          run_in_dataset=True,
+                          batch_length=None):
+    function, name = func_fixture
+    if batch_length is None:
+        for input in inputs:
+            try:
+                batch_length = len(inputs)
+            except TypeError:
+                pass
+    expected_output = function(mock_udf_context(batch_length), *inputs)
+    func = pc.get_function(name)
+    assert func.name == name
+
+    result = pc.call_function(name, inputs)
+    assert result == expected_output
+    # At the moment there is an issue when handling nullary functions.
+    # See: ARROW-15286 and ARROW-16290.
+    if run_in_dataset:
+        field_names = [f'field{index}' for index, in_arr in inputs]
+        table = pa.Table.from_arrays(inputs, field_names)
+        dataset = ds.dataset(table)
+        func_args = [ds.field(field_name) for field_name in field_names]
+        result_table = dataset.to_table(
+            columns={'result': ds.field('')._call(name, func_args)})
+        assert result_table.column(0).chunks[0] == expected_output
+
+
+def test_scalar_udf_array_unary(unary_func_fixture):
+    check_scalar_function(unary_func_fixture,
+                          [
+                              pa.array([10, 20], pa.int64())
+                          ]
+                          )
+
+
+def test_scalar_udf_array_binary(binary_func_fixture):
+    check_scalar_function(binary_func_fixture,
+                          [
+                              pa.array([10, 20], pa.int64()),
+                              pa.array([2, 4], pa.int64())
+                          ]
+                          )
+
+
+def test_scalar_udf_array_ternary(ternary_func_fixture):
+    check_scalar_function(ternary_func_fixture,
+                          [
+                              pa.array([10, 20], pa.int64()),
+                              pa.array([2, 4], pa.int64()),
+                              pa.array([5, 10], pa.int64())
+                          ]
+                          )
+
+
+def test_scalar_udf_array_varargs(varargs_func_fixture):
+    check_scalar_function(varargs_func_fixture,
+                          [
+                              pa.array([2, 3], pa.int64()),
+                              pa.array([10, 20], pa.int64()),
+                              pa.array([3, 7], pa.int64()),
+                              pa.array([20, 30], pa.int64()),
+                              pa.array([5, 10], pa.int64())
+                          ]
+                          )
+
+
+def test_registration_errors():
+    # validate function name
+    doc = {
+        "summary": "test udf input",
+        "description": "parameters are validated"
+    }
+    in_types = {"scalar": pa.int64()}
+    out_type = pa.int64()
+
+    def test_reg_function(context):
+        return pa.array([10])
+
+    with pytest.raises(TypeError):
+        pc.register_scalar_function(test_reg_function,
+                                    None, doc, in_types,
+                                    out_type)
+
+    # validate function
+    with pytest.raises(TypeError, match="func must be a callable"):
+        pc.register_scalar_function(None, "test_none_function", doc, in_types,
+                                    out_type)
+
+    # validate output type
+    expected_expr = "DataType expected, got <class 'NoneType'>"
+    with pytest.raises(TypeError, match=expected_expr):
+        pc.register_scalar_function(test_reg_function,
+                                    "test_output_function", doc, in_types,
+                                    None)
+
+    # validate input type
+    expected_expr = "in_types must be a dictionary of DataType"
+    with pytest.raises(TypeError, match=expected_expr):
+        pc.register_scalar_function(test_reg_function,
+                                    "test_input_function", doc, None,
+                                    out_type)
+
+    # register an already registered function
+    # first registration
+    pc.register_scalar_function(test_reg_function,
+                                "test_reg_function", doc, {},
+                                out_type)
+    # second registration
+    expected_expr = "Already have a function registered with name:" \
+        + " test_reg_function"
+    with pytest.raises(KeyError, match=expected_expr):
+        pc.register_scalar_function(test_reg_function,
+                                    "test_reg_function", doc, {},
+                                    out_type)
+
+
+def test_varargs_function_validation(varargs_func_fixture):
+    _, func_name = varargs_func_fixture
+
+    error_msg = r"VarArgs function 'z=ax\+by\+c' needs at least 2 arguments"
+
+    with pytest.raises(ValueError, match=error_msg):
+        pc.call_function(func_name, [42])
+
+
+def test_function_doc_validation():
+    # validate arity
+    in_types = {"scalar": pa.int64()}
+    out_type = pa.int64()
+
+    # doc with no summary
+    func_doc = {
+        "description": "desc"
+    }
+
+    def add_const(ctx, scalar):
+        return pc.call_function("add", [scalar, 1])
+
+    with pytest.raises(ValueError,
+                       match="Function doc must contain a summary"):
+        pc.register_scalar_function(add_const, "test_no_summary",
+                                    func_doc, in_types,
+                                    out_type)
+
+    # doc with no decription
+    func_doc = {
+        "summary": "test summary"
+    }
+
+    with pytest.raises(ValueError,
+                       match="Function doc must contain a description"):
+        pc.register_scalar_function(add_const, "test_no_desc",
+                                    func_doc, in_types,
+                                    out_type)
+
+
+def test_nullary_function(nullary_func_fixture):
+    # XXX the Python compute layer API doesn't let us override batch_length,
+    # so only test with the default value of 1.
+    check_scalar_function(nullary_func_fixture, [], run_in_dataset=False,
+                          batch_length=1)
+
+
+def test_wrong_output_type(wrong_output_type_func_fixture):
+    _, func_name = wrong_output_type_func_fixture
+
+    with pytest.raises(TypeError,
+                       match="Unexpected output type: int"):
+        pc.call_function(func_name, [])
+
+
+def test_wrong_output_datatype(wrong_output_datatype_func_fixture):
+    _, func_name = wrong_output_datatype_func_fixture
+
+    expected_expr = ("Expected output datatype int16, "
+                     "but function returned datatype int64")
+
+    with pytest.raises(TypeError, match=expected_expr):
+        pc.call_function(func_name, [pa.array([20, 30])])
+
+
+def test_wrong_signature(wrong_signature_func_fixture):
+    _, func_name = wrong_signature_func_fixture
+
+    expected_expr = (r"wrong_signature\(\) takes 0 positional arguments "
+                     "but 1 was given")
+
+    with pytest.raises(TypeError, match=expected_expr):
+        pc.call_function(func_name, [])
+
+
+def test_wrong_datatype_declaration():
+    def identity(ctx, val):
+        return val
+
+    func_name = "test_wrong_datatype_declaration"
+    in_types = {"array": pa.int64()}
+    out_type = {}
+    doc = {
+        "summary": "test output value",
+        "description": "test output"
+    }
+    with pytest.raises(TypeError,
+                       match="DataType expected, got <class 'dict'>"):
+        pc.register_scalar_function(identity, func_name,
+                                    doc, in_types, out_type)
+
+
+def test_wrong_input_type_declaration():
+    def identity(ctx, val):
+        return val
+
+    func_name = "test_wrong_input_type_declaration"
+    in_types = {"array": None}
+    out_type = pa.int64()
+    doc = {
+        "summary": "test invalid input type",
+        "description": "invalid input function"
+    }
+    with pytest.raises(TypeError,
+                       match="DataType expected, got <class 'NoneType'>"):
+        pc.register_scalar_function(identity, func_name, doc,
+                                    in_types, out_type)
+
+
+def test_udf_context(unary_func_fixture):
+    # Check the memory_pool argument is properly propagated
+    proxy_pool = pa.proxy_memory_pool(pa.default_memory_pool())
+    _, func_name = unary_func_fixture
+
+    res = pc.call_function(func_name,
+                           [pa.array([1] * 1000, type=pa.int64())],
+                           memory_pool=proxy_pool)
+    assert res == pa.array([2] * 1000, type=pa.int64())
+    assert proxy_pool.bytes_allocated() == 1000 * 8
+    # Destroying Python array should destroy underlying C++ memory
+    res = None
+    assert proxy_pool.bytes_allocated() == 0
+
+
+def test_raising_func(raising_func_fixture):
+    _, func_name = raising_func_fixture
+    with pytest.raises(MyError, match="error raised by scalar UDF"):
+        pc.call_function(func_name, [])
+
+
+def test_scalar_input(unary_func_fixture):
+    function, func_name = unary_func_fixture
+    res = pc.call_function(func_name, [pa.scalar(10)])
+    assert res == pa.scalar(11)
+
+
+def test_input_lifetime(unary_func_fixture):
+    function, func_name = unary_func_fixture
+
+    proxy_pool = pa.proxy_memory_pool(pa.default_memory_pool())
+    assert proxy_pool.bytes_allocated() == 0
+
+    v = pa.array([1] * 1000, type=pa.int64(), memory_pool=proxy_pool)
+    assert proxy_pool.bytes_allocated() == 1000 * 8
+    pc.call_function(func_name, [v])
+    assert proxy_pool.bytes_allocated() == 1000 * 8
+    # Calling a UDF should not have kept `v` alive longer than required
+    v = None
+    assert proxy_pool.bytes_allocated() == 0