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 2020/04/07 09:36:50 UTC

[arrow] branch master updated: ARROW-8347: [C++] Migrate Array methods to Result

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 cda6a4b  ARROW-8347: [C++] Migrate Array methods to Result<T>
cda6a4b is described below

commit cda6a4bd45275338603c0977160cc17382f23372
Author: Antoine Pitrou <an...@python.org>
AuthorDate: Tue Apr 7 11:36:31 2020 +0200

    ARROW-8347: [C++] Migrate Array methods to Result<T>
    
    Closes #6851 from pitrou/ARROW-8347-array-result-apis
    
    Authored-by: Antoine Pitrou <an...@python.org>
    Signed-off-by: Antoine Pitrou <an...@python.org>
---
 c_glib/arrow-glib/basic-array.cpp                 |   7 +-
 c_glib/arrow-glib/composite-array.cpp             |  83 ++++-----
 cpp/src/arrow/array.cc                            | 217 +++++++++++++---------
 cpp/src/arrow/array.h                             | 184 ++++++++++--------
 cpp/src/arrow/array/concatenate_test.cc           |   9 +-
 cpp/src/arrow/array/dict_internal.cc              |  41 ++--
 cpp/src/arrow/array/diff_test.cc                  |  16 +-
 cpp/src/arrow/array_dict_test.cc                  |  50 +++--
 cpp/src/arrow/array_list_test.cc                  |  29 ++-
 cpp/src/arrow/array_struct_test.cc                |   3 +-
 cpp/src/arrow/array_test.cc                       |   6 +-
 cpp/src/arrow/array_union_test.cc                 |  97 +++++-----
 cpp/src/arrow/array_view_test.cc                  |  48 ++---
 cpp/src/arrow/c/bridge_benchmark.cc               |   3 +-
 cpp/src/arrow/c/bridge_test.cc                    |  82 ++++----
 cpp/src/arrow/compute/kernels/cast_test.cc        |  25 +--
 cpp/src/arrow/compute/kernels/filter_test.cc      |  11 +-
 cpp/src/arrow/compute/kernels/take_test.cc        |  11 +-
 cpp/src/arrow/dataset/dataset_test.cc             |   3 +-
 cpp/src/arrow/dataset/filter.cc                   |  12 +-
 cpp/src/arrow/dataset/filter_test.cc              |   6 +-
 cpp/src/arrow/dataset/projector.cc                |  13 +-
 cpp/src/arrow/extension_type_test.cc              |   4 +-
 cpp/src/arrow/ipc/json_simple.cc                  |   4 +-
 cpp/src/arrow/ipc/json_simple_test.cc             |  68 +++----
 cpp/src/arrow/ipc/read_write_test.cc              |   4 +-
 cpp/src/arrow/json/converter.cc                   |   8 +-
 cpp/src/arrow/python/python_to_arrow.cc           |   2 +-
 cpp/src/arrow/table.cc                            |  49 ++---
 cpp/src/arrow/table.h                             |   8 +-
 cpp/src/arrow/table_test.cc                       |  14 +-
 cpp/src/arrow/testing/random.cc                   |   3 +-
 cpp/src/parquet/arrow/arrow_reader_writer_test.cc |  15 +-
 cpp/src/parquet/arrow/generate_fuzz_corpus.cc     |   4 +-
 cpp/src/parquet/arrow/reader_internal.cc          |   5 +-
 cpp/src/parquet/encoding_test.cc                  |   3 +-
 python/pyarrow/array.pxi                          |  36 ++--
 python/pyarrow/includes/libarrow.pxd              |  56 +++---
 python/pyarrow/table.pxi                          |   2 +-
 r/src/array.cpp                                   |   9 +-
 r/src/array_from_vector.cpp                       |   8 +-
 r/src/array_to_vector.cpp                         |   3 +-
 r/src/chunkedarray.cpp                            |   4 +-
 43 files changed, 654 insertions(+), 611 deletions(-)

diff --git a/c_glib/arrow-glib/basic-array.cpp b/c_glib/arrow-glib/basic-array.cpp
index d99dd82..8a60abd 100644
--- a/c_glib/arrow-glib/basic-array.cpp
+++ b/c_glib/arrow-glib/basic-array.cpp
@@ -547,10 +547,9 @@ garrow_array_view(GArrowArray *array,
 {
   auto arrow_array_raw = garrow_array_get_raw(array);
   auto arrow_return_type = garrow_data_type_get_raw(return_type);
-  std::shared_ptr<arrow::Array> arrow_array;
-  auto status = arrow_array_raw->View(arrow_return_type, &arrow_array);
-  if (garrow_error_check(error, status, "[array][view]")) {
-    return garrow_array_new_raw(&arrow_array);
+  auto arrow_array = arrow_array_raw->View(arrow_return_type);
+  if (garrow::check(error, arrow_array, "[array][view]")) {
+    return garrow_array_new_raw(&(*arrow_array));
   } else {
     return NULL;
   }
diff --git a/c_glib/arrow-glib/composite-array.cpp b/c_glib/arrow-glib/composite-array.cpp
index 0b21002..711bb51 100644
--- a/c_glib/arrow-glib/composite-array.cpp
+++ b/c_glib/arrow-glib/composite-array.cpp
@@ -360,14 +360,13 @@ garrow_struct_array_flatten(GArrowStructArray *array, GError **error)
     std::static_pointer_cast<arrow::StructArray>(arrow_array);
 
   auto memory_pool = arrow::default_memory_pool();
-  arrow::ArrayVector arrow_arrays;
-  auto status = arrow_struct_array->Flatten(memory_pool, &arrow_arrays);
-  if (!garrow_error_check(error, status, "[struct-array][flatten]")) {
+  auto arrow_arrays = arrow_struct_array->Flatten(memory_pool);
+  if (!garrow::check(error, arrow_arrays, "[struct-array][flatten]")) {
     return NULL;
   }
 
   GList *arrays = NULL;
-  for (auto arrow_array : arrow_arrays) {
+  for (auto arrow_array : *arrow_arrays) {
     auto array = garrow_array_new_raw(&arrow_array);
     arrays = g_list_prepend(arrays, array);
   }
@@ -411,15 +410,13 @@ garrow_map_array_new(GArrowArray *offsets,
   const auto arrow_offsets = garrow_array_get_raw(offsets);
   const auto arrow_keys = garrow_array_get_raw(keys);
   const auto arrow_items = garrow_array_get_raw(items);
-  std::shared_ptr<arrow::Array> arrow_array;
   auto arrow_memory_pool = arrow::default_memory_pool();
-  auto status = arrow::MapArray::FromArrays(arrow_offsets,
-                                            arrow_keys,
-                                            arrow_items,
-                                            arrow_memory_pool,
-                                            &arrow_array);
-  if (garrow::check(error, status, "[map-array][new]")) {
-    return GARROW_MAP_ARRAY(garrow_array_new_raw(&arrow_array));
+  auto arrow_array = arrow::MapArray::FromArrays(arrow_offsets,
+                                                 arrow_keys,
+                                                 arrow_items,
+                                                 arrow_memory_pool);
+  if (garrow::check(error, arrow_array, "[map-array][new]")) {
+    return GARROW_MAP_ARRAY(garrow_array_new_raw(&(*arrow_array)));
   } else {
     return NULL;
   }
@@ -543,12 +540,10 @@ garrow_sparse_union_array_new(GArrowInt8Array *type_ids,
     auto *field = GARROW_ARRAY(node->data);
     arrow_fields.push_back(garrow_array_get_raw(field));
   }
-  std::shared_ptr<arrow::Array> arrow_union_array;
-  auto status = arrow::UnionArray::MakeSparse(*arrow_type_ids,
-                                              arrow_fields,
-                                              &arrow_union_array);
-  if (garrow_error_check(error, status, "[sparse-union-array][new]")) {
-    return GARROW_SPARSE_UNION_ARRAY(garrow_array_new_raw(&arrow_union_array));
+  auto arrow_union_array = arrow::UnionArray::MakeSparse(*arrow_type_ids,
+                                                         arrow_fields);
+  if (garrow::check(error, arrow_union_array, "[sparse-union-array][new]")) {
+    return GARROW_SPARSE_UNION_ARRAY(garrow_array_new_raw(&(*arrow_union_array)));
   } else {
     return NULL;
   }
@@ -586,16 +581,11 @@ garrow_sparse_union_array_new_data_type(GArrowSparseUnionDataType *data_type,
     auto *field = GARROW_ARRAY(node->data);
     arrow_fields.push_back(garrow_array_get_raw(field));
   }
-  std::shared_ptr<arrow::Array> arrow_union_array;
-  auto status = arrow::UnionArray::MakeSparse(*arrow_type_ids,
-                                              arrow_fields,
-                                              arrow_field_names,
-                                              arrow_union_data_type->type_codes(),
-                                              &arrow_union_array);
-  if (garrow_error_check(error,
-                         status,
-                         "[sparse-union-array][new][data-type]")) {
-    return GARROW_SPARSE_UNION_ARRAY(garrow_array_new_raw(&arrow_union_array));
+  auto arrow_union_array = arrow::UnionArray::MakeSparse(
+    *arrow_type_ids, arrow_fields, arrow_field_names,
+    arrow_union_data_type->type_codes());
+  if (garrow::check(error, arrow_union_array, "[sparse-union-array][new][data-type]")) {
+    return GARROW_SPARSE_UNION_ARRAY(garrow_array_new_raw(&(*arrow_union_array)));
   } else {
     return NULL;
   }
@@ -643,13 +633,11 @@ garrow_dense_union_array_new(GArrowInt8Array *type_ids,
     auto *field = GARROW_ARRAY(node->data);
     arrow_fields.push_back(garrow_array_get_raw(field));
   }
-  std::shared_ptr<arrow::Array> arrow_union_array;
-  auto status = arrow::UnionArray::MakeDense(*arrow_type_ids,
-                                             *arrow_value_offsets,
-                                             arrow_fields,
-                                             &arrow_union_array);
-  if (garrow_error_check(error, status, "[dense-union-array][new]")) {
-    return GARROW_DENSE_UNION_ARRAY(garrow_array_new_raw(&arrow_union_array));
+  auto arrow_union_array = arrow::UnionArray::MakeDense(*arrow_type_ids,
+                                                        *arrow_value_offsets,
+                                                        arrow_fields);
+  if (garrow::check(error, arrow_union_array, "[dense-union-array][new]")) {
+    return GARROW_DENSE_UNION_ARRAY(garrow_array_new_raw(&(*arrow_union_array)));
   } else {
     return NULL;
   }
@@ -691,15 +679,11 @@ garrow_dense_union_array_new_data_type(GArrowDenseUnionDataType *data_type,
     auto *field = GARROW_ARRAY(node->data);
     arrow_fields.push_back(garrow_array_get_raw(field));
   }
-  std::shared_ptr<arrow::Array> arrow_union_array;
-  auto status = arrow::UnionArray::MakeDense(*arrow_type_ids,
-                                             *arrow_value_offsets,
-                                             arrow_fields,
-                                             arrow_field_names,
-                                             arrow_union_data_type->type_codes(),
-                                             &arrow_union_array);
-  if (garrow_error_check(error, status, "[dense-union-array][new][data-type]")) {
-    return GARROW_DENSE_UNION_ARRAY(garrow_array_new_raw(&arrow_union_array));
+  auto arrow_union_array = arrow::UnionArray::MakeDense(
+    *arrow_type_ids, *arrow_value_offsets, arrow_fields, arrow_field_names,
+    arrow_union_data_type->type_codes());
+  if (garrow::check(error, arrow_union_array, "[dense-union-array][new][data-type]")) {
+    return GARROW_DENSE_UNION_ARRAY(garrow_array_new_raw(&(*arrow_union_array)));
   } else {
     return NULL;
   }
@@ -741,14 +725,11 @@ garrow_dictionary_array_new(GArrowDataType *data_type,
   const auto arrow_data_type = garrow_data_type_get_raw(data_type);
   const auto arrow_indices = garrow_array_get_raw(indices);
   const auto arrow_dictionary = garrow_array_get_raw(dictionary);
-  std::shared_ptr<arrow::Array> arrow_dictionary_array;
-  auto status = arrow::DictionaryArray::FromArrays(arrow_data_type,
-                                                   arrow_indices,
-                                                   arrow_dictionary,
-                                                   &arrow_dictionary_array);
-  if (garrow_error_check(error, status, "[dictionary-array][new]")) {
+  auto arrow_dictionary_array = arrow::DictionaryArray::FromArrays(
+    arrow_data_type, arrow_indices, arrow_dictionary);
+  if (garrow::check(error, arrow_dictionary_array, "[dictionary-array][new]")) {
     auto arrow_array =
-      std::static_pointer_cast<arrow::Array>(arrow_dictionary_array);
+      std::static_pointer_cast<arrow::Array>(*arrow_dictionary_array);
     return GARROW_DICTIONARY_ARRAY(garrow_array_new_raw(&arrow_array));
   } else {
     return NULL;
diff --git a/cpp/src/arrow/array.cc b/cpp/src/arrow/array.cc
index d9c4a54..0671713 100644
--- a/cpp/src/arrow/array.cc
+++ b/cpp/src/arrow/array.cc
@@ -287,8 +287,9 @@ Status CleanListOffsets(const Array& offsets, MemoryPool* pool,
 }
 
 template <typename TYPE>
-Status ListArrayFromArrays(const Array& offsets, const Array& values, MemoryPool* pool,
-                           std::shared_ptr<Array>* out) {
+Result<std::shared_ptr<Array>> ListArrayFromArrays(const Array& offsets,
+                                                   const Array& values,
+                                                   MemoryPool* pool) {
   using offset_type = typename TYPE::offset_type;
   using ArrayType = typename TypeTraits<TYPE>::ArrayType;
   using OffsetArrowType = typename CTypeTraits<offset_type>::ArrowType;
@@ -311,8 +312,7 @@ Status ListArrayFromArrays(const Array& offsets, const Array& values, MemoryPool
                       offsets.null_count(), offsets.offset());
   internal_data->child_data.push_back(values.data());
 
-  *out = std::make_shared<ArrayType>(internal_data);
-  return Status::OK();
+  return std::make_shared<ArrayType>(internal_data);
 }
 
 static std::shared_ptr<Array> SliceArrayWithOffsets(const Array& array, int64_t begin,
@@ -426,14 +426,26 @@ void LargeListArray::SetData(const std::shared_ptr<ArrayData>& data) {
   values_ = MakeArray(data_->child_data[0]);
 }
 
+Result<std::shared_ptr<Array>> ListArray::FromArrays(const Array& offsets,
+                                                     const Array& values,
+                                                     MemoryPool* pool) {
+  return ListArrayFromArrays<ListType>(offsets, values, pool);
+}
+
+Result<std::shared_ptr<Array>> LargeListArray::FromArrays(const Array& offsets,
+                                                          const Array& values,
+                                                          MemoryPool* pool) {
+  return ListArrayFromArrays<LargeListType>(offsets, values, pool);
+}
+
 Status ListArray::FromArrays(const Array& offsets, const Array& values, MemoryPool* pool,
                              std::shared_ptr<Array>* out) {
-  return ListArrayFromArrays<ListType>(offsets, values, pool, out);
+  return FromArrays(offsets, values, pool).Value(out);
 }
 
 Status LargeListArray::FromArrays(const Array& offsets, const Array& values,
                                   MemoryPool* pool, std::shared_ptr<Array>* out) {
-  return ListArrayFromArrays<LargeListType>(offsets, values, pool, out);
+  return FromArrays(offsets, values, pool).Value(out);
 }
 
 Result<std::shared_ptr<Array>> ListArray::Flatten(MemoryPool* memory_pool) const {
@@ -470,10 +482,10 @@ MapArray::MapArray(const std::shared_ptr<DataType>& type, int64_t length,
   SetData(map_data);
 }
 
-Status MapArray::FromArrays(const std::shared_ptr<Array>& offsets,
-                            const std::shared_ptr<Array>& keys,
-                            const std::shared_ptr<Array>& items, MemoryPool* pool,
-                            std::shared_ptr<Array>* out) {
+Result<std::shared_ptr<Array>> MapArray::FromArrays(const std::shared_ptr<Array>& offsets,
+                                                    const std::shared_ptr<Array>& keys,
+                                                    const std::shared_ptr<Array>& items,
+                                                    MemoryPool* pool) {
   using offset_type = typename MapType::offset_type;
   using OffsetArrowType = typename CTypeTraits<offset_type>::ArrowType;
 
@@ -497,10 +509,16 @@ Status MapArray::FromArrays(const std::shared_ptr<Array>& offsets,
   RETURN_NOT_OK(CleanListOffsets<MapType>(*offsets, pool, &offset_buf, &validity_buf));
 
   auto map_type = std::make_shared<MapType>(keys->type(), items->type());
-  *out =
-      std::make_shared<MapArray>(map_type, offsets->length() - 1, offset_buf, keys, items,
-                                 validity_buf, offsets->null_count(), offsets->offset());
-  return Status::OK();
+  return std::make_shared<MapArray>(map_type, offsets->length() - 1, offset_buf, keys,
+                                    items, validity_buf, offsets->null_count(),
+                                    offsets->offset());
+}
+
+Status MapArray::FromArrays(const std::shared_ptr<Array>& offsets,
+                            const std::shared_ptr<Array>& keys,
+                            const std::shared_ptr<Array>& items, MemoryPool* pool,
+                            std::shared_ptr<Array>* out) {
+  return FromArrays(offsets, keys, items, pool).Value(out);
 }
 
 Status MapArray::ValidateChildData(
@@ -791,7 +809,7 @@ std::shared_ptr<Array> StructArray::GetFieldByName(const std::string& name) cons
   return i == -1 ? nullptr : field(i);
 }
 
-Status StructArray::Flatten(MemoryPool* pool, ArrayVector* out) const {
+Result<ArrayVector> StructArray::Flatten(MemoryPool* pool) const {
   ArrayVector flattened;
   flattened.reserve(data_->child_data.size());
   std::shared_ptr<Buffer> null_bitmap = data_->buffers[0];
@@ -839,8 +857,11 @@ Status StructArray::Flatten(MemoryPool* pool, ArrayVector* out) const {
     flattened.push_back(MakeArray(flattened_data));
   }
 
-  *out = flattened;
-  return Status::OK();
+  return flattened;
+}
+
+Status StructArray::Flatten(MemoryPool* pool, ArrayVector* out) const {
+  return Flatten(pool).Value(out);
 }
 
 // ----------------------------------------------------------------------
@@ -880,11 +901,10 @@ UnionArray::UnionArray(const std::shared_ptr<DataType>& type, int64_t length,
   SetData(internal_data);
 }
 
-Status UnionArray::MakeDense(const Array& type_ids, const Array& value_offsets,
-                             const std::vector<std::shared_ptr<Array>>& children,
-                             const std::vector<std::string>& field_names,
-                             const std::vector<int8_t>& type_codes,
-                             std::shared_ptr<Array>* out) {
+Result<std::shared_ptr<Array>> UnionArray::MakeDense(
+    const Array& type_ids, const Array& value_offsets,
+    const std::vector<std::shared_ptr<Array>>& children,
+    const std::vector<std::string>& field_names, const std::vector<int8_t>& type_codes) {
   if (value_offsets.length() == 0) {
     return Status::Invalid("UnionArray offsets must have non-zero length");
   }
@@ -920,15 +940,12 @@ Status UnionArray::MakeDense(const Array& type_ids, const Array& value_offsets,
   for (const auto& child : children) {
     internal_data->child_data.push_back(child->data());
   }
-  *out = std::make_shared<UnionArray>(internal_data);
-  return Status::OK();
+  return std::make_shared<UnionArray>(internal_data);
 }
 
-Status UnionArray::MakeSparse(const Array& type_ids,
-                              const std::vector<std::shared_ptr<Array>>& children,
-                              const std::vector<std::string>& field_names,
-                              const std::vector<int8_t>& type_codes,
-                              std::shared_ptr<Array>* out) {
+Result<std::shared_ptr<Array>> UnionArray::MakeSparse(
+    const Array& type_ids, const std::vector<std::shared_ptr<Array>>& children,
+    const std::vector<std::string>& field_names, const std::vector<int8_t>& type_codes) {
   if (type_ids.type_id() != Type::INT8) {
     return Status::TypeError("UnionArray type_ids must be signed int8");
   }
@@ -954,8 +971,7 @@ Status UnionArray::MakeSparse(const Array& type_ids,
           "Sparse UnionArray must have len(child) == len(type_ids) for all children");
     }
   }
-  *out = std::make_shared<UnionArray>(internal_data);
-  return Status::OK();
+  return std::make_shared<UnionArray>(internal_data);
 }
 
 std::shared_ptr<Array> UnionArray::child(int i) const {
@@ -1045,10 +1061,9 @@ DictionaryArray::DictionaryArray(const std::shared_ptr<DataType>& type,
 
 std::shared_ptr<Array> DictionaryArray::dictionary() const { return data_->dictionary; }
 
-Status DictionaryArray::FromArrays(const std::shared_ptr<DataType>& type,
-                                   const std::shared_ptr<Array>& indices,
-                                   const std::shared_ptr<Array>& dictionary,
-                                   std::shared_ptr<Array>* out) {
+Result<std::shared_ptr<Array>> DictionaryArray::FromArrays(
+    const std::shared_ptr<DataType>& type, const std::shared_ptr<Array>& indices,
+    const std::shared_ptr<Array>& dictionary) {
   if (type->id() != Type::DICTIONARY) {
     return Status::TypeError("Expected a dictionary type");
   }
@@ -1076,8 +1091,14 @@ Status DictionaryArray::FromArrays(const std::shared_ptr<DataType>& type,
                                     indices->type()->ToString());
   }
   RETURN_NOT_OK(is_valid);
-  *out = std::make_shared<DictionaryArray>(type, indices, dictionary);
-  return Status::OK();
+  return std::make_shared<DictionaryArray>(type, indices, dictionary);
+}
+
+Status DictionaryArray::FromArrays(const std::shared_ptr<DataType>& type,
+                                   const std::shared_ptr<Array>& indices,
+                                   const std::shared_ptr<Array>& dictionary,
+                                   std::shared_ptr<Array>* out) {
+  return FromArrays(type, indices, dictionary).Value(out);
 }
 
 bool DictionaryArray::CanCompareIndices(const DictionaryArray& other) const {
@@ -1167,12 +1188,12 @@ struct ViewDataImpl {
     return Status::OK();
   }
 
-  Status GetDictionaryView(const DataType& out_type, std::shared_ptr<Array>* out) {
+  Result<std::shared_ptr<Array>> GetDictionaryView(const DataType& out_type) {
     if (in_data[in_layout_idx]->type->id() != Type::DICTIONARY) {
       return InvalidView("Cannot get view as dictionary type");
     }
     const auto& dict_out_type = static_cast<const DictionaryType&>(out_type);
-    return in_data[in_layout_idx]->dictionary->View(dict_out_type.value_type(), out);
+    return in_data[in_layout_idx]->dictionary->View(dict_out_type.value_type());
   }
 
   Status MakeDataView(const std::shared_ptr<Field>& out_field,
@@ -1187,7 +1208,7 @@ struct ViewDataImpl {
 
     std::shared_ptr<Array> dictionary;
     if (out_type->id() == Type::DICTIONARY) {
-      RETURN_NOT_OK(GetDictionaryView(*out_type, &dictionary));
+      ARROW_ASSIGN_OR_RAISE(dictionary, GetDictionaryView(*out_type));
     }
 
     // No type has a purely empty layout
@@ -1276,8 +1297,8 @@ struct ViewDataImpl {
 
 }  // namespace
 
-Status Array::View(const std::shared_ptr<DataType>& out_type,
-                   std::shared_ptr<Array>* out) const {
+Result<std::shared_ptr<Array>> Array::View(
+    const std::shared_ptr<DataType>& out_type) const {
   ViewDataImpl impl;
   impl.root_in_type = data_->type;
   impl.root_out_type = out_type;
@@ -1290,8 +1311,12 @@ Status Array::View(const std::shared_ptr<DataType>& out_type,
   auto out_field = field("", out_type);
   RETURN_NOT_OK(impl.MakeDataView(out_field, &out_data));
   RETURN_NOT_OK(impl.CheckInputExhausted());
-  *out = MakeArray(out_data);
-  return Status::OK();
+  return MakeArray(out_data);
+}
+
+Status Array::View(const std::shared_ptr<DataType>& out_type,
+                   std::shared_ptr<Array>* out) const {
+  return View(out_type).Value(out);
 }
 
 // ----------------------------------------------------------------------
@@ -1439,8 +1464,8 @@ class NullArrayFactory {
   };
 
   NullArrayFactory(MemoryPool* pool, const std::shared_ptr<DataType>& type,
-                   int64_t length, std::shared_ptr<ArrayData>* out)
-      : pool_(pool), type_(type), length_(length), out_(out) {}
+                   int64_t length)
+      : pool_(pool), type_(type), length_(length) {}
 
   Status CreateBuffer() {
     ARROW_ASSIGN_OR_RAISE(int64_t buffer_length,
@@ -1450,71 +1475,74 @@ class NullArrayFactory {
     return Status::OK();
   }
 
-  Status Create() {
+  Result<std::shared_ptr<ArrayData>> Create() {
     if (buffer_ == nullptr) {
       RETURN_NOT_OK(CreateBuffer());
     }
     std::vector<std::shared_ptr<ArrayData>> child_data(type_->num_children());
-    *out_ = ArrayData::Make(type_, length_, {buffer_}, child_data, length_, 0);
-    return VisitTypeInline(*type_, this);
+    out_ = ArrayData::Make(type_, length_, {buffer_}, child_data, length_, 0);
+    RETURN_NOT_OK(VisitTypeInline(*type_, this));
+    return out_;
   }
 
   Status Visit(const NullType&) { return Status::OK(); }
 
   Status Visit(const FixedWidthType&) {
-    (*out_)->buffers.resize(2, buffer_);
+    out_->buffers.resize(2, buffer_);
     return Status::OK();
   }
 
   template <typename T>
   enable_if_base_binary<T, Status> Visit(const T&) {
-    (*out_)->buffers.resize(3, buffer_);
+    out_->buffers.resize(3, buffer_);
     return Status::OK();
   }
 
   template <typename T>
   enable_if_var_size_list<T, Status> Visit(const T& type) {
-    (*out_)->buffers.resize(2, buffer_);
-    return CreateChild(0, length_, &(*out_)->child_data[0]);
+    out_->buffers.resize(2, buffer_);
+    ARROW_ASSIGN_OR_RAISE(out_->child_data[0], CreateChild(0, length_));
+    return Status::OK();
   }
 
   Status Visit(const FixedSizeListType& type) {
-    return CreateChild(0, length_ * type.list_size(), &(*out_)->child_data[0]);
+    ARROW_ASSIGN_OR_RAISE(out_->child_data[0],
+                          CreateChild(0, length_ * type.list_size()));
+    return Status::OK();
   }
 
   Status Visit(const StructType& type) {
     for (int i = 0; i < type_->num_children(); ++i) {
-      RETURN_NOT_OK(CreateChild(i, length_, &(*out_)->child_data[i]));
+      ARROW_ASSIGN_OR_RAISE(out_->child_data[i], CreateChild(i, length_));
     }
     return Status::OK();
   }
 
   Status Visit(const UnionType& type) {
     if (type.mode() == UnionMode::DENSE) {
-      (*out_)->buffers.resize(3, buffer_);
+      out_->buffers.resize(3, buffer_);
     } else {
-      (*out_)->buffers = {buffer_, buffer_, nullptr};
+      out_->buffers = {buffer_, buffer_, nullptr};
     }
 
     for (int i = 0; i < type_->num_children(); ++i) {
-      RETURN_NOT_OK(CreateChild(i, length_, &(*out_)->child_data[i]));
+      ARROW_ASSIGN_OR_RAISE(out_->child_data[i], CreateChild(i, length_));
     }
     return Status::OK();
   }
 
   Status Visit(const DictionaryType& type) {
-    (*out_)->buffers.resize(2, buffer_);
-    std::shared_ptr<ArrayData> dictionary_data;
-    return MakeArrayOfNull(type.value_type(), 0, &(*out_)->dictionary);
+    out_->buffers.resize(2, buffer_);
+    ARROW_ASSIGN_OR_RAISE(out_->dictionary, MakeArrayOfNull(type.value_type(), 0));
+    return Status::OK();
   }
 
   Status Visit(const DataType& type) {
     return Status::NotImplemented("construction of all-null ", type);
   }
 
-  Status CreateChild(int i, int64_t length, std::shared_ptr<ArrayData>* out) {
-    NullArrayFactory child_factory(pool_, type_->child(i)->type(), length,
-                                   &(*out_)->child_data[i]);
+  Result<std::shared_ptr<ArrayData>> CreateChild(int i, int64_t length) {
+    NullArrayFactory child_factory(pool_, type_->child(i)->type(), length);
     child_factory.buffer_ = buffer_;
     return child_factory.Create();
   }
@@ -1522,17 +1550,19 @@ class NullArrayFactory {
   MemoryPool* pool_;
   std::shared_ptr<DataType> type_;
   int64_t length_;
-  std::shared_ptr<ArrayData>* out_;
+  std::shared_ptr<ArrayData> out_;
   std::shared_ptr<Buffer> buffer_;
 };
 
 class RepeatedArrayFactory {
  public:
-  RepeatedArrayFactory(MemoryPool* pool, const Scalar& scalar, int64_t length,
-                       std::shared_ptr<Array>* out)
-      : pool_(pool), scalar_(scalar), length_(length), out_(out) {}
+  RepeatedArrayFactory(MemoryPool* pool, const Scalar& scalar, int64_t length)
+      : pool_(pool), scalar_(scalar), length_(length) {}
 
-  Status Create() { return VisitTypeInline(*scalar_.type, this); }
+  Result<std::shared_ptr<Array>> Create() {
+    RETURN_NOT_OK(VisitTypeInline(*scalar_.type, this));
+    return out_;
+  }
 
   Status Visit(const NullType&) { return Status::OK(); }
 
@@ -1541,7 +1571,7 @@ class RepeatedArrayFactory {
     RETURN_NOT_OK(AllocateBitmap(pool_, length_, &buffer));
     BitUtil::SetBitsTo(buffer->mutable_data(), 0, length_,
                        checked_cast<const BooleanScalar&>(scalar_).value);
-    *out_ = std::make_shared<BooleanArray>(length_, buffer);
+    out_ = std::make_shared<BooleanArray>(length_, buffer);
     return Status::OK();
   }
 
@@ -1559,8 +1589,8 @@ class RepeatedArrayFactory {
     RETURN_NOT_OK(CreateBufferOf(value->data(), value->size(), &values_buffer));
     auto size = static_cast<typename T::offset_type>(value->size());
     RETURN_NOT_OK(CreateOffsetsBuffer(size, &offsets_buffer));
-    *out_ = std::make_shared<typename TypeTraits<T>::ArrayType>(length_, offsets_buffer,
-                                                                values_buffer);
+    out_ = std::make_shared<typename TypeTraits<T>::ArrayType>(length_, offsets_buffer,
+                                                               values_buffer);
     return Status::OK();
   }
 
@@ -1576,16 +1606,14 @@ class RepeatedArrayFactory {
   }
 
   Status Visit(const DictionaryType& type) {
-    std::shared_ptr<Array> dictionary, indices;
-
     const auto& value = checked_cast<const DictionaryScalar&>(scalar_).value;
-    RETURN_NOT_OK(MakeArrayFromScalar(pool_, *value, 1, &dictionary));
+    ARROW_ASSIGN_OR_RAISE(auto dictionary, MakeArrayFromScalar(*value, 1, pool_));
 
     ARROW_ASSIGN_OR_RAISE(auto zero, MakeScalar(type.index_type(), 0));
-    RETURN_NOT_OK(MakeArrayFromScalar(pool_, *zero, length_, &indices));
+    ARROW_ASSIGN_OR_RAISE(auto indices, MakeArrayFromScalar(*zero, length_, pool_));
 
-    *out_ = std::make_shared<DictionaryArray>(scalar_.type, std::move(indices),
-                                              std::move(dictionary));
+    out_ = std::make_shared<DictionaryArray>(scalar_.type, std::move(indices),
+                                             std::move(dictionary));
     return Status::OK();
   }
 
@@ -1617,7 +1645,7 @@ class RepeatedArrayFactory {
   Status FinishFixedWidth(const void* data, size_t data_length) {
     std::shared_ptr<Buffer> buffer;
     RETURN_NOT_OK(CreateBufferOf(data, data_length, &buffer));
-    *out_ = MakeArray(
+    out_ = MakeArray(
         ArrayData::Make(scalar_.type, length_, {nullptr, std::move(buffer)}, 0));
     return Status::OK();
   }
@@ -1625,35 +1653,44 @@ class RepeatedArrayFactory {
   MemoryPool* pool_;
   const Scalar& scalar_;
   int64_t length_;
-  std::shared_ptr<Array>* out_;
+  std::shared_ptr<Array> out_;
 };
 
 }  // namespace internal
 
+Result<std::shared_ptr<Array>> MakeArrayOfNull(const std::shared_ptr<DataType>& type,
+                                               int64_t length, MemoryPool* pool) {
+  ARROW_ASSIGN_OR_RAISE(auto data,
+                        internal::NullArrayFactory(pool, type, length).Create());
+  return MakeArray(data);
+}
+
+Result<std::shared_ptr<Array>> MakeArrayFromScalar(const Scalar& scalar, int64_t length,
+                                                   MemoryPool* pool) {
+  if (!scalar.is_valid) {
+    return MakeArrayOfNull(scalar.type, length, pool);
+  }
+  return internal::RepeatedArrayFactory(pool, scalar, length).Create();
+}
+
 Status MakeArrayOfNull(MemoryPool* pool, const std::shared_ptr<DataType>& type,
                        int64_t length, std::shared_ptr<Array>* out) {
-  std::shared_ptr<ArrayData> out_data;
-  RETURN_NOT_OK(internal::NullArrayFactory(pool, type, length, &out_data).Create());
-  *out = MakeArray(out_data);
-  return Status::OK();
+  return MakeArrayOfNull(type, length, pool).Value(out);
 }
 
 Status MakeArrayOfNull(const std::shared_ptr<DataType>& type, int64_t length,
                        std::shared_ptr<Array>* out) {
-  return MakeArrayOfNull(default_memory_pool(), type, length, out);
+  return MakeArrayOfNull(type, length).Value(out);
 }
 
 Status MakeArrayFromScalar(MemoryPool* pool, const Scalar& scalar, int64_t length,
                            std::shared_ptr<Array>* out) {
-  if (!scalar.is_valid) {
-    return MakeArrayOfNull(pool, scalar.type, length, out);
-  }
-  return internal::RepeatedArrayFactory(pool, scalar, length, out).Create();
+  return MakeArrayFromScalar(scalar, length, pool).Value(out);
 }
 
 Status MakeArrayFromScalar(const Scalar& scalar, int64_t length,
                            std::shared_ptr<Array>* out) {
-  return MakeArrayFromScalar(default_memory_pool(), scalar, length, out);
+  return MakeArrayFromScalar(scalar, length).Value(out);
 }
 
 namespace internal {
diff --git a/cpp/src/arrow/array.h b/cpp/src/arrow/array.h
index 69f7e4e..1d2ad0f 100644
--- a/cpp/src/arrow/array.h
+++ b/cpp/src/arrow/array.h
@@ -240,7 +240,25 @@ std::shared_ptr<Array> MakeArray(const std::shared_ptr<ArrayData>& data);
 /// \brief Create a strongly-typed Array instance with all elements null
 /// \param[in] type the array type
 /// \param[in] length the array length
+/// \param[in] pool the memory pool to allocate memory from
+ARROW_EXPORT
+Result<std::shared_ptr<Array>> MakeArrayOfNull(
+    const std::shared_ptr<DataType>& type, int64_t length,
+    MemoryPool* pool ARROW_MEMORY_POOL_DEFAULT);
+
+/// \brief Create an Array instance whose slots are the given scalar
+/// \param[in] scalar the value with which to fill the array
+/// \param[in] length the array length
+/// \param[in] pool the memory pool to allocate memory from
+ARROW_EXPORT
+Result<std::shared_ptr<Array>> MakeArrayFromScalar(
+    const Scalar& scalar, int64_t length, MemoryPool* pool ARROW_MEMORY_POOL_DEFAULT);
+
+/// \brief Create a strongly-typed Array instance with all elements null
+/// \param[in] type the array type
+/// \param[in] length the array length
 /// \param[out] out resulting Array instance
+ARROW_DEPRECATED("Use Result-returning version")
 ARROW_EXPORT
 Status MakeArrayOfNull(const std::shared_ptr<DataType>& type, int64_t length,
                        std::shared_ptr<Array>* out);
@@ -250,6 +268,7 @@ Status MakeArrayOfNull(const std::shared_ptr<DataType>& type, int64_t length,
 /// \param[in] type the array type
 /// \param[in] length the array length
 /// \param[out] out resulting Array instance
+ARROW_DEPRECATED("Use Result-returning version")
 ARROW_EXPORT
 Status MakeArrayOfNull(MemoryPool* pool, const std::shared_ptr<DataType>& type,
                        int64_t length, std::shared_ptr<Array>* out);
@@ -258,6 +277,7 @@ Status MakeArrayOfNull(MemoryPool* pool, const std::shared_ptr<DataType>& type,
 /// \param[in] scalar the value with which to fill the array
 /// \param[in] length the array length
 /// \param[out] out resulting Array instance
+ARROW_DEPRECATED("Use Result-returning version")
 ARROW_EXPORT
 Status MakeArrayFromScalar(const Scalar& scalar, int64_t length,
                            std::shared_ptr<Array>* out);
@@ -267,6 +287,7 @@ Status MakeArrayFromScalar(const Scalar& scalar, int64_t length,
 /// \param[in] scalar the value with which to fill the array
 /// \param[in] length the array length
 /// \param[out] out resulting Array instance
+ARROW_DEPRECATED("Use Result-returning version")
 ARROW_EXPORT
 Status MakeArrayFromScalar(MemoryPool* pool, const Scalar& scalar, int64_t length,
                            std::shared_ptr<Array>* out);
@@ -366,6 +387,9 @@ class ARROW_EXPORT Array {
   /// Nested types are traversed in depth-first order. Data buffers must have
   /// the same item sizes, even though the logical types may be different.
   /// An error is returned if the types are not layout-compatible.
+  Result<std::shared_ptr<Array>> View(const std::shared_ptr<DataType>& type) const;
+
+  ARROW_DEPRECATED("Use Result-returning version")
   Status View(const std::shared_ptr<DataType>& type, std::shared_ptr<Array>* out) const;
 
   /// Construct a zero-copy slice of the array with the indicated offset and
@@ -611,7 +635,11 @@ class ARROW_EXPORT ListArray : public BaseListArray<ListType> {
   /// \param[in] values Array containing list values
   /// \param[in] pool MemoryPool in case new offsets array needs to be
   /// allocated because of null values
-  /// \param[out] out Will have length equal to offsets.length() - 1
+  static Result<std::shared_ptr<Array>> FromArrays(
+      const Array& offsets, const Array& values,
+      MemoryPool* pool ARROW_MEMORY_POOL_DEFAULT);
+
+  ARROW_DEPRECATED("Use Result-returning version")
   static Status FromArrays(const Array& offsets, const Array& values, MemoryPool* pool,
                            std::shared_ptr<Array>* out);
 
@@ -653,7 +681,11 @@ class ARROW_EXPORT LargeListArray : public BaseListArray<LargeListType> {
   /// \param[in] values Array containing list values
   /// \param[in] pool MemoryPool in case new offsets array needs to be
   /// allocated because of null values
-  /// \param[out] out Will have length equal to offsets.length() - 1
+  static Result<std::shared_ptr<Array>> FromArrays(
+      const Array& offsets, const Array& values,
+      MemoryPool* pool ARROW_MEMORY_POOL_DEFAULT);
+
+  ARROW_DEPRECATED("Use Result-returning version")
   static Status FromArrays(const Array& offsets, const Array& values, MemoryPool* pool,
                            std::shared_ptr<Array>* out);
 
@@ -706,7 +738,11 @@ class ARROW_EXPORT MapArray : public ListArray {
   /// \param[in] items Array containing item values
   /// \param[in] pool MemoryPool in case new offsets array needs to be
   /// allocated because of null values
-  /// \param[out] out Will have length equal to offsets.length() - 1
+  static Result<std::shared_ptr<Array>> FromArrays(
+      const std::shared_ptr<Array>& offsets, const std::shared_ptr<Array>& keys,
+      const std::shared_ptr<Array>& items, MemoryPool* pool ARROW_MEMORY_POOL_DEFAULT);
+
+  ARROW_DEPRECATED("Use Result-returning version")
   static Status FromArrays(const std::shared_ptr<Array>& offsets,
                            const std::shared_ptr<Array>& keys,
                            const std::shared_ptr<Array>& items, MemoryPool* pool,
@@ -1051,7 +1087,9 @@ class ARROW_EXPORT StructArray : public Array {
   /// \brief Flatten this array as a vector of arrays, one for each field
   ///
   /// \param[in] pool The pool to allocate null bitmaps from, if necessary
-  /// \param[out] out The resulting vector of arrays
+  Result<ArrayVector> Flatten(MemoryPool* pool ARROW_MEMORY_POOL_DEFAULT) const;
+
+  ARROW_DEPRECATED("Use Result-returning version")
   Status Flatten(MemoryPool* pool, ArrayVector* out) const;
 
  private:
@@ -1091,12 +1129,11 @@ class ARROW_EXPORT UnionArray : public Array {
   /// \param[in] children Vector of children Arrays containing the data for each type.
   /// \param[in] field_names Vector of strings containing the name of each field.
   /// \param[in] type_codes Vector of type codes.
-  /// \param[out] out Will have length equal to value_offsets.length()
-  static Status MakeDense(const Array& type_ids, const Array& value_offsets,
-                          const std::vector<std::shared_ptr<Array>>& children,
-                          const std::vector<std::string>& field_names,
-                          const std::vector<type_code_t>& type_codes,
-                          std::shared_ptr<Array>* out);
+  static Result<std::shared_ptr<Array>> MakeDense(
+      const Array& type_ids, const Array& value_offsets,
+      const std::vector<std::shared_ptr<Array>>& children,
+      const std::vector<std::string>& field_names = {},
+      const std::vector<type_code_t>& type_codes = {});
 
   /// \brief Construct Dense UnionArray from types_ids, value_offsets and children
   ///
@@ -1108,51 +1145,46 @@ class ARROW_EXPORT UnionArray : public Array {
   /// relative offset into the respective child array for the type in a given slot.
   /// The respective offsets for each child value array must be in order / increasing.
   /// \param[in] children Vector of children Arrays containing the data for each type.
-  /// \param[in] field_names Vector of strings containing the name of each field.
-  /// \param[out] out Will have length equal to value_offsets.length()
+  /// \param[in] type_codes Vector of type codes.
+  static Result<std::shared_ptr<Array>> MakeDense(
+      const Array& type_ids, const Array& value_offsets,
+      const std::vector<std::shared_ptr<Array>>& children,
+      const std::vector<type_code_t>& type_codes) {
+    return MakeDense(type_ids, value_offsets, children, std::vector<std::string>{},
+                     type_codes);
+  }
+
+  ARROW_DEPRECATED("Use Result-returning version")
   static Status MakeDense(const Array& type_ids, const Array& value_offsets,
                           const std::vector<std::shared_ptr<Array>>& children,
                           const std::vector<std::string>& field_names,
+                          const std::vector<type_code_t>& type_codes,
                           std::shared_ptr<Array>* out) {
-    return MakeDense(type_ids, value_offsets, children, field_names, {}, out);
+    return MakeDense(type_ids, value_offsets, children, field_names, type_codes)
+        .Value(out);
   }
 
-  /// \brief Construct Dense UnionArray from types_ids, value_offsets and children
-  ///
-  /// This function does the bare minimum of validation of the offsets and
-  /// input types. The value_offsets are assumed to be well-formed.
-  ///
-  /// \param[in] type_ids An array of logical type ids for the union type
-  /// \param[in] value_offsets An array of signed int32 values indicating the
-  /// relative offset into the respective child array for the type in a given slot.
-  /// The respective offsets for each child value array must be in order / increasing.
-  /// \param[in] children Vector of children Arrays containing the data for each type.
-  /// \param[in] type_codes Vector of type codes.
-  /// \param[out] out Will have length equal to value_offsets.length()
+  ARROW_DEPRECATED("Use Result-returning version")
+  static Status MakeDense(const Array& type_ids, const Array& value_offsets,
+                          const std::vector<std::shared_ptr<Array>>& children,
+                          const std::vector<std::string>& field_names,
+                          std::shared_ptr<Array>* out) {
+    return MakeDense(type_ids, value_offsets, children, field_names).Value(out);
+  }
+
+  ARROW_DEPRECATED("Use Result-returning version")
   static Status MakeDense(const Array& type_ids, const Array& value_offsets,
                           const std::vector<std::shared_ptr<Array>>& children,
                           const std::vector<type_code_t>& type_codes,
                           std::shared_ptr<Array>* out) {
-    return MakeDense(type_ids, value_offsets, children, {}, type_codes, out);
+    return MakeDense(type_ids, value_offsets, children, type_codes).Value(out);
   }
 
-  /// \brief Construct Dense UnionArray from types_ids, value_offsets and children
-  ///
-  /// This function does the bare minimum of validation of the offsets and
-  /// input types. The value_offsets are assumed to be well-formed.
-  ///
-  /// The name of each field is filled by the index of the field.
-  ///
-  /// \param[in] type_ids An array of logical type ids for the union type
-  /// \param[in] value_offsets An array of signed int32 values indicating the
-  /// relative offset into the respective child array for the type in a given slot.
-  /// The respective offsets for each child value array must be in order / increasing.
-  /// \param[in] children Vector of children Arrays containing the data for each type.
-  /// \param[out] out Will have length equal to value_offsets.length()
+  ARROW_DEPRECATED("Use Result-returning version")
   static Status MakeDense(const Array& type_ids, const Array& value_offsets,
                           const std::vector<std::shared_ptr<Array>>& children,
                           std::shared_ptr<Array>* out) {
-    return MakeDense(type_ids, value_offsets, children, {}, {}, out);
+    return MakeDense(type_ids, value_offsets, children).Value(out);
   }
 
   /// \brief Construct Sparse UnionArray from type_ids and children
@@ -1164,12 +1196,10 @@ class ARROW_EXPORT UnionArray : public Array {
   /// \param[in] children Vector of children Arrays containing the data for each type.
   /// \param[in] field_names Vector of strings containing the name of each field.
   /// \param[in] type_codes Vector of type codes.
-  /// \param[out] out Will have length equal to type_ids.length()
-  static Status MakeSparse(const Array& type_ids,
-                           const std::vector<std::shared_ptr<Array>>& children,
-                           const std::vector<std::string>& field_names,
-                           const std::vector<type_code_t>& type_codes,
-                           std::shared_ptr<Array>* out);
+  static Result<std::shared_ptr<Array>> MakeSparse(
+      const Array& type_ids, const std::vector<std::shared_ptr<Array>>& children,
+      const std::vector<std::string>& field_names = {},
+      const std::vector<type_code_t>& type_codes = {});
 
   /// \brief Construct Sparse UnionArray from type_ids and children
   ///
@@ -1178,45 +1208,43 @@ class ARROW_EXPORT UnionArray : public Array {
   ///
   /// \param[in] type_ids An array of logical type ids for the union type
   /// \param[in] children Vector of children Arrays containing the data for each type.
-  /// \param[in] field_names Vector of strings containing the name of each field.
-  /// \param[out] out Will have length equal to type_ids.length()
+  /// \param[in] type_codes Vector of type codes.
+  static Result<std::shared_ptr<Array>> MakeSparse(
+      const Array& type_ids, const std::vector<std::shared_ptr<Array>>& children,
+      const std::vector<type_code_t>& type_codes) {
+    return MakeSparse(type_ids, children, std::vector<std::string>{}, type_codes);
+  }
+
+  ARROW_DEPRECATED("Use Result-returning version")
   static Status MakeSparse(const Array& type_ids,
                            const std::vector<std::shared_ptr<Array>>& children,
                            const std::vector<std::string>& field_names,
+                           const std::vector<type_code_t>& type_codes,
                            std::shared_ptr<Array>* out) {
-    return MakeSparse(type_ids, children, field_names, {}, out);
+    return MakeSparse(type_ids, children, field_names, type_codes).Value(out);
   }
 
-  /// \brief Construct Sparse UnionArray from type_ids and children
-  ///
-  /// This function does the bare minimum of validation of the offsets and
-  /// input types.
-  ///
-  /// \param[in] type_ids An array of logical type ids for the union type
-  /// \param[in] children Vector of children Arrays containing the data for each type.
-  /// \param[in] type_codes Vector of type codes.
-  /// \param[out] out Will have length equal to type_ids.length()
+  ARROW_DEPRECATED("Use Result-returning version")
+  static Status MakeSparse(const Array& type_ids,
+                           const std::vector<std::shared_ptr<Array>>& children,
+                           const std::vector<std::string>& field_names,
+                           std::shared_ptr<Array>* out) {
+    return MakeSparse(type_ids, children, field_names).Value(out);
+  }
+
+  ARROW_DEPRECATED("Use Result-returning version")
   static Status MakeSparse(const Array& type_ids,
                            const std::vector<std::shared_ptr<Array>>& children,
                            const std::vector<type_code_t>& type_codes,
                            std::shared_ptr<Array>* out) {
-    return MakeSparse(type_ids, children, {}, type_codes, out);
+    return MakeSparse(type_ids, children, type_codes).Value(out);
   }
 
-  /// \brief Construct Sparse UnionArray from type_ids and children
-  ///
-  /// This function does the bare minimum of validation of the offsets and
-  /// input types.
-  ///
-  /// The name of each field is filled by the index of the field.
-  ///
-  /// \param[in] type_ids An array of logical type ids for the union type
-  /// \param[in] children Vector of children Arrays containing the data for each type.
-  /// \param[out] out Will have length equal to type_ids.length()
+  ARROW_DEPRECATED("Use Result-returning version")
   static Status MakeSparse(const Array& type_ids,
                            const std::vector<std::shared_ptr<Array>>& children,
                            std::shared_ptr<Array>* out) {
-    return MakeSparse(type_ids, children, {}, {}, out);
+    return MakeSparse(type_ids, children).Value(out);
   }
 
   /// Note that this buffer does not account for any slice offset
@@ -1302,7 +1330,11 @@ class ARROW_EXPORT DictionaryArray : public Array {
   /// type object
   /// \param[in] indices an array of non-negative signed
   /// integers smaller than the size of the dictionary
-  /// \param[out] out the resulting DictionaryArray instance
+  static Result<std::shared_ptr<Array>> FromArrays(
+      const std::shared_ptr<DataType>& type, const std::shared_ptr<Array>& indices,
+      const std::shared_ptr<Array>& dictionary);
+
+  ARROW_DEPRECATED("Use Result-returning version")
   static Status FromArrays(const std::shared_ptr<DataType>& type,
                            const std::shared_ptr<Array>& indices,
                            const std::shared_ptr<Array>& dictionary,
@@ -1315,12 +1347,16 @@ class ARROW_EXPORT DictionaryArray : public Array {
   /// The type and the transpose map are typically computed using
   /// DictionaryUnifier.
   ///
-  /// \param[in] pool a pool to allocate the array data from
   /// \param[in] type the new type object
   /// \param[in] dictionary the new dictionary
   /// \param[in] transpose_map transposition array of this array's indices
-  /// into the target array's indices
-  /// \param[out] out the resulting DictionaryArray instance
+  ///   into the target array's indices
+  /// \param[in] pool a pool to allocate the array data from
+  Result<std::shared_ptr<Array>> Transpose(
+      const std::shared_ptr<DataType>& type, const std::shared_ptr<Array>& dictionary,
+      const int32_t* transpose_map, MemoryPool* pool ARROW_MEMORY_POOL_DEFAULT) const;
+
+  ARROW_DEPRECATED("Use Result-returning version")
   Status Transpose(MemoryPool* pool, const std::shared_ptr<DataType>& type,
                    const std::shared_ptr<Array>& dictionary, const int32_t* transpose_map,
                    std::shared_ptr<Array>* out) const;
diff --git a/cpp/src/arrow/array/concatenate_test.cc b/cpp/src/arrow/array/concatenate_test.cc
index e66e8a1..04e3a8f 100644
--- a/cpp/src/arrow/array/concatenate_test.cc
+++ b/cpp/src/arrow/array/concatenate_test.cc
@@ -171,7 +171,7 @@ TEST_F(ConcatenateTest, ListType) {
     offsets_vector.back() = static_cast<int32_t>(values_size);
     std::shared_ptr<Array> offsets;
     ArrayFromVector<Int32Type>(offsets_vector, &offsets);
-    ASSERT_OK(ListArray::FromArrays(*offsets, *values, default_memory_pool(), out));
+    ASSERT_OK_AND_ASSIGN(*out, ListArray::FromArrays(*offsets, *values));
     ASSERT_OK((**out).ValidateFull());
   });
 }
@@ -186,7 +186,7 @@ TEST_F(ConcatenateTest, LargeListType) {
     offsets_vector.back() = static_cast<int64_t>(values_size);
     std::shared_ptr<Array> offsets;
     ArrayFromVector<Int64Type>(offsets_vector, &offsets);
-    ASSERT_OK(LargeListArray::FromArrays(*offsets, *values, default_memory_pool(), out));
+    ASSERT_OK_AND_ASSIGN(*out, LargeListArray::FromArrays(*offsets, *values));
     ASSERT_OK((**out).ValidateFull());
   });
 }
@@ -218,7 +218,7 @@ TEST_F(ConcatenateTest, DISABLED_UnionType) {
     auto bar = this->GeneratePrimitive<DoubleType>(size, null_probability);
     auto baz = this->GeneratePrimitive<BooleanType>(size, null_probability);
     auto type_ids = rng_.Numeric<Int8Type>(size, 0, 2, null_probability);
-    ASSERT_OK(UnionArray::MakeSparse(*type_ids, {foo, bar, baz}, out));
+    ASSERT_OK_AND_ASSIGN(*out, UnionArray::MakeSparse(*type_ids, {foo, bar, baz}));
   });
   // dense mode
   Check([this](int32_t size, double null_probability, std::shared_ptr<Array>* out) {
@@ -227,7 +227,8 @@ TEST_F(ConcatenateTest, DISABLED_UnionType) {
     auto baz = this->GeneratePrimitive<BooleanType>(size, null_probability);
     auto type_ids = rng_.Numeric<Int8Type>(size, 0, 2, null_probability);
     auto value_offsets = rng_.Numeric<Int32Type>(size, 0, size, 0);
-    ASSERT_OK(UnionArray::MakeDense(*type_ids, *value_offsets, {foo, bar, baz}, out));
+    ASSERT_OK_AND_ASSIGN(
+        *out, UnionArray::MakeDense(*type_ids, *value_offsets, {foo, bar, baz}));
   });
 }
 
diff --git a/cpp/src/arrow/array/dict_internal.cc b/cpp/src/arrow/array/dict_internal.cc
index b1c0b01..afd0167 100644
--- a/cpp/src/arrow/array/dict_internal.cc
+++ b/cpp/src/arrow/array/dict_internal.cc
@@ -146,7 +146,9 @@ Status DictionaryUnifier::Make(MemoryPool* pool, std::shared_ptr<DataType> value
 // ----------------------------------------------------------------------
 // DictionaryArray transposition
 
-static bool IsTrivialTransposition(const int32_t* transpose_map,
+namespace {
+
+inline bool IsTrivialTransposition(const int32_t* transpose_map,
                                    int64_t input_dict_size) {
   for (int64_t i = 0; i < input_dict_size; ++i) {
     if (transpose_map[i] != i) {
@@ -157,23 +159,22 @@ static bool IsTrivialTransposition(const int32_t* transpose_map,
 }
 
 template <typename InType, typename OutType>
-static Status TransposeDictIndices(MemoryPool* pool, const ArrayData& in_data,
-                                   const int32_t* transpose_map,
-                                   const std::shared_ptr<ArrayData>& out_data,
-                                   std::shared_ptr<Array>* out) {
+Result<std::shared_ptr<Array>> TransposeDictIndices(
+    MemoryPool* pool, const ArrayData& in_data, const int32_t* transpose_map,
+    const std::shared_ptr<ArrayData>& out_data) {
   using in_c_type = typename InType::c_type;
   using out_c_type = typename OutType::c_type;
   internal::TransposeInts(in_data.GetValues<in_c_type>(1),
                           out_data->GetMutableValues<out_c_type>(1), in_data.length,
                           transpose_map);
-  *out = MakeArray(out_data);
-  return Status::OK();
+  return MakeArray(out_data);
 }
 
-Status DictionaryArray::Transpose(MemoryPool* pool, const std::shared_ptr<DataType>& type,
-                                  const std::shared_ptr<Array>& dictionary,
-                                  const int32_t* transpose_map,
-                                  std::shared_ptr<Array>* out) const {
+}  // namespace
+
+Result<std::shared_ptr<Array>> DictionaryArray::Transpose(
+    const std::shared_ptr<DataType>& type, const std::shared_ptr<Array>& dictionary,
+    const int32_t* transpose_map, MemoryPool* pool) const {
   if (type->id() != Type::DICTIONARY) {
     return Status::TypeError("Expected dictionary type");
   }
@@ -193,8 +194,7 @@ Status DictionaryArray::Transpose(MemoryPool* pool, const std::shared_ptr<DataTy
         ArrayData::Make(type, data_->length, {data_->buffers[0], data_->buffers[1]},
                         data_->null_count, data_->offset);
     out_data->dictionary = dictionary;
-    *out = MakeArray(out_data);
-    return Status::OK();
+    return MakeArray(out_data);
   }
 
   // Default path: compute a buffer of transposed indices.
@@ -215,10 +215,10 @@ Status DictionaryArray::Transpose(MemoryPool* pool, const std::shared_ptr<DataTy
       ArrayData::Make(type, data_->length, {null_bitmap, out_buffer}, data_->null_count);
   out_data->dictionary = dictionary;
 
-#define TRANSPOSE_IN_OUT_CASE(IN_INDEX_TYPE, OUT_INDEX_TYPE)    \
-  case OUT_INDEX_TYPE::type_id:                                 \
-    return TransposeDictIndices<IN_INDEX_TYPE, OUT_INDEX_TYPE>( \
-        pool, *data_, transpose_map, out_data, out);
+#define TRANSPOSE_IN_OUT_CASE(IN_INDEX_TYPE, OUT_INDEX_TYPE)                 \
+  case OUT_INDEX_TYPE::type_id:                                              \
+    return TransposeDictIndices<IN_INDEX_TYPE, OUT_INDEX_TYPE>(pool, *data_, \
+                                                               transpose_map, out_data);
 
 #define TRANSPOSE_IN_CASE(IN_INDEX_TYPE)                        \
   case IN_INDEX_TYPE::type_id:                                  \
@@ -243,4 +243,11 @@ Status DictionaryArray::Transpose(MemoryPool* pool, const std::shared_ptr<DataTy
 #undef TRANSPOSE_IN_OUT_CASE
 }
 
+Status DictionaryArray::Transpose(MemoryPool* pool, const std::shared_ptr<DataType>& type,
+                                  const std::shared_ptr<Array>& dictionary,
+                                  const int32_t* transpose_map,
+                                  std::shared_ptr<Array>* out) const {
+  return Transpose(type, dictionary, transpose_map, pool).Value(out);
+}
+
 }  // namespace arrow
diff --git a/cpp/src/arrow/array/diff_test.cc b/cpp/src/arrow/array/diff_test.cc
index 2e71333..c85e9da 100644
--- a/cpp/src/arrow/array/diff_test.cc
+++ b/cpp/src/arrow/array/diff_test.cc
@@ -566,15 +566,16 @@ TEST_F(DiffTest, DictionaryDiffFormatter) {
   // differing indices
   auto base_dict = ArrayFromJSON(utf8(), R"(["a", "b", "c"])");
   auto base_indices = ArrayFromJSON(int8(), "[0, 1, 2, 2, 0, 1]");
-  ASSERT_OK(
-      DictionaryArray::FromArrays(dictionary(base_indices->type(), base_dict->type()),
-                                  base_indices, base_dict, &base_));
+  ASSERT_OK_AND_ASSIGN(base_, DictionaryArray::FromArrays(
+                                  dictionary(base_indices->type(), base_dict->type()),
+                                  base_indices, base_dict));
 
   auto target_dict = base_dict;
   auto target_indices = ArrayFromJSON(int8(), "[0, 1, 2, 2, 1, 1]");
-  ASSERT_OK(
+  ASSERT_OK_AND_ASSIGN(
+      target_,
       DictionaryArray::FromArrays(dictionary(target_indices->type(), target_dict->type()),
-                                  target_indices, target_dict, &target_));
+                                  target_indices, target_dict));
 
   base_->Equals(*target_, EqualOptions().diff_sink(&formatted));
   auto formatted_expected_indices = R"(# Dictionary arrays differed
@@ -590,9 +591,10 @@ TEST_F(DiffTest, DictionaryDiffFormatter) {
   // differing dictionaries
   target_dict = ArrayFromJSON(utf8(), R"(["b", "c", "a"])");
   target_indices = base_indices;
-  ASSERT_OK(
+  ASSERT_OK_AND_ASSIGN(
+      target_,
       DictionaryArray::FromArrays(dictionary(target_indices->type(), target_dict->type()),
-                                  target_indices, target_dict, &target_));
+                                  target_indices, target_dict));
 
   formatted.str("");
   base_->Equals(*target_, EqualOptions().diff_sink(&formatted));
diff --git a/cpp/src/arrow/array_dict_test.cc b/cpp/src/arrow/array_dict_test.cc
index 52e5e98..28a072b 100644
--- a/cpp/src/arrow/array_dict_test.cc
+++ b/cpp/src/arrow/array_dict_test.cc
@@ -1017,11 +1017,10 @@ TEST(TestDictionary, FromArray) {
   // Index out of bounds
   auto indices4 = ArrayFromJSON(int16(), "[1, 2, null, 3, 2, 0]");
 
-  std::shared_ptr<Array> arr1, arr2, arr3, arr4;
-  ASSERT_OK(DictionaryArray::FromArrays(dict_type, indices1, dict, &arr1));
-  ASSERT_RAISES(Invalid, DictionaryArray::FromArrays(dict_type, indices2, dict, &arr2));
-  ASSERT_OK(DictionaryArray::FromArrays(dict_type, indices3, dict, &arr3));
-  ASSERT_RAISES(Invalid, DictionaryArray::FromArrays(dict_type, indices4, dict, &arr4));
+  ASSERT_OK_AND_ASSIGN(auto arr1, DictionaryArray::FromArrays(dict_type, indices1, dict));
+  ASSERT_RAISES(Invalid, DictionaryArray::FromArrays(dict_type, indices2, dict));
+  ASSERT_OK_AND_ASSIGN(auto arr3, DictionaryArray::FromArrays(dict_type, indices3, dict));
+  ASSERT_RAISES(Invalid, DictionaryArray::FromArrays(dict_type, indices4, dict));
 }
 
 static void CheckTranspose(const std::shared_ptr<Array>& input,
@@ -1029,26 +1028,24 @@ static void CheckTranspose(const std::shared_ptr<Array>& input,
                            const std::shared_ptr<DataType>& out_dict_type,
                            const std::shared_ptr<Array>& out_dict,
                            const std::shared_ptr<Array>& expected_indices) {
-  std::shared_ptr<Array> out, expected;
-  ASSERT_OK(internal::checked_cast<const DictionaryArray&>(*input).Transpose(
-      default_memory_pool(), out_dict_type, out_dict, transpose_map, &out));
-  ASSERT_OK(out->ValidateFull());
-
-  ASSERT_OK(
-      DictionaryArray::FromArrays(out_dict_type, expected_indices, out_dict, &expected));
-  AssertArraysEqual(*out, *expected);
+  ASSERT_OK_AND_ASSIGN(auto transposed,
+                       internal::checked_cast<const DictionaryArray&>(*input).Transpose(
+                           out_dict_type, out_dict, transpose_map));
+  ASSERT_OK(transposed->ValidateFull());
+
+  ASSERT_OK_AND_ASSIGN(auto expected, DictionaryArray::FromArrays(
+                                          out_dict_type, expected_indices, out_dict));
+  AssertArraysEqual(*transposed, *expected);
 }
 
 TEST(TestDictionary, TransposeBasic) {
-  std::shared_ptr<Array> arr, sliced;
-
   auto dict = ArrayFromJSON(utf8(), "[\"A\", \"B\", \"C\"]");
   auto dict_type = dictionary(int16(), utf8());
   auto indices = ArrayFromJSON(int16(), "[1, 2, 0, 0]");
   // ["B", "C", "A", "A"]
-  ASSERT_OK(DictionaryArray::FromArrays(dict_type, indices, dict, &arr));
+  ASSERT_OK_AND_ASSIGN(auto arr, DictionaryArray::FromArrays(dict_type, indices, dict));
   // ["C", "A"]
-  sliced = arr->Slice(1, 2);
+  auto sliced = arr->Slice(1, 2);
 
   // Transpose to same index type
   {
@@ -1081,15 +1078,14 @@ TEST(TestDictionary, TransposeBasic) {
 
 TEST(TestDictionary, TransposeTrivial) {
   // Test a trivial transposition, possibly optimized away
-  std::shared_ptr<Array> arr, sliced;
 
   auto dict = ArrayFromJSON(utf8(), "[\"A\", \"B\", \"C\"]");
   auto dict_type = dictionary(int16(), utf8());
   auto indices = ArrayFromJSON(int16(), "[1, 2, 0, 0]");
   // ["B", "C", "A", "A"]
-  ASSERT_OK(DictionaryArray::FromArrays(dict_type, indices, dict, &arr));
+  ASSERT_OK_AND_ASSIGN(auto arr, DictionaryArray::FromArrays(dict_type, indices, dict));
   // ["C", "A"]
-  sliced = arr->Slice(1, 2);
+  auto sliced = arr->Slice(1, 2);
 
   std::vector<int32_t> transpose_map = {0, 1, 2};
 
@@ -1121,15 +1117,13 @@ TEST(TestDictionary, TransposeTrivial) {
 }
 
 TEST(TestDictionary, TransposeNulls) {
-  std::shared_ptr<Array> arr, sliced;
-
   auto dict = ArrayFromJSON(utf8(), "[\"A\", \"B\", \"C\"]");
   auto dict_type = dictionary(int16(), utf8());
   auto indices = ArrayFromJSON(int16(), "[1, 2, null, 0]");
   // ["B", "C", null, "A"]
-  ASSERT_OK(DictionaryArray::FromArrays(dict_type, indices, dict, &arr));
+  ASSERT_OK_AND_ASSIGN(auto arr, DictionaryArray::FromArrays(dict_type, indices, dict));
   // ["C", null]
-  sliced = arr->Slice(1, 2);
+  auto sliced = arr->Slice(1, 2);
 
   auto out_dict = ArrayFromJSON(utf8(), "[\"Z\", \"A\", \"C\", \"B\"]");
   auto out_dict_type = dictionary(int16(), utf8());
@@ -1195,9 +1189,11 @@ TEST(TestDictionary, CanCompareIndices) {
   auto make_dict = [](std::shared_ptr<DataType> index_type,
                       std::shared_ptr<DataType> value_type, std::string dictionary_json) {
     std::shared_ptr<Array> out;
-    ARROW_EXPECT_OK(DictionaryArray::FromArrays(
-        dictionary(index_type, value_type), ArrayFromJSON(index_type, "[]"),
-        ArrayFromJSON(value_type, dictionary_json), &out));
+    ARROW_EXPECT_OK(
+        DictionaryArray::FromArrays(dictionary(index_type, value_type),
+                                    ArrayFromJSON(index_type, "[]"),
+                                    ArrayFromJSON(value_type, dictionary_json))
+            .Value(&out));
     return checked_pointer_cast<DictionaryArray>(out);
   };
 
diff --git a/cpp/src/arrow/array_list_test.cc b/cpp/src/arrow/array_list_test.cc
index 714f546..158d2c0 100644
--- a/cpp/src/arrow/array_list_test.cc
+++ b/cpp/src/arrow/array_list_test.cc
@@ -209,10 +209,9 @@ class TestListArray : public TestBuilder {
 
     auto list_type = std::make_shared<T>(int8());
 
-    std::shared_ptr<Array> list1, list3, list4;
-    ASSERT_OK(ArrayType::FromArrays(*offsets1, *values, pool_, &list1));
-    ASSERT_OK(ArrayType::FromArrays(*offsets3, *values, pool_, &list3));
-    ASSERT_OK(ArrayType::FromArrays(*offsets4, *values, pool_, &list4));
+    ASSERT_OK_AND_ASSIGN(auto list1, ArrayType::FromArrays(*offsets1, *values, pool_));
+    ASSERT_OK_AND_ASSIGN(auto list3, ArrayType::FromArrays(*offsets3, *values, pool_));
+    ASSERT_OK_AND_ASSIGN(auto list4, ArrayType::FromArrays(*offsets4, *values, pool_));
     ASSERT_OK(list1->ValidateFull());
     ASSERT_OK(list3->ValidateFull());
     ASSERT_OK(list4->ValidateFull());
@@ -238,11 +237,10 @@ class TestListArray : public TestBuilder {
     std::shared_ptr<Array> tmp;
 
     // Zero-length offsets
-    ASSERT_RAISES(Invalid,
-                  ArrayType::FromArrays(*offsets1->Slice(0, 0), *values, pool_, &tmp));
+    ASSERT_RAISES(Invalid, ArrayType::FromArrays(*offsets1->Slice(0, 0), *values, pool_));
 
     // Offsets not the right type
-    ASSERT_RAISES(TypeError, ArrayType::FromArrays(*values, *offsets1, pool_, &tmp));
+    ASSERT_RAISES(TypeError, ArrayType::FromArrays(*values, *offsets1, pool_));
   }
 
   void TestAppendNull() {
@@ -669,10 +667,9 @@ TEST_F(TestMapArray, FromArrays) {
 
   auto map_type = map(int8(), int16());
 
-  std::shared_ptr<Array> map1, map3, map4;
-  ASSERT_OK(MapArray::FromArrays(offsets1, keys, items, pool_, &map1));
-  ASSERT_OK(MapArray::FromArrays(offsets3, keys, items, pool_, &map3));
-  ASSERT_OK(MapArray::FromArrays(offsets4, keys, items, pool_, &map4));
+  ASSERT_OK_AND_ASSIGN(auto map1, MapArray::FromArrays(offsets1, keys, items, pool_));
+  ASSERT_OK_AND_ASSIGN(auto map3, MapArray::FromArrays(offsets3, keys, items, pool_));
+  ASSERT_OK_AND_ASSIGN(auto map4, MapArray::FromArrays(offsets4, keys, items, pool_));
   ASSERT_OK(map1->Validate());
   ASSERT_OK(map3->Validate());
   ASSERT_OK(map4->Validate());
@@ -698,22 +695,20 @@ TEST_F(TestMapArray, FromArrays) {
   std::shared_ptr<Array> tmp;
 
   // Zero-length offsets
-  ASSERT_RAISES(Invalid,
-                MapArray::FromArrays(offsets1->Slice(0, 0), keys, items, pool_, &tmp));
+  ASSERT_RAISES(Invalid, MapArray::FromArrays(offsets1->Slice(0, 0), keys, items, pool_));
 
   // Offsets not the right type
-  ASSERT_RAISES(TypeError, MapArray::FromArrays(keys, offsets1, items, pool_, &tmp));
+  ASSERT_RAISES(TypeError, MapArray::FromArrays(keys, offsets1, items, pool_));
 
   // Keys and Items different lengths
-  ASSERT_RAISES(Invalid,
-                MapArray::FromArrays(offsets1, keys->Slice(0, 1), items, pool_, &tmp));
+  ASSERT_RAISES(Invalid, MapArray::FromArrays(offsets1, keys->Slice(0, 1), items, pool_));
 
   // Keys contains null values
   std::shared_ptr<Array> keys_with_null = offsets3;
   std::shared_ptr<Array> tmp_items = items->Slice(0, offsets3->length());
   ASSERT_EQ(keys_with_null->length(), tmp_items->length());
   ASSERT_RAISES(Invalid,
-                MapArray::FromArrays(offsets1, keys_with_null, tmp_items, pool_, &tmp));
+                MapArray::FromArrays(offsets1, keys_with_null, tmp_items, pool_));
 }
 
 namespace {
diff --git a/cpp/src/arrow/array_struct_test.cc b/cpp/src/arrow/array_struct_test.cc
index cd9242a..e4f50b1 100644
--- a/cpp/src/arrow/array_struct_test.cc
+++ b/cpp/src/arrow/array_struct_test.cc
@@ -207,8 +207,7 @@ TEST(StructArray, FlattenOfSlice) {
   auto slice = internal::checked_pointer_cast<StructArray>(arr->Slice(0, 1));
   ASSERT_OK(slice->ValidateFull());
 
-  ArrayVector flattened;
-  ASSERT_OK(slice->Flatten(default_memory_pool(), &flattened));
+  ASSERT_OK_AND_ASSIGN(auto flattened, slice->Flatten(default_memory_pool()));
 
   ASSERT_OK(slice->ValidateFull());
   ASSERT_OK(arr->ValidateFull());
diff --git a/cpp/src/arrow/array_test.cc b/cpp/src/arrow/array_test.cc
index bab85d8..842914f 100644
--- a/cpp/src/arrow/array_test.cc
+++ b/cpp/src/arrow/array_test.cc
@@ -282,8 +282,7 @@ TEST_F(TestArray, TestMakeArrayOfNull) {
 
   for (int64_t length : {16}) {
     for (auto type : types) {
-      std::shared_ptr<Array> array;
-      ASSERT_OK(MakeArrayOfNull(type, length, &array));
+      ASSERT_OK_AND_ASSIGN(auto array, MakeArrayOfNull(type, length));
       ASSERT_OK(array->ValidateFull());
       ASSERT_EQ(array->length(), length);
       ASSERT_EQ(array->null_count(), length);
@@ -310,8 +309,7 @@ TEST_F(TestArray, TestMakeArrayFromScalar) {
 
   for (int64_t length : {16}) {
     for (auto scalar : scalars) {
-      std::shared_ptr<Array> array;
-      ASSERT_OK(MakeArrayFromScalar(*scalar, length, &array));
+      ASSERT_OK_AND_ASSIGN(auto array, MakeArrayFromScalar(*scalar, length));
       ASSERT_OK(array->ValidateFull());
       ASSERT_EQ(array->length(), length);
       ASSERT_EQ(array->null_count(), 0);
diff --git a/cpp/src/arrow/array_union_test.cc b/cpp/src/arrow/array_union_test.cc
index 62288ca..e3b9a77 100644
--- a/cpp/src/arrow/array_union_test.cc
+++ b/cpp/src/arrow/array_union_test.cc
@@ -162,54 +162,56 @@ TEST_F(TestUnionArrayFactories, TestMakeDense) {
   const UnionArray* union_array;
 
   // without field names and type codes
-  ASSERT_OK(UnionArray::MakeDense(*type_ids_, *value_offsets, children, &result));
+  ASSERT_OK_AND_ASSIGN(result,
+                       UnionArray::MakeDense(*type_ids_, *value_offsets, children));
   ASSERT_OK(result->ValidateFull());
   union_array = checked_cast<const UnionArray*>(result.get());
   CheckUnionArray(*union_array, UnionMode::DENSE, {"0", "1", "2", "3"}, {0, 1, 2, 3});
 
   // with field name
-  ASSERT_RAISES(Invalid, UnionArray::MakeDense(*type_ids_, *value_offsets, children,
-                                               {"one"}, &result));
-  ASSERT_OK(
-      UnionArray::MakeDense(*type_ids_, *value_offsets, children, field_names, &result));
+  ASSERT_RAISES(Invalid,
+                UnionArray::MakeDense(*type_ids_, *value_offsets, children, {"one"}));
+  ASSERT_OK_AND_ASSIGN(
+      result, UnionArray::MakeDense(*type_ids_, *value_offsets, children, field_names));
   ASSERT_OK(result->ValidateFull());
   union_array = checked_cast<const UnionArray*>(result.get());
   CheckUnionArray(*union_array, UnionMode::DENSE, field_names, {0, 1, 2, 3});
 
   // with type codes
-  ASSERT_RAISES(Invalid,
-                UnionArray::MakeDense(*logical_type_ids_, *value_offsets, children,
-                                      std::vector<int8_t>{0}, &result));
-  ASSERT_OK(UnionArray::MakeDense(*logical_type_ids_, *value_offsets, children,
-                                  type_codes_, &result));
+  ASSERT_RAISES(Invalid, UnionArray::MakeDense(*logical_type_ids_, *value_offsets,
+                                               children, std::vector<int8_t>{0}));
+  ASSERT_OK_AND_ASSIGN(result, UnionArray::MakeDense(*logical_type_ids_, *value_offsets,
+                                                     children, type_codes_));
   ASSERT_OK(result->ValidateFull());
   union_array = checked_cast<const UnionArray*>(result.get());
   CheckUnionArray(*union_array, UnionMode::DENSE, {"0", "1", "2", "3"}, type_codes_);
 
   // with field names and type codes
   ASSERT_RAISES(Invalid, UnionArray::MakeDense(*logical_type_ids_, *value_offsets,
-                                               children, {"one"}, type_codes_, &result));
-  ASSERT_OK(UnionArray::MakeDense(*logical_type_ids_, *value_offsets, children,
-                                  field_names, type_codes_, &result));
+                                               children, {"one"}, type_codes_));
+  ASSERT_OK_AND_ASSIGN(result, UnionArray::MakeDense(*logical_type_ids_, *value_offsets,
+                                                     children, field_names, type_codes_));
   ASSERT_OK(result->ValidateFull());
   union_array = checked_cast<const UnionArray*>(result.get());
   CheckUnionArray(*union_array, UnionMode::DENSE, field_names, type_codes_);
 
   // Invalid type codes
-  ASSERT_OK(UnionArray::MakeDense(*invalid_type_ids1_, *value_offsets, children,
-                                  type_codes_, &result));
+  ASSERT_OK_AND_ASSIGN(result, UnionArray::MakeDense(*invalid_type_ids1_, *value_offsets,
+                                                     children, type_codes_));
   ASSERT_RAISES(Invalid, result->ValidateFull());
-  ASSERT_OK(UnionArray::MakeDense(*invalid_type_ids2_, *value_offsets, children,
-                                  type_codes_, &result));
+  ASSERT_OK_AND_ASSIGN(result, UnionArray::MakeDense(*invalid_type_ids2_, *value_offsets,
+                                                     children, type_codes_));
   ASSERT_RAISES(Invalid, result->ValidateFull());
 
   // Invalid offsets
   std::shared_ptr<Array> invalid_offsets;
   ArrayFromVector<Int32Type, int32_t>({1, 0, 0, 0, 1, 1, 1, 2, 1, 2}, &invalid_offsets);
-  ASSERT_OK(UnionArray::MakeDense(*type_ids_, *invalid_offsets, children, &result));
+  ASSERT_OK_AND_ASSIGN(result,
+                       UnionArray::MakeDense(*type_ids_, *invalid_offsets, children));
   ASSERT_RAISES(Invalid, result->ValidateFull());
   ArrayFromVector<Int32Type, int32_t>({1, 0, 0, 0, 1, -1, 1, 2, 1, 2}, &invalid_offsets);
-  ASSERT_OK(UnionArray::MakeDense(*type_ids_, *invalid_offsets, children, &result));
+  ASSERT_OK_AND_ASSIGN(result,
+                       UnionArray::MakeDense(*type_ids_, *invalid_offsets, children));
   ASSERT_RAISES(Invalid, result->ValidateFull());
 }
 
@@ -227,44 +229,47 @@ TEST_F(TestUnionArrayFactories, TestMakeSparse) {
   std::shared_ptr<Array> result;
 
   // without field names and type codes
-  ASSERT_OK(UnionArray::MakeSparse(*type_ids_, children, &result));
+  ASSERT_OK_AND_ASSIGN(result, UnionArray::MakeSparse(*type_ids_, children));
   ASSERT_OK(result->ValidateFull());
   CheckUnionArray(checked_cast<UnionArray&>(*result), UnionMode::SPARSE,
                   {"0", "1", "2", "3"}, {0, 1, 2, 3});
 
   // with field names
-  ASSERT_RAISES(Invalid, UnionArray::MakeSparse(*type_ids_, children, {"one"}, &result));
-  ASSERT_OK(UnionArray::MakeSparse(*type_ids_, children, field_names, &result));
+  ASSERT_RAISES(Invalid, UnionArray::MakeSparse(*type_ids_, children, {"one"}));
+  ASSERT_OK_AND_ASSIGN(result, UnionArray::MakeSparse(*type_ids_, children, field_names));
   ASSERT_OK(result->ValidateFull());
   CheckUnionArray(checked_cast<UnionArray&>(*result), UnionMode::SPARSE, field_names,
                   {0, 1, 2, 3});
 
   // with type codes
   ASSERT_RAISES(Invalid, UnionArray::MakeSparse(*logical_type_ids_, children,
-                                                std::vector<int8_t>{0}, &result));
-  ASSERT_OK(UnionArray::MakeSparse(*logical_type_ids_, children, type_codes_, &result));
+                                                std::vector<int8_t>{0}));
+  ASSERT_OK_AND_ASSIGN(result,
+                       UnionArray::MakeSparse(*logical_type_ids_, children, type_codes_));
   ASSERT_OK(result->ValidateFull());
   CheckUnionArray(checked_cast<UnionArray&>(*result), UnionMode::SPARSE,
                   {"0", "1", "2", "3"}, type_codes_);
 
   // with field names and type codes
   ASSERT_RAISES(Invalid, UnionArray::MakeSparse(*logical_type_ids_, children, {"one"},
-                                                type_codes_, &result));
-  ASSERT_OK(UnionArray::MakeSparse(*logical_type_ids_, children, field_names, type_codes_,
-                                   &result));
+                                                type_codes_));
+  ASSERT_OK_AND_ASSIGN(result, UnionArray::MakeSparse(*logical_type_ids_, children,
+                                                      field_names, type_codes_));
   ASSERT_OK(result->ValidateFull());
   CheckUnionArray(checked_cast<UnionArray&>(*result), UnionMode::SPARSE, field_names,
                   type_codes_);
 
   // Invalid type codes
-  ASSERT_OK(UnionArray::MakeSparse(*invalid_type_ids1_, children, type_codes_, &result));
+  ASSERT_OK_AND_ASSIGN(
+      result, UnionArray::MakeSparse(*invalid_type_ids1_, children, type_codes_));
   ASSERT_RAISES(Invalid, result->ValidateFull());
-  ASSERT_OK(UnionArray::MakeSparse(*invalid_type_ids2_, children, type_codes_, &result));
+  ASSERT_OK_AND_ASSIGN(
+      result, UnionArray::MakeSparse(*invalid_type_ids2_, children, type_codes_));
   ASSERT_RAISES(Invalid, result->ValidateFull());
 
   // Invalid child length
   ArrayFromVector<Int8Type>({0, 0, 0, 0, 0, -12, 0, 0, 0}, &children[3]);
-  ASSERT_RAISES(Invalid, UnionArray::MakeSparse(*type_ids_, children, &result));
+  ASSERT_RAISES(Invalid, UnionArray::MakeSparse(*type_ids_, children));
 }
 
 template <typename B>
@@ -399,10 +404,10 @@ TEST_F(DenseUnionBuilderTest, Basics) {
 
   auto expected_offsets = ArrayFromJSON(int32(), "[0, 0, 0, 1, 1, 1, 2, 2, 2]");
 
-  std::shared_ptr<Array> expected;
-  ASSERT_OK(UnionArray::MakeDense(*expected_types, *expected_offsets,
-                                  {expected_i8, expected_str, expected_dbl},
-                                  {"i8", "str", "dbl"}, {I8, STR, DBL}, &expected));
+  ASSERT_OK_AND_ASSIGN(auto expected,
+                       UnionArray::MakeDense(*expected_types, *expected_offsets,
+                                             {expected_i8, expected_str, expected_dbl},
+                                             {"i8", "str", "dbl"}, {I8, STR, DBL}));
 
   ASSERT_EQ(expected->type()->ToString(), actual->type()->ToString());
   ASSERT_ARRAYS_EQUAL(*expected, *actual);
@@ -417,10 +422,10 @@ TEST_F(DenseUnionBuilderTest, InferredType) {
 
   auto expected_offsets = ArrayFromJSON(int32(), "[0, 1, 0, 1, 2, 2, 0, 1, 2]");
 
-  std::shared_ptr<Array> expected;
-  ASSERT_OK(UnionArray::MakeDense(*expected_types, *expected_offsets,
-                                  {expected_i8, expected_str, expected_dbl},
-                                  {"i8", "str", "dbl"}, {I8, STR, DBL}, &expected));
+  ASSERT_OK_AND_ASSIGN(auto expected,
+                       UnionArray::MakeDense(*expected_types, *expected_offsets,
+                                             {expected_i8, expected_str, expected_dbl},
+                                             {"i8", "str", "dbl"}, {I8, STR, DBL}));
 
   ASSERT_EQ(expected->type()->ToString(), actual->type()->ToString());
   ASSERT_ARRAYS_EQUAL(*expected, *actual);
@@ -451,10 +456,10 @@ TEST_F(SparseUnionBuilderTest, Basics) {
   auto expected_dbl =
       ArrayFromJSON(float64(), "[null, null, 1.0, -1.0, null, null, null, null, 0.5]");
 
-  std::shared_ptr<Array> expected;
-  ASSERT_OK(UnionArray::MakeSparse(*expected_types,
-                                   {expected_i8, expected_str, expected_dbl},
-                                   {"i8", "str", "dbl"}, {I8, STR, DBL}, &expected));
+  ASSERT_OK_AND_ASSIGN(
+      auto expected,
+      UnionArray::MakeSparse(*expected_types, {expected_i8, expected_str, expected_dbl},
+                             {"i8", "str", "dbl"}, {I8, STR, DBL}));
 
   ASSERT_EQ(expected->type()->ToString(), actual->type()->ToString());
   ASSERT_ARRAYS_EQUAL(*expected, *actual);
@@ -470,10 +475,10 @@ TEST_F(SparseUnionBuilderTest, InferredType) {
   auto expected_dbl =
       ArrayFromJSON(float64(), "[null, null, null, null, null, null, 1.0, -1.0, 0.5]");
 
-  std::shared_ptr<Array> expected;
-  ASSERT_OK(UnionArray::MakeSparse(*expected_types,
-                                   {expected_i8, expected_str, expected_dbl},
-                                   {"i8", "str", "dbl"}, {I8, STR, DBL}, &expected));
+  ASSERT_OK_AND_ASSIGN(
+      auto expected,
+      UnionArray::MakeSparse(*expected_types, {expected_i8, expected_str, expected_dbl},
+                             {"i8", "str", "dbl"}, {I8, STR, DBL}));
 
   ASSERT_EQ(expected->type()->ToString(), actual->type()->ToString());
   ASSERT_ARRAYS_EQUAL(*expected, *actual);
diff --git a/cpp/src/arrow/array_view_test.cc b/cpp/src/arrow/array_view_test.cc
index 5657077..8707ce3 100644
--- a/cpp/src/arrow/array_view_test.cc
+++ b/cpp/src/arrow/array_view_test.cc
@@ -31,8 +31,7 @@ namespace arrow {
 void CheckView(const std::shared_ptr<Array>& input,
                const std::shared_ptr<DataType>& view_type,
                const std::shared_ptr<Array>& expected) {
-  std::shared_ptr<Array> result;
-  ASSERT_OK(input->View(view_type, &result));
+  ASSERT_OK_AND_ASSIGN(auto result, input->View(view_type));
   ASSERT_OK(result->ValidateFull());
   AssertArraysEqual(*expected, *result);
 }
@@ -44,8 +43,7 @@ void CheckView(const std::shared_ptr<Array>& input,
 
 void CheckViewFails(const std::shared_ptr<Array>& input,
                     const std::shared_ptr<DataType>& view_type) {
-  std::shared_ptr<Array> result;
-  ASSERT_RAISES(Invalid, input->View(view_type, &result));
+  ASSERT_RAISES(Invalid, input->View(view_type));
 }
 
 class IPv4Type : public ExtensionType {
@@ -321,8 +319,7 @@ TEST(TestArrayView, SparseUnionAsStruct) {
   auto child1 = ArrayFromJSON(int16(), "[0, -1, 42]");
   auto child2 = ArrayFromJSON(int32(), "[0, 1069547520, -1071644672]");
   auto indices = ArrayFromJSON(int8(), "[0, 0, 1]");
-  std::shared_ptr<Array> arr;
-  ASSERT_OK(UnionArray::MakeSparse(*indices, {child1, child2}, &arr));
+  ASSERT_OK_AND_ASSIGN(auto arr, UnionArray::MakeSparse(*indices, {child1, child2}));
   ASSERT_OK(arr->ValidateFull());
 
   auto ty1 = struct_({field("a", int8()), field("b", uint16()), field("c", float32())});
@@ -332,7 +329,7 @@ TEST(TestArrayView, SparseUnionAsStruct) {
 
   // With nulls
   indices = ArrayFromJSON(int8(), "[null, 0, 1]");
-  ASSERT_OK(UnionArray::MakeSparse(*indices, {child1, child2}, &arr));
+  ASSERT_OK_AND_ASSIGN(arr, UnionArray::MakeSparse(*indices, {child1, child2}));
   ASSERT_OK(arr->ValidateFull());
   expected = ArrayFromJSON(ty1, "[null, [0, 65535, 1.5], [1, 42, -2.5]]");
   CheckView(arr, expected);
@@ -341,7 +338,7 @@ TEST(TestArrayView, SparseUnionAsStruct) {
   // With nested nulls
   child1 = ArrayFromJSON(int16(), "[0, -1, null]");
   child2 = ArrayFromJSON(int32(), "[0, null, -1071644672]");
-  ASSERT_OK(UnionArray::MakeSparse(*indices, {child1, child2}, &arr));
+  ASSERT_OK_AND_ASSIGN(arr, UnionArray::MakeSparse(*indices, {child1, child2}));
   ASSERT_OK(arr->ValidateFull());
   expected = ArrayFromJSON(ty1, "[null, [0, 65535, null], [1, null, -2.5]]");
   CheckView(arr, expected);
@@ -353,10 +350,9 @@ TEST(TestArrayView, DecimalRoundTrip) {
   auto arr = ArrayFromJSON(ty1, R"(["123.4567", "-78.9000", null])");
 
   auto ty2 = fixed_size_binary(16);
-  std::shared_ptr<Array> v, w;
-  ASSERT_OK(arr->View(ty2, &v));
+  ASSERT_OK_AND_ASSIGN(auto v, arr->View(ty2));
   ASSERT_OK(v->ValidateFull());
-  ASSERT_OK(v->View(ty1, &w));
+  ASSERT_OK_AND_ASSIGN(auto w, v->View(ty1));
   ASSERT_OK(w->ValidateFull());
   AssertArraysEqual(*arr, *w);
 }
@@ -369,10 +365,10 @@ TEST(TestArrayView, Dictionaries) {
   auto indices = ArrayFromJSON(int8(), "[0, 2, null, 1]");
   auto values = ArrayFromJSON(float32(), "[0.0, 1.5, -2.5]");
 
-  std::shared_ptr<Array> arr, expected, expected_dict;
-  ASSERT_OK(values->View(int32(), &expected_dict));
-  ASSERT_OK(DictionaryArray::FromArrays(ty1, indices, values, &arr));
-  ASSERT_OK(DictionaryArray::FromArrays(ty2, indices, expected_dict, &expected));
+  ASSERT_OK_AND_ASSIGN(auto expected_dict, values->View(int32()));
+  ASSERT_OK_AND_ASSIGN(auto arr, DictionaryArray::FromArrays(ty1, indices, values));
+  ASSERT_OK_AND_ASSIGN(auto expected,
+                       DictionaryArray::FromArrays(ty2, indices, expected_dict));
 
   CheckView(arr, expected);
   CheckView(expected, arr);
@@ -387,10 +383,9 @@ TEST(TestArrayView, Dictionaries) {
 
   // Check dictionary-encoded child
   auto offsets = ArrayFromJSON(int32(), "[0, 2, 2, 4]");
-  std::shared_ptr<Array> list_arr, expected_list_arr;
-  ASSERT_OK(ListArray::FromArrays(*offsets, *arr, default_memory_pool(), &list_arr));
-  ASSERT_OK(ListArray::FromArrays(*offsets, *expected, default_memory_pool(),
-                                  &expected_list_arr));
+  ASSERT_OK_AND_ASSIGN(auto list_arr, ListArray::FromArrays(*offsets, *arr));
+  ASSERT_OK_AND_ASSIGN(auto expected_list_arr,
+                       ListArray::FromArrays(*offsets, *expected));
   CheckView(list_arr, expected_list_arr);
   CheckView(expected_list_arr, list_arr);
 }
@@ -408,8 +403,7 @@ TEST(TestArrayView, ExtensionType) {
 TEST(TestArrayView, NonZeroOffset) {
   auto arr = ArrayFromJSON(int16(), "[10, 11, 12, 13]");
 
-  std::shared_ptr<Array> expected;
-  ASSERT_OK(arr->View(fixed_size_binary(2), &expected));
+  ASSERT_OK_AND_ASSIGN(auto expected, arr->View(fixed_size_binary(2)));
   CheckView(arr->Slice(1), expected->Slice(1));
 }
 
@@ -419,17 +413,15 @@ TEST(TestArrayView, NonZeroNestedOffset) {
 
   auto list_offsets = ArrayFromJSON(int32(), "[0, 2, 3]");
 
-  std::shared_ptr<Array> arr, expected;
-  ASSERT_OK(ListArray::FromArrays(*list_offsets, *list_values->Slice(2),
-                                  default_memory_pool(), &arr));
-  ASSERT_OK(ListArray::FromArrays(*list_offsets, *view_values->Slice(2),
-                                  default_memory_pool(), &expected));
+  ASSERT_OK_AND_ASSIGN(auto arr,
+                       ListArray::FromArrays(*list_offsets, *list_values->Slice(2)));
+  ASSERT_OK_AND_ASSIGN(auto expected,
+                       ListArray::FromArrays(*list_offsets, *view_values->Slice(2)));
   ASSERT_OK(arr->ValidateFull());
   CheckView(arr->Slice(1), expected->Slice(1));
 
   // Be extra paranoid about checking offsets
-  std::shared_ptr<Array> result;
-  ASSERT_OK(arr->Slice(1)->View(expected->type(), &result));
+  ASSERT_OK_AND_ASSIGN(auto result, arr->Slice(1)->View(expected->type()));
   ASSERT_EQ(1, result->offset());
   ASSERT_EQ(2, static_cast<const ListArray&>(*result).values()->offset());
 }
diff --git a/cpp/src/arrow/c/bridge_benchmark.cc b/cpp/src/arrow/c/bridge_benchmark.cc
index 94d5e06..1ae4657 100644
--- a/cpp/src/arrow/c/bridge_benchmark.cc
+++ b/cpp/src/arrow/c/bridge_benchmark.cc
@@ -49,8 +49,7 @@ std::shared_ptr<RecordBatch> ExampleRecordBatch() {
   int64_t length = 1000;
   std::vector<std::shared_ptr<Array>> columns;
   for (const auto& field : schema->fields()) {
-    std::shared_ptr<Array> array;
-    ABORT_NOT_OK(MakeArrayOfNull(field->type(), length, &array));
+    auto array = *MakeArrayOfNull(field->type(), length);
     columns.push_back(array);
   }
   return RecordBatch::Make(schema, length, columns);
diff --git a/cpp/src/arrow/c/bridge_test.cc b/cpp/src/arrow/c/bridge_test.cc
index 4f1feb3..b29cd13 100644
--- a/cpp/src/arrow/c/bridge_test.cc
+++ b/cpp/src/arrow/c/bridge_test.cc
@@ -699,7 +699,7 @@ TEST_F(TestArrayExport, ListSliced) {
     auto factory = [](std::shared_ptr<Array>* out) -> Status {
       auto values = ArrayFromJSON(int16(), "[1, 2, 3, 4, null, 5, 6, 7, 8]")->Slice(1, 6);
       auto offsets = ArrayFromJSON(int32(), "[0, 2, 3, 5, 6]")->Slice(2, 4);
-      return ListArray::FromArrays(*offsets, *values, default_memory_pool(), out);
+      return ListArray::FromArrays(*offsets, *values).Value(out);
     };
     TestNested(factory);
   }
@@ -737,7 +737,7 @@ TEST_F(TestArrayExport, Dictionary) {
       auto values = ArrayFromJSON(utf8(), R"(["foo", "bar", "quux"])");
       auto indices = ArrayFromJSON(int32(), "[0, 2, 1, null, 1]");
       return DictionaryArray::FromArrays(dictionary(indices->type(), values->type()),
-                                         indices, values, out);
+                                         indices, values).Value(out);
     };
     TestNested(factory);
   }
@@ -746,8 +746,8 @@ TEST_F(TestArrayExport, Dictionary) {
       auto values = ArrayFromJSON(list(utf8()), R"([["abc", "def"], ["efg"], []])");
       auto indices = ArrayFromJSON(int32(), "[0, 2, 1, null, 1]");
       return DictionaryArray::FromArrays(
-          dictionary(indices->type(), values->type(), /*ordered=*/true), indices, values,
-          out);
+          dictionary(indices->type(), values->type(), /*ordered=*/true),
+          indices, values).Value(out);
     };
     TestNested(factory);
   }
@@ -755,12 +755,13 @@ TEST_F(TestArrayExport, Dictionary) {
     auto factory = [](std::shared_ptr<Array>* out) -> Status {
       auto values = ArrayFromJSON(list(utf8()), R"([["abc", "def"], ["efg"], []])");
       auto indices = ArrayFromJSON(int32(), "[0, 2, 1, null, 1]");
-      std::shared_ptr<Array> dict_array;
-      RETURN_NOT_OK(DictionaryArray::FromArrays(
-          dictionary(indices->type(), values->type()), indices, values, &dict_array));
+      ARROW_ASSIGN_OR_RAISE(
+          auto dict_array,
+          DictionaryArray::FromArrays(dictionary(indices->type(), values->type()),
+                                      indices, values));
       auto offsets = ArrayFromJSON(int64(), "[0, 2, 5]");
       RETURN_NOT_OK(
-          LargeListArray::FromArrays(*offsets, *dict_array, default_memory_pool(), out));
+          LargeListArray::FromArrays(*offsets, *dict_array).Value(out));
       return (*out)->ValidateFull();
     };
     TestNested(factory);
@@ -786,7 +787,7 @@ TEST_F(TestArrayExport, MoveDictionary) {
       auto values = ArrayFromJSON(utf8(), R"(["foo", "bar", "quux"])");
       auto indices = ArrayFromJSON(int32(), "[0, 2, 1, null, 1]");
       return DictionaryArray::FromArrays(dictionary(indices->type(), values->type()),
-                                         indices, values, out);
+                                         indices, values).Value(out);
     };
     TestMoveNested(factory);
   }
@@ -794,12 +795,13 @@ TEST_F(TestArrayExport, MoveDictionary) {
     auto factory = [](std::shared_ptr<Array>* out) -> Status {
       auto values = ArrayFromJSON(list(utf8()), R"([["abc", "def"], ["efg"], []])");
       auto indices = ArrayFromJSON(int32(), "[0, 2, 1, null, 1]");
-      std::shared_ptr<Array> dict_array;
-      RETURN_NOT_OK(DictionaryArray::FromArrays(
-          dictionary(indices->type(), values->type()), indices, values, &dict_array));
+      ARROW_ASSIGN_OR_RAISE(
+          auto dict_array,
+          DictionaryArray::FromArrays(dictionary(indices->type(), values->type()),
+                                      indices, values));
       auto offsets = ArrayFromJSON(int64(), "[0, 2, 5]");
       RETURN_NOT_OK(
-          LargeListArray::FromArrays(*offsets, *dict_array, default_memory_pool(), out));
+          LargeListArray::FromArrays(*offsets, *dict_array).Value(out));
       return (*out)->ValidateFull();
     };
     TestMoveNested(factory);
@@ -820,12 +822,13 @@ TEST_F(TestArrayExport, MoveChild) {
     auto factory = [](std::shared_ptr<Array>* out) -> Status {
       auto values = ArrayFromJSON(list(utf8()), R"([["abc", "def"], ["efg"], []])");
       auto indices = ArrayFromJSON(int32(), "[0, 2, 1, null, 1]");
-      std::shared_ptr<Array> dict_array;
-      RETURN_NOT_OK(DictionaryArray::FromArrays(
-          dictionary(indices->type(), values->type()), indices, values, &dict_array));
+      ARROW_ASSIGN_OR_RAISE(
+          auto dict_array,
+          DictionaryArray::FromArrays(dictionary(indices->type(), values->type()),
+                                      indices, values));
       auto offsets = ArrayFromJSON(int64(), "[0, 2, 5]");
       RETURN_NOT_OK(
-          LargeListArray::FromArrays(*offsets, *dict_array, default_memory_pool(), out));
+          LargeListArray::FromArrays(*offsets, *dict_array).Value(out));
       return (*out)->ValidateFull();
     };
     TestMoveChild(factory, /*child_id=*/0);
@@ -1965,17 +1968,19 @@ TEST_F(TestArrayImport, Dictionary) {
 
   auto dict_values = ArrayFromJSON(utf8(), R"(["foo", "", "bar", "quux"])");
   auto indices = ArrayFromJSON(int8(), "[1, 2, 0, 1, 3, 0]");
-  std::shared_ptr<Array> expected;
-  ASSERT_OK(DictionaryArray::FromArrays(dictionary(int8(), utf8()), indices, dict_values,
-                                        &expected));
+  ASSERT_OK_AND_ASSIGN(auto expected,
+                       DictionaryArray::FromArrays(dictionary(int8(), utf8()),
+                                                   indices, dict_values));
   CheckImport(expected);
 
   FillStringLike(AddChild(), 4, 0, 0, string_buffers_no_nulls1);
   FillPrimitive(6, 0, 0, primitive_buffers_no_nulls4);
   FillDictionary();
 
-  ASSERT_OK(DictionaryArray::FromArrays(dictionary(int8(), utf8(), /*ordered=*/true),
-                                        indices, dict_values, &expected));
+  ASSERT_OK_AND_ASSIGN(
+      expected,
+      DictionaryArray::FromArrays(dictionary(int8(), utf8(), /*ordered=*/true),
+                                  indices, dict_values));
   CheckImport(expected);
 }
 
@@ -1987,9 +1992,9 @@ TEST_F(TestArrayImport, NestedDictionary) {
 
   auto dict_values = ArrayFromJSON(list(int8()), "[[1, 2], [], [3, 4, 5], [6]]");
   auto indices = ArrayFromJSON(int8(), "[1, 2, 0, 1, 3, 0]");
-  std::shared_ptr<Array> expected;
-  ASSERT_OK(DictionaryArray::FromArrays(dictionary(int8(), list(int8())),
-                                        indices, dict_values, &expected));
+  ASSERT_OK_AND_ASSIGN(auto expected,
+                       DictionaryArray::FromArrays(dictionary(int8(), list(int8())),
+                                                   indices, dict_values));
   CheckImport(expected);
 
   FillStringLike(AddChild(), 4, 0, 0, string_buffers_no_nulls1);
@@ -1999,12 +2004,12 @@ TEST_F(TestArrayImport, NestedDictionary) {
 
   dict_values = ArrayFromJSON(utf8(), R"(["foo", "", "bar", "quux"])");
   indices = ArrayFromJSON(int8(), "[1, 2, 0, 1, 3, 0]");
-  std::shared_ptr<Array> dict_array;
-  ASSERT_OK(DictionaryArray::FromArrays(dictionary(int8(), utf8()), indices, dict_values,
-                                        &dict_array));
+  ASSERT_OK_AND_ASSIGN(auto dict_array,
+                       DictionaryArray::FromArrays(dictionary(int8(), utf8()),
+                                                   indices, dict_values));
   auto offsets = ArrayFromJSON(int32(), "[0, 2, 2, 5]");
-  ASSERT_OK(ListArray::FromArrays(*offsets, *dict_array, default_memory_pool(),
-                                  &expected));
+  ASSERT_OK_AND_ASSIGN(expected,
+                       ListArray::FromArrays(*offsets, *dict_array));
   CheckImport(expected);
 }
 
@@ -2015,9 +2020,9 @@ TEST_F(TestArrayImport, DictionaryWithOffset) {
 
   auto dict_values = ArrayFromJSON(utf8(), R"(["", "bar", "quux"])");
   auto indices = ArrayFromJSON(int8(), "[1, 2, 0]");
-  std::shared_ptr<Array> expected;
-  ASSERT_OK(DictionaryArray::FromArrays(dictionary(int8(), utf8()), indices, dict_values,
-                                        &expected));
+  ASSERT_OK_AND_ASSIGN(auto expected,
+                       DictionaryArray::FromArrays(dictionary(int8(), utf8()),
+                                                   indices, dict_values));
   CheckImport(expected);
 
   FillStringLike(AddChild(), 4, 0, 0, string_buffers_no_nulls1);
@@ -2026,8 +2031,9 @@ TEST_F(TestArrayImport, DictionaryWithOffset) {
 
   dict_values = ArrayFromJSON(utf8(), R"(["foo", "", "bar", "quux"])");
   indices = ArrayFromJSON(int8(), "[0, 1, 3, 0]");
-  ASSERT_OK(DictionaryArray::FromArrays(dictionary(int8(), utf8()), indices, dict_values,
-                                        &expected));
+  ASSERT_OK_AND_ASSIGN(expected,
+                       DictionaryArray::FromArrays(dictionary(int8(), utf8()),
+                                                   indices, dict_values));
   CheckImport(expected);
 }
 
@@ -2502,7 +2508,7 @@ TEST_F(TestArrayRoundtrip, Dictionary) {
       auto values = ArrayFromJSON(utf8(), R"(["foo", "bar", "quux"])");
       auto indices = ArrayFromJSON(int32(), "[0, 2, 1, null, 1]");
       return DictionaryArray::FromArrays(dictionary(indices->type(), values->type()),
-                                         indices, values, out);
+                                         indices, values).Value(out);
     };
     TestWithArrayFactory(factory);
     TestWithArrayFactory(SlicedArrayFactory(factory));
@@ -2512,8 +2518,8 @@ TEST_F(TestArrayRoundtrip, Dictionary) {
       auto values = ArrayFromJSON(list(utf8()), R"([["abc", "def"], ["efg"], []])");
       auto indices = ArrayFromJSON(int32(), "[0, 2, 1, null, 1]");
       return DictionaryArray::FromArrays(
-          dictionary(indices->type(), values->type(), /*ordered=*/true), indices, values,
-          out);
+          dictionary(indices->type(), values->type(), /*ordered=*/true),
+          indices, values).Value(out);
     };
     TestWithArrayFactory(factory);
     TestWithArrayFactory(SlicedArrayFactory(factory));
diff --git a/cpp/src/arrow/compute/kernels/cast_test.cc b/cpp/src/arrow/compute/kernels/cast_test.cc
index 9a5e2b3..94f3689 100644
--- a/cpp/src/arrow/compute/kernels/cast_test.cc
+++ b/cpp/src/arrow/compute/kernels/cast_test.cc
@@ -1446,22 +1446,19 @@ TEST_F(TestCast, ListToList) {
 
   std::shared_ptr<Array> int32_plain_array =
       TestBase::MakeRandomArray<typename TypeTraits<Int32Type>::ArrayType>(10, 2);
-  std::shared_ptr<Array> int32_list_array;
-  ASSERT_OK(
-      ListArray::FromArrays(*offsets, *int32_plain_array, pool_, &int32_list_array));
+  ASSERT_OK_AND_ASSIGN(auto int32_list_array,
+                       ListArray::FromArrays(*offsets, *int32_plain_array, pool_));
 
   std::shared_ptr<Array> int64_plain_array;
   ASSERT_OK(Cast(&this->ctx_, *int32_plain_array, int64(), options, &int64_plain_array));
-  std::shared_ptr<Array> int64_list_array;
-  ASSERT_OK(
-      ListArray::FromArrays(*offsets, *int64_plain_array, pool_, &int64_list_array));
+  ASSERT_OK_AND_ASSIGN(auto int64_list_array,
+                       ListArray::FromArrays(*offsets, *int64_plain_array, pool_));
 
   std::shared_ptr<Array> float64_plain_array;
   ASSERT_OK(
       Cast(&this->ctx_, *int32_plain_array, float64(), options, &float64_plain_array));
-  std::shared_ptr<Array> float64_list_array;
-  ASSERT_OK(
-      ListArray::FromArrays(*offsets, *float64_plain_array, pool_, &float64_list_array));
+  ASSERT_OK_AND_ASSIGN(auto float64_list_array,
+                       ListArray::FromArrays(*offsets, *float64_plain_array, pool_));
 
   CheckPass(*int32_list_array, *int64_list_array, int64_list_array->type(), options);
   CheckPass(*int32_list_array, *float64_list_array, float64_list_array->type(), options);
@@ -1484,16 +1481,14 @@ TEST_F(TestCast, LargeListToLargeList) {
 
   std::shared_ptr<Array> int32_plain_array =
       TestBase::MakeRandomArray<typename TypeTraits<Int32Type>::ArrayType>(10, 2);
-  std::shared_ptr<Array> int32_list_array;
-  ASSERT_OK(
-      LargeListArray::FromArrays(*offsets, *int32_plain_array, pool_, &int32_list_array));
+  ASSERT_OK_AND_ASSIGN(auto int32_list_array,
+                       LargeListArray::FromArrays(*offsets, *int32_plain_array, pool_));
 
   std::shared_ptr<Array> float64_plain_array;
   ASSERT_OK(
       Cast(&this->ctx_, *int32_plain_array, float64(), options, &float64_plain_array));
-  std::shared_ptr<Array> float64_list_array;
-  ASSERT_OK(LargeListArray::FromArrays(*offsets, *float64_plain_array, pool_,
-                                       &float64_list_array));
+  ASSERT_OK_AND_ASSIGN(auto float64_list_array,
+                       LargeListArray::FromArrays(*offsets, *float64_plain_array, pool_));
 
   CheckPass(*int32_list_array, *float64_list_array, float64_list_array->type(), options);
 
diff --git a/cpp/src/arrow/compute/kernels/filter_test.cc b/cpp/src/arrow/compute/kernels/filter_test.cc
index a75d740..45c690e 100644
--- a/cpp/src/arrow/compute/kernels/filter_test.cc
+++ b/cpp/src/arrow/compute/kernels/filter_test.cc
@@ -369,11 +369,12 @@ class TestFilterKernelWithString : public TestFilterKernel<TypeClass> {
                               const std::string& expected_filter) {
     auto dict = ArrayFromJSON(value_type(), dictionary_values);
     auto type = dictionary(int8(), value_type());
-    std::shared_ptr<Array> values, actual, expected;
-    ASSERT_OK(DictionaryArray::FromArrays(type, ArrayFromJSON(int8(), dictionary_filter),
-                                          dict, &values));
-    ASSERT_OK(DictionaryArray::FromArrays(type, ArrayFromJSON(int8(), expected_filter),
-                                          dict, &expected));
+    ASSERT_OK_AND_ASSIGN(auto values,
+                         DictionaryArray::FromArrays(
+                             type, ArrayFromJSON(int8(), dictionary_filter), dict));
+    ASSERT_OK_AND_ASSIGN(
+        auto expected,
+        DictionaryArray::FromArrays(type, ArrayFromJSON(int8(), expected_filter), dict));
     auto take_filter = ArrayFromJSON(boolean(), filter);
     TestFilterKernel<TypeClass>::AssertFilter(values, take_filter, expected);
   }
diff --git a/cpp/src/arrow/compute/kernels/take_test.cc b/cpp/src/arrow/compute/kernels/take_test.cc
index d5a6a75..239ca6d 100644
--- a/cpp/src/arrow/compute/kernels/take_test.cc
+++ b/cpp/src/arrow/compute/kernels/take_test.cc
@@ -202,11 +202,12 @@ class TestTakeKernelWithString : public TestTakeKernel<TypeClass> {
                             const std::string& expected_indices) {
     auto dict = ArrayFromJSON(value_type(), dictionary_values);
     auto type = dictionary(int8(), value_type());
-    std::shared_ptr<Array> values, actual, expected;
-    ASSERT_OK(DictionaryArray::FromArrays(type, ArrayFromJSON(int8(), dictionary_indices),
-                                          dict, &values));
-    ASSERT_OK(DictionaryArray::FromArrays(type, ArrayFromJSON(int8(), expected_indices),
-                                          dict, &expected));
+    ASSERT_OK_AND_ASSIGN(auto values,
+                         DictionaryArray::FromArrays(
+                             type, ArrayFromJSON(int8(), dictionary_indices), dict));
+    ASSERT_OK_AND_ASSIGN(
+        auto expected,
+        DictionaryArray::FromArrays(type, ArrayFromJSON(int8(), expected_indices), dict));
     auto take_indices = ArrayFromJSON(int8(), indices);
     this->AssertTakeArrays(values, take_indices, expected);
   }
diff --git a/cpp/src/arrow/dataset/dataset_test.cc b/cpp/src/arrow/dataset/dataset_test.cc
index 7f82b0d..01c5b54 100644
--- a/cpp/src/arrow/dataset/dataset_test.cc
+++ b/cpp/src/arrow/dataset/dataset_test.cc
@@ -151,8 +151,7 @@ TEST(TestProjector, AugmentWithNull) {
 
   RecordBatchProjector projector(to_schema);
 
-  std::shared_ptr<Array> null_i32;
-  ASSERT_OK(MakeArrayOfNull(int32(), batch->num_rows(), &null_i32));
+  ASSERT_OK_AND_ASSIGN(auto null_i32, MakeArrayOfNull(int32(), batch->num_rows()));
   auto expected_batch =
       RecordBatch::Make(to_schema, batch->num_rows(), {null_i32, batch->column(0)});
 
diff --git a/cpp/src/arrow/dataset/filter.cc b/cpp/src/arrow/dataset/filter.cc
index 7ec27c2..d81501d 100644
--- a/cpp/src/arrow/dataset/filter.cc
+++ b/cpp/src/arrow/dataset/filter.cc
@@ -1122,16 +1122,16 @@ struct TreeEvaluator::Impl {
     ARROW_ASSIGN_OR_RAISE(auto rhs, Evaluate(*expr.right_operand()));
 
     if (lhs.is_scalar()) {
-      std::shared_ptr<Array> lhs_array;
-      RETURN_NOT_OK(MakeArrayFromScalar(ctx_.memory_pool(), *lhs.scalar(),
-                                        batch_.num_rows(), &lhs_array));
+      ARROW_ASSIGN_OR_RAISE(
+          auto lhs_array,
+          MakeArrayFromScalar(*lhs.scalar(), batch_.num_rows(), ctx_.memory_pool()));
       lhs = Datum(std::move(lhs_array));
     }
 
     if (rhs.is_scalar()) {
-      std::shared_ptr<Array> rhs_array;
-      RETURN_NOT_OK(MakeArrayFromScalar(ctx_.memory_pool(), *rhs.scalar(),
-                                        batch_.num_rows(), &rhs_array));
+      ARROW_ASSIGN_OR_RAISE(
+          auto rhs_array,
+          MakeArrayFromScalar(*rhs.scalar(), batch_.num_rows(), ctx_.memory_pool()));
       rhs = Datum(std::move(rhs_array));
     }
 
diff --git a/cpp/src/arrow/dataset/filter_test.cc b/cpp/src/arrow/dataset/filter_test.cc
index d71c856..0776dde 100644
--- a/cpp/src/arrow/dataset/filter_test.cc
+++ b/cpp/src/arrow/dataset/filter_test.cc
@@ -458,9 +458,9 @@ class TakeExpression : public CustomExpression {
       ARROW_ASSIGN_OR_RAISE(auto indices, Evaluate(*take_expr.operand_, batch, pool));
 
       if (indices.kind() == Datum::SCALAR) {
-        std::shared_ptr<Array> indices_array;
-        RETURN_NOT_OK(MakeArrayFromScalar(default_memory_pool(), *indices.scalar(),
-                                          batch.num_rows(), &indices_array));
+        ARROW_ASSIGN_OR_RAISE(auto indices_array,
+                              MakeArrayFromScalar(*indices.scalar(), batch.num_rows(),
+                                                  default_memory_pool()));
         indices = compute::Datum(indices_array->data());
       }
 
diff --git a/cpp/src/arrow/dataset/projector.cc b/cpp/src/arrow/dataset/projector.cc
index 6787b6e..531c4a5 100644
--- a/cpp/src/arrow/dataset/projector.cc
+++ b/cpp/src/arrow/dataset/projector.cc
@@ -96,8 +96,8 @@ Status RecordBatchProjector::SetInputSchema(std::shared_ptr<Schema> from,
     if (matches.empty()) {
       // Mark column i as missing by setting missing_columns_[i]
       // to a non-null placeholder.
-      RETURN_NOT_OK(
-          MakeArrayOfNull(pool, to_->field(i)->type(), 0, &missing_columns_[i]));
+      ARROW_ASSIGN_OR_RAISE(missing_columns_[i],
+                            MakeArrayOfNull(to_->field(i)->type(), 0, pool));
       column_indices_[i] = kNoMatch;
     } else {
       RETURN_NOT_OK(ref.CheckNonMultiple(matches, *from_));
@@ -125,12 +125,13 @@ Status RecordBatchProjector::ResizeMissingColumns(int64_t new_length, MemoryPool
       continue;
     }
     if (scalars_[i] == nullptr) {
-      RETURN_NOT_OK(MakeArrayOfNull(pool, missing_columns_[i]->type(), new_length,
-                                    &missing_columns_[i]));
+      ARROW_ASSIGN_OR_RAISE(
+          missing_columns_[i],
+          MakeArrayOfNull(missing_columns_[i]->type(), new_length, pool));
       continue;
     }
-    RETURN_NOT_OK(
-        MakeArrayFromScalar(pool, *scalars_[i], new_length, &missing_columns_[i]));
+    ARROW_ASSIGN_OR_RAISE(missing_columns_[i],
+                          MakeArrayFromScalar(*scalars_[i], new_length, pool));
   }
   missing_columns_length_ = new_length;
   return Status::OK();
diff --git a/cpp/src/arrow/extension_type_test.cc b/cpp/src/arrow/extension_type_test.cc
index ed44706..3f57daf 100644
--- a/cpp/src/arrow/extension_type_test.cc
+++ b/cpp/src/arrow/extension_type_test.cc
@@ -239,9 +239,7 @@ TEST_F(TestExtensionType, IpcRoundtrip) {
 
   // Wrap type in a ListArray and ensure it also makes it
   auto offsets_arr = ArrayFromJSON(int32(), "[0, 0, 2, 4]");
-  std::shared_ptr<Array> list_arr;
-  ASSERT_OK(
-      ListArray::FromArrays(*offsets_arr, *ext_arr, default_memory_pool(), &list_arr));
+  ASSERT_OK_AND_ASSIGN(auto list_arr, ListArray::FromArrays(*offsets_arr, *ext_arr));
   batch = RecordBatch::Make(schema({field("f0", list(uuid()))}), 3, {list_arr});
   RoundtripBatch(batch, &read_batch);
   CompareBatch(*batch, *read_batch, false /* compare_metadata */);
diff --git a/cpp/src/arrow/ipc/json_simple.cc b/cpp/src/arrow/ipc/json_simple.cc
index 1972ae4..cf04e31 100644
--- a/cpp/src/arrow/ipc/json_simple.cc
+++ b/cpp/src/arrow/ipc/json_simple.cc
@@ -867,8 +867,8 @@ Status DictArrayFromJSON(const std::shared_ptr<DataType>& type,
   RETURN_NOT_OK(
       ArrayFromJSON(dictionary_type.value_type(), dictionary_json, &dictionary));
 
-  return DictionaryArray::FromArrays(type, std::move(indices), std::move(dictionary),
-                                     out);
+  return DictionaryArray::FromArrays(type, std::move(indices), std::move(dictionary))
+      .Value(out);
 }
 
 }  // namespace json
diff --git a/cpp/src/arrow/ipc/json_simple_test.cc b/cpp/src/arrow/ipc/json_simple_test.cc
index 940f985..c962498 100644
--- a/cpp/src/arrow/ipc/json_simple_test.cc
+++ b/cpp/src/arrow/ipc/json_simple_test.cc
@@ -491,14 +491,14 @@ TEST(TestList, IntegerList) {
   ASSERT_OK(actual->ValidateFull());
   ArrayFromVector<Int32Type>({0}, &offsets);
   ArrayFromVector<Int64Type>({}, &values);
-  ASSERT_OK(ListArray::FromArrays(*offsets, *values, pool, &expected));
+  ASSERT_OK_AND_ASSIGN(expected, ListArray::FromArrays(*offsets, *values, pool));
   AssertArraysEqual(*expected, *actual);
 
   ASSERT_OK(ArrayFromJSON(type, "[[4, 5], [], [6]]", &actual));
   ASSERT_OK(actual->ValidateFull());
   ArrayFromVector<Int32Type>({0, 2, 2, 3}, &offsets);
   ArrayFromVector<Int64Type>({4, 5, 6}, &values);
-  ASSERT_OK(ListArray::FromArrays(*offsets, *values, pool, &expected));
+  ASSERT_OK_AND_ASSIGN(expected, ListArray::FromArrays(*offsets, *values, pool));
   AssertArraysEqual(*expected, *actual);
 
   ASSERT_OK(ArrayFromJSON(type, "[[], [null], [6, null]]", &actual));
@@ -506,7 +506,7 @@ TEST(TestList, IntegerList) {
   ArrayFromVector<Int32Type>({0, 0, 1, 3}, &offsets);
   auto is_valid = std::vector<bool>{false, true, false};
   ArrayFromVector<Int64Type>(is_valid, {0, 6, 0}, &values);
-  ASSERT_OK(ListArray::FromArrays(*offsets, *values, pool, &expected));
+  ASSERT_OK_AND_ASSIGN(expected, ListArray::FromArrays(*offsets, *values, pool));
   AssertArraysEqual(*expected, *actual);
 
   ASSERT_OK(ArrayFromJSON(type, "[null, [], null]", &actual));
@@ -541,14 +541,14 @@ TEST(TestList, NullList) {
   ASSERT_OK(actual->ValidateFull());
   ArrayFromVector<Int32Type>({0}, &offsets);
   values = std::make_shared<NullArray>(0);
-  ASSERT_OK(ListArray::FromArrays(*offsets, *values, pool, &expected));
+  ASSERT_OK_AND_ASSIGN(expected, ListArray::FromArrays(*offsets, *values, pool));
   AssertArraysEqual(*expected, *actual);
 
   ASSERT_OK(ArrayFromJSON(type, "[[], [null], [null, null]]", &actual));
   ASSERT_OK(actual->ValidateFull());
   ArrayFromVector<Int32Type>({0, 0, 1, 3}, &offsets);
   values = std::make_shared<NullArray>(3);
-  ASSERT_OK(ListArray::FromArrays(*offsets, *values, pool, &expected));
+  ASSERT_OK_AND_ASSIGN(expected, ListArray::FromArrays(*offsets, *values, pool));
   AssertArraysEqual(*expected, *actual);
 
   ASSERT_OK(ArrayFromJSON(type, "[null, [], null]", &actual));
@@ -574,9 +574,9 @@ TEST(TestList, IntegerListList) {
   ASSERT_OK(actual->ValidateFull());
   ArrayFromVector<Int32Type>({0, 1, 3, 6}, &offsets);
   ArrayFromVector<UInt8Type>({4, 5, 6, 7, 8, 9}, &values);
-  ASSERT_OK(ListArray::FromArrays(*offsets, *values, pool, &nested));
+  ASSERT_OK_AND_ASSIGN(nested, ListArray::FromArrays(*offsets, *values, pool));
   ArrayFromVector<Int32Type>({0, 2, 3}, &offsets);
-  ASSERT_OK(ListArray::FromArrays(*offsets, *nested, pool, &expected));
+  ASSERT_OK_AND_ASSIGN(expected, ListArray::FromArrays(*offsets, *nested, pool));
   ASSERT_EQ(actual->length(), 2);
   AssertArraysEqual(*expected, *actual);
 
@@ -584,9 +584,9 @@ TEST(TestList, IntegerListList) {
   ASSERT_OK(actual->ValidateFull());
   ArrayFromVector<Int32Type>({0, 0, 1, 1, 3, 6}, &offsets);
   ArrayFromVector<UInt8Type>({4, 5, 6, 7, 8, 9}, &values);
-  ASSERT_OK(ListArray::FromArrays(*offsets, *values, pool, &nested));
+  ASSERT_OK_AND_ASSIGN(nested, ListArray::FromArrays(*offsets, *values, pool));
   ArrayFromVector<Int32Type>({0, 0, 1, 4, 5}, &offsets);
-  ASSERT_OK(ListArray::FromArrays(*offsets, *nested, pool, &expected));
+  ASSERT_OK_AND_ASSIGN(expected, ListArray::FromArrays(*offsets, *nested, pool));
   ASSERT_EQ(actual->length(), 4);
   AssertArraysEqual(*expected, *actual);
 
@@ -617,7 +617,7 @@ TEST(TestLargeList, Basics) {
   ArrayFromVector<Int64Type>({0, 0, 1, 3}, &offsets);
   auto is_valid = std::vector<bool>{false, true, false};
   ArrayFromVector<Int16Type>(is_valid, {0, 6, 0}, &values);
-  ASSERT_OK(LargeListArray::FromArrays(*offsets, *values, pool, &expected));
+  ASSERT_OK_AND_ASSIGN(expected, LargeListArray::FromArrays(*offsets, *values, pool));
   AssertArraysEqual(*expected, *actual);
 }
 
@@ -1001,10 +1001,9 @@ TEST(TestDenseUnion, Basics) {
   auto expected_a = ArrayFromJSON(int8(), "[122, null]");
   auto expected_b = ArrayFromJSON(boolean(), "[true, false]");
 
-  std::shared_ptr<Array> expected;
-  ASSERT_OK(UnionArray::MakeDense(*expected_types, *expected_offsets,
-                                  {expected_a, expected_b}, {"a", "b"}, {4, 8},
-                                  &expected));
+  ASSERT_OK_AND_ASSIGN(
+      auto expected, UnionArray::MakeDense(*expected_types, *expected_offsets,
+                                           {expected_a, expected_b}, {"a", "b"}, {4, 8}));
 
   ASSERT_ARRAYS_EQUAL(*expected, *array);
 
@@ -1025,9 +1024,9 @@ TEST(TestSparseUnion, Basics) {
   auto expected_a = ArrayFromJSON(int8(), "[122, null, null, null, null]");
   auto expected_b = ArrayFromJSON(boolean(), "[null, true, null, null, false]");
 
-  std::shared_ptr<Array> expected;
-  ASSERT_OK(UnionArray::MakeSparse(*expected_types, {expected_a, expected_b}, {"a", "b"},
-                                   {4, 8}, &expected));
+  ASSERT_OK_AND_ASSIGN(auto expected,
+                       UnionArray::MakeSparse(*expected_types, {expected_a, expected_b},
+                                              {"a", "b"}, {4, 8}));
 
   ASSERT_ARRAYS_EQUAL(*expected, *array);
 }
@@ -1049,13 +1048,13 @@ TEST(TestDenseUnion, ListOfUnion) {
   auto expected_a = ArrayFromJSON(int8(), "[122, null]");
   auto expected_b = ArrayFromJSON(boolean(), "[true, false]");
 
-  std::shared_ptr<Array> expected_values, expected;
-  ASSERT_OK(UnionArray::MakeDense(*expected_types, *expected_offsets,
-                                  {expected_a, expected_b}, {"a", "b"}, {4, 8},
-                                  &expected_values));
+  ASSERT_OK_AND_ASSIGN(
+      auto expected_values,
+      UnionArray::MakeDense(*expected_types, *expected_offsets, {expected_a, expected_b},
+                            {"a", "b"}, {4, 8}));
   auto expected_list_offsets = ArrayFromJSON(int32(), "[0, 2, 5]");
-  ASSERT_OK(ListArray::FromArrays(*expected_list_offsets, *expected_values,
-                                  default_memory_pool(), &expected));
+  ASSERT_OK_AND_ASSIGN(auto expected,
+                       ListArray::FromArrays(*expected_list_offsets, *expected_values));
 
   ASSERT_ARRAYS_EQUAL(*expected, *array);
 
@@ -1082,12 +1081,12 @@ TEST(TestSparseUnion, ListOfUnion) {
   auto expected_a = ArrayFromJSON(int8(), "[122, null, null, null, null]");
   auto expected_b = ArrayFromJSON(boolean(), "[null, true, null, null, false]");
 
-  std::shared_ptr<Array> expected_values, expected;
-  ASSERT_OK(UnionArray::MakeSparse(*expected_types, {expected_a, expected_b}, {"a", "b"},
-                                   {4, 8}, &expected_values));
+  ASSERT_OK_AND_ASSIGN(auto expected_values,
+                       UnionArray::MakeSparse(*expected_types, {expected_a, expected_b},
+                                              {"a", "b"}, {4, 8}));
   auto expected_list_offsets = ArrayFromJSON(int32(), "[0, 2, 5]");
-  ASSERT_OK(ListArray::FromArrays(*expected_list_offsets, *expected_values,
-                                  default_memory_pool(), &expected));
+  ASSERT_OK_AND_ASSIGN(auto expected,
+                       ListArray::FromArrays(*expected_list_offsets, *expected_values));
 
   ASSERT_ARRAYS_EQUAL(*expected, *array);
 }
@@ -1119,9 +1118,10 @@ TEST(TestDenseUnion, UnionOfStructs) {
     ])"),
                                  ArrayFromJSON(fields[2]->type(), "[]")};
 
-  std::shared_ptr<Array> expected;
-  ASSERT_OK(UnionArray::MakeDense(*expected_types, *expected_offsets, expected_fields,
-                                  {"ab", "wtf", "q"}, {0, 23, 47}, &expected));
+  ASSERT_OK_AND_ASSIGN(
+      auto expected,
+      UnionArray::MakeDense(*expected_types, *expected_offsets, expected_fields,
+                            {"ab", "wtf", "q"}, {0, 23, 47}));
 
   ASSERT_ARRAYS_EQUAL(*expected, *array);
 
@@ -1166,9 +1166,9 @@ TEST(TestSparseUnion, UnionOfStructs) {
     ])"),
       ArrayFromJSON(fields[2]->type(), "[null, null, null, null, null]")};
 
-  std::shared_ptr<Array> expected;
-  ASSERT_OK(UnionArray::MakeSparse(*expected_types, expected_fields, {"ab", "wtf", "q"},
-                                   {0, 23, 47}, &expected));
+  ASSERT_OK_AND_ASSIGN(auto expected,
+                       UnionArray::MakeSparse(*expected_types, expected_fields,
+                                              {"ab", "wtf", "q"}, {0, 23, 47}));
 
   ASSERT_ARRAYS_EQUAL(*expected, *array);
 }
diff --git a/cpp/src/arrow/ipc/read_write_test.cc b/cpp/src/arrow/ipc/read_write_test.cc
index d1e39d8..4620f4a 100644
--- a/cpp/src/arrow/ipc/read_write_test.cc
+++ b/cpp/src/arrow/ipc/read_write_test.cc
@@ -493,8 +493,8 @@ TEST_F(TestWriteRecordBatch, WriteWithCompression) {
 
   auto dict_type = dictionary(int32(), utf8());
   auto dict_field = field("f1", dict_type);
-  std::shared_ptr<Array> dict_array;
-  ASSERT_OK(DictionaryArray::FromArrays(dict_type, indices, dict, &dict_array));
+  ASSERT_OK_AND_ASSIGN(auto dict_array,
+                       DictionaryArray::FromArrays(dict_type, indices, dict));
 
   auto schema = ::arrow::schema({field("f0", utf8()), dict_field});
   auto batch =
diff --git a/cpp/src/arrow/json/converter.cc b/cpp/src/arrow/json/converter.cc
index 25bb314..d3f04ac 100644
--- a/cpp/src/arrow/json/converter.cc
+++ b/cpp/src/arrow/json/converter.cc
@@ -92,7 +92,7 @@ class BooleanConverter : public PrimitiveConverter {
 
   Status Convert(const std::shared_ptr<Array>& in, std::shared_ptr<Array>* out) override {
     if (in->type_id() == Type::NA) {
-      return MakeArrayOfNull(pool_, boolean(), in->length(), out);
+      return MakeArrayOfNull(boolean(), in->length(), pool_).Value(out);
     }
     if (in->type_id() != Type::BOOL) {
       return GenericConversionError(*out_type_, " from ", *in->type());
@@ -112,7 +112,7 @@ class NumericConverter : public PrimitiveConverter {
 
   Status Convert(const std::shared_ptr<Array>& in, std::shared_ptr<Array>* out) override {
     if (in->type_id() == Type::NA) {
-      return MakeArrayOfNull(pool_, out_type_, in->length(), out);
+      return MakeArrayOfNull(out_type_, in->length(), pool_).Value(out);
     }
     const auto& dict_array = GetDictionaryArray(in);
 
@@ -150,7 +150,7 @@ class DateTimeConverter : public PrimitiveConverter {
 
   Status Convert(const std::shared_ptr<Array>& in, std::shared_ptr<Array>* out) override {
     if (in->type_id() == Type::NA) {
-      return MakeArrayOfNull(pool_, out_type_, in->length(), out);
+      return MakeArrayOfNull(out_type_, in->length(), pool_).Value(out);
     }
 
     std::shared_ptr<Array> repr;
@@ -178,7 +178,7 @@ class BinaryConverter : public PrimitiveConverter {
 
   Status Convert(const std::shared_ptr<Array>& in, std::shared_ptr<Array>* out) override {
     if (in->type_id() == Type::NA) {
-      return MakeArrayOfNull(pool_, out_type_, in->length(), out);
+      return MakeArrayOfNull(out_type_, in->length(), pool_).Value(out);
     }
     const auto& dict_array = GetDictionaryArray(in);
 
diff --git a/cpp/src/arrow/python/python_to_arrow.cc b/cpp/src/arrow/python/python_to_arrow.cc
index d2a5806..e531ce3 100644
--- a/cpp/src/arrow/python/python_to_arrow.cc
+++ b/cpp/src/arrow/python/python_to_arrow.cc
@@ -644,7 +644,7 @@ class StringConverter
 
       auto binary_type =
           TypeTraits<typename TypeClass::EquivalentBinaryType>::type_singleton();
-      return (*out)->View(binary_type, out);
+      return (*out)->View(binary_type).Value(out);
     }
     return Status::OK();
   }
diff --git a/cpp/src/arrow/table.cc b/cpp/src/arrow/table.cc
index 3ce7217..765280e 100644
--- a/cpp/src/arrow/table.cc
+++ b/cpp/src/arrow/table.cc
@@ -132,39 +132,48 @@ std::shared_ptr<ChunkedArray> ChunkedArray::Slice(int64_t offset) const {
 
 Status ChunkedArray::Flatten(MemoryPool* pool,
                              std::vector<std::shared_ptr<ChunkedArray>>* out) const {
-  out->clear();
+  return Flatten(pool).Value(out);
+}
+
+Result<std::vector<std::shared_ptr<ChunkedArray>>> ChunkedArray::Flatten(
+    MemoryPool* pool) const {
   if (type()->id() != Type::STRUCT) {
     // Emulate nonexistent copy constructor
-    *out = {std::make_shared<ChunkedArray>(chunks_, type_)};
-    return Status::OK();
+    return std::vector<std::shared_ptr<ChunkedArray>>{
+        std::make_shared<ChunkedArray>(chunks_, type_)};
   }
 
   std::vector<ArrayVector> flattened_chunks(type()->num_children());
   for (const auto& chunk : chunks_) {
-    ArrayVector res;
-    RETURN_NOT_OK(checked_cast<const StructArray&>(*chunk).Flatten(pool, &res));
-    DCHECK_EQ(res.size(), flattened_chunks.size());
-    for (size_t i = 0; i < res.size(); ++i) {
-      flattened_chunks[i].push_back(res[i]);
+    ARROW_ASSIGN_OR_RAISE(auto arrays,
+                          checked_cast<const StructArray&>(*chunk).Flatten(pool));
+    DCHECK_EQ(arrays.size(), flattened_chunks.size());
+    for (size_t i = 0; i < arrays.size(); ++i) {
+      flattened_chunks[i].push_back(arrays[i]);
     }
   }
 
-  out->resize(type()->num_children());
-  for (size_t i = 0; i < out->size(); ++i) {
+  std::vector<std::shared_ptr<ChunkedArray>> flattened(type()->num_children());
+  for (size_t i = 0; i < flattened.size(); ++i) {
     auto child_type = type()->child(static_cast<int>(i))->type();
-    out->at(i) = std::make_shared<ChunkedArray>(flattened_chunks[i], child_type);
+    flattened[i] =
+        std::make_shared<ChunkedArray>(std::move(flattened_chunks[i]), child_type);
   }
-  return Status::OK();
+  return flattened;
 }
 
-Status ChunkedArray::View(const std::shared_ptr<DataType>& type,
-                          std::shared_ptr<ChunkedArray>* out) const {
+Result<std::shared_ptr<ChunkedArray>> ChunkedArray::View(
+    const std::shared_ptr<DataType>& type) const {
   ArrayVector out_chunks(this->num_chunks());
   for (int i = 0; i < this->num_chunks(); ++i) {
-    RETURN_NOT_OK(chunks_[i]->View(type, &out_chunks[i]));
+    ARROW_ASSIGN_OR_RAISE(out_chunks[i], chunks_[i]->View(type));
   }
-  *out = std::make_shared<ChunkedArray>(out_chunks, type);
-  return Status::OK();
+  return std::make_shared<ChunkedArray>(out_chunks, type);
+}
+
+Status ChunkedArray::View(const std::shared_ptr<DataType>& type,
+                          std::shared_ptr<ChunkedArray>* out) const {
+  return View(type).Value(out);
 }
 
 Status ChunkedArray::Validate() const {
@@ -360,9 +369,8 @@ class SimpleTable : public Table {
     std::vector<std::shared_ptr<Field>> flattened_fields;
     std::vector<std::shared_ptr<ChunkedArray>> flattened_columns;
     for (int i = 0; i < num_columns(); ++i) {
-      std::vector<std::shared_ptr<ChunkedArray>> new_columns;
       std::vector<std::shared_ptr<Field>> new_fields = field(i)->Flatten();
-      RETURN_NOT_OK(column(i)->Flatten(pool, &new_columns));
+      ARROW_ASSIGN_OR_RAISE(auto new_columns, column(i)->Flatten(pool));
       DCHECK_EQ(new_columns.size(), new_fields.size());
       for (size_t j = 0; j < new_columns.size(); ++j) {
         flattened_fields.push_back(new_fields[j]);
@@ -631,10 +639,9 @@ Result<std::shared_ptr<Table>> PromoteTableToSchema(const std::shared_ptr<Table>
   const int64_t num_rows = table->num_rows();
   auto AppendColumnOfNulls = [pool, &columns,
                               num_rows](const std::shared_ptr<DataType>& type) {
-    std::shared_ptr<Array> array_of_nulls;
     // TODO(bkietz): share the zero-filled buffers as much as possible across
     // the null-filled arrays created here.
-    RETURN_NOT_OK(MakeArrayOfNull(pool, type, num_rows, &array_of_nulls));
+    ARROW_ASSIGN_OR_RAISE(auto array_of_nulls, MakeArrayOfNull(type, num_rows, pool));
     columns.push_back(std::make_shared<ChunkedArray>(array_of_nulls));
     return Status::OK();
   };
diff --git a/cpp/src/arrow/table.h b/cpp/src/arrow/table.h
index baeebf8..fc5d499 100644
--- a/cpp/src/arrow/table.h
+++ b/cpp/src/arrow/table.h
@@ -82,12 +82,18 @@ class ARROW_EXPORT ChunkedArray {
   /// for each struct field
   ///
   /// \param[in] pool The pool for buffer allocations, if any
-  /// \param[out] out The resulting vector of arrays
+  Result<std::vector<std::shared_ptr<ChunkedArray>>> Flatten(
+      MemoryPool* pool ARROW_MEMORY_POOL_DEFAULT) const;
+
+  ARROW_DEPRECATED("Use Result-returning version")
   Status Flatten(MemoryPool* pool, std::vector<std::shared_ptr<ChunkedArray>>* out) const;
 
   /// Construct a zero-copy view of this chunked array with the given
   /// type. Calls Array::View on each constituent chunk. Always succeeds if
   /// there are zero chunks
+  Result<std::shared_ptr<ChunkedArray>> View(const std::shared_ptr<DataType>& type) const;
+
+  ARROW_DEPRECATED("Use Result-returning version")
   Status View(const std::shared_ptr<DataType>& type,
               std::shared_ptr<ChunkedArray>* out) const;
 
diff --git a/cpp/src/arrow/table_test.cc b/cpp/src/arrow/table_test.cc
index 2284768..652d690 100644
--- a/cpp/src/arrow/table_test.cc
+++ b/cpp/src/arrow/table_test.cc
@@ -185,15 +185,14 @@ TEST_F(TestChunkedArray, View) {
   auto carr = std::make_shared<ChunkedArray>(chunks);
   auto expected = std::make_shared<ChunkedArray>(ex_chunks);
 
-  std::shared_ptr<ChunkedArray> result;
-  ASSERT_OK(carr->View(out_ty, &result));
+  ASSERT_OK_AND_ASSIGN(auto result, carr->View(out_ty));
   AssertChunkedEqual(*expected, *result);
 
   // Zero length
   ArrayVector empty = {};
   carr = std::make_shared<ChunkedArray>(empty, in_ty);
   expected = std::make_shared<ChunkedArray>(empty, out_ty);
-  ASSERT_OK(carr->View(out_ty, &result));
+  ASSERT_OK_AND_ASSIGN(result, carr->View(out_ty));
   AssertChunkedEqual(*expected, *result);
 }
 
@@ -454,8 +453,7 @@ TEST_F(TestTable, ConcatenateTables) {
 std::shared_ptr<Table> MakeTableWithOneNullFilledColumn(
     const std::string& column_name, const std::shared_ptr<DataType>& data_type,
     const int length) {
-  std::shared_ptr<Array> array_of_nulls;
-  DCHECK_OK(MakeArrayOfNull(data_type, length, &array_of_nulls));
+  auto array_of_nulls = *MakeArrayOfNull(data_type, length);
   return Table::Make(schema({field(column_name, data_type)}), {array_of_nulls});
 }
 
@@ -605,10 +603,8 @@ TEST_F(ConcatenateTablesWithPromotionTest, Simple) {
   MakeExample1(length);
   auto batch1 = RecordBatch::Make(schema_, length, arrays_);
 
-  std::shared_ptr<Array> f1_nulls;
-  ASSERT_OK(MakeArrayOfNull(schema_->field(1)->type(), length, &f1_nulls));
-  std::shared_ptr<Array> f2_nulls;
-  ASSERT_OK(MakeArrayOfNull(schema_->field(2)->type(), length, &f2_nulls));
+  ASSERT_OK_AND_ASSIGN(auto f1_nulls, MakeArrayOfNull(schema_->field(1)->type(), length));
+  ASSERT_OK_AND_ASSIGN(auto f2_nulls, MakeArrayOfNull(schema_->field(2)->type(), length));
 
   MakeExample2(length);
   auto batch2 = RecordBatch::Make(schema_, length, arrays_);
diff --git a/cpp/src/arrow/testing/random.cc b/cpp/src/arrow/testing/random.cc
index c6b25d1..80c813a 100644
--- a/cpp/src/arrow/testing/random.cc
+++ b/cpp/src/arrow/testing/random.cc
@@ -208,8 +208,7 @@ std::shared_ptr<Array> RandomArrayGenerator::BinaryWithRepeats(int64_t size,
   auto strings =
       StringWithRepeats(size, unique, min_length, max_length, null_probability);
   std::shared_ptr<Array> out;
-  ABORT_NOT_OK(strings->View(binary(), &out));
-  return out;
+  return *strings->View(binary());
 }
 
 std::shared_ptr<Array> RandomArrayGenerator::StringWithRepeats(int64_t size,
diff --git a/cpp/src/parquet/arrow/arrow_reader_writer_test.cc b/cpp/src/parquet/arrow/arrow_reader_writer_test.cc
index 558e218..0d8cd2d 100644
--- a/cpp/src/parquet/arrow/arrow_reader_writer_test.cc
+++ b/cpp/src/parquet/arrow/arrow_reader_writer_test.cc
@@ -1155,8 +1155,8 @@ TEST_F(TestNullParquetIO, NullListColumn) {
     std::shared_ptr<Array> offsets_array, values_array, list_array;
     ::arrow::ArrayFromVector<::arrow::Int32Type, int32_t>(offsets, &offsets_array);
     values_array = std::make_shared<::arrow::NullArray>(offsets.back());
-    ASSERT_OK(::arrow::ListArray::FromArrays(*offsets_array, *values_array,
-                                             default_memory_pool(), &list_array));
+    ASSERT_OK_AND_ASSIGN(list_array,
+                         ::arrow::ListArray::FromArrays(*offsets_array, *values_array));
 
     std::shared_ptr<Table> table = MakeSimpleTable(list_array, false /* nullable */);
     this->ResetSink();
@@ -1183,8 +1183,7 @@ TEST_F(TestNullParquetIO, NullDictionaryColumn) {
   ASSERT_OK(::arrow::AllocateEmptyBitmap(::arrow::default_memory_pool(), SMALL_SIZE,
                                          &null_bitmap));
 
-  std::shared_ptr<Array> indices;
-  ASSERT_OK(MakeArrayOfNull(::arrow::int8(), SMALL_SIZE, &indices));
+  ASSERT_OK_AND_ASSIGN(auto indices, MakeArrayOfNull(::arrow::int8(), SMALL_SIZE));
   std::shared_ptr<::arrow::DictionaryType> dict_type =
       std::make_shared<::arrow::DictionaryType>(::arrow::int8(), ::arrow::null());
 
@@ -3113,11 +3112,11 @@ TEST(TestArrowWriteDictionaries, NestedSubfield) {
   auto indices = ::arrow::ArrayFromJSON(::arrow::int32(), "[0, 0, 0]");
   auto dict = ::arrow::ArrayFromJSON(::arrow::utf8(), "[\"foo\"]");
 
-  std::shared_ptr<Array> dict_values, values;
   auto dict_ty = ::arrow::dictionary(::arrow::int32(), ::arrow::utf8());
-  ASSERT_OK(::arrow::DictionaryArray::FromArrays(dict_ty, indices, dict, &dict_values));
-  ASSERT_OK(::arrow::ListArray::FromArrays(*offsets, *dict_values,
-                                           ::arrow::default_memory_pool(), &values));
+  ASSERT_OK_AND_ASSIGN(auto dict_values,
+                       ::arrow::DictionaryArray::FromArrays(dict_ty, indices, dict));
+  ASSERT_OK_AND_ASSIGN(auto values,
+                       ::arrow::ListArray::FromArrays(*offsets, *dict_values));
 
   auto dense_ty = ::arrow::list(::arrow::utf8());
   auto dense_values =
diff --git a/cpp/src/parquet/arrow/generate_fuzz_corpus.cc b/cpp/src/parquet/arrow/generate_fuzz_corpus.cc
index c4d3638..e5d8660 100644
--- a/cpp/src/parquet/arrow/generate_fuzz_corpus.cc
+++ b/cpp/src/parquet/arrow/generate_fuzz_corpus.cc
@@ -61,10 +61,10 @@ Result<std::shared_ptr<RecordBatch>> ExampleBatch1() {
   {
     auto values = gen.Int64(kBatchSize * 10, -10000, 10000, /*null_probability=*/0.2);
     auto offsets = gen.Offsets(kBatchSize + 1, 0, static_cast<int32_t>(values->length()));
-    RETURN_NOT_OK(ListArray::FromArrays(*offsets, *values, default_memory_pool(), &d));
+    ARROW_ASSIGN_OR_RAISE(d, ListArray::FromArrays(*offsets, *values));
   }
   // A column of a repeated constant that will hopefully trigger RLE encoding
-  RETURN_NOT_OK(MakeArrayFromScalar(Int16Scalar(42), kBatchSize, &e));
+  ARROW_ASSIGN_OR_RAISE(e, MakeArrayFromScalar(Int16Scalar(42), kBatchSize));
   // A non-dict-encoded column
   no_dict = gen.String(kBatchSize, 0, 30, /*null_probability=*/0.2);
 
diff --git a/cpp/src/parquet/arrow/reader_internal.cc b/cpp/src/parquet/arrow/reader_internal.cc
index 25c7dad..2986e60 100644
--- a/cpp/src/parquet/arrow/reader_internal.cc
+++ b/cpp/src/parquet/arrow/reader_internal.cc
@@ -890,7 +890,7 @@ Status TransferDictionary(RecordReader* reader,
   DCHECK(dict_reader);
   *out = dict_reader->GetResult();
   if (!logical_value_type->Equals(*(*out)->type())) {
-    RETURN_NOT_OK((*out)->View(logical_value_type, out));
+    ARROW_ASSIGN_OR_RAISE(*out, (*out)->View(logical_value_type));
   }
   return Status::OK();
 }
@@ -907,7 +907,8 @@ Status TransferBinary(RecordReader* reader,
   auto chunks = binary_reader->GetBuilderChunks();
   for (const auto& chunk : chunks) {
     if (!chunk->type()->Equals(*logical_value_type)) {
-      return ChunkedArray(chunks).View(logical_value_type, out);
+      ARROW_ASSIGN_OR_RAISE(*out, ChunkedArray(chunks).View(logical_value_type));
+      return Status::OK();
     }
   }
   *out = std::make_shared<ChunkedArray>(chunks, logical_value_type);
diff --git a/cpp/src/parquet/encoding_test.cc b/cpp/src/parquet/encoding_test.cc
index 3d5fdb3..216da3e 100644
--- a/cpp/src/parquet/encoding_test.cc
+++ b/cpp/src/parquet/encoding_test.cc
@@ -739,7 +739,8 @@ std::shared_ptr<arrow::Array> EncodingAdHocTyped<FLBAType>::GetValues(int seed)
   std::shared_ptr<arrow::Array> values;
   ARROW_EXPECT_OK(
       rag.UInt64(size_, 0, std::numeric_limits<uint64_t>::max(), null_probability_)
-          ->View(arrow_type(), &values));
+          ->View(arrow_type())
+          .Value(&values));
   return values;
 }
 
diff --git a/python/pyarrow/array.pxi b/python/pyarrow/array.pxi
index 21016fe..39663da 100644
--- a/python/pyarrow/array.pxi
+++ b/python/pyarrow/array.pxi
@@ -715,7 +715,7 @@ cdef class Array(_PandasConvertible):
         cdef DataType type = ensure_type(target_type)
         cdef shared_ptr[CArray] result
         with nogil:
-            check_status(self.ap.View(type.sp_type, &result))
+            result = GetResultValue(self.ap.View(type.sp_type))
         return pyarrow_wrap_array(result)
 
     def sum(self):
@@ -1462,8 +1462,8 @@ cdef class ListArray(Array):
         _values = asarray(values)
 
         with nogil:
-            check_status(CListArray.FromArrays(_offsets.ap[0], _values.ap[0],
-                                               cpool, &out))
+            out = GetResultValue(
+                CListArray.FromArrays(_offsets.ap[0], _values.ap[0], cpool))
         cdef Array result = pyarrow_wrap_array(out)
         result.validate()
         return result
@@ -1544,9 +1544,9 @@ cdef class LargeListArray(Array):
         _values = asarray(values)
 
         with nogil:
-            check_status(CLargeListArray.FromArrays(_offsets.ap[0],
-                                                    _values.ap[0],
-                                                    cpool, &out))
+            out = GetResultValue(
+                CLargeListArray.FromArrays(_offsets.ap[0], _values.ap[0],
+                                           cpool))
         cdef Array result = pyarrow_wrap_array(out)
         result.validate()
         return result
@@ -1627,9 +1627,10 @@ cdef class MapArray(Array):
         _items = asarray(items)
 
         with nogil:
-            check_status(CMapArray.FromArrays(_offsets.sp_array,
-                                              _keys.sp_array, _items.sp_array,
-                                              cpool, &out))
+            out = GetResultValue(
+                CMapArray.FromArrays(_offsets.sp_array,
+                                     _keys.sp_array,
+                                     _items.sp_array, cpool))
         cdef Array result = pyarrow_wrap_array(out)
         result.validate()
         return result
@@ -1730,9 +1731,9 @@ cdef class UnionArray(Array):
             for x in type_codes:
                 c_type_codes.push_back(x)
         with nogil:
-            check_status(CUnionArray.MakeDense(
+            out = GetResultValue(CUnionArray.MakeDense(
                 deref(types.ap), deref(value_offsets.ap), c, c_field_names,
-                c_type_codes, &out))
+                c_type_codes))
         cdef Array result = pyarrow_wrap_array(out)
         result.validate()
         return result
@@ -1769,10 +1770,8 @@ cdef class UnionArray(Array):
             for x in type_codes:
                 c_type_codes.push_back(x)
         with nogil:
-            check_status(CUnionArray.MakeSparse(deref(types.ap), c,
-                                                c_field_names,
-                                                c_type_codes,
-                                                &out))
+            out = GetResultValue(CUnionArray.MakeSparse(
+                deref(types.ap), c, c_field_names, c_type_codes))
         cdef Array result = pyarrow_wrap_array(out)
         result.validate()
         return result
@@ -1942,10 +1941,9 @@ cdef class DictionaryArray(Array):
 
         if safe:
             with nogil:
-                check_status(
+                c_result = GetResultValue(
                     CDictionaryArray.FromArrays(c_type, _indices.sp_array,
-                                                _dictionary.sp_array,
-                                                &c_result))
+                                                _dictionary.sp_array))
         else:
             c_result.reset(new CDictionaryArray(c_type, _indices.sp_array,
                                                 _dictionary.sp_array))
@@ -2008,7 +2006,7 @@ cdef class StructArray(Array):
             CStructArray* sarr = <CStructArray*> self.ap
 
         with nogil:
-            check_status(sarr.Flatten(pool, &arrays))
+            arrays = GetResultValue(sarr.Flatten(pool))
 
         return [pyarrow_wrap_array(arr) for arr in arrays]
 
diff --git a/python/pyarrow/includes/libarrow.pxd b/python/pyarrow/includes/libarrow.pxd
index 489da44..ba356ce 100644
--- a/python/pyarrow/includes/libarrow.pxd
+++ b/python/pyarrow/includes/libarrow.pxd
@@ -179,8 +179,7 @@ cdef extern from "arrow/api.h" namespace "arrow" nogil:
 
         CStatus Validate() const
         CStatus ValidateFull() const
-        CStatus View(const shared_ptr[CDataType]& type,
-                     shared_ptr[CArray]* out)
+        CResult[shared_ptr[CArray]] View(const shared_ptr[CDataType]& type)
 
     shared_ptr[CArray] MakeArray(const shared_ptr[CArrayData]& data)
 
@@ -198,10 +197,10 @@ cdef extern from "arrow/api.h" namespace "arrow" nogil:
                          const shared_ptr[CArray]& dictionary)
 
         @staticmethod
-        CStatus FromArrays(const shared_ptr[CDataType]& type,
-                           const shared_ptr[CArray]& indices,
-                           const shared_ptr[CArray]& dictionary,
-                           shared_ptr[CArray]* out)
+        CResult[shared_ptr[CArray]] FromArrays(
+            const shared_ptr[CDataType]& type,
+            const shared_ptr[CArray]& indices,
+            const shared_ptr[CArray]& dictionary)
 
         shared_ptr[CArray] indices()
         shared_ptr[CArray] dictionary()
@@ -502,8 +501,8 @@ cdef extern from "arrow/api.h" namespace "arrow" nogil:
 
     cdef cppclass CListArray" arrow::ListArray"(CArray):
         @staticmethod
-        CStatus FromArrays(const CArray& offsets, const CArray& values,
-                           CMemoryPool* pool, shared_ptr[CArray]* out)
+        CResult[shared_ptr[CArray]] FromArrays(
+            const CArray& offsets, const CArray& values, CMemoryPool* pool)
 
         const int32_t* raw_value_offsets()
         int32_t value_offset(int i)
@@ -514,8 +513,8 @@ cdef extern from "arrow/api.h" namespace "arrow" nogil:
 
     cdef cppclass CLargeListArray" arrow::LargeListArray"(CArray):
         @staticmethod
-        CStatus FromArrays(const CArray& offsets, const CArray& values,
-                           CMemoryPool* pool, shared_ptr[CArray]* out)
+        CResult[shared_ptr[CArray]] FromArrays(
+            const CArray& offsets, const CArray& values, CMemoryPool* pool)
 
         int64_t value_offset(int i)
         int64_t value_length(int i)
@@ -535,10 +534,11 @@ cdef extern from "arrow/api.h" namespace "arrow" nogil:
 
     cdef cppclass CMapArray" arrow::MapArray"(CArray):
         @staticmethod
-        CStatus FromArrays(const shared_ptr[CArray]& offsets,
-                           const shared_ptr[CArray]& keys,
-                           const shared_ptr[CArray]& items,
-                           CMemoryPool* pool, shared_ptr[CArray]* out)
+        CResult[shared_ptr[CArray]] FromArrays(
+            const shared_ptr[CArray]& offsets,
+            const shared_ptr[CArray]& keys,
+            const shared_ptr[CArray]& items,
+            CMemoryPool* pool)
 
         shared_ptr[CArray] keys()
         shared_ptr[CArray] items()
@@ -550,19 +550,20 @@ cdef extern from "arrow/api.h" namespace "arrow" nogil:
 
     cdef cppclass CUnionArray" arrow::UnionArray"(CArray):
         @staticmethod
-        CStatus MakeSparse(const CArray& type_codes,
-                           const vector[shared_ptr[CArray]]& children,
-                           const vector[c_string]& field_names,
-                           const vector[int8_t]& type_codes,
-                           shared_ptr[CArray]* out)
+        CResult[shared_ptr[CArray]] MakeSparse(
+            const CArray& type_codes,
+            const vector[shared_ptr[CArray]]& children,
+            const vector[c_string]& field_names,
+            const vector[int8_t]& type_codes)
 
         @staticmethod
-        CStatus MakeDense(const CArray& type_codes,
-                          const CArray& value_offsets,
-                          const vector[shared_ptr[CArray]]& children,
-                          const vector[c_string]& field_names,
-                          const vector[int8_t]& type_codes,
-                          shared_ptr[CArray]* out)
+        CResult[shared_ptr[CArray]] MakeDense(
+            const CArray& type_codes,
+            const CArray& value_offsets,
+            const vector[shared_ptr[CArray]]& children,
+            const vector[c_string]& field_names,
+            const vector[int8_t]& type_codes)
+
         int8_t* raw_type_codes()
         int32_t value_offset(int i)
         int child_id(int64_t index)
@@ -629,7 +630,7 @@ cdef extern from "arrow/api.h" namespace "arrow" nogil:
         shared_ptr[CArray] field(int pos)
         shared_ptr[CArray] GetFieldByName(const c_string& name) const
 
-        CStatus Flatten(CMemoryPool* pool, vector[shared_ptr[CArray]]* out)
+        CResult[vector[shared_ptr[CArray]]] Flatten(CMemoryPool* pool)
 
     cdef cppclass CChunkedArray" arrow::ChunkedArray":
         CChunkedArray(const vector[shared_ptr[CArray]]& arrays)
@@ -645,8 +646,7 @@ cdef extern from "arrow/api.h" namespace "arrow" nogil:
         shared_ptr[CChunkedArray] Slice(int64_t offset, int64_t length) const
         shared_ptr[CChunkedArray] Slice(int64_t offset) const
 
-        CStatus Flatten(CMemoryPool* pool,
-                        vector[shared_ptr[CChunkedArray]]* out)
+        CResult[vector[shared_ptr[CChunkedArray]]] Flatten(CMemoryPool* pool)
 
         CStatus Validate() const
         CStatus ValidateFull() const
diff --git a/python/pyarrow/table.pxi b/python/pyarrow/table.pxi
index 4613eff..1976dde 100644
--- a/python/pyarrow/table.pxi
+++ b/python/pyarrow/table.pxi
@@ -289,7 +289,7 @@ cdef class ChunkedArray(_PandasConvertible):
             CMemoryPool* pool = maybe_unbox_memory_pool(memory_pool)
 
         with nogil:
-            check_status(self.chunked_array.Flatten(pool, &flattened))
+            flattened = GetResultValue(self.chunked_array.Flatten(pool))
 
         return [pyarrow_wrap_chunked_array(col) for col in flattened]
 
diff --git a/r/src/array.cpp b/r/src/array.cpp
index 4f3a411..6244674 100644
--- a/r/src/array.cpp
+++ b/r/src/array.cpp
@@ -143,9 +143,7 @@ bool Array__RangeEquals(const std::shared_ptr<arrow::Array>& self,
 // [[arrow::export]]
 std::shared_ptr<arrow::Array> Array__View(const std::shared_ptr<arrow::Array>& array,
                                           const std::shared_ptr<arrow::DataType>& type) {
-  std::shared_ptr<arrow::Array> out;
-  STOP_IF_NOT_OK(array->View(type, &out));
-  return out;
+  return VALUE_OR_STOP(array->View(type));
 }
 
 // [[arrow::export]]
@@ -196,10 +194,7 @@ std::shared_ptr<arrow::Array> StructArray__GetFieldByName(
 // [[arrow::export]]
 arrow::ArrayVector StructArray__Flatten(
     const std::shared_ptr<arrow::StructArray>& array) {
-  int nf = array->num_fields();
-  arrow::ArrayVector out(nf);
-  STOP_IF_NOT_OK(array->Flatten(arrow::default_memory_pool(), &out));
-  return out;
+  return VALUE_OR_STOP(array->Flatten());
 }
 
 // [[arrow::export]]
diff --git a/r/src/array_from_vector.cpp b/r/src/array_from_vector.cpp
index 9b73fc0..e1f44eb 100644
--- a/r/src/array_from_vector.cpp
+++ b/r/src/array_from_vector.cpp
@@ -274,9 +274,7 @@ std::shared_ptr<Array> MakeFactorArrayImpl(Rcpp::IntegerVector_ factor,
   SEXP levels = Rf_getAttrib(factor, R_LevelsSymbol);
   auto dict = MakeStringArray(levels, utf8());
 
-  std::shared_ptr<Array> out;
-  STOP_IF_NOT_OK(DictionaryArray::FromArrays(type, array_indices, dict, &out));
-  return out;
+  return VALUE_OR_STOP(DictionaryArray::FromArrays(type, array_indices, dict));
 }
 
 std::shared_ptr<Array> MakeFactorArray(Rcpp::IntegerVector_ factor,
@@ -1287,9 +1285,7 @@ std::shared_ptr<arrow::Array> DictionaryArray__FromArrays(
     const std::shared_ptr<arrow::DataType>& type,
     const std::shared_ptr<arrow::Array>& indices,
     const std::shared_ptr<arrow::Array>& dict) {
-  std::shared_ptr<arrow::Array> out;
-  STOP_IF_NOT_OK(arrow::DictionaryArray::FromArrays(type, indices, dict, &out));
-  return out;
+  return VALUE_OR_STOP(arrow::DictionaryArray::FromArrays(type, indices, dict));
 }
 
 #endif
diff --git a/r/src/array_to_vector.cpp b/r/src/array_to_vector.cpp
index 732f6d1..4162c82 100644
--- a/r/src/array_to_vector.cpp
+++ b/r/src/array_to_vector.cpp
@@ -397,8 +397,7 @@ class Converter_Struct : public Converter {
     auto struct_array = internal::checked_cast<arrow::StructArray*>(array.get());
     int nf = converters.size();
     // Flatten() deals with merging of nulls
-    ArrayVector arrays(nf);
-    STOP_IF_NOT_OK(struct_array->Flatten(default_memory_pool(), &arrays));
+    auto arrays = VALUE_OR_STOP(struct_array->Flatten(default_memory_pool()));
     for (int i = 0; i < nf; i++) {
       STOP_IF_NOT_OK(
           converters[i]->Ingest_some_nulls(VECTOR_ELT(data, i), arrays[i], start, n));
diff --git a/r/src/chunkedarray.cpp b/r/src/chunkedarray.cpp
index dcbde2d..749dc2d 100644
--- a/r/src/chunkedarray.cpp
+++ b/r/src/chunkedarray.cpp
@@ -74,9 +74,7 @@ std::shared_ptr<arrow::ChunkedArray> ChunkedArray__Slice2(
 std::shared_ptr<arrow::ChunkedArray> ChunkedArray__View(
     const std::shared_ptr<arrow::ChunkedArray>& array,
     const std::shared_ptr<arrow::DataType>& type) {
-  std::shared_ptr<arrow::ChunkedArray> out;
-  STOP_IF_NOT_OK(array->View(type, &out));
-  return out;
+  return VALUE_OR_STOP(array->View(type));
 }
 
 // [[arrow::export]]