You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@arrow.apache.org by bk...@apache.org on 2020/04/07 19:11:20 UTC

[arrow] branch master updated: ARROW-8361: [C++] Add Result APIs to Buffer methods and functions

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

bkietz 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 b665b47  ARROW-8361: [C++] Add Result<T> APIs to Buffer methods and functions
b665b47 is described below

commit b665b4731a21f06e908228765375acdda0c4f9a0
Author: Antoine Pitrou <an...@python.org>
AuthorDate: Tue Apr 7 15:11:01 2020 -0400

    ARROW-8361: [C++] Add Result<T> APIs to Buffer methods and functions
    
    Closes #6863 from pitrou/ARROW-8361-buffer-result-apis
    
    Authored-by: Antoine Pitrou <an...@python.org>
    Signed-off-by: Benjamin Kietzman <be...@gmail.com>
---
 c_glib/arrow-glib/buffer.cpp                      |  14 +--
 c_glib/arrow-glib/input-stream.cpp                |   6 +-
 cpp/src/arrow/array.cc                            |  17 ++-
 cpp/src/arrow/array/builder_adaptive.cc           |   2 +-
 cpp/src/arrow/array/concatenate.cc                |  12 +--
 cpp/src/arrow/array/dict_internal.cc              |  17 ++-
 cpp/src/arrow/array/dict_internal.h               |  23 ++--
 cpp/src/arrow/array/diff.cc                       |  13 +--
 cpp/src/arrow/array_dict_test.cc                  |   2 +-
 cpp/src/arrow/array_test.cc                       |   5 +-
 cpp/src/arrow/buffer.cc                           | 115 ++++++++++++--------
 cpp/src/arrow/buffer.h                            | 126 +++++++++-------------
 cpp/src/arrow/buffer_builder.h                    |   4 +-
 cpp/src/arrow/buffer_test.cc                      |  42 +++-----
 cpp/src/arrow/compute/context.cc                  |   3 +-
 cpp/src/arrow/compute/kernels/aggregate.cc        |   9 +-
 cpp/src/arrow/compute/kernels/boolean.cc          |   3 +-
 cpp/src/arrow/compute/kernels/nth_to_indices.cc   |   6 +-
 cpp/src/arrow/compute/kernels/sort_to_indices.cc  |   5 +-
 cpp/src/arrow/compute/kernels/take_internal.h     |   6 +-
 cpp/src/arrow/compute/kernels/take_test.cc        |   3 +-
 cpp/src/arrow/csv/parser.cc                       |   9 +-
 cpp/src/arrow/csv/reader.cc                       |   3 +-
 cpp/src/arrow/dataset/file_ipc_test.cc            |   6 +-
 cpp/src/arrow/device.cc                           |  14 +--
 cpp/src/arrow/filesystem/mockfs.cc                |   3 +-
 cpp/src/arrow/filesystem/s3fs.cc                  |   9 +-
 cpp/src/arrow/filesystem/util_internal.cc         |   3 +-
 cpp/src/arrow/flight/client.cc                    |   2 +-
 cpp/src/arrow/flight/flight_test.cc               |  27 ++---
 cpp/src/arrow/flight/internal.cc                  |   4 +-
 cpp/src/arrow/flight/test_util.cc                 |   2 +-
 cpp/src/arrow/gpu/cuda_memory.cc                  |   6 +-
 cpp/src/arrow/gpu/cuda_test.cc                    |  11 +-
 cpp/src/arrow/io/buffered.cc                      |   7 +-
 cpp/src/arrow/io/buffered_test.cc                 |   5 +-
 cpp/src/arrow/io/compressed.cc                    |  10 +-
 cpp/src/arrow/io/compressed_test.cc               |   7 +-
 cpp/src/arrow/io/file.cc                          |  10 +-
 cpp/src/arrow/io/hdfs.cc                          |  10 +-
 cpp/src/arrow/io/hdfs_test.cc                     |   6 +-
 cpp/src/arrow/io/memory.cc                        |   2 +-
 cpp/src/arrow/io/memory_benchmark.cc              |  11 +-
 cpp/src/arrow/io/memory_test.cc                   |  18 ++--
 cpp/src/arrow/ipc/json_internal.cc                |  14 +--
 cpp/src/arrow/ipc/message.cc                      |   2 +-
 cpp/src/arrow/ipc/metadata_internal.h             |   6 +-
 cpp/src/arrow/ipc/read_write_benchmark.cc         |   6 +-
 cpp/src/arrow/ipc/read_write_test.cc              |  10 +-
 cpp/src/arrow/ipc/reader.cc                       |   9 +-
 cpp/src/arrow/ipc/test_common.cc                  |   7 +-
 cpp/src/arrow/ipc/writer.cc                       |  44 ++++----
 cpp/src/arrow/json/chunked_builder.cc             |  19 ++--
 cpp/src/arrow/json/chunker_test.cc                |   8 +-
 cpp/src/arrow/json/reader.cc                      |   3 +-
 cpp/src/arrow/python/arrow_to_pandas.cc           |   5 +-
 cpp/src/arrow/python/numpy_to_arrow.cc            |  23 ++--
 cpp/src/arrow/sparse_tensor.cc                    |  13 ++-
 cpp/src/arrow/tensor/coo_converter.cc             |  14 ++-
 cpp/src/arrow/tensor/csc_converter.cc             |  14 +--
 cpp/src/arrow/tensor/csf_converter.cc             |   7 +-
 cpp/src/arrow/tensor/csr_converter.cc             |  14 +--
 cpp/src/arrow/tensor_test.cc                      |  28 +++--
 cpp/src/arrow/testing/gtest_common.h              |  14 ++-
 cpp/src/arrow/testing/gtest_util.h                |   3 +-
 cpp/src/arrow/testing/random.cc                   |  10 +-
 cpp/src/arrow/testing/util.cc                     |   5 +-
 cpp/src/arrow/testing/util.h                      |   5 +-
 cpp/src/arrow/util/bit_util.cc                    |  16 ++-
 cpp/src/arrow/util/bit_util_benchmark.cc          |   7 +-
 cpp/src/arrow/util/bit_util_test.cc               |  30 ++----
 cpp/src/arrow/util/io_util.cc                     |   5 +-
 cpp/src/gandiva/annotator_test.cc                 |  13 +--
 cpp/src/gandiva/projector.cc                      |  17 ++-
 cpp/src/gandiva/selection_vector.cc               |   2 +-
 cpp/src/gandiva/selection_vector_test.cc          |  27 ++---
 cpp/src/parquet/arrow/arrow_reader_writer_test.cc |   4 +-
 cpp/src/parquet/arrow/reader.cc                   |   6 +-
 cpp/src/parquet/arrow/reader_internal.cc          |  44 ++++----
 cpp/src/parquet/arrow/test_util.h                 |  15 +--
 cpp/src/parquet/bloom_filter.cc                   |   4 +-
 cpp/src/parquet/column_writer.cc                  |  10 +-
 cpp/src/parquet/column_writer_test.cc             |   3 +-
 cpp/src/parquet/encoding.cc                       |  17 ++-
 cpp/src/parquet/platform.cc                       |   6 +-
 python/pyarrow/includes/common.pxd                |  20 ++++
 python/pyarrow/includes/libarrow.pxd              |   8 +-
 python/pyarrow/io.pxi                             |  19 ++--
 r/src/array_from_vector.cpp                       |  10 +-
 89 files changed, 549 insertions(+), 635 deletions(-)

diff --git a/c_glib/arrow-glib/buffer.cpp b/c_glib/arrow-glib/buffer.cpp
index 4a0acd7..3e9b721 100644
--- a/c_glib/arrow-glib/buffer.cpp
+++ b/c_glib/arrow-glib/buffer.cpp
@@ -386,10 +386,9 @@ garrow_buffer_copy(GArrowBuffer *buffer,
                    GError **error)
 {
   auto arrow_buffer = garrow_buffer_get_raw(buffer);
-  std::shared_ptr<arrow::Buffer> arrow_copied_buffer;
-  auto status = arrow_buffer->Copy(start, size, &arrow_copied_buffer);
-  if (garrow_error_check(error, status, "[buffer][copy]")) {
-    return garrow_buffer_new_raw(&arrow_copied_buffer);
+  auto maybe_copied_buffer = arrow_buffer->CopySlice(start, size);
+  if (garrow::check(error, maybe_copied_buffer, "[buffer][copy]")) {
+    return garrow_buffer_new_raw(&(*maybe_copied_buffer));
   } else {
     return NULL;
   }
@@ -567,9 +566,10 @@ GArrowResizableBuffer *
 garrow_resizable_buffer_new(gint64 initial_size,
                             GError **error)
 {
-  std::shared_ptr<arrow::ResizableBuffer> arrow_buffer;
-  auto status = arrow::AllocateResizableBuffer(initial_size, &arrow_buffer);
-  if (garrow_error_check(error, status, "[resizable-buffer][new]")) {
+  auto maybe_buffer = arrow::AllocateResizableBuffer(initial_size);
+  if (garrow::check(error, maybe_buffer, "[resizable-buffer][new]")) {
+    auto arrow_buffer = std::shared_ptr<arrow::ResizableBuffer>(
+      *std::move(maybe_buffer));
     return garrow_resizable_buffer_new_raw(&arrow_buffer);
   } else {
     return NULL;
diff --git a/c_glib/arrow-glib/input-stream.cpp b/c_glib/arrow-glib/input-stream.cpp
index 41c5893..c98442e 100644
--- a/c_glib/arrow-glib/input-stream.cpp
+++ b/c_glib/arrow-glib/input-stream.cpp
@@ -698,9 +698,7 @@ namespace garrow {
 
     arrow::Result<std::shared_ptr<arrow::Buffer>>
     Read(int64_t n_bytes) override {
-      arrow::MemoryPool *pool = arrow::default_memory_pool();
-      std::shared_ptr<arrow::ResizableBuffer> buffer;
-      RETURN_NOT_OK(AllocateResizableBuffer(pool, n_bytes, &buffer));
+      ARROW_ASSIGN_OR_RAISE(auto buffer, arrow::AllocateResizableBuffer(n_bytes));
 
       std::lock_guard<std::mutex> guard(lock_);
       GError *error = NULL;
@@ -717,7 +715,7 @@ namespace garrow {
         if (n_read_bytes < n_bytes) {
           RETURN_NOT_OK(buffer->Resize(n_read_bytes));
         }
-        return buffer;
+        return std::move(buffer);
       }
     }
 
diff --git a/cpp/src/arrow/array.cc b/cpp/src/arrow/array.cc
index 0671713..7aa8f94 100644
--- a/cpp/src/arrow/array.cc
+++ b/cpp/src/arrow/array.cc
@@ -253,14 +253,14 @@ Status CleanListOffsets(const Array& offsets, MemoryPool* pool,
       return Status::Invalid("Last list offset should be non-null");
     }
 
-    std::shared_ptr<Buffer> clean_offsets, clean_valid_bits;
-    RETURN_NOT_OK(
-        AllocateBuffer(pool, num_offsets * sizeof(offset_type), &clean_offsets));
+    ARROW_ASSIGN_OR_RAISE(auto clean_offsets,
+                          AllocateBuffer(num_offsets * sizeof(offset_type), pool));
 
     // Copy valid bits, zero out the bit for the final offset
     // XXX why?
-    RETURN_NOT_OK(offsets.null_bitmap()->Copy(0, BitUtil::BytesForBits(num_offsets - 1),
-                                              &clean_valid_bits));
+    ARROW_ASSIGN_OR_RAISE(
+        auto clean_valid_bits,
+        offsets.null_bitmap()->CopySlice(0, BitUtil::BytesForBits(num_offsets - 1)));
     BitUtil::ClearBit(clean_valid_bits->mutable_data(), num_offsets);
     *validity_buf_out = clean_valid_bits;
 
@@ -277,7 +277,7 @@ Status CleanListOffsets(const Array& offsets, MemoryPool* pool,
       clean_raw_offsets[i] = current_offset;
     }
 
-    *offset_buf_out = clean_offsets;
+    *offset_buf_out = std::move(clean_offsets);
   } else {
     *validity_buf_out = offsets.null_bitmap();
     *offset_buf_out = typed_offsets.values();
@@ -1470,7 +1470,7 @@ class NullArrayFactory {
   Status CreateBuffer() {
     ARROW_ASSIGN_OR_RAISE(int64_t buffer_length,
                           GetBufferLength(type_, length_).Finish());
-    RETURN_NOT_OK(AllocateBuffer(pool_, buffer_length, &buffer_));
+    ARROW_ASSIGN_OR_RAISE(buffer_, AllocateBuffer(buffer_length, pool_));
     std::memset(buffer_->mutable_data(), 0, buffer_->size());
     return Status::OK();
   }
@@ -1567,8 +1567,7 @@ class RepeatedArrayFactory {
   Status Visit(const NullType&) { return Status::OK(); }
 
   Status Visit(const BooleanType&) {
-    std::shared_ptr<Buffer> buffer;
-    RETURN_NOT_OK(AllocateBitmap(pool_, length_, &buffer));
+    ARROW_ASSIGN_OR_RAISE(auto buffer, AllocateBitmap(length_, pool_));
     BitUtil::SetBitsTo(buffer->mutable_data(), 0, length_,
                        checked_cast<const BooleanScalar&>(scalar_).value);
     out_ = std::make_shared<BooleanArray>(length_, buffer);
diff --git a/cpp/src/arrow/array/builder_adaptive.cc b/cpp/src/arrow/array/builder_adaptive.cc
index a6bdb9c..7a75988 100644
--- a/cpp/src/arrow/array/builder_adaptive.cc
+++ b/cpp/src/arrow/array/builder_adaptive.cc
@@ -52,7 +52,7 @@ Status AdaptiveIntBuilderBase::Resize(int64_t capacity) {
 
   int64_t nbytes = capacity * int_size_;
   if (capacity_ == 0) {
-    RETURN_NOT_OK(AllocateResizableBuffer(pool_, nbytes, &data_));
+    ARROW_ASSIGN_OR_RAISE(data_, AllocateResizableBuffer(nbytes, pool_));
   } else {
     RETURN_NOT_OK(data_->Resize(nbytes));
   }
diff --git a/cpp/src/arrow/array/concatenate.cc b/cpp/src/arrow/array/concatenate.cc
index 4428e4b..30f1ec9 100644
--- a/cpp/src/arrow/array/concatenate.cc
+++ b/cpp/src/arrow/array/concatenate.cc
@@ -60,7 +60,7 @@ static Status ConcatenateBitmaps(const std::vector<Bitmap>& bitmaps, MemoryPool*
   for (size_t i = 0; i < bitmaps.size(); ++i) {
     out_length += bitmaps[i].range.length;
   }
-  RETURN_NOT_OK(AllocateBitmap(pool, out_length, out));
+  ARROW_ASSIGN_OR_RAISE(*out, AllocateBitmap(out_length, pool));
   uint8_t* dst = (*out)->mutable_data();
 
   int64_t bitmap_offset = 0;
@@ -101,7 +101,7 @@ static Status ConcatenateOffsets(const BufferVector& buffers, MemoryPool* pool,
   for (size_t i = 0; i < buffers.size(); ++i) {
     out_length += buffers[i]->size() / sizeof(Offset);
   }
-  RETURN_NOT_OK(AllocateBuffer(pool, (out_length + 1) * sizeof(Offset), out));
+  ARROW_ASSIGN_OR_RAISE(*out, AllocateBuffer((out_length + 1) * sizeof(Offset), pool));
   auto dst = reinterpret_cast<Offset*>((*out)->mutable_data());
 
   int64_t elements_length = 0;
@@ -179,21 +179,21 @@ class ConcatenateImpl {
 
   Status Visit(const FixedWidthType& fixed) {
     // handles numbers, decimal128, fixed_size_binary
-    return ConcatenateBuffers(Buffers(1, fixed), pool_, &out_.buffers[1]);
+    return ConcatenateBuffers(Buffers(1, fixed), pool_).Value(&out_.buffers[1]);
   }
 
   Status Visit(const BinaryType&) {
     std::vector<Range> value_ranges;
     RETURN_NOT_OK(ConcatenateOffsets<int32_t>(Buffers(1, sizeof(int32_t)), pool_,
                                               &out_.buffers[1], &value_ranges));
-    return ConcatenateBuffers(Buffers(2, value_ranges), pool_, &out_.buffers[2]);
+    return ConcatenateBuffers(Buffers(2, value_ranges), pool_).Value(&out_.buffers[2]);
   }
 
   Status Visit(const LargeBinaryType&) {
     std::vector<Range> value_ranges;
     RETURN_NOT_OK(ConcatenateOffsets<int64_t>(Buffers(1, sizeof(int64_t)), pool_,
                                               &out_.buffers[1], &value_ranges));
-    return ConcatenateBuffers(Buffers(2, value_ranges), pool_, &out_.buffers[2]);
+    return ConcatenateBuffers(Buffers(2, value_ranges), pool_).Value(&out_.buffers[2]);
   }
 
   Status Visit(const ListType&) {
@@ -240,7 +240,7 @@ class ConcatenateImpl {
 
     if (dictionaries_same) {
       out_.dictionary = in_[0].dictionary;
-      return ConcatenateBuffers(Buffers(1, *fixed), pool_, &out_.buffers[1]);
+      return ConcatenateBuffers(Buffers(1, *fixed), pool_).Value(&out_.buffers[1]);
     } else {
       return Status::NotImplemented("Concat with dictionary unification NYI");
     }
diff --git a/cpp/src/arrow/array/dict_internal.cc b/cpp/src/arrow/array/dict_internal.cc
index afd0167..63176a4 100644
--- a/cpp/src/arrow/array/dict_internal.cc
+++ b/cpp/src/arrow/array/dict_internal.cc
@@ -64,14 +64,13 @@ class DictionaryUnifierImpl : public DictionaryUnifier {
     }
     const ArrayType& values = checked_cast<const ArrayType&>(dictionary);
     if (out != nullptr) {
-      std::shared_ptr<Buffer> result;
-      RETURN_NOT_OK(
-          AllocateBuffer(pool_, dictionary.length() * sizeof(int32_t), &result));
+      ARROW_ASSIGN_OR_RAISE(auto result,
+                            AllocateBuffer(dictionary.length() * sizeof(int32_t), pool_));
       auto result_raw = reinterpret_cast<int32_t*>(result->mutable_data());
       for (int64_t i = 0; i < values.length(); ++i) {
         RETURN_NOT_OK(memo_table_.GetOrInsert(values.GetView(i), &result_raw[i]));
       }
-      *out = result;
+      *out = std::move(result);
     } else {
       for (int64_t i = 0; i < values.length(); ++i) {
         int32_t unused_memo_index;
@@ -198,9 +197,9 @@ Result<std::shared_ptr<Array>> DictionaryArray::Transpose(
   }
 
   // Default path: compute a buffer of transposed indices.
-  std::shared_ptr<Buffer> out_buffer;
-  RETURN_NOT_OK(AllocateBuffer(
-      pool, data_->length * out_index_type.bit_width() * CHAR_BIT, &out_buffer));
+  ARROW_ASSIGN_OR_RAISE(
+      auto out_buffer,
+      AllocateBuffer(data_->length * out_index_type.bit_width() * CHAR_BIT, pool));
 
   // Shift null buffer if the original offset is non-zero
   std::shared_ptr<Buffer> null_bitmap;
@@ -211,8 +210,8 @@ Result<std::shared_ptr<Array>> DictionaryArray::Transpose(
     null_bitmap = data_->buffers[0];
   }
 
-  auto out_data =
-      ArrayData::Make(type, data_->length, {null_bitmap, out_buffer}, data_->null_count);
+  auto out_data = ArrayData::Make(
+      type, data_->length, {null_bitmap, std::move(out_buffer)}, data_->null_count);
   out_data->dictionary = dictionary;
 
 #define TRANSPOSE_IN_OUT_CASE(IN_INDEX_TYPE, OUT_INDEX_TYPE)                 \
diff --git a/cpp/src/arrow/array/dict_internal.h b/cpp/src/arrow/array/dict_internal.h
index a9ac06c..5bf584c 100644
--- a/cpp/src/arrow/array/dict_internal.h
+++ b/cpp/src/arrow/array/dict_internal.h
@@ -97,14 +97,14 @@ struct DictionaryTraits<T, enable_if_has_c_type<T>> {
                                        const MemoTableType& memo_table,
                                        int64_t start_offset,
                                        std::shared_ptr<ArrayData>* out) {
-    std::shared_ptr<Buffer> dict_buffer;
     auto dict_length = static_cast<int64_t>(memo_table.size()) - start_offset;
     // This makes a copy, but we assume a dictionary array is usually small
     // compared to the size of the dictionary-using array.
     // (also, copying the dictionary values is cheap compared to the cost
     //  of building the memo table)
-    RETURN_NOT_OK(
-        AllocateBuffer(pool, TypeTraits<T>::bytes_required(dict_length), &dict_buffer));
+    ARROW_ASSIGN_OR_RAISE(
+        std::shared_ptr<Buffer> dict_buffer,
+        AllocateBuffer(TypeTraits<T>::bytes_required(dict_length), pool));
     memo_table.CopyValues(static_cast<int32_t>(start_offset),
                           reinterpret_cast<c_type*>(dict_buffer->mutable_data()));
 
@@ -128,19 +128,17 @@ struct DictionaryTraits<T, enable_if_base_binary<T>> {
                                        int64_t start_offset,
                                        std::shared_ptr<ArrayData>* out) {
     using offset_type = typename T::offset_type;
-    std::shared_ptr<Buffer> dict_offsets;
-    std::shared_ptr<Buffer> dict_data;
 
     // Create the offsets buffer
     auto dict_length = static_cast<int64_t>(memo_table.size() - start_offset);
-    RETURN_NOT_OK(
-        AllocateBuffer(pool, sizeof(offset_type) * (dict_length + 1), &dict_offsets));
+    ARROW_ASSIGN_OR_RAISE(auto dict_offsets,
+                          AllocateBuffer(sizeof(offset_type) * (dict_length + 1), pool));
     auto raw_offsets = reinterpret_cast<offset_type*>(dict_offsets->mutable_data());
     memo_table.CopyOffsets(static_cast<int32_t>(start_offset), raw_offsets);
 
     // Create the data buffer
     auto values_size = memo_table.values_size();
-    RETURN_NOT_OK(AllocateBuffer(pool, values_size, &dict_data));
+    ARROW_ASSIGN_OR_RAISE(auto dict_data, AllocateBuffer(values_size, pool));
     if (values_size > 0) {
       memo_table.CopyValues(static_cast<int32_t>(start_offset), dict_data->size(),
                             dict_data->mutable_data());
@@ -151,7 +149,8 @@ struct DictionaryTraits<T, enable_if_base_binary<T>> {
     RETURN_NOT_OK(
         ComputeNullBitmap(pool, memo_table, start_offset, &null_count, &null_bitmap));
 
-    *out = ArrayData::Make(type, dict_length, {null_bitmap, dict_offsets, dict_data},
+    *out = ArrayData::Make(type, dict_length,
+                           {null_bitmap, std::move(dict_offsets), std::move(dict_data)},
                            null_count);
 
     return Status::OK();
@@ -168,13 +167,12 @@ struct DictionaryTraits<T, enable_if_fixed_size_binary<T>> {
                                        int64_t start_offset,
                                        std::shared_ptr<ArrayData>* out) {
     const T& concrete_type = internal::checked_cast<const T&>(*type);
-    std::shared_ptr<Buffer> dict_data;
 
     // Create the data buffer
     auto dict_length = static_cast<int64_t>(memo_table.size() - start_offset);
     auto width_length = concrete_type.byte_width();
     auto data_length = dict_length * width_length;
-    RETURN_NOT_OK(AllocateBuffer(pool, data_length, &dict_data));
+    ARROW_ASSIGN_OR_RAISE(auto dict_data, AllocateBuffer(data_length, pool));
     auto data = dict_data->mutable_data();
 
     memo_table.CopyFixedWidthValues(static_cast<int32_t>(start_offset), width_length,
@@ -185,7 +183,8 @@ struct DictionaryTraits<T, enable_if_fixed_size_binary<T>> {
     RETURN_NOT_OK(
         ComputeNullBitmap(pool, memo_table, start_offset, &null_count, &null_bitmap));
 
-    *out = ArrayData::Make(type, dict_length, {null_bitmap, dict_data}, null_count);
+    *out = ArrayData::Make(type, dict_length, {null_bitmap, std::move(dict_data)},
+                           null_count);
     return Status::OK();
   }
 };
diff --git a/cpp/src/arrow/array/diff.cc b/cpp/src/arrow/array/diff.cc
index 7843533..690f0c1 100644
--- a/cpp/src/arrow/array/diff.cc
+++ b/cpp/src/arrow/array/diff.cc
@@ -305,9 +305,9 @@ class QuadraticSpaceMyersDiff {
     DCHECK(Done());
 
     int64_t length = edit_count_ + 1;
-    std::shared_ptr<Buffer> insert_buf, run_length_buf;
-    RETURN_NOT_OK(AllocateEmptyBitmap(pool, length, &insert_buf));
-    RETURN_NOT_OK(AllocateBuffer(pool, length * sizeof(int64_t), &run_length_buf));
+    ARROW_ASSIGN_OR_RAISE(auto insert_buf, AllocateEmptyBitmap(length, pool));
+    ARROW_ASSIGN_OR_RAISE(auto run_length_buf,
+                          AllocateBuffer(length * sizeof(int64_t), pool));
     auto run_length = reinterpret_cast<int64_t*>(run_length_buf->mutable_data());
 
     auto index = finish_index_;
@@ -336,9 +336,10 @@ class QuadraticSpaceMyersDiff {
     BitUtil::SetBitTo(insert_buf->mutable_data(), 0, false);
     run_length[0] = endpoint.base - base_begin_;
 
-    return StructArray::Make({std::make_shared<BooleanArray>(length, insert_buf),
-                              std::make_shared<Int64Array>(length, run_length_buf)},
-                             {field("insert", boolean()), field("run_length", int64())});
+    return StructArray::Make(
+        {std::make_shared<BooleanArray>(length, std::move(insert_buf)),
+         std::make_shared<Int64Array>(length, std::move(run_length_buf))},
+        {field("insert", boolean()), field("run_length", int64())});
   }
 
  private:
diff --git a/cpp/src/arrow/array_dict_test.cc b/cpp/src/arrow/array_dict_test.cc
index 28a072b..2b555c2 100644
--- a/cpp/src/arrow/array_dict_test.cc
+++ b/cpp/src/arrow/array_dict_test.cc
@@ -989,7 +989,7 @@ TEST(TestDictionary, Validate) {
   ASSERT_RAISES(Invalid, arr->ValidateFull());
 
   // Make the data buffer non-null
-  ASSERT_OK(AllocateBuffer(0, &buffers[2]));
+  ASSERT_OK_AND_ASSIGN(buffers[2], AllocateBuffer(0));
   arr = std::make_shared<DictionaryArray>(dict_type, indices, MakeArray(invalid_data));
   ASSERT_RAISES(Invalid, arr->ValidateFull());
 
diff --git a/cpp/src/arrow/array_test.cc b/cpp/src/arrow/array_test.cc
index 842914f..04e96cf 100644
--- a/cpp/src/arrow/array_test.cc
+++ b/cpp/src/arrow/array_test.cc
@@ -171,12 +171,11 @@ TEST_F(TestArray, SliceRecomputeNullCount) {
   ASSERT_EQ(5, slice->null_count());
 
   // No bitmap, compute 0
-  std::shared_ptr<Buffer> data;
   const int kBufferSize = 64;
-  ASSERT_OK(AllocateBuffer(pool_, kBufferSize, &data));
+  ASSERT_OK_AND_ASSIGN(auto data, AllocateBuffer(kBufferSize, pool_));
   memset(data->mutable_data(), 0, kBufferSize);
 
-  auto arr = std::make_shared<Int32Array>(16, data, nullptr, -1);
+  auto arr = std::make_shared<Int32Array>(16, std::move(data), nullptr, -1);
   ASSERT_EQ(0, arr->null_count());
 }
 
diff --git a/cpp/src/arrow/buffer.cc b/cpp/src/arrow/buffer.cc
index da8aee3..edb4796 100644
--- a/cpp/src/arrow/buffer.cc
+++ b/cpp/src/arrow/buffer.cc
@@ -29,24 +29,26 @@
 
 namespace arrow {
 
-Status Buffer::Copy(const int64_t start, const int64_t nbytes, MemoryPool* pool,
-                    std::shared_ptr<Buffer>* out) const {
+Result<std::shared_ptr<Buffer>> Buffer::CopySlice(const int64_t start,
+                                                  const int64_t nbytes,
+                                                  MemoryPool* pool) const {
   // Sanity checks
   ARROW_CHECK_LT(start, size_);
   ARROW_CHECK_LE(nbytes, size_ - start);
 
-  std::shared_ptr<ResizableBuffer> new_buffer;
-  RETURN_NOT_OK(AllocateResizableBuffer(pool, nbytes, &new_buffer));
-
+  ARROW_ASSIGN_OR_RAISE(auto new_buffer, AllocateResizableBuffer(nbytes, pool));
   std::memcpy(new_buffer->mutable_data(), data() + start, static_cast<size_t>(nbytes));
+  return std::move(new_buffer);
+}
 
-  *out = new_buffer;
-  return Status::OK();
+Status Buffer::Copy(const int64_t start, const int64_t nbytes, MemoryPool* pool,
+                    std::shared_ptr<Buffer>* out) const {
+  return CopySlice(start, nbytes, pool).Value(out);
 }
 
 Status Buffer::Copy(const int64_t start, const int64_t nbytes,
                     std::shared_ptr<Buffer>* out) const {
-  return Copy(start, nbytes, default_memory_pool(), out);
+  return CopySlice(start, nbytes).Value(out);
 }
 
 std::string Buffer::ToHexString() {
@@ -65,16 +67,21 @@ bool Buffer::Equals(const Buffer& other) const {
                              !memcmp(data_, other.data_, static_cast<size_t>(size_))));
 }
 
+Result<std::shared_ptr<Buffer>> Buffer::FromString(const std::string& data,
+                                                   MemoryPool* pool) {
+  auto size = static_cast<int64_t>(data.size());
+  ARROW_ASSIGN_OR_RAISE(auto buf, AllocateBuffer(size, pool));
+  std::copy(data.c_str(), data.c_str() + size, buf->mutable_data());
+  return std::move(buf);
+}
+
 Status Buffer::FromString(const std::string& data, MemoryPool* pool,
                           std::shared_ptr<Buffer>* out) {
-  auto size = static_cast<int64_t>(data.size());
-  RETURN_NOT_OK(AllocateBuffer(pool, size, out));
-  std::copy(data.c_str(), data.c_str() + size, (*out)->mutable_data());
-  return Status::OK();
+  return FromString(data, pool).Value(out);
 }
 
 Status Buffer::FromString(const std::string& data, std::shared_ptr<Buffer>* out) {
-  return FromString(data, default_memory_pool(), out);
+  return FromString(data).Value(out);
 }
 
 std::string Buffer::ToString() const {
@@ -232,84 +239,106 @@ class PoolBuffer : public ResizableBuffer {
 
 namespace {
 // A utility that does most of the work of the `AllocateBuffer` and
-// `AllocateResizableBuffer` methods. The argument `buffer` should be a smart pointer to a
-// PoolBuffer.
-template <typename PoolBufferPtr, typename BufferPtr>
-inline Status ResizePoolBuffer(PoolBufferPtr&& buffer, const int64_t size,
-                               BufferPtr* out) {
+// `AllocateResizableBuffer` methods. The argument `buffer` should be a smart pointer to
+// a PoolBuffer.
+template <typename BufferPtr, typename PoolBufferPtr>
+inline Result<BufferPtr> ResizePoolBuffer(PoolBufferPtr&& buffer, const int64_t size) {
   RETURN_NOT_OK(buffer->Resize(size));
   buffer->ZeroPadding();
-  *out = std::move(buffer);
-  return Status::OK();
+  return std::move(buffer);
 }
+
 }  // namespace
 
+Result<std::unique_ptr<Buffer>> AllocateBuffer(const int64_t size, MemoryPool* pool) {
+  return ResizePoolBuffer<std::unique_ptr<Buffer>>(PoolBuffer::MakeUnique(pool), size);
+}
+
 Status AllocateBuffer(MemoryPool* pool, const int64_t size,
                       std::shared_ptr<Buffer>* out) {
-  return ResizePoolBuffer(PoolBuffer::MakeShared(pool), size, out);
+  return AllocateBuffer(size, pool).Value(out);
+}
+
+Status AllocateBuffer(const int64_t size, std::shared_ptr<Buffer>* out) {
+  return AllocateBuffer(size).Value(out);
 }
 
 Status AllocateBuffer(MemoryPool* pool, const int64_t size,
                       std::unique_ptr<Buffer>* out) {
-  return ResizePoolBuffer(PoolBuffer::MakeUnique(pool), size, out);
+  return AllocateBuffer(size, pool).Value(out);
 }
 
-Status AllocateBuffer(const int64_t size, std::shared_ptr<Buffer>* out) {
-  return AllocateBuffer(nullptr, size, out);
+Status AllocateBuffer(const int64_t size, std::unique_ptr<Buffer>* out) {
+  return AllocateBuffer(size).Value(out);
 }
 
-Status AllocateBuffer(const int64_t size, std::unique_ptr<Buffer>* out) {
-  return AllocateBuffer(nullptr, size, out);
+Result<std::unique_ptr<ResizableBuffer>> AllocateResizableBuffer(const int64_t size,
+                                                                 MemoryPool* pool) {
+  return ResizePoolBuffer<std::unique_ptr<ResizableBuffer>>(PoolBuffer::MakeUnique(pool),
+                                                            size);
 }
 
 Status AllocateResizableBuffer(MemoryPool* pool, const int64_t size,
                                std::shared_ptr<ResizableBuffer>* out) {
-  return ResizePoolBuffer(PoolBuffer::MakeShared(pool), size, out);
+  return AllocateResizableBuffer(size, pool).Value(out);
+}
+
+Status AllocateResizableBuffer(const int64_t size,
+                               std::shared_ptr<ResizableBuffer>* out) {
+  return AllocateResizableBuffer(size).Value(out);
 }
 
 Status AllocateResizableBuffer(MemoryPool* pool, const int64_t size,
                                std::unique_ptr<ResizableBuffer>* out) {
-  return ResizePoolBuffer(PoolBuffer::MakeUnique(pool), size, out);
+  return AllocateResizableBuffer(size, pool).Value(out);
 }
 
 Status AllocateResizableBuffer(const int64_t size,
-                               std::shared_ptr<ResizableBuffer>* out) {
-  return AllocateResizableBuffer(nullptr, size, out);
+                               std::unique_ptr<ResizableBuffer>* out) {
+  return AllocateResizableBuffer(size).Value(out);
 }
 
-Status AllocateResizableBuffer(const int64_t size,
-                               std::unique_ptr<ResizableBuffer>* out) {
-  return AllocateResizableBuffer(nullptr, size, out);
+Result<std::shared_ptr<Buffer>> AllocateBitmap(int64_t length, MemoryPool* pool) {
+  return AllocateBuffer(BitUtil::BytesForBits(length), pool);
 }
 
 Status AllocateBitmap(MemoryPool* pool, int64_t length, std::shared_ptr<Buffer>* out) {
-  return AllocateBuffer(pool, BitUtil::BytesForBits(length), out);
+  return AllocateBitmap(length, pool).Value(out);
+}
+
+Result<std::shared_ptr<Buffer>> AllocateEmptyBitmap(int64_t length, MemoryPool* pool) {
+  ARROW_ASSIGN_OR_RAISE(auto buf, AllocateBitmap(length, pool));
+  memset(buf->mutable_data(), 0, static_cast<size_t>(buf->size()));
+  return buf;
 }
 
 Status AllocateEmptyBitmap(MemoryPool* pool, int64_t length,
                            std::shared_ptr<Buffer>* out) {
-  RETURN_NOT_OK(AllocateBitmap(pool, length, out));
-  memset((*out)->mutable_data(), 0, static_cast<size_t>((*out)->size()));
-  return Status::OK();
+  return AllocateEmptyBitmap(length, pool).Value(out);
 }
 
 Status AllocateEmptyBitmap(int64_t length, std::shared_ptr<Buffer>* out) {
-  return AllocateEmptyBitmap(default_memory_pool(), length, out);
+  return AllocateEmptyBitmap(length).Value(out);
 }
 
-Status ConcatenateBuffers(const std::vector<std::shared_ptr<Buffer>>& buffers,
-                          MemoryPool* pool, std::shared_ptr<Buffer>* out) {
+Result<std::shared_ptr<Buffer>> ConcatenateBuffers(
+    const std::vector<std::shared_ptr<Buffer>>& buffers, MemoryPool* pool) {
   int64_t out_length = 0;
   for (const auto& buffer : buffers) {
     out_length += buffer->size();
   }
-  RETURN_NOT_OK(AllocateBuffer(pool, out_length, out));
-  auto out_data = (*out)->mutable_data();
+  ARROW_ASSIGN_OR_RAISE(auto out, AllocateBuffer(out_length, pool));
+  auto out_data = out->mutable_data();
   for (const auto& buffer : buffers) {
     std::memcpy(out_data, buffer->data(), buffer->size());
     out_data += buffer->size();
   }
-  return Status::OK();
+  return std::move(out);
+}
+
+Status ConcatenateBuffers(const std::vector<std::shared_ptr<Buffer>>& buffers,
+                          MemoryPool* pool, std::shared_ptr<Buffer>* out) {
+  return ConcatenateBuffers(buffers, pool).Value(out);
 }
 
 }  // namespace arrow
diff --git a/cpp/src/arrow/buffer.h b/cpp/src/arrow/buffer.h
index fbc7b6b..807b16e 100644
--- a/cpp/src/arrow/buffer.h
+++ b/cpp/src/arrow/buffer.h
@@ -122,10 +122,15 @@ class ARROW_EXPORT Buffer {
   bool Equals(const Buffer& other) const;
 
   /// Copy a section of the buffer into a new Buffer.
+  Result<std::shared_ptr<Buffer>> CopySlice(
+      const int64_t start, const int64_t nbytes,
+      MemoryPool* pool = default_memory_pool()) const;
+
+  ARROW_DEPRECATED("Use CopySlice")
   Status Copy(const int64_t start, const int64_t nbytes, MemoryPool* pool,
               std::shared_ptr<Buffer>* out) const;
 
-  /// Copy a section of the buffer using the default memory pool into a new Buffer.
+  ARROW_DEPRECATED("Use CopySlice")
   Status Copy(const int64_t start, const int64_t nbytes,
               std::shared_ptr<Buffer>* out) const;
 
@@ -142,16 +147,18 @@ class ARROW_EXPORT Buffer {
 
   /// \brief Construct a new buffer that owns its memory from a std::string
   ///
+  /// The string data is copied into the newly-allocated buffer memory.
+  ///
   /// \param[in] data a std::string object
   /// \param[in] pool a memory pool
-  /// \param[out] out the created buffer
-  ///
-  /// \return Status message
+  static Result<std::shared_ptr<Buffer>> FromString(
+      const std::string& data, MemoryPool* pool ARROW_MEMORY_POOL_DEFAULT);
+
+  ARROW_DEPRECATED("Use Result-returning version")
   static Status FromString(const std::string& data, MemoryPool* pool,
                            std::shared_ptr<Buffer>* out);
 
-  /// \brief Construct a new buffer that owns its memory from a std::string
-  /// using the default memory pool
+  ARROW_DEPRECATED("Use Result-returning version")
   static Status FromString(const std::string& data, std::shared_ptr<Buffer>* out);
 
   /// \brief Construct an immutable buffer that takes ownership of the contents
@@ -447,110 +454,80 @@ class ARROW_EXPORT ResizableBuffer : public MutableBuffer {
 
 /// \brief Allocate a fixed size mutable buffer from a memory pool, zero its padding.
 ///
-/// \param[in] pool a memory pool
 /// \param[in] size size of buffer to allocate
-/// \param[out] out the allocated buffer (contains padding)
-///
-/// \return Status message
+/// \param[in] pool a memory pool
 ARROW_EXPORT
-Status AllocateBuffer(MemoryPool* pool, const int64_t size, std::shared_ptr<Buffer>* out);
+Result<std::unique_ptr<Buffer>> AllocateBuffer(const int64_t size,
+                                               MemoryPool* pool = NULLPTR);
 
-/// \brief Allocate a fixed size mutable buffer from a memory pool, zero its padding.
-///
-/// \param[in] pool a memory pool
-/// \param[in] size size of buffer to allocate
-/// \param[out] out the allocated buffer (contains padding)
-///
-/// \return Status message
+ARROW_DEPRECATED("Use Result-returning version")
 ARROW_EXPORT
-Status AllocateBuffer(MemoryPool* pool, const int64_t size, std::unique_ptr<Buffer>* out);
+Status AllocateBuffer(MemoryPool* pool, const int64_t size, std::shared_ptr<Buffer>* out);
 
-/// \brief Allocate a fixed-size mutable buffer from the default memory pool
-///
-/// \param[in] size size of buffer to allocate
-/// \param[out] out the allocated buffer (contains padding)
-///
-/// \return Status message
+ARROW_DEPRECATED("Use Result-returning version")
 ARROW_EXPORT
 Status AllocateBuffer(const int64_t size, std::shared_ptr<Buffer>* out);
 
-/// \brief Allocate a fixed-size mutable buffer from the default memory pool
-///
-/// \param[in] size size of buffer to allocate
-/// \param[out] out the allocated buffer (contains padding)
-///
-/// \return Status message
+ARROW_DEPRECATED("Use Result-returning version")
+ARROW_EXPORT
+Status AllocateBuffer(MemoryPool* pool, const int64_t size, std::unique_ptr<Buffer>* out);
+
+ARROW_DEPRECATED("Use Result-returning version")
 ARROW_EXPORT
 Status AllocateBuffer(const int64_t size, std::unique_ptr<Buffer>* out);
 
 /// \brief Allocate a resizeable buffer from a memory pool, zero its padding.
 ///
-/// \param[in] pool a memory pool
 /// \param[in] size size of buffer to allocate
-/// \param[out] out the allocated buffer
-///
-/// \return Status message
+/// \param[in] pool a memory pool
 ARROW_EXPORT
-Status AllocateResizableBuffer(MemoryPool* pool, const int64_t size,
-                               std::shared_ptr<ResizableBuffer>* out);
+Result<std::unique_ptr<ResizableBuffer>> AllocateResizableBuffer(
+    const int64_t size, MemoryPool* pool = NULLPTR);
 
-/// \brief Allocate a resizeable buffer from a memory pool, zero its padding.
-///
-/// \param[in] pool a memory pool
-/// \param[in] size size of buffer to allocate
-/// \param[out] out the allocated buffer
-///
-/// \return Status message
+ARROW_DEPRECATED("Use Result-returning version")
 ARROW_EXPORT
 Status AllocateResizableBuffer(MemoryPool* pool, const int64_t size,
-                               std::unique_ptr<ResizableBuffer>* out);
+                               std::shared_ptr<ResizableBuffer>* out);
 
-/// \brief Allocate a resizeable buffer from the default memory pool
-///
-/// \param[in] size size of buffer to allocate
-/// \param[out] out the allocated buffer
-///
-/// \return Status message
+ARROW_DEPRECATED("Use Result-returning version")
 ARROW_EXPORT
 Status AllocateResizableBuffer(const int64_t size, std::shared_ptr<ResizableBuffer>* out);
 
-/// \brief Allocate a resizeable buffer from the default memory pool
-///
-/// \param[in] size size of buffer to allocate
-/// \param[out] out the allocated buffer
-///
-/// \return Status message
+ARROW_DEPRECATED("Use Result-returning version")
+ARROW_EXPORT
+Status AllocateResizableBuffer(MemoryPool* pool, const int64_t size,
+                               std::unique_ptr<ResizableBuffer>* out);
+
+ARROW_DEPRECATED("Use Result-returning version")
 ARROW_EXPORT
 Status AllocateResizableBuffer(const int64_t size, std::unique_ptr<ResizableBuffer>* out);
 
 /// \brief Allocate a bitmap buffer from a memory pool
 /// no guarantee on values is provided.
 ///
-/// \param[in] pool memory pool to allocate memory from
 /// \param[in] length size in bits of bitmap to allocate
-/// \param[out] out the resulting buffer
-///
-/// \return Status message
+/// \param[in] pool memory pool to allocate memory from
+ARROW_EXPORT
+Result<std::shared_ptr<Buffer>> AllocateBitmap(int64_t length,
+                                               MemoryPool* pool = NULLPTR);
+
 ARROW_EXPORT
 Status AllocateBitmap(MemoryPool* pool, int64_t length, std::shared_ptr<Buffer>* out);
 
 /// \brief Allocate a zero-initialized bitmap buffer from a memory pool
 ///
-/// \param[in] pool memory pool to allocate memory from
 /// \param[in] length size in bits of bitmap to allocate
-/// \param[out] out the resulting buffer (zero-initialized).
-///
-/// \return Status message
+/// \param[in] pool memory pool to allocate memory from
+ARROW_EXPORT
+Result<std::shared_ptr<Buffer>> AllocateEmptyBitmap(int64_t length,
+                                                    MemoryPool* pool = NULLPTR);
+
+ARROW_DEPRECATED("Use Result-returning version")
 ARROW_EXPORT
 Status AllocateEmptyBitmap(MemoryPool* pool, int64_t length,
                            std::shared_ptr<Buffer>* out);
 
-/// \brief Allocate a zero-initialized bitmap buffer from the default memory pool
-///
-/// \param[in] length size in bits of bitmap to allocate
-/// \param[out] out the resulting buffer
-///
-/// \return Status message
+ARROW_DEPRECATED("Use Result-returning version")
 ARROW_EXPORT
 Status AllocateEmptyBitmap(int64_t length, std::shared_ptr<Buffer>* out);
 
@@ -558,9 +535,10 @@ Status AllocateEmptyBitmap(int64_t length, std::shared_ptr<Buffer>* out);
 ///
 /// \param[in] buffers to be concatenated
 /// \param[in] pool memory pool to allocate the new buffer from
-/// \param[out] out the concatenated buffer
-///
-/// \return Status
+ARROW_EXPORT
+Result<std::shared_ptr<Buffer>> ConcatenateBuffers(const BufferVector& buffers,
+                                                   MemoryPool* pool = NULLPTR);
+
 ARROW_EXPORT
 Status ConcatenateBuffers(const BufferVector& buffers, MemoryPool* pool,
                           std::shared_ptr<Buffer>* out);
diff --git a/cpp/src/arrow/buffer_builder.h b/cpp/src/arrow/buffer_builder.h
index 20c981d..cd59db6 100644
--- a/cpp/src/arrow/buffer_builder.h
+++ b/cpp/src/arrow/buffer_builder.h
@@ -73,7 +73,7 @@ class ARROW_EXPORT BufferBuilder {
       return Status::OK();
     }
     if (buffer_ == NULLPTR) {
-      ARROW_RETURN_NOT_OK(AllocateResizableBuffer(pool_, new_capacity, &buffer_));
+      ARROW_ASSIGN_OR_RAISE(buffer_, AllocateResizableBuffer(new_capacity, pool_));
     } else {
       ARROW_RETURN_NOT_OK(buffer_->Resize(new_capacity, shrink_to_fit));
     }
@@ -155,7 +155,7 @@ class ARROW_EXPORT BufferBuilder {
     if (size_ != 0) buffer_->ZeroPadding();
     *out = buffer_;
     if (*out == NULLPTR) {
-      ARROW_RETURN_NOT_OK(AllocateBuffer(pool_, 0, out));
+      ARROW_ASSIGN_OR_RAISE(*out, AllocateBuffer(0, pool_));
     }
     Reset();
     return Status::OK();
diff --git a/cpp/src/arrow/buffer_test.cc b/cpp/src/arrow/buffer_test.cc
index 5076f55..460da3d 100644
--- a/cpp/src/arrow/buffer_test.cc
+++ b/cpp/src/arrow/buffer_test.cc
@@ -319,8 +319,7 @@ TEST_F(TestDevice, View) {
 }
 
 TEST(TestAllocate, Basics) {
-  std::shared_ptr<Buffer> new_buffer;
-  ASSERT_OK(AllocateBuffer(1024, &new_buffer));
+  ASSERT_OK_AND_ASSIGN(auto new_buffer, AllocateBuffer(1024));
   auto mm = new_buffer->memory_manager();
   ASSERT_TRUE(mm->is_cpu());
   ASSERT_EQ(mm.get(), default_cpu_memory_manager().get());
@@ -328,7 +327,7 @@ TEST(TestAllocate, Basics) {
   ASSERT_EQ(cpu_mm->pool(), default_memory_pool());
 
   auto pool = std::make_shared<ProxyMemoryPool>(default_memory_pool());
-  ASSERT_OK(AllocateBuffer(pool.get(), 1024, &new_buffer));
+  ASSERT_OK_AND_ASSIGN(new_buffer, AllocateBuffer(1024, pool.get()));
   mm = new_buffer->memory_manager();
   ASSERT_TRUE(mm->is_cpu());
   cpu_mm = checked_pointer_cast<CPUMemoryManager>(mm);
@@ -337,16 +336,14 @@ TEST(TestAllocate, Basics) {
 }
 
 TEST(TestAllocate, Bitmap) {
-  std::shared_ptr<Buffer> new_buffer;
-  ARROW_EXPECT_OK(AllocateBitmap(default_memory_pool(), 100, &new_buffer));
+  ASSERT_OK_AND_ASSIGN(auto new_buffer, AllocateBitmap(100));
   AssertIsCPUBuffer(*new_buffer);
   EXPECT_GE(new_buffer->size(), 13);
   EXPECT_EQ(new_buffer->capacity() % 8, 0);
 }
 
 TEST(TestAllocate, EmptyBitmap) {
-  std::shared_ptr<Buffer> new_buffer;
-  ARROW_EXPECT_OK(AllocateEmptyBitmap(default_memory_pool(), 100, &new_buffer));
+  ASSERT_OK_AND_ASSIGN(auto new_buffer, AllocateEmptyBitmap(100));
   AssertIsCPUBuffer(*new_buffer);
   EXPECT_EQ(new_buffer->size(), 13);
   EXPECT_EQ(new_buffer->capacity() % 8, 0);
@@ -369,7 +366,7 @@ TEST(TestBuffer, FromStdStringWithMemory) {
 
   {
     std::string temp = "hello, world";
-    ASSERT_OK(Buffer::FromString(temp, &buf));
+    ASSERT_OK_AND_ASSIGN(buf, Buffer::FromString(temp));
     AssertIsCPUBuffer(*buf);
     ASSERT_EQ(0, memcmp(buf->data(), temp.c_str(), temp.size()));
     ASSERT_EQ(static_cast<int64_t>(temp.size()), buf->size());
@@ -424,16 +421,14 @@ TEST(TestBuffer, EqualsWithSameBuffer) {
   pool->Free(rawBuffer, bufferSize);
 }
 
-TEST(TestBuffer, Copy) {
+TEST(TestBuffer, CopySlice) {
   std::string data_str = "some data to copy";
 
   auto data = reinterpret_cast<const uint8_t*>(data_str.c_str());
 
   Buffer buf(data, data_str.size());
 
-  std::shared_ptr<Buffer> out;
-
-  ASSERT_OK(buf.Copy(5, 4, &out));
+  ASSERT_OK_AND_ASSIGN(auto out, buf.CopySlice(5, 4));
   AssertIsCPUBuffer(*out);
 
   Buffer expected(data + 5, 4);
@@ -499,8 +494,7 @@ TEST(TestBuffer, SliceMutableBuffer) {
   std::string data_str = "some data to slice";
   auto data = reinterpret_cast<const uint8_t*>(data_str.c_str());
 
-  std::shared_ptr<Buffer> buffer;
-  ASSERT_OK(AllocateBuffer(50, &buffer));
+  ASSERT_OK_AND_ASSIGN(std::shared_ptr<Buffer> buffer, AllocateBuffer(50));
 
   memcpy(buffer->mutable_data(), data, data_str.size());
 
@@ -525,8 +519,7 @@ TEST(TestBuffer, GetReader) {
 }
 
 TEST(TestBuffer, GetWriter) {
-  std::shared_ptr<Buffer> buf;
-  ASSERT_OK(AllocateBuffer(9, &buf));
+  ASSERT_OK_AND_ASSIGN(std::shared_ptr<Buffer> buf, AllocateBuffer(9));
   ASSERT_OK_AND_ASSIGN(auto writer, Buffer::GetWriter(buf));
   ASSERT_OK(writer->Write(reinterpret_cast<const uint8_t*>("some data"), 9));
   AssertBufferEqual(*buf, "some data");
@@ -564,7 +557,7 @@ void TestZeroSizeAllocateBuffer(MemoryPool* pool, AllocateFunction&& allocate_fu
 TEST(TestAllocateBuffer, ZeroSize) {
   MemoryPool* pool = default_memory_pool();
   auto allocate_func = [](MemoryPool* pool, int64_t size, std::shared_ptr<Buffer>* out) {
-    return AllocateBuffer(pool, size, out);
+    return AllocateBuffer(size, pool).Value(out);
   };
   TestZeroSizeAllocateBuffer(pool, allocate_func);
 }
@@ -572,9 +565,8 @@ TEST(TestAllocateBuffer, ZeroSize) {
 TEST(TestAllocateResizableBuffer, ZeroSize) {
   MemoryPool* pool = default_memory_pool();
   auto allocate_func = [](MemoryPool* pool, int64_t size, std::shared_ptr<Buffer>* out) {
-    std::shared_ptr<ResizableBuffer> res;
-    RETURN_NOT_OK(AllocateResizableBuffer(pool, size, &res));
-    *out = res;
+    ARROW_ASSIGN_OR_RAISE(auto resizable, AllocateResizableBuffer(size, pool));
+    *out = std::move(resizable);
     return Status::OK();
   };
   TestZeroSizeAllocateBuffer(pool, allocate_func);
@@ -586,7 +578,7 @@ TEST(TestAllocateResizableBuffer, ZeroResize) {
   {
     std::shared_ptr<ResizableBuffer> buffer;
 
-    ASSERT_OK(AllocateResizableBuffer(pool, 1000, &buffer));
+    ASSERT_OK_AND_ASSIGN(buffer, AllocateResizableBuffer(1000, pool));
     ASSERT_EQ(buffer->size(), 1000);
     ASSERT_NE(buffer->data(), nullptr);
     ASSERT_EQ(buffer->mutable_data(), buffer->data());
@@ -745,14 +737,14 @@ TYPED_TEST(TypedTestBuffer, IsMutableFlag) {
   AssertIsCPUBuffer(mbuf);
 
   TypeParam pool_buf;
-  ASSERT_OK(AllocateResizableBuffer(0, &pool_buf));
+  ASSERT_OK_AND_ASSIGN(pool_buf, AllocateResizableBuffer(0));
   ASSERT_TRUE(pool_buf->is_mutable());
   AssertIsCPUBuffer(*pool_buf);
 }
 
 TYPED_TEST(TypedTestBuffer, Resize) {
   TypeParam buf;
-  ASSERT_OK(AllocateResizableBuffer(0, &buf));
+  ASSERT_OK_AND_ASSIGN(buf, AllocateResizableBuffer(0));
   AssertIsCPUBuffer(*buf);
 
   ASSERT_EQ(0, buf->size());
@@ -777,7 +769,7 @@ TYPED_TEST(TypedTestBuffer, Resize) {
 
 TYPED_TEST(TypedTestBuffer, TypedResize) {
   TypeParam buf;
-  ASSERT_OK(AllocateResizableBuffer(0, &buf));
+  ASSERT_OK_AND_ASSIGN(buf, AllocateResizableBuffer(0));
 
   ASSERT_EQ(0, buf->size());
   ASSERT_OK(buf->template TypedResize<double>(100));
@@ -800,7 +792,7 @@ TYPED_TEST(TypedTestBuffer, ResizeOOM) {
 #ifndef ADDRESS_SANITIZER
   // realloc fails, even though there may be no explicit limit
   TypeParam buf;
-  ASSERT_OK(AllocateResizableBuffer(0, &buf));
+  ASSERT_OK_AND_ASSIGN(buf, AllocateResizableBuffer(0));
   ASSERT_OK(buf->Resize(100));
   int64_t to_alloc = std::min<uint64_t>(std::numeric_limits<int64_t>::max(),
                                         std::numeric_limits<size_t>::max());
diff --git a/cpp/src/arrow/compute/context.cc b/cpp/src/arrow/compute/context.cc
index 56ec597..dade246 100644
--- a/cpp/src/arrow/compute/context.cc
+++ b/cpp/src/arrow/compute/context.cc
@@ -20,6 +20,7 @@
 #include <memory>
 
 #include "arrow/buffer.h"
+#include "arrow/result.h"
 #include "arrow/util/cpu_info.h"
 
 namespace arrow {
@@ -31,7 +32,7 @@ FunctionContext::FunctionContext(MemoryPool* pool)
 MemoryPool* FunctionContext::memory_pool() const { return pool_; }
 
 Status FunctionContext::Allocate(const int64_t nbytes, std::shared_ptr<Buffer>* out) {
-  return AllocateBuffer(pool_, nbytes, out);
+  return AllocateBuffer(nbytes, pool_).Value(out);
 }
 
 void FunctionContext::SetStatus(const Status& status) {
diff --git a/cpp/src/arrow/compute/kernels/aggregate.cc b/cpp/src/arrow/compute/kernels/aggregate.cc
index 7a458e3..9033758 100644
--- a/cpp/src/arrow/compute/kernels/aggregate.cc
+++ b/cpp/src/arrow/compute/kernels/aggregate.cc
@@ -38,10 +38,11 @@ class ManagedAggregateState {
 
   static std::shared_ptr<ManagedAggregateState> Make(
       std::shared_ptr<AggregateFunction>& desc, MemoryPool* pool) {
-    std::shared_ptr<Buffer> buf;
-    if (!AllocateBuffer(pool, desc->Size(), &buf).ok()) return nullptr;
-
-    return std::make_shared<ManagedAggregateState>(desc, std::move(buf));
+    auto maybe_buf = AllocateBuffer(desc->Size(), pool);
+    if (!maybe_buf.ok()) {
+      return nullptr;
+    }
+    return std::make_shared<ManagedAggregateState>(desc, *std::move(maybe_buf));
   }
 
  private:
diff --git a/cpp/src/arrow/compute/kernels/boolean.cc b/cpp/src/arrow/compute/kernels/boolean.cc
index 2218a68..29fe09f 100644
--- a/cpp/src/arrow/compute/kernels/boolean.cc
+++ b/cpp/src/arrow/compute/kernels/boolean.cc
@@ -119,7 +119,8 @@ class BinaryBooleanKernel : public BinaryKernel {
     bitmaps[RIGHT_VALID] = {right.buffers[0], right.offset, right.length};
     bitmaps[RIGHT_DATA] = {right.buffers[1], right.offset, right.length};
 
-    RETURN_NOT_OK(AllocateEmptyBitmap(ctx->memory_pool(), out->length, &out->buffers[0]));
+    ARROW_ASSIGN_OR_RAISE(out->buffers[0],
+                          AllocateEmptyBitmap(out->length, ctx->memory_pool()));
 
     auto out_validity = out->GetMutableValues<uint64_t>(0);
     auto out_data = out->GetMutableValues<uint64_t>(1);
diff --git a/cpp/src/arrow/compute/kernels/nth_to_indices.cc b/cpp/src/arrow/compute/kernels/nth_to_indices.cc
index 5972395..bd28add 100644
--- a/cpp/src/arrow/compute/kernels/nth_to_indices.cc
+++ b/cpp/src/arrow/compute/kernels/nth_to_indices.cc
@@ -18,6 +18,7 @@
 #include "arrow/compute/kernels/nth_to_indices.h"
 
 #include <algorithm>
+#include <utility>
 
 #include "arrow/builder.h"
 #include "arrow/compute/context.h"
@@ -62,15 +63,14 @@ class NthToIndicesKernelImpl final : public NthToIndicesKernel {
       return Status::IndexError("NthToIndices index out of bound");
     }
 
-    std::shared_ptr<Buffer> indices_buf;
     int64_t buf_size = values->length() * sizeof(uint64_t);
-    RETURN_NOT_OK(AllocateBuffer(ctx->memory_pool(), buf_size, &indices_buf));
+    ARROW_ASSIGN_OR_RAISE(auto indices_buf, AllocateBuffer(buf_size, ctx->memory_pool()));
 
     int64_t* indices_begin = reinterpret_cast<int64_t*>(indices_buf->mutable_data());
     int64_t* indices_end = indices_begin + values->length();
 
     std::iota(indices_begin, indices_end, 0);
-    *offsets = std::make_shared<UInt64Array>(values->length(), indices_buf);
+    *offsets = std::make_shared<UInt64Array>(values->length(), std::move(indices_buf));
 
     if (n == values->length()) {
       return Status::OK();
diff --git a/cpp/src/arrow/compute/kernels/sort_to_indices.cc b/cpp/src/arrow/compute/kernels/sort_to_indices.cc
index 805826a..01bd570 100644
--- a/cpp/src/arrow/compute/kernels/sort_to_indices.cc
+++ b/cpp/src/arrow/compute/kernels/sort_to_indices.cc
@@ -227,15 +227,14 @@ class SortToIndicesKernelImpl : public SortToIndicesKernel {
 
   Status SortToIndicesImpl(FunctionContext* ctx, const std::shared_ptr<ArrayType>& values,
                            std::shared_ptr<Array>* offsets) {
-    std::shared_ptr<Buffer> indices_buf;
     int64_t buf_size = values->length() * sizeof(uint64_t);
-    RETURN_NOT_OK(AllocateBuffer(ctx->memory_pool(), buf_size, &indices_buf));
+    ARROW_ASSIGN_OR_RAISE(auto indices_buf, AllocateBuffer(buf_size, ctx->memory_pool()));
 
     int64_t* indices_begin = reinterpret_cast<int64_t*>(indices_buf->mutable_data());
     int64_t* indices_end = indices_begin + values->length();
 
     sorter_.Sort(indices_begin, indices_end, *values.get());
-    *offsets = std::make_shared<UInt64Array>(values->length(), indices_buf);
+    *offsets = std::make_shared<UInt64Array>(values->length(), std::move(indices_buf));
     return Status::OK();
   }
 };
diff --git a/cpp/src/arrow/compute/kernels/take_internal.h b/cpp/src/arrow/compute/kernels/take_internal.h
index d40ff06..f36fd81 100644
--- a/cpp/src/arrow/compute/kernels/take_internal.h
+++ b/cpp/src/arrow/compute/kernels/take_internal.h
@@ -585,9 +585,9 @@ class TakerImpl<IndexSequence, UnionType> : public Taker<IndexSequence> {
       // Allocate temporary storage for the offsets of all valid slots
       auto child_offsets_storage_size =
           std::accumulate(child_counts.begin(), child_counts.end(), 0);
-      std::shared_ptr<Buffer> child_offsets_storage;
-      RETURN_NOT_OK(AllocateBuffer(pool_, child_offsets_storage_size * sizeof(int32_t),
-                                   &child_offsets_storage));
+      ARROW_ASSIGN_OR_RAISE(
+          std::shared_ptr<Buffer> child_offsets_storage,
+          AllocateBuffer(child_offsets_storage_size * sizeof(int32_t), pool_));
 
       // Partition offsets by type_code: child_offset_partitions[type_code] will
       // point to storage for child_counts[type_code] offsets
diff --git a/cpp/src/arrow/compute/kernels/take_test.cc b/cpp/src/arrow/compute/kernels/take_test.cc
index 239ca6d..eaaa579 100644
--- a/cpp/src/arrow/compute/kernels/take_test.cc
+++ b/cpp/src/arrow/compute/kernels/take_test.cc
@@ -445,8 +445,7 @@ class TestPermutationsWithTake : public ComputeFixture, public TestBase {
   template <typename Rng>
   void Shuffle(const Int16Array& array, Rng& gen, std::shared_ptr<Int16Array>* shuffled) {
     auto byte_length = array.length() * sizeof(int16_t);
-    std::shared_ptr<Buffer> data;
-    ASSERT_OK(array.values()->Copy(0, byte_length, &data));
+    ASSERT_OK_AND_ASSIGN(auto data, array.values()->CopySlice(0, byte_length));
     auto mutable_data = reinterpret_cast<int16_t*>(data->mutable_data());
     std::shuffle(mutable_data, mutable_data + array.length(), gen);
     shuffled->reset(new Int16Array(array.length(), data));
diff --git a/cpp/src/arrow/csv/parser.cc b/cpp/src/arrow/csv/parser.cc
index 7b06bce..77ad402 100644
--- a/cpp/src/arrow/csv/parser.cc
+++ b/cpp/src/arrow/csv/parser.cc
@@ -23,6 +23,7 @@
 #include <utility>
 
 #include "arrow/memory_pool.h"
+#include "arrow/result.h"
 #include "arrow/status.h"
 #include "arrow/util/logging.h"
 
@@ -80,7 +81,7 @@ class BlockParser::PresizedParsedWriter {
  public:
   PresizedParsedWriter(MemoryPool* pool, uint32_t size)
       : parsed_size_(0), parsed_capacity_(size) {
-    ARROW_CHECK_OK(AllocateResizableBuffer(pool, parsed_capacity_, &parsed_buffer_));
+    parsed_buffer_ = *AllocateResizableBuffer(parsed_capacity_, pool);
     parsed_ = parsed_buffer_->mutable_data();
   }
 
@@ -117,8 +118,7 @@ class BlockParser::ResizableValuesWriter {
  public:
   explicit ResizableValuesWriter(MemoryPool* pool)
       : values_size_(0), values_capacity_(256) {
-    ARROW_CHECK_OK(AllocateResizableBuffer(pool, values_capacity_ * sizeof(*values_),
-                                           &values_buffer_));
+    values_buffer_ = *AllocateResizableBuffer(values_capacity_ * sizeof(*values_), pool);
     values_ = reinterpret_cast<ValueDesc*>(values_buffer_->mutable_data());
   }
 
@@ -171,8 +171,7 @@ class BlockParser::PresizedValuesWriter {
  public:
   PresizedValuesWriter(MemoryPool* pool, int32_t num_rows, int32_t num_cols)
       : values_size_(0), values_capacity_(1 + num_rows * num_cols) {
-    ARROW_CHECK_OK(AllocateResizableBuffer(pool, values_capacity_ * sizeof(*values_),
-                                           &values_buffer_));
+    values_buffer_ = *AllocateResizableBuffer(values_capacity_ * sizeof(*values_), pool);
     values_ = reinterpret_cast<ValueDesc*>(values_buffer_->mutable_data());
   }
 
diff --git a/cpp/src/arrow/csv/reader.cc b/cpp/src/arrow/csv/reader.cc
index fed4cf4..facf267 100644
--- a/cpp/src/arrow/csv/reader.cc
+++ b/cpp/src/arrow/csv/reader.cc
@@ -273,7 +273,8 @@ class BaseTableReader : public csv::TableReader {
       } else if (completion->size() == 0) {
         straddling = partial;
       } else {
-        RETURN_NOT_OK(ConcatenateBuffers({partial, completion}, pool_, &straddling));
+        ARROW_ASSIGN_OR_RAISE(straddling,
+                              ConcatenateBuffers({partial, completion}, pool_));
       }
       views = {util::string_view(*straddling), util::string_view(*block)};
     } else {
diff --git a/cpp/src/arrow/dataset/file_ipc_test.cc b/cpp/src/arrow/dataset/file_ipc_test.cc
index f50a1a4..8b1d884 100644
--- a/cpp/src/arrow/dataset/file_ipc_test.cc
+++ b/cpp/src/arrow/dataset/file_ipc_test.cc
@@ -87,9 +87,9 @@ class TestIpcFileFormat : public ArrowIpcWriterMixin {
   }
 
   Result<FileSource> GetFileSink() {
-    std::shared_ptr<ResizableBuffer> buffer;
-    RETURN_NOT_OK(AllocateResizableBuffer(0, &buffer));
-    return FileSource(buffer);
+    ARROW_ASSIGN_OR_RAISE(std::shared_ptr<ResizableBuffer> buffer,
+                          AllocateResizableBuffer(0));
+    return FileSource(std::move(buffer));
   }
 
   RecordBatchIterator Batches(ScanTaskIterator scan_task_it) {
diff --git a/cpp/src/arrow/device.cc b/cpp/src/arrow/device.cc
index e5b880d..1aead49 100644
--- a/cpp/src/arrow/device.cc
+++ b/cpp/src/arrow/device.cc
@@ -136,9 +136,7 @@ Result<std::shared_ptr<io::OutputStream>> CPUMemoryManager::GetBufferWriter(
 }
 
 Result<std::shared_ptr<Buffer>> CPUMemoryManager::AllocateBuffer(int64_t size) {
-  std::shared_ptr<Buffer> buf;
-  RETURN_NOT_OK(::arrow::AllocateBuffer(pool_, size, &buf));
-  return buf;
+  return ::arrow::AllocateBuffer(size, pool_);
 }
 
 Result<std::shared_ptr<Buffer>> CPUMemoryManager::CopyBufferFrom(
@@ -146,12 +144,11 @@ Result<std::shared_ptr<Buffer>> CPUMemoryManager::CopyBufferFrom(
   if (!from->is_cpu()) {
     return nullptr;
   }
-  std::shared_ptr<Buffer> dest;
-  RETURN_NOT_OK(::arrow::AllocateBuffer(pool_, buf->size(), &dest));
+  ARROW_ASSIGN_OR_RAISE(auto dest, ::arrow::AllocateBuffer(buf->size(), pool_));
   if (buf->size() > 0) {
     memcpy(dest->mutable_data(), buf->data(), static_cast<size_t>(buf->size()));
   }
-  return dest;
+  return std::move(dest);
 }
 
 Result<std::shared_ptr<Buffer>> CPUMemoryManager::ViewBufferFrom(
@@ -167,12 +164,11 @@ Result<std::shared_ptr<Buffer>> CPUMemoryManager::CopyBufferTo(
   if (!to->is_cpu()) {
     return nullptr;
   }
-  std::shared_ptr<Buffer> dest;
-  RETURN_NOT_OK(::arrow::AllocateBuffer(pool_, buf->size(), &dest));
+  ARROW_ASSIGN_OR_RAISE(auto dest, ::arrow::AllocateBuffer(buf->size(), pool_));
   if (buf->size() > 0) {
     memcpy(dest->mutable_data(), buf->data(), static_cast<size_t>(buf->size()));
   }
-  return dest;
+  return std::move(dest);
 }
 
 Result<std::shared_ptr<Buffer>> CPUMemoryManager::ViewBufferTo(
diff --git a/cpp/src/arrow/filesystem/mockfs.cc b/cpp/src/arrow/filesystem/mockfs.cc
index c7a17d5..521ac66 100644
--- a/cpp/src/arrow/filesystem/mockfs.cc
+++ b/cpp/src/arrow/filesystem/mockfs.cc
@@ -371,8 +371,7 @@ class MockFileSystem::Impl {
     if (!entry->is_file()) {
       return NotAFile(path);
     }
-    std::shared_ptr<Buffer> buffer;
-    RETURN_NOT_OK(Buffer::FromString(entry->as_file().data, &buffer));
+    ARROW_ASSIGN_OR_RAISE(auto buffer, Buffer::FromString(entry->as_file().data));
     return std::make_shared<io::BufferReader>(buffer);
   }
 };
diff --git a/cpp/src/arrow/filesystem/s3fs.cc b/cpp/src/arrow/filesystem/s3fs.cc
index 71511bd..3870c39 100644
--- a/cpp/src/arrow/filesystem/s3fs.cc
+++ b/cpp/src/arrow/filesystem/s3fs.cc
@@ -460,15 +460,14 @@ class ObjectInputFile : public io::RandomAccessFile {
     // No need to allocate more than the remaining number of bytes
     nbytes = std::min(nbytes, content_length_ - position);
 
-    std::shared_ptr<ResizableBuffer> buf;
-    RETURN_NOT_OK(AllocateResizableBuffer(nbytes, &buf));
+    ARROW_ASSIGN_OR_RAISE(auto buf, AllocateResizableBuffer(nbytes));
     if (nbytes > 0) {
       ARROW_ASSIGN_OR_RAISE(int64_t bytes_read,
                             ReadAt(position, nbytes, buf->mutable_data()));
       DCHECK_LE(bytes_read, nbytes);
       RETURN_NOT_OK(buf->Resize(bytes_read));
     }
-    return buf;
+    return std::move(buf);
   }
 
   Result<int64_t> Read(int64_t nbytes, void* out) override {
@@ -480,7 +479,7 @@ class ObjectInputFile : public io::RandomAccessFile {
   Result<std::shared_ptr<Buffer>> Read(int64_t nbytes) override {
     ARROW_ASSIGN_OR_RAISE(auto buffer, ReadAt(pos_, nbytes));
     pos_ += buffer->size();
-    return buffer;
+    return std::move(buffer);
   }
 
  protected:
@@ -708,7 +707,7 @@ class ObjectOutputStream : public io::OutputStream {
 
       // If the data isn't owned, make an immutable copy for the lifetime of the closure
       if (owned_buffer == nullptr) {
-        RETURN_NOT_OK(AllocateBuffer(nbytes, &owned_buffer));
+        ARROW_ASSIGN_OR_RAISE(owned_buffer, AllocateBuffer(nbytes));
         memcpy(owned_buffer->mutable_data(), data, nbytes);
       } else {
         DCHECK_EQ(data, owned_buffer->data());
diff --git a/cpp/src/arrow/filesystem/util_internal.cc b/cpp/src/arrow/filesystem/util_internal.cc
index 6484862..f4d90ba 100644
--- a/cpp/src/arrow/filesystem/util_internal.cc
+++ b/cpp/src/arrow/filesystem/util_internal.cc
@@ -32,9 +32,8 @@ TimePoint CurrentTimePoint() {
 
 Status CopyStream(const std::shared_ptr<io::InputStream>& src,
                   const std::shared_ptr<io::OutputStream>& dest, int64_t chunk_size) {
-  std::shared_ptr<Buffer> chunk;
+  ARROW_ASSIGN_OR_RAISE(auto chunk, AllocateBuffer(chunk_size));
 
-  RETURN_NOT_OK(AllocateBuffer(chunk_size, &chunk));
   while (true) {
     ARROW_ASSIGN_OR_RAISE(int64_t bytes_read,
                           src->Read(chunk_size, chunk->mutable_data()));
diff --git a/cpp/src/arrow/flight/client.cc b/cpp/src/arrow/flight/client.cc
index 2b52e60..0346294 100644
--- a/cpp/src/arrow/flight/client.cc
+++ b/cpp/src/arrow/flight/client.cc
@@ -433,7 +433,7 @@ class DoPutPayloadWriter : public ipc::internal::IpcPayloadWriter {
           return Status::UnknownError("Failed to serialized Flight descriptor");
         }
       }
-      RETURN_NOT_OK(Buffer::FromString(str_descr, &payload.descriptor));
+      payload.descriptor = Buffer::FromString(std::move(str_descr));
       first_payload_ = false;
     } else if (ipc_payload.type == ipc::Message::RECORD_BATCH &&
                stream_writer_->app_metadata_) {
diff --git a/cpp/src/arrow/flight/flight_test.cc b/cpp/src/arrow/flight/flight_test.cc
index 5ed3ac5..de8b1e3 100644
--- a/cpp/src/arrow/flight/flight_test.cc
+++ b/cpp/src/arrow/flight/flight_test.cc
@@ -390,8 +390,7 @@ class TestFlightClient : public ::testing::Test {
 class AuthTestServer : public FlightServerBase {
   Status DoAction(const ServerCallContext& context, const Action& action,
                   std::unique_ptr<ResultStream>* result) override {
-    std::shared_ptr<Buffer> buf;
-    RETURN_NOT_OK(Buffer::FromString(context.peer_identity(), &buf));
+    ARROW_ASSIGN_OR_RAISE(auto buf, Buffer::FromString(context.peer_identity()));
     *result = std::unique_ptr<ResultStream>(new SimpleResultStream({Result{buf}}));
     return Status::OK();
   }
@@ -400,8 +399,7 @@ class AuthTestServer : public FlightServerBase {
 class TlsTestServer : public FlightServerBase {
   Status DoAction(const ServerCallContext& context, const Action& action,
                   std::unique_ptr<ResultStream>* result) override {
-    std::shared_ptr<Buffer> buf;
-    RETURN_NOT_OK(Buffer::FromString("Hello, world!", &buf));
+    auto buf = Buffer::FromString("Hello, world!");
     *result = std::unique_ptr<ResultStream>(new SimpleResultStream({Result{buf}}));
     return Status::OK();
   }
@@ -717,10 +715,10 @@ class ReportContextTestServer : public FlightServerBase {
     std::shared_ptr<Buffer> buf;
     const ServerMiddleware* middleware = context.GetMiddleware("tracing");
     if (middleware == nullptr || middleware->name() != "TracingServerMiddleware") {
-      RETURN_NOT_OK(Buffer::FromString("", &buf));
+      buf = Buffer::FromString("");
     } else {
-      RETURN_NOT_OK(Buffer::FromString(
-          ((const TracingServerMiddleware*)middleware)->span_id, &buf));
+      ARROW_ASSIGN_OR_RAISE(
+          buf, Buffer::FromString(((const TracingServerMiddleware*)middleware)->span_id));
     }
     *result = std::unique_ptr<ResultStream>(new SimpleResultStream({Result{buf}}));
     return Status::OK();
@@ -730,16 +728,13 @@ class ReportContextTestServer : public FlightServerBase {
 class ErrorMiddlewareServer : public FlightServerBase {
   Status DoAction(const ServerCallContext& context, const Action& action,
                   std::unique_ptr<ResultStream>* result) override {
-    std::shared_ptr<Buffer> buf;
     std::string msg = "error_message";
-    Status s = Buffer::FromString("", &buf);
+    auto buf = Buffer::FromString("");
 
     std::shared_ptr<FlightStatusDetail> flightStatusDetail(
         new FlightStatusDetail(FlightStatusCode::Failed, msg));
     *result = std::unique_ptr<ResultStream>(new SimpleResultStream({Result{buf}}));
-    Status s_err = Status(StatusCode::ExecutionError, "test failed", flightStatusDetail);
-    RETURN_NOT_OK(s_err);
-    return Status::OK();
+    return Status(StatusCode::ExecutionError, "test failed", flightStatusDetail);
   }
 };
 
@@ -895,8 +890,7 @@ TEST_F(TestErrorMiddleware, TestMetadata) {
   // Run action1
   action.type = "action1";
 
-  const std::string action1_value = "action1-content";
-  ASSERT_OK(Buffer::FromString(action1_value, &action.body));
+  action.body = Buffer::FromString("action1-content");
   Status s = client_->DoAction(action, &stream);
   ASSERT_FALSE(s.ok());
   std::shared_ptr<FlightStatusDetail> flightStatusDetail =
@@ -1009,7 +1003,7 @@ TEST_F(TestFlightClient, DoAction) {
   action.type = "action1";
 
   const std::string action1_value = "action1-content";
-  ASSERT_OK(Buffer::FromString(action1_value, &action.body));
+  action.body = *Buffer::FromString(action1_value);
   ASSERT_OK(client_->DoAction(action, &stream));
 
   for (int i = 0; i < 3; ++i) {
@@ -1473,8 +1467,7 @@ TEST_F(TestPropagatingMiddleware, Propagate) {
   client_middleware_->Reset();
 
   action.type = "action1";
-  const std::string action1_value = "action1-content";
-  ASSERT_OK(Buffer::FromString(action1_value, &action.body));
+  action.body = Buffer::FromString("action1-content");
   ASSERT_OK(client_->DoAction(action, &stream));
 
   ASSERT_OK(stream->Next(&result));
diff --git a/cpp/src/arrow/flight/internal.cc b/cpp/src/arrow/flight/internal.cc
index c649925..bd5afa7 100644
--- a/cpp/src/arrow/flight/internal.cc
+++ b/cpp/src/arrow/flight/internal.cc
@@ -305,7 +305,7 @@ Status ToProto(const ActionType& type, pb::ActionType* pb_type) {
 
 Status FromProto(const pb::Action& pb_action, Action* action) {
   action->type = pb_action.type();
-  return Buffer::FromString(pb_action.body(), &action->body);
+  return Buffer::FromString(pb_action.body()).Value(&action->body);
 }
 
 Status ToProto(const Action& action, pb::Action* pb_action) {
@@ -321,7 +321,7 @@ Status ToProto(const Action& action, pb::Action* pb_action) {
 Status FromProto(const pb::Result& pb_result, Result* result) {
   // ARROW-3250; can avoid copy. Can also write custom deserializer if it
   // becomes an issue
-  return Buffer::FromString(pb_result.body(), &result->body);
+  return Buffer::FromString(pb_result.body()).Value(&result->body);
 }
 
 Status ToProto(const Result& result, pb::Result* pb_result) {
diff --git a/cpp/src/arrow/flight/test_util.cc b/cpp/src/arrow/flight/test_util.cc
index 64e8bd2..9ed1015 100644
--- a/cpp/src/arrow/flight/test_util.cc
+++ b/cpp/src/arrow/flight/test_util.cc
@@ -208,7 +208,7 @@ class FlightTestServer : public FlightServerBase {
     for (int i = 0; i < 3; ++i) {
       Result result;
       std::string value = action.body->ToString() + "-part" + std::to_string(i);
-      RETURN_NOT_OK(Buffer::FromString(value, &result.body));
+      result.body = Buffer::FromString(std::move(value));
       results.push_back(result);
     }
     *out = std::unique_ptr<ResultStream>(new SimpleResultStream(std::move(results)));
diff --git a/cpp/src/arrow/gpu/cuda_memory.cc b/cpp/src/arrow/gpu/cuda_memory.cc
index 4e82e05..a33511d 100644
--- a/cpp/src/arrow/gpu/cuda_memory.cc
+++ b/cpp/src/arrow/gpu/cuda_memory.cc
@@ -82,18 +82,18 @@ Status CudaIpcMemHandle::FromBuffer(const void* opaque_handle,
 }
 
 Result<std::shared_ptr<Buffer>> CudaIpcMemHandle::Serialize(MemoryPool* pool) const {
-  std::shared_ptr<Buffer> buffer;
   int64_t size = impl_->memory_size;
   const size_t handle_size =
       (size > 0 ? sizeof(int64_t) + sizeof(CUipcMemHandle) : sizeof(int64_t));
 
-  RETURN_NOT_OK(AllocateBuffer(pool, static_cast<int64_t>(handle_size), &buffer));
+  ARROW_ASSIGN_OR_RAISE(auto buffer,
+                        AllocateBuffer(static_cast<int64_t>(handle_size), pool));
   memcpy(buffer->mutable_data(), &impl_->memory_size, sizeof(impl_->memory_size));
   if (size > 0) {
     memcpy(buffer->mutable_data() + sizeof(impl_->memory_size), &impl_->ipc_handle,
            sizeof(impl_->ipc_handle));
   }
-  return buffer;
+  return std::move(buffer);
 }
 
 Status CudaIpcMemHandle::Serialize(MemoryPool* pool, std::shared_ptr<Buffer>* out) const {
diff --git a/cpp/src/arrow/gpu/cuda_test.cc b/cpp/src/arrow/gpu/cuda_test.cc
index e5f6a7d..5a09b2a 100644
--- a/cpp/src/arrow/gpu/cuda_test.cc
+++ b/cpp/src/arrow/gpu/cuda_test.cc
@@ -52,8 +52,7 @@ constexpr int kOtherGpuNumber = 1;
 
 template <typename Expected>
 void AssertCudaBufferEquals(const CudaBuffer& buffer, Expected&& expected) {
-  std::shared_ptr<Buffer> result;
-  ASSERT_OK(AllocateBuffer(default_memory_pool(), buffer.size(), &result));
+  ASSERT_OK_AND_ASSIGN(auto result, AllocateBuffer(buffer.size()));
   ASSERT_OK(buffer.CopyToHost(0, buffer.size(), result->mutable_data()));
   AssertBufferEqual(*result, expected);
 }
@@ -326,8 +325,7 @@ TEST_F(TestCudaBuffer, DISABLED_ExportForIpc) {
 
   ASSERT_EQ(kSize, ipc_buffer->size());
 
-  std::shared_ptr<Buffer> ipc_data;
-  ASSERT_OK(AllocateBuffer(default_memory_pool(), kSize, &ipc_data));
+  ASSERT_OK_AND_ASSIGN(auto ipc_data, AllocateBuffer(kSize));
   ASSERT_OK(ipc_buffer->CopyToHost(0, kSize, ipc_data->mutable_data()));
   ASSERT_EQ(0, std::memcmp(ipc_buffer->data(), host_buffer->data(), kSize));
 }
@@ -567,13 +565,12 @@ TEST_F(TestCudaArrowIpc, BasicWriteRead) {
   ASSERT_OK_AND_ASSIGN(device_batch, ReadRecordBatch(batch->schema(), device_serialized));
 
   // Copy data from device, read batch, and compare
-  std::shared_ptr<Buffer> host_buffer;
   int64_t size = device_serialized->size();
-  ASSERT_OK(AllocateBuffer(pool_, size, &host_buffer));
+  ASSERT_OK_AND_ASSIGN(auto host_buffer, AllocateBuffer(size, pool_));
   ASSERT_OK(device_serialized->CopyToHost(0, size, host_buffer->mutable_data()));
 
   std::shared_ptr<RecordBatch> cpu_batch;
-  io::BufferReader cpu_reader(host_buffer);
+  io::BufferReader cpu_reader(std::move(host_buffer));
   ipc::DictionaryMemo unused_memo;
   ASSERT_OK_AND_ASSIGN(
       cpu_batch, ipc::ReadRecordBatch(batch->schema(), &unused_memo,
diff --git a/cpp/src/arrow/io/buffered.cc b/cpp/src/arrow/io/buffered.cc
index cd436da..92d0fc3 100644
--- a/cpp/src/arrow/io/buffered.cc
+++ b/cpp/src/arrow/io/buffered.cc
@@ -55,7 +55,7 @@ class BufferedBase {
     if (!buffer_) {
       // On first invocation, or if the buffer has been released, we allocate a
       // new buffer
-      RETURN_NOT_OK(AllocateResizableBuffer(pool_, buffer_size_, &buffer_));
+      ARROW_ASSIGN_OR_RAISE(buffer_, AllocateResizableBuffer(buffer_size_, pool_));
     } else if (buffer_->size() != buffer_size_) {
       RETURN_NOT_OK(buffer_->Resize(buffer_size_));
     }
@@ -400,8 +400,7 @@ class BufferedInputStream::Impl : public BufferedBase {
   }
 
   Result<std::shared_ptr<Buffer>> Read(int64_t nbytes) {
-    std::shared_ptr<ResizableBuffer> buffer;
-    RETURN_NOT_OK(AllocateResizableBuffer(pool_, nbytes, &buffer));
+    ARROW_ASSIGN_OR_RAISE(auto buffer, AllocateResizableBuffer(nbytes, pool_));
 
     ARROW_ASSIGN_OR_RAISE(int64_t bytes_read, Read(nbytes, buffer->mutable_data()));
 
@@ -410,7 +409,7 @@ class BufferedInputStream::Impl : public BufferedBase {
       RETURN_NOT_OK(buffer->Resize(bytes_read, false /* shrink_to_fit */));
       buffer->ZeroPadding();
     }
-    return buffer;
+    return std::move(buffer);
   }
 
   // For providing access to the raw file handles
diff --git a/cpp/src/arrow/io/buffered_test.cc b/cpp/src/arrow/io/buffered_test.cc
index cf85ef5..65f7113 100644
--- a/cpp/src/arrow/io/buffered_test.cc
+++ b/cpp/src/arrow/io/buffered_test.cc
@@ -454,13 +454,12 @@ class TestBufferedInputStreamBound : public ::testing::Test {
   void CreateExample(bool bounded = true) {
     // Create a buffer larger than source size, to check that the
     // stream end is respected
-    std::shared_ptr<ResizableBuffer> buf;
-    ASSERT_OK(AllocateResizableBuffer(default_memory_pool(), source_size_ + 10, &buf));
+    ASSERT_OK_AND_ASSIGN(auto buf, AllocateResizableBuffer(source_size_ + 10));
     ASSERT_LT(source_size_, buf->size());
     for (int i = 0; i < source_size_; i++) {
       buf->mutable_data()[i] = static_cast<uint8_t>(i);
     }
-    source_ = std::make_shared<BufferReader>(buf);
+    source_ = std::make_shared<BufferReader>(std::move(buf));
     ASSERT_OK(source_->Advance(stream_offset_));
     ASSERT_OK_AND_ASSIGN(
         stream_, BufferedInputStream::Create(chunk_size_, default_memory_pool(), source_,
diff --git a/cpp/src/arrow/io/compressed.cc b/cpp/src/arrow/io/compressed.cc
index 5908169..d0aebda 100644
--- a/cpp/src/arrow/io/compressed.cc
+++ b/cpp/src/arrow/io/compressed.cc
@@ -49,7 +49,7 @@ class CompressedOutputStream::Impl {
 
   Status Init(Codec* codec) {
     ARROW_ASSIGN_OR_RAISE(compressor_, codec->MakeCompressor());
-    RETURN_NOT_OK(AllocateResizableBuffer(pool_, kChunkSize, &compressed_));
+    ARROW_ASSIGN_OR_RAISE(compressed_, AllocateResizableBuffer(kChunkSize, pool_));
     compressed_pos_ = 0;
     is_open_ = true;
     return Status::OK();
@@ -282,7 +282,8 @@ class CompressedInputStream::Impl {
     int64_t decompress_size = kDecompressSize;
 
     while (true) {
-      RETURN_NOT_OK(AllocateResizableBuffer(pool_, decompress_size, &decompressed_));
+      ARROW_ASSIGN_OR_RAISE(decompressed_,
+                            AllocateResizableBuffer(decompress_size, pool_));
       decompressed_pos_ = 0;
 
       int64_t input_len = compressed_->size() - compressed_pos_;
@@ -376,11 +377,10 @@ class CompressedInputStream::Impl {
   }
 
   Result<std::shared_ptr<Buffer>> Read(int64_t nbytes) {
-    std::shared_ptr<ResizableBuffer> buf;
-    RETURN_NOT_OK(AllocateResizableBuffer(pool_, nbytes, &buf));
+    ARROW_ASSIGN_OR_RAISE(auto buf, AllocateResizableBuffer(nbytes, pool_));
     ARROW_ASSIGN_OR_RAISE(int64_t bytes_read, Read(nbytes, buf->mutable_data()));
     RETURN_NOT_OK(buf->Resize(bytes_read));
-    return buf;
+    return std::move(buf);
   }
 
   std::shared_ptr<InputStream> raw() const { return raw_; }
diff --git a/cpp/src/arrow/io/compressed_test.cc b/cpp/src/arrow/io/compressed_test.cc
index 669430b..7e21ba8 100644
--- a/cpp/src/arrow/io/compressed_test.cc
+++ b/cpp/src/arrow/io/compressed_test.cc
@@ -72,8 +72,7 @@ std::shared_ptr<Buffer> CompressDataOneShot(Codec* codec,
                                             const std::vector<uint8_t>& data) {
   int64_t max_compressed_len, compressed_len;
   max_compressed_len = codec->MaxCompressedLen(data.size(), data.data());
-  std::shared_ptr<ResizableBuffer> compressed;
-  ABORT_NOT_OK(AllocateResizableBuffer(max_compressed_len, &compressed));
+  auto compressed = *AllocateResizableBuffer(max_compressed_len);
   compressed_len = *codec->Compress(data.size(), data.data(), max_compressed_len,
                                     compressed->mutable_data());
   ABORT_NOT_OK(compressed->Resize(compressed_len));
@@ -212,9 +211,7 @@ TEST_P(CompressedInputStreamTest, ConcatenatedStreams) {
   auto compressed1 = CompressDataOneShot(codec.get(), data1);
   auto compressed2 = CompressDataOneShot(codec.get(), data2);
 
-  std::shared_ptr<Buffer> concatenated;
-  ASSERT_OK(ConcatenateBuffers({compressed1, compressed2}, default_memory_pool(),
-                               &concatenated));
+  ASSERT_OK_AND_ASSIGN(auto concatenated, ConcatenateBuffers({compressed1, compressed2}));
   std::vector<uint8_t> decompressed, expected;
   ASSERT_OK(RunCompressedInputStream(codec.get(), concatenated, &decompressed));
   std::copy(data1.begin(), data1.end(), std::back_inserter(expected));
diff --git a/cpp/src/arrow/io/file.cc b/cpp/src/arrow/io/file.cc
index 1300734..55f1b59 100644
--- a/cpp/src/arrow/io/file.cc
+++ b/cpp/src/arrow/io/file.cc
@@ -240,20 +240,18 @@ class ReadableFile::ReadableFileImpl : public OSFile {
   Status Open(int fd) { return OpenReadable(fd); }
 
   Result<std::shared_ptr<Buffer>> ReadBuffer(int64_t nbytes) {
-    std::shared_ptr<ResizableBuffer> buffer;
-    RETURN_NOT_OK(AllocateResizableBuffer(pool_, nbytes, &buffer));
+    ARROW_ASSIGN_OR_RAISE(auto buffer, AllocateResizableBuffer(nbytes, pool_));
 
     ARROW_ASSIGN_OR_RAISE(int64_t bytes_read, Read(nbytes, buffer->mutable_data()));
     if (bytes_read < nbytes) {
       RETURN_NOT_OK(buffer->Resize(bytes_read));
       buffer->ZeroPadding();
     }
-    return buffer;
+    return std::move(buffer);
   }
 
   Result<std::shared_ptr<Buffer>> ReadBufferAt(int64_t position, int64_t nbytes) {
-    std::shared_ptr<ResizableBuffer> buffer;
-    RETURN_NOT_OK(AllocateResizableBuffer(pool_, nbytes, &buffer));
+    ARROW_ASSIGN_OR_RAISE(auto buffer, AllocateResizableBuffer(nbytes, pool_));
 
     ARROW_ASSIGN_OR_RAISE(int64_t bytes_read,
                           ReadAt(position, nbytes, buffer->mutable_data()));
@@ -261,7 +259,7 @@ class ReadableFile::ReadableFileImpl : public OSFile {
       RETURN_NOT_OK(buffer->Resize(bytes_read));
       buffer->ZeroPadding();
     }
-    return buffer;
+    return std::move(buffer);
   }
 
  private:
diff --git a/cpp/src/arrow/io/hdfs.cc b/cpp/src/arrow/io/hdfs.cc
index 75b7a83..2de25be 100644
--- a/cpp/src/arrow/io/hdfs.cc
+++ b/cpp/src/arrow/io/hdfs.cc
@@ -151,8 +151,7 @@ class HdfsReadableFile::HdfsReadableFileImpl : public HdfsAnyFileImpl {
   }
 
   Result<std::shared_ptr<Buffer>> ReadAt(int64_t position, int64_t nbytes) {
-    std::shared_ptr<ResizableBuffer> buffer;
-    RETURN_NOT_OK(AllocateResizableBuffer(pool_, nbytes, &buffer));
+    ARROW_ASSIGN_OR_RAISE(auto buffer, AllocateResizableBuffer(nbytes, pool_));
 
     ARROW_ASSIGN_OR_RAISE(int64_t bytes_read,
                           ReadAt(position, nbytes, buffer->mutable_data()));
@@ -161,7 +160,7 @@ class HdfsReadableFile::HdfsReadableFileImpl : public HdfsAnyFileImpl {
       RETURN_NOT_OK(buffer->Resize(bytes_read));
       buffer->ZeroPadding();
     }
-    return buffer;
+    return std::move(buffer);
   }
 
   Result<int64_t> Read(int64_t nbytes, void* buffer) {
@@ -180,14 +179,13 @@ class HdfsReadableFile::HdfsReadableFileImpl : public HdfsAnyFileImpl {
   }
 
   Result<std::shared_ptr<Buffer>> Read(int64_t nbytes) {
-    std::shared_ptr<ResizableBuffer> buffer;
-    RETURN_NOT_OK(AllocateResizableBuffer(pool_, nbytes, &buffer));
+    ARROW_ASSIGN_OR_RAISE(auto buffer, AllocateResizableBuffer(nbytes, pool_));
 
     ARROW_ASSIGN_OR_RAISE(int64_t bytes_read, Read(nbytes, buffer->mutable_data()));
     if (bytes_read < nbytes) {
       RETURN_NOT_OK(buffer->Resize(bytes_read));
     }
-    return buffer;
+    return std::move(buffer);
   }
 
   Result<int64_t> GetSize() {
diff --git a/cpp/src/arrow/io/hdfs_test.cc b/cpp/src/arrow/io/hdfs_test.cc
index 0998f4e..ac71007 100644
--- a/cpp/src/arrow/io/hdfs_test.cc
+++ b/cpp/src/arrow/io/hdfs_test.cc
@@ -358,8 +358,7 @@ TEST_F(TestHadoopFileSystem, LargeFile) {
 
   ASSERT_FALSE(file->closed());
 
-  std::shared_ptr<Buffer> buffer;
-  ASSERT_OK(AllocateBuffer(nullptr, size, &buffer));
+  ASSERT_OK_AND_ASSIGN(auto buffer, AllocateBuffer(size));
 
   ASSERT_OK_AND_EQ(size, file->Read(size, buffer->mutable_data()));
   ASSERT_EQ(0, std::memcmp(buffer->data(), data.data(), size));
@@ -368,8 +367,7 @@ TEST_F(TestHadoopFileSystem, LargeFile) {
   std::shared_ptr<HdfsReadableFile> file2;
   ASSERT_OK(this->client_->OpenReadable(path, 1 << 18, &file2));
 
-  std::shared_ptr<Buffer> buffer2;
-  ASSERT_OK(AllocateBuffer(nullptr, size, &buffer2));
+  ASSERT_OK_AND_ASSIGN(auto buffer2, AllocateBuffer(size));
 
   ASSERT_OK_AND_EQ(size, file2->Read(size, buffer2->mutable_data()));
   ASSERT_EQ(0, std::memcmp(buffer2->data(), data.data(), size));
diff --git a/cpp/src/arrow/io/memory.cc b/cpp/src/arrow/io/memory.cc
index 8089800..4e6c13c 100644
--- a/cpp/src/arrow/io/memory.cc
+++ b/cpp/src/arrow/io/memory.cc
@@ -59,7 +59,7 @@ Result<std::shared_ptr<BufferOutputStream>> BufferOutputStream::Create(
 }
 
 Status BufferOutputStream::Reset(int64_t initial_capacity, MemoryPool* pool) {
-  RETURN_NOT_OK(AllocateResizableBuffer(pool, initial_capacity, &buffer_));
+  ARROW_ASSIGN_OR_RAISE(buffer_, AllocateResizableBuffer(initial_capacity, pool));
   is_open_ = true;
   capacity_ = initial_capacity;
   position_ = 0;
diff --git a/cpp/src/arrow/io/memory_benchmark.cc b/cpp/src/arrow/io/memory_benchmark.cc
index 643d890..d27fec0 100644
--- a/cpp/src/arrow/io/memory_benchmark.cc
+++ b/cpp/src/arrow/io/memory_benchmark.cc
@@ -250,8 +250,8 @@ static void MemoryBandwidth(benchmark::State& state) {  // NOLINT non-const refe
   const size_t buffer_size = state.range(0);
   BufferPtr src, dst;
 
-  ABORT_NOT_OK(AllocateBuffer(buffer_size, &dst));
-  ABORT_NOT_OK(AllocateBuffer(buffer_size, &src));
+  dst = *AllocateBuffer(buffer_size);
+  src = *AllocateBuffer(buffer_size);
   random_bytes(buffer_size, 0, src->mutable_data());
 
   while (state.KeepRunning()) {
@@ -294,14 +294,13 @@ static void ParallelMemoryCopy(benchmark::State& state) {  // NOLINT non-const r
   const int64_t n_threads = state.range(0);
   const int64_t buffer_size = kMemoryPerCore;
 
-  std::shared_ptr<Buffer> src, dst;
-  ABORT_NOT_OK(AllocateBuffer(buffer_size, &src));
-  ABORT_NOT_OK(AllocateBuffer(buffer_size, &dst));
+  auto src = *AllocateBuffer(buffer_size);
+  auto dst = *AllocateBuffer(buffer_size);
 
   random_bytes(buffer_size, 0, src->mutable_data());
 
   while (state.KeepRunning()) {
-    io::FixedSizeBufferWriter writer(dst);
+    io::FixedSizeBufferWriter writer(std::move(dst));
     writer.set_memcopy_threads(static_cast<int>(n_threads));
     ABORT_NOT_OK(writer.Write(src->data(), src->size()));
   }
diff --git a/cpp/src/arrow/io/memory_test.cc b/cpp/src/arrow/io/memory_test.cc
index 710cd63..7a96130 100644
--- a/cpp/src/arrow/io/memory_test.cc
+++ b/cpp/src/arrow/io/memory_test.cc
@@ -52,7 +52,7 @@ std::ostream& operator<<(std::ostream& os, const ReadRange& range) {
 class TestBufferOutputStream : public ::testing::Test {
  public:
   void SetUp() {
-    ASSERT_OK(AllocateResizableBuffer(0, &buffer_));
+    ASSERT_OK_AND_ASSIGN(buffer_, AllocateResizableBuffer(0));
     stream_.reset(new BufferOutputStream(buffer_));
   }
 
@@ -115,8 +115,7 @@ TEST_F(TestBufferOutputStream, Reset) {
 }
 
 TEST(TestFixedSizeBufferWriter, Basics) {
-  std::shared_ptr<Buffer> buffer;
-  ASSERT_OK(AllocateBuffer(1024, &buffer));
+  ASSERT_OK_AND_ASSIGN(std::shared_ptr<Buffer> buffer, AllocateBuffer(1024));
 
   FixedSizeBufferWriter writer(buffer);
 
@@ -145,8 +144,7 @@ TEST(TestFixedSizeBufferWriter, Basics) {
 }
 
 TEST(TestFixedSizeBufferWriter, InvalidWrites) {
-  std::shared_ptr<Buffer> buffer;
-  ASSERT_OK(AllocateBuffer(1024, &buffer));
+  ASSERT_OK_AND_ASSIGN(std::shared_ptr<Buffer> buffer, AllocateBuffer(1024));
 
   FixedSizeBufferWriter writer(buffer);
   const uint8_t data[10]{};
@@ -242,8 +240,8 @@ TEST(TestBufferReader, RetainParentReference) {
   std::shared_ptr<Buffer> slice1;
   std::shared_ptr<Buffer> slice2;
   {
-    std::shared_ptr<Buffer> buffer;
-    ASSERT_OK(AllocateBuffer(nullptr, static_cast<int64_t>(data.size()), &buffer));
+    ASSERT_OK_AND_ASSIGN(std::shared_ptr<Buffer> buffer,
+                         AllocateBuffer(static_cast<int64_t>(data.size())));
     std::memcpy(buffer->mutable_data(), data.c_str(), data.size());
     BufferReader reader(buffer);
     ASSERT_OK_AND_ASSIGN(slice1, reader.Read(4));
@@ -323,10 +321,8 @@ TEST(TestMemcopy, ParallelMemcopy) {
     // randomize size so the memcopy alignment is tested
     int64_t total_size = 3 * THRESHOLD + std::rand() % 100;
 
-    std::shared_ptr<Buffer> buffer1, buffer2;
-
-    ASSERT_OK(AllocateBuffer(total_size, &buffer1));
-    ASSERT_OK(AllocateBuffer(total_size, &buffer2));
+    ASSERT_OK_AND_ASSIGN(std::shared_ptr<Buffer> buffer1, AllocateBuffer(total_size));
+    ASSERT_OK_AND_ASSIGN(std::shared_ptr<Buffer> buffer2, AllocateBuffer(total_size));
 
     random_bytes(total_size, 0, buffer2->mutable_data());
 
diff --git a/cpp/src/arrow/ipc/json_internal.cc b/cpp/src/arrow/ipc/json_internal.cc
index 4e3b450..133681c 100644
--- a/cpp/src/arrow/ipc/json_internal.cc
+++ b/cpp/src/arrow/ipc/json_internal.cc
@@ -1182,8 +1182,7 @@ class ArrayReader {
         }
         const auto value_len = static_cast<int64_t>(hex_string.size()) / 2;
 
-        std::shared_ptr<Buffer> byte_buffer;
-        RETURN_NOT_OK(AllocateBuffer(pool_, value_len, &byte_buffer));
+        ARROW_ASSIGN_OR_RAISE(auto byte_buffer, AllocateBuffer(value_len, pool_));
 
         const char* hex_data = hex_string.c_str();
         uint8_t* byte_buffer_data = byte_buffer->mutable_data();
@@ -1239,8 +1238,7 @@ class ArrayReader {
     int32_t byte_width = type.byte_width();
 
     // Allocate space for parsed values
-    std::shared_ptr<Buffer> byte_buffer;
-    RETURN_NOT_OK(AllocateBuffer(pool_, byte_width, &byte_buffer));
+    ARROW_ASSIGN_OR_RAISE(auto byte_buffer, AllocateBuffer(byte_width, pool_));
     uint8_t* byte_buffer_data = byte_buffer->mutable_data();
 
     for (int i = 0; i < length_; ++i) {
@@ -1298,8 +1296,7 @@ class ArrayReader {
   template <typename T>
   Status GetIntArray(const RjArray& json_array, const int32_t length,
                      std::shared_ptr<Buffer>* out) {
-    std::shared_ptr<Buffer> buffer;
-    RETURN_NOT_OK(AllocateBuffer(pool_, length * sizeof(T), &buffer));
+    ARROW_ASSIGN_OR_RAISE(auto buffer, AllocateBuffer(length * sizeof(T), pool_));
 
     T* values = reinterpret_cast<T*>(buffer->mutable_data());
     for (int i = 0; i < length; ++i) {
@@ -1312,7 +1309,7 @@ class ArrayReader {
       }
     }
 
-    *out = buffer;
+    *out = std::move(buffer);
     return Status::OK();
   }
 
@@ -1447,8 +1444,7 @@ class ArrayReader {
                            std::shared_ptr<Buffer>* validity_buffer) {
     int length = static_cast<int>(is_valid.size());
 
-    std::shared_ptr<Buffer> out_buffer;
-    RETURN_NOT_OK(AllocateEmptyBitmap(pool_, length, &out_buffer));
+    ARROW_ASSIGN_OR_RAISE(auto out_buffer, AllocateEmptyBitmap(length, pool_));
     uint8_t* bitmap = out_buffer->mutable_data();
 
     *null_count = 0;
diff --git a/cpp/src/arrow/ipc/message.cc b/cpp/src/arrow/ipc/message.cc
index 6192138..7280781 100644
--- a/cpp/src/arrow/ipc/message.cc
+++ b/cpp/src/arrow/ipc/message.cc
@@ -170,7 +170,7 @@ Status MaybeAlignMetadata(std::shared_ptr<Buffer>* metadata) {
   if (reinterpret_cast<uintptr_t>((*metadata)->data()) % 8 != 0) {
     // If the metadata memory is not aligned, we copy it here to avoid
     // potential UBSAN issues from Flatbuffers
-    RETURN_NOT_OK((*metadata)->Copy(0, (*metadata)->size(), metadata));
+    ARROW_ASSIGN_OR_RAISE(*metadata, (*metadata)->CopySlice(0, (*metadata)->size()));
   }
   return Status::OK();
 }
diff --git a/cpp/src/arrow/ipc/metadata_internal.h b/cpp/src/arrow/ipc/metadata_internal.h
index 617f9f4..2bb3790 100644
--- a/cpp/src/arrow/ipc/metadata_internal.h
+++ b/cpp/src/arrow/ipc/metadata_internal.h
@@ -23,6 +23,7 @@
 #include <cstring>
 #include <memory>
 #include <string>
+#include <utility>
 #include <vector>
 
 #include <flatbuffers/flatbuffers.h>
@@ -203,12 +204,11 @@ static inline Result<std::shared_ptr<Buffer>> WriteFlatbufferBuilder(
     flatbuffers::FlatBufferBuilder& fbb) {
   int32_t size = fbb.GetSize();
 
-  std::shared_ptr<Buffer> result;
-  RETURN_NOT_OK(AllocateBuffer(default_memory_pool(), size, &result));
+  ARROW_ASSIGN_OR_RAISE(auto result, AllocateBuffer(size));
 
   uint8_t* dst = result->mutable_data();
   memcpy(dst, fbb.GetBufferPointer(), size);
-  return result;
+  return std::move(result);
 }
 
 }  // namespace internal
diff --git a/cpp/src/arrow/ipc/read_write_benchmark.cc b/cpp/src/arrow/ipc/read_write_benchmark.cc
index cad10cb..c76c166 100644
--- a/cpp/src/arrow/ipc/read_write_benchmark.cc
+++ b/cpp/src/arrow/ipc/read_write_benchmark.cc
@@ -52,8 +52,7 @@ static void WriteRecordBatch(benchmark::State& state) {  // NOLINT non-const ref
   constexpr int64_t kTotalSize = 1 << 20;
   auto options = ipc::IpcWriteOptions::Defaults();
 
-  std::shared_ptr<ResizableBuffer> buffer;
-  ABORT_NOT_OK(AllocateResizableBuffer(kTotalSize & 2, &buffer));
+  std::shared_ptr<ResizableBuffer> buffer = *AllocateResizableBuffer(kTotalSize & 2);
   auto record_batch = MakeRecordBatch(kTotalSize, state.range(0));
 
   while (state.KeepRunning()) {
@@ -74,8 +73,7 @@ static void ReadRecordBatch(benchmark::State& state) {  // NOLINT non-const refe
   constexpr int64_t kTotalSize = 1 << 20;
   auto options = ipc::IpcWriteOptions::Defaults();
 
-  std::shared_ptr<ResizableBuffer> buffer;
-  ABORT_NOT_OK(AllocateResizableBuffer(kTotalSize & 2, &buffer));
+  std::shared_ptr<ResizableBuffer> buffer = *AllocateResizableBuffer(kTotalSize & 2);
   auto record_batch = MakeRecordBatch(kTotalSize, state.range(0));
 
   io::BufferOutputStream stream(buffer);
diff --git a/cpp/src/arrow/ipc/read_write_test.cc b/cpp/src/arrow/ipc/read_write_test.cc
index 4620f4a..94d1d8e 100644
--- a/cpp/src/arrow/ipc/read_write_test.cc
+++ b/cpp/src/arrow/ipc/read_write_test.cc
@@ -462,12 +462,12 @@ TEST_P(TestIpcRoundTrip, ZeroLengthArrays) {
   CheckRoundtrip(*zero_length_batch);
 
   // ARROW-544: check binary array
-  std::shared_ptr<Buffer> value_offsets;
-  ASSERT_OK(AllocateBuffer(options_.memory_pool, sizeof(int32_t), &value_offsets));
+  ASSERT_OK_AND_ASSIGN(auto value_offsets,
+                       AllocateBuffer(sizeof(int32_t), options_.memory_pool));
   *reinterpret_cast<int32_t*>(value_offsets->mutable_data()) = 0;
 
   std::shared_ptr<Array> bin_array = std::make_shared<BinaryArray>(
-      0, value_offsets, std::make_shared<Buffer>(nullptr, 0),
+      0, std::move(value_offsets), std::make_shared<Buffer>(nullptr, 0),
       std::make_shared<Buffer>(nullptr, 0));
 
   // null value_offsets
@@ -793,7 +793,7 @@ struct FileWriterHelper {
               const std::shared_ptr<const KeyValueMetadata>& metadata = nullptr) {
     num_batches_written_ = 0;
 
-    RETURN_NOT_OK(AllocateResizableBuffer(0, &buffer_));
+    ARROW_ASSIGN_OR_RAISE(buffer_, AllocateResizableBuffer(0));
     sink_.reset(new io::BufferOutputStream(buffer_));
     ARROW_ASSIGN_OR_RAISE(writer_, NewFileWriter(sink_.get(), schema, options, metadata));
     return Status::OK();
@@ -853,7 +853,7 @@ struct FileWriterHelper {
 
 struct StreamWriterHelper {
   Status Init(const std::shared_ptr<Schema>& schema, const IpcWriteOptions& options) {
-    RETURN_NOT_OK(AllocateResizableBuffer(0, &buffer_));
+    ARROW_ASSIGN_OR_RAISE(buffer_, AllocateResizableBuffer(0));
     sink_.reset(new io::BufferOutputStream(buffer_));
     ARROW_ASSIGN_OR_RAISE(writer_, NewStreamWriter(sink_.get(), schema, options));
     return Status::OK();
diff --git a/cpp/src/arrow/ipc/reader.cc b/cpp/src/arrow/ipc/reader.cc
index 9cefb51..09da403 100644
--- a/cpp/src/arrow/ipc/reader.cc
+++ b/cpp/src/arrow/ipc/reader.cc
@@ -157,7 +157,7 @@ class ArrayLoader {
     if (buffer->length() == 0) {
       // Should never return a null buffer here.
       // (zero-sized buffer allocations are cheap)
-      return AllocateBuffer(0, out);
+      return AllocateBuffer(0).Value(out);
     } else {
       return ReadBuffer(buffer->offset(), buffer->length(), out);
     }
@@ -366,9 +366,8 @@ Status DecompressBuffers(Compression::type compression, const IpcReadOptions& op
       int64_t uncompressed_size =
           BitUtil::FromLittleEndian(util::SafeLoadAs<int64_t>(data));
 
-      std::shared_ptr<Buffer> uncompressed;
-      RETURN_NOT_OK(
-          AllocateBuffer(options.memory_pool, uncompressed_size, &uncompressed));
+      ARROW_ASSIGN_OR_RAISE(auto uncompressed,
+                            AllocateBuffer(uncompressed_size, options.memory_pool));
 
       int64_t actual_decompressed;
       ARROW_ASSIGN_OR_RAISE(
@@ -380,7 +379,7 @@ Status DecompressBuffers(Compression::type compression, const IpcReadOptions& op
                                uncompressed_size, " bytes but decompressed ",
                                actual_decompressed);
       }
-      arr->buffers[i] = uncompressed;
+      arr->buffers[i] = std::move(uncompressed);
     }
     return Status::OK();
   };
diff --git a/cpp/src/arrow/ipc/test_common.cc b/cpp/src/arrow/ipc/test_common.cc
index 1f93eff..6ce40b0 100644
--- a/cpp/src/arrow/ipc/test_common.cc
+++ b/cpp/src/arrow/ipc/test_common.cc
@@ -717,15 +717,16 @@ Status MakeDecimal(std::shared_ptr<RecordBatch>* out) {
   constexpr int kDecimalSize = 16;
   constexpr int length = 10;
 
-  std::shared_ptr<Buffer> data, is_valid;
   std::vector<uint8_t> is_valid_bytes(length);
 
-  RETURN_NOT_OK(AllocateBuffer(kDecimalSize * length, &data));
+  ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Buffer> data,
+                        AllocateBuffer(kDecimalSize * length));
 
   random_decimals(length, 1, kDecimalPrecision, data->mutable_data());
   random_null_bytes(length, 0.1, is_valid_bytes.data());
 
-  ARROW_ASSIGN_OR_RAISE(is_valid, BitUtil::BytesToBits(is_valid_bytes));
+  ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Buffer> is_valid,
+                        BitUtil::BytesToBits(is_valid_bytes));
 
   auto a1 = std::make_shared<Decimal128Array>(f0->type(), length, data, is_valid,
                                               kUnknownNullCount);
diff --git a/cpp/src/arrow/ipc/writer.cc b/cpp/src/arrow/ipc/writer.cc
index 2dc0c8d..d0dae14 100644
--- a/cpp/src/arrow/ipc/writer.cc
+++ b/cpp/src/arrow/ipc/writer.cc
@@ -168,15 +168,14 @@ class RecordBatchSerializer {
                         std::shared_ptr<Buffer>* out) {
     // Convert buffer to uncompressed-length-prefixed compressed buffer
     int64_t maximum_length = codec->MaxCompressedLen(buffer.size(), buffer.data());
-    std::shared_ptr<Buffer> result;
-    RETURN_NOT_OK(AllocateBuffer(maximum_length + sizeof(int64_t), &result));
+    ARROW_ASSIGN_OR_RAISE(auto result, AllocateBuffer(maximum_length + sizeof(int64_t)));
 
     int64_t actual_length;
     ARROW_ASSIGN_OR_RAISE(actual_length,
                           codec->Compress(buffer.size(), buffer.data(), maximum_length,
                                           result->mutable_data() + sizeof(int64_t)));
     *reinterpret_cast<int64_t*>(result->mutable_data()) = buffer.size();
-    *out = SliceBuffer(result, /*offset=*/0, actual_length + sizeof(int64_t));
+    *out = SliceBuffer(std::move(result), /*offset=*/0, actual_length + sizeof(int64_t));
     return Status::OK();
   }
 
@@ -274,9 +273,8 @@ class RecordBatchSerializer {
       // zero. We must a) create a new offsets array with shifted offsets and
       // b) slice the values array accordingly
 
-      std::shared_ptr<Buffer> shifted_offsets;
-      RETURN_NOT_OK(
-          AllocateBuffer(options_.memory_pool, required_bytes, &shifted_offsets));
+      ARROW_ASSIGN_OR_RAISE(auto shifted_offsets,
+                            AllocateBuffer(required_bytes, options_.memory_pool));
 
       offset_type* dest_offsets =
           reinterpret_cast<offset_type*>(shifted_offsets->mutable_data());
@@ -287,7 +285,7 @@ class RecordBatchSerializer {
       }
       // Final offset
       dest_offsets[array.length()] = array.value_offset(array.length()) - start_offset;
-      offsets = shifted_offsets;
+      offsets = std::move(shifted_offsets);
     } else {
       // ARROW-6046: Slice offsets to used extent, in case we have a truncated
       // slice
@@ -295,7 +293,7 @@ class RecordBatchSerializer {
         offsets = SliceBuffer(offsets, 0, required_bytes);
       }
     }
-    *value_offsets = offsets;
+    *value_offsets = std::move(offsets);
     return Status::OK();
   }
 
@@ -446,9 +444,9 @@ class RecordBatchSerializer {
         const int8_t* type_codes = array.raw_type_codes();
 
         // Allocate the shifted offsets
-        std::shared_ptr<Buffer> shifted_offsets_buffer;
-        RETURN_NOT_OK(AllocateBuffer(options_.memory_pool, length * sizeof(int32_t),
-                                     &shifted_offsets_buffer));
+        ARROW_ASSIGN_OR_RAISE(
+            auto shifted_offsets_buffer,
+            AllocateBuffer(length * sizeof(int32_t), options_.memory_pool));
         int32_t* shifted_offsets =
             reinterpret_cast<int32_t*>(shifted_offsets_buffer->mutable_data());
 
@@ -471,7 +469,7 @@ class RecordBatchSerializer {
           child_lengths[code] = std::max(child_lengths[code], shifted_offsets[i] + 1);
         }
 
-        value_offsets = shifted_offsets_buffer;
+        value_offsets = std::move(shifted_offsets_buffer);
       }
       out_->body_buffers.emplace_back(value_offsets);
 
@@ -676,13 +674,12 @@ Status GetContiguousTensor(const Tensor& tensor, MemoryPool* pool,
   const auto& type = checked_cast<const FixedWidthType&>(*tensor.type());
   const int elem_size = type.bit_width() / 8;
 
-  std::shared_ptr<Buffer> scratch_space;
-  RETURN_NOT_OK(AllocateBuffer(pool, tensor.shape()[tensor.ndim() - 1] * elem_size,
-                               &scratch_space));
+  ARROW_ASSIGN_OR_RAISE(
+      auto scratch_space,
+      AllocateBuffer(tensor.shape()[tensor.ndim() - 1] * elem_size, pool));
 
-  std::shared_ptr<ResizableBuffer> contiguous_data;
-  RETURN_NOT_OK(
-      AllocateResizableBuffer(pool, tensor.size() * elem_size, &contiguous_data));
+  ARROW_ASSIGN_OR_RAISE(std::shared_ptr<ResizableBuffer> contiguous_data,
+                        AllocateResizableBuffer(tensor.size() * elem_size, pool));
 
   io::BufferOutputStream stream(contiguous_data);
   RETURN_NOT_OK(WriteStridedTensorData(0, 0, elem_size, tensor,
@@ -718,9 +715,8 @@ Status WriteTensor(const Tensor& tensor, io::OutputStream* dst, int32_t* metadat
 
     // TODO: Do we care enough about this temporary allocation to pass in a
     // MemoryPool to this function?
-    std::shared_ptr<Buffer> scratch_space;
-    RETURN_NOT_OK(
-        AllocateBuffer(tensor.shape()[tensor.ndim() - 1] * elem_size, &scratch_space));
+    ARROW_ASSIGN_OR_RAISE(auto scratch_space,
+                          AllocateBuffer(tensor.shape()[tensor.ndim() - 1] * elem_size));
 
     RETURN_NOT_OK(WriteStridedTensorData(0, 0, elem_size, tensor,
                                          scratch_space->mutable_data(), dst));
@@ -1242,12 +1238,12 @@ Status SerializeRecordBatch(const RecordBatch& batch, const IpcWriteOptions& opt
                             std::shared_ptr<Buffer>* out) {
   int64_t size = 0;
   RETURN_NOT_OK(GetRecordBatchSize(batch, &size));
-  std::shared_ptr<Buffer> buffer;
-  RETURN_NOT_OK(AllocateBuffer(options.memory_pool, size, &buffer));
+  ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Buffer> buffer,
+                        AllocateBuffer(size, options.memory_pool));
 
   io::FixedSizeBufferWriter stream(buffer);
   RETURN_NOT_OK(SerializeRecordBatch(batch, options, &stream));
-  *out = buffer;
+  *out = std::move(buffer);
   return Status::OK();
 }
 
diff --git a/cpp/src/arrow/json/chunked_builder.cc b/cpp/src/arrow/json/chunked_builder.cc
index e99be0a..8364ade 100644
--- a/cpp/src/arrow/json/chunked_builder.cc
+++ b/cpp/src/arrow/json/chunked_builder.cc
@@ -245,12 +245,14 @@ class ChunkedListArrayBuilder : public ChunkedArrayBuilder {
   Status InsertNull(int64_t block_index, int64_t length) {
     value_builder_->Insert(block_index, value_field_, std::make_shared<NullArray>(0));
 
-    RETURN_NOT_OK(AllocateBitmap(pool_, length, &null_bitmap_chunks_[block_index]));
+    ARROW_ASSIGN_OR_RAISE(null_bitmap_chunks_[block_index],
+                          AllocateBitmap(length, pool_));
     std::memset(null_bitmap_chunks_[block_index]->mutable_data(), 0,
                 null_bitmap_chunks_[block_index]->size());
 
     int64_t offsets_length = (length + 1) * sizeof(int32_t);
-    RETURN_NOT_OK(AllocateBuffer(pool_, offsets_length, &offset_chunks_[block_index]));
+    ARROW_ASSIGN_OR_RAISE(offset_chunks_[block_index],
+                          AllocateBuffer(offsets_length, pool_));
     std::memset(offset_chunks_[block_index]->mutable_data(), 0, offsets_length);
 
     return Status::OK();
@@ -291,12 +293,13 @@ class ChunkedStructArrayBuilder : public ChunkedArrayBuilder {
     chunk_lengths_[block_index] = unconverted->length();
 
     if (unconverted->type_id() == Type::NA) {
-      auto st =
-          AllocateBitmap(pool_, unconverted->length(), &null_bitmap_chunks_[block_index]);
-      std::memset(null_bitmap_chunks_[block_index]->mutable_data(), 0,
-                  null_bitmap_chunks_[block_index]->size());
-
-      if (!st.ok()) {
+      auto maybe_buffer = AllocateBitmap(unconverted->length(), pool_);
+      if (maybe_buffer.ok()) {
+        null_bitmap_chunks_[block_index] = *std::move(maybe_buffer);
+        std::memset(null_bitmap_chunks_[block_index]->mutable_data(), 0,
+                    null_bitmap_chunks_[block_index]->size());
+      } else {
+        Status st = maybe_buffer.status();
         task_group_->Append([st] { return st; });
       }
 
diff --git a/cpp/src/arrow/json/chunker_test.cc b/cpp/src/arrow/json/chunker_test.cc
index 57b9a6c..1b4ea4d 100644
--- a/cpp/src/arrow/json/chunker_test.cc
+++ b/cpp/src/arrow/json/chunker_test.cc
@@ -52,8 +52,7 @@ static std::shared_ptr<Buffer> join(Lines&& lines, std::string delimiter,
   if (!delimiter_at_end) {
     line_buffers.pop_back();
   }
-  ABORT_NOT_OK(ConcatenateBuffers(line_buffers, default_memory_pool(), &joined));
-  return joined;
+  return *ConcatenateBuffers(line_buffers);
 }
 
 static bool WhitespaceOnly(string_view s) {
@@ -137,7 +136,7 @@ void AssertChunkingBlockSize(Chunker& chunker, std::shared_ptr<Buffer> buf,
       ASSERT_OK(chunker.Process(starts_with_whole, &whole, &next_partial));
     }
     // partial + completion should be a valid JSON block
-    ASSERT_OK(ConcatenateBuffers({partial, completion}, default_memory_pool(), &partial));
+    ASSERT_OK_AND_ASSIGN(partial, ConcatenateBuffers({partial, completion}));
     AssertOnlyWholeObjects(chunker, partial, &count);
     total_count += count;
     // whole should be a valid JSON block
@@ -160,8 +159,7 @@ void AssertStraddledChunking(Chunker& chunker, const std::shared_ptr<Buffer>& bu
   ASSERT_OK(chunker.ProcessWithPartial(partial, second_half, &completion, &rest));
   ASSERT_TRUE(string_view(*second_half).starts_with(string_view(*completion)));
   std::shared_ptr<Buffer> straddling;
-  ASSERT_OK(
-      ConcatenateBuffers({partial, completion}, default_memory_pool(), &straddling));
+  ASSERT_OK_AND_ASSIGN(straddling, ConcatenateBuffers({partial, completion}));
   auto length = ConsumeWholeObject(&straddling);
   ASSERT_NE(length, string_view::npos);
   ASSERT_NE(length, 0);
diff --git a/cpp/src/arrow/json/reader.cc b/cpp/src/arrow/json/reader.cc
index 188ba55..4f4ac21 100644
--- a/cpp/src/arrow/json/reader.cc
+++ b/cpp/src/arrow/json/reader.cc
@@ -140,7 +140,8 @@ class TableReaderImpl : public TableReader,
       } else if (completion->size() == 0) {
         straddling = partial;
       } else {
-        RETURN_NOT_OK(ConcatenateBuffers({partial, completion}, pool_, &straddling));
+        ARROW_ASSIGN_OR_RAISE(straddling,
+                              ConcatenateBuffers({partial, completion}, pool_));
       }
       RETURN_NOT_OK(parser->Parse(straddling));
     }
diff --git a/cpp/src/arrow/python/arrow_to_pandas.cc b/cpp/src/arrow/python/arrow_to_pandas.cc
index d6c626a..3c989b5 100644
--- a/cpp/src/arrow/python/arrow_to_pandas.cc
+++ b/cpp/src/arrow/python/arrow_to_pandas.cc
@@ -239,8 +239,7 @@ Status PyArray_NewFromPool(int nd, npy_intp* dims, PyArray_Descr* descr, MemoryP
     total_size *= dims[i];
   }
 
-  std::shared_ptr<Buffer> buffer;
-  RETURN_NOT_OK(AllocateBuffer(pool, total_size, &buffer));
+  ARROW_ASSIGN_OR_RAISE(auto buffer, AllocateBuffer(total_size, pool));
   *out = PyArray_NewFromDescr(&PyArray_Type, descr, nd, dims,
                               /*strides=*/nullptr,
                               /*data=*/buffer->mutable_data(),
@@ -250,7 +249,7 @@ Status PyArray_NewFromPool(int nd, npy_intp* dims, PyArray_Descr* descr, MemoryP
     RETURN_IF_PYERROR();
     // Trust that error set if NULL returned
   }
-  return SetBufferBase(reinterpret_cast<PyArrayObject*>(*out), buffer);
+  return SetBufferBase(reinterpret_cast<PyArrayObject*>(*out), std::move(buffer));
 }
 
 template <typename T = void>
diff --git a/cpp/src/arrow/python/numpy_to_arrow.cc b/cpp/src/arrow/python/numpy_to_arrow.cc
index 00b41f5..756707b 100644
--- a/cpp/src/arrow/python/numpy_to_arrow.cc
+++ b/cpp/src/arrow/python/numpy_to_arrow.cc
@@ -74,12 +74,11 @@ namespace {
 Status AllocateNullBitmap(MemoryPool* pool, int64_t length,
                           std::shared_ptr<ResizableBuffer>* out) {
   int64_t null_bytes = BitUtil::BytesForBits(length);
-  std::shared_ptr<ResizableBuffer> null_bitmap;
-  RETURN_NOT_OK(AllocateResizableBuffer(pool, null_bytes, &null_bitmap));
+  ARROW_ASSIGN_OR_RAISE(auto null_bitmap, AllocateResizableBuffer(null_bytes, pool));
 
   // Padding zeroed by AllocateResizableBuffer
   memset(null_bitmap->mutable_data(), 0, static_cast<size_t>(null_bytes));
-  *out = null_bitmap;
+  *out = std::move(null_bitmap);
   return Status::OK();
 }
 
@@ -356,15 +355,14 @@ Status CastBuffer(const std::shared_ptr<DataType>& in_type,
 template <typename FromType, typename ToType>
 Status StaticCastBuffer(const Buffer& input, const int64_t length, MemoryPool* pool,
                         std::shared_ptr<Buffer>* out) {
-  std::shared_ptr<Buffer> result;
-  RETURN_NOT_OK(AllocateBuffer(pool, sizeof(ToType) * length, &result));
+  ARROW_ASSIGN_OR_RAISE(auto result, AllocateBuffer(sizeof(ToType) * length, pool));
 
   auto in_values = reinterpret_cast<const FromType*>(input.data());
   auto out_values = reinterpret_cast<ToType*>(result->mutable_data());
   for (int64_t i = 0; i < length; ++i) {
     *out_values++ = static_cast<ToType>(*in_values++);
   }
-  *out = result;
+  *out = std::move(result);
   return Status::OK();
 }
 
@@ -402,7 +400,7 @@ class NumPyStridedConverter {
     using traits = internal::npy_traits<TYPE>;
     using T = typename traits::value_type;
 
-    RETURN_NOT_OK(AllocateBuffer(pool_, sizeof(T) * length_, &buffer_));
+    ARROW_ASSIGN_OR_RAISE(buffer_, AllocateBuffer(sizeof(T) * length_, pool_));
 
     const int64_t stride = PyArray_STRIDES(arr)[0];
     if (stride % sizeof(T) == 0) {
@@ -436,15 +434,14 @@ inline Status NumPyConverter::PrepareInputData(std::shared_ptr<Buffer>* data) {
 
   if (dtype_->type_num == NPY_BOOL) {
     int64_t nbytes = BitUtil::BytesForBits(length_);
-    std::shared_ptr<Buffer> buffer;
-    RETURN_NOT_OK(AllocateBuffer(pool_, nbytes, &buffer));
+    ARROW_ASSIGN_OR_RAISE(auto buffer, AllocateBuffer(nbytes, pool_));
 
     Ndarray1DIndexer<uint8_t> values(arr_);
     int64_t i = 0;
     const auto generate = [&values, &i]() -> bool { return values[i++] > 0; };
     GenerateBitsUnrolled(buffer->mutable_data(), 0, length_, generate);
 
-    *data = buffer;
+    *data = std::move(buffer);
   } else if (is_strided()) {
     RETURN_NOT_OK(NumPyStridedConverter::Convert(arr_, length_, pool_, data));
   } else {
@@ -518,15 +515,15 @@ inline Status NumPyConverter::ConvertData<Date64Type>(std::shared_ptr<Buffer>* d
     // separately here from int64_t to int32_t, because this data is not
     // supported in compute::Cast
     if (date_dtype->meta.base == NPY_FR_D) {
-      std::shared_ptr<Buffer> result;
-      RETURN_NOT_OK(AllocateBuffer(pool_, sizeof(int64_t) * length_, &result));
+      ARROW_ASSIGN_OR_RAISE(auto result,
+                            AllocateBuffer(sizeof(int64_t) * length_, pool_));
 
       auto in_values = reinterpret_cast<const int64_t*>((*data)->data());
       auto out_values = reinterpret_cast<int64_t*>(result->mutable_data());
       for (int64_t i = 0; i < length_; ++i) {
         *out_values++ = kMillisecondsInDay * (*in_values++);
       }
-      *data = result;
+      *data = std::move(result);
     } else {
       RETURN_NOT_OK(NumPyDtypeToArrow(reinterpret_cast<PyObject*>(dtype_), &input_type));
       if (!input_type->Equals(*type_)) {
diff --git a/cpp/src/arrow/sparse_tensor.cc b/cpp/src/arrow/sparse_tensor.cc
index 52c13ad..8da6a8b 100644
--- a/cpp/src/arrow/sparse_tensor.cc
+++ b/cpp/src/arrow/sparse_tensor.cc
@@ -109,9 +109,8 @@ Status MakeTensorFromSparseTensor(MemoryPool* pool, const SparseTensor* sparse_t
   using NumericTensorType = NumericTensor<TYPE>;
   using value_type = typename NumericTensorType::value_type;
 
-  std::shared_ptr<Buffer> values_buffer;
-  RETURN_NOT_OK(
-      AllocateBuffer(pool, sizeof(value_type) * sparse_tensor->size(), &values_buffer));
+  ARROW_ASSIGN_OR_RAISE(auto values_buffer,
+                        AllocateBuffer(sizeof(value_type) * sparse_tensor->size(), pool));
   auto values = reinterpret_cast<value_type*>(values_buffer->mutable_data());
 
   std::fill_n(values, sparse_tensor->size(), static_cast<value_type>(0));
@@ -139,7 +138,7 @@ Status MakeTensorFromSparseTensor(MemoryPool* pool, const SparseTensor* sparse_t
         }
         values[offset] = raw_data[i];
       }
-      *out = std::make_shared<Tensor>(sparse_tensor->type(), values_buffer,
+      *out = std::make_shared<Tensor>(sparse_tensor->type(), std::move(values_buffer),
                                       sparse_tensor->shape(), empty_strides,
                                       sparse_tensor->dim_names());
       return Status::OK();
@@ -160,7 +159,7 @@ Status MakeTensorFromSparseTensor(MemoryPool* pool, const SparseTensor* sparse_t
           values[offset] = raw_data[j];
         }
       }
-      *out = std::make_shared<Tensor>(sparse_tensor->type(), values_buffer,
+      *out = std::make_shared<Tensor>(sparse_tensor->type(), std::move(values_buffer),
                                       sparse_tensor->shape(), empty_strides,
                                       sparse_tensor->dim_names());
       return Status::OK();
@@ -181,7 +180,7 @@ Status MakeTensorFromSparseTensor(MemoryPool* pool, const SparseTensor* sparse_t
           values[offset] = raw_data[i];
         }
       }
-      *out = std::make_shared<Tensor>(sparse_tensor->type(), values_buffer,
+      *out = std::make_shared<Tensor>(sparse_tensor->type(), std::move(values_buffer),
                                       sparse_tensor->shape(), empty_strides,
                                       sparse_tensor->dim_names());
       return Status::OK();
@@ -194,7 +193,7 @@ Status MakeTensorFromSparseTensor(MemoryPool* pool, const SparseTensor* sparse_t
       ExpandSparseCSFTensorValues<value_type, IndexValueType>(
           0, 0, 0, sparse_index.indptr()[0]->size() - 1, sparse_index, raw_data, strides,
           sparse_index.axis_order(), values);
-      *out = std::make_shared<Tensor>(sparse_tensor->type(), values_buffer,
+      *out = std::make_shared<Tensor>(sparse_tensor->type(), std::move(values_buffer),
                                       sparse_tensor->shape(), empty_strides,
                                       sparse_tensor->dim_names());
       return Status::OK();
diff --git a/cpp/src/arrow/tensor/coo_converter.cc b/cpp/src/arrow/tensor/coo_converter.cc
index 844d895..5a63874 100644
--- a/cpp/src/arrow/tensor/coo_converter.cc
+++ b/cpp/src/arrow/tensor/coo_converter.cc
@@ -71,15 +71,13 @@ class SparseCOOTensorConverter {
     int64_t nonzero_count = -1;
     RETURN_NOT_OK(tensor_.CountNonZero(&nonzero_count));
 
-    std::shared_ptr<Buffer> indices_buffer;
-    RETURN_NOT_OK(
-        AllocateBuffer(pool_, indices_elsize * ndim * nonzero_count, &indices_buffer));
+    ARROW_ASSIGN_OR_RAISE(auto indices_buffer,
+                          AllocateBuffer(indices_elsize * ndim * nonzero_count, pool_));
     c_index_value_type* indices =
         reinterpret_cast<c_index_value_type*>(indices_buffer->mutable_data());
 
-    std::shared_ptr<Buffer> values_buffer;
-    RETURN_NOT_OK(
-        AllocateBuffer(pool_, sizeof(value_type) * nonzero_count, &values_buffer));
+    ARROW_ASSIGN_OR_RAISE(auto values_buffer,
+                          AllocateBuffer(sizeof(value_type) * nonzero_count, pool_));
     value_type* values = reinterpret_cast<value_type*>(values_buffer->mutable_data());
 
     if (ndim <= 1) {
@@ -112,8 +110,8 @@ class SparseCOOTensorConverter {
     const std::vector<int64_t> indices_shape = {nonzero_count, ndim};
     const std::vector<int64_t> indices_strides = {indices_elsize * ndim, indices_elsize};
     sparse_index = std::make_shared<SparseCOOIndex>(std::make_shared<Tensor>(
-        index_value_type_, indices_buffer, indices_shape, indices_strides));
-    data = values_buffer;
+        index_value_type_, std::move(indices_buffer), indices_shape, indices_strides));
+    data = std::move(values_buffer);
 
     return Status::OK();
   }
diff --git a/cpp/src/arrow/tensor/csc_converter.cc b/cpp/src/arrow/tensor/csc_converter.cc
index edeaeef..ccc4bf2 100644
--- a/cpp/src/arrow/tensor/csc_converter.cc
+++ b/cpp/src/arrow/tensor/csc_converter.cc
@@ -70,19 +70,19 @@ class SparseCSCMatrixConverter {
     std::shared_ptr<Buffer> indptr_buffer;
     std::shared_ptr<Buffer> indices_buffer;
 
-    std::shared_ptr<Buffer> values_buffer;
-    RETURN_NOT_OK(
-        AllocateBuffer(pool_, sizeof(value_type) * nonzero_count, &values_buffer));
+    ARROW_ASSIGN_OR_RAISE(auto values_buffer,
+                          AllocateBuffer(sizeof(value_type) * nonzero_count, pool_));
     value_type* values = reinterpret_cast<value_type*>(values_buffer->mutable_data());
 
     if (ndim <= 1) {
       return Status::NotImplemented("TODO for ndim <= 1");
     } else {
-      RETURN_NOT_OK(AllocateBuffer(pool_, indices_elsize * (nc + 1), &indptr_buffer));
+      ARROW_ASSIGN_OR_RAISE(indptr_buffer,
+                            AllocateBuffer(indices_elsize * (nc + 1), pool_));
       auto* indptr = reinterpret_cast<c_index_value_type*>(indptr_buffer->mutable_data());
 
-      RETURN_NOT_OK(
-          AllocateBuffer(pool_, indices_elsize * nonzero_count, &indices_buffer));
+      ARROW_ASSIGN_OR_RAISE(indices_buffer,
+                            AllocateBuffer(indices_elsize * nonzero_count, pool_));
       auto* indices =
           reinterpret_cast<c_index_value_type*>(indices_buffer->mutable_data());
 
@@ -110,7 +110,7 @@ class SparseCSCMatrixConverter {
         std::make_shared<Tensor>(index_value_type_, indices_buffer, indices_shape);
 
     sparse_index = std::make_shared<SparseCSCIndex>(indptr_tensor, indices_tensor);
-    data = values_buffer;
+    data = std::move(values_buffer);
 
     return Status::OK();
   }
diff --git a/cpp/src/arrow/tensor/csf_converter.cc b/cpp/src/arrow/tensor/csf_converter.cc
index 7fe40b2..1c9ee2c 100644
--- a/cpp/src/arrow/tensor/csf_converter.cc
+++ b/cpp/src/arrow/tensor/csf_converter.cc
@@ -80,9 +80,8 @@ class SparseCSFTensorConverter {
     int64_t nonzero_count = -1;
     RETURN_NOT_OK(tensor_.CountNonZero(&nonzero_count));
 
-    std::shared_ptr<Buffer> values_buffer;
-    RETURN_NOT_OK(
-        AllocateBuffer(pool_, sizeof(value_type) * nonzero_count, &values_buffer));
+    ARROW_ASSIGN_OR_RAISE(auto values_buffer,
+                          AllocateBuffer(sizeof(value_type) * nonzero_count, pool_));
     value_type* values = reinterpret_cast<value_type*>(values_buffer->mutable_data());
 
     std::vector<int64_t> counts(ndim, 0);
@@ -128,7 +127,7 @@ class SparseCSFTensorConverter {
     }
 
     // make results
-    data = values_buffer;
+    data = std::move(values_buffer);
 
     std::vector<std::shared_ptr<Buffer>> indptr_buffers(ndim - 1);
     std::vector<std::shared_ptr<Buffer>> indices_buffers(ndim);
diff --git a/cpp/src/arrow/tensor/csr_converter.cc b/cpp/src/arrow/tensor/csr_converter.cc
index f5239bd..ec9cbd7 100644
--- a/cpp/src/arrow/tensor/csr_converter.cc
+++ b/cpp/src/arrow/tensor/csr_converter.cc
@@ -70,19 +70,19 @@ class SparseCSRMatrixConverter {
     std::shared_ptr<Buffer> indptr_buffer;
     std::shared_ptr<Buffer> indices_buffer;
 
-    std::shared_ptr<Buffer> values_buffer;
-    RETURN_NOT_OK(
-        AllocateBuffer(pool_, sizeof(value_type) * nonzero_count, &values_buffer));
+    ARROW_ASSIGN_OR_RAISE(auto values_buffer,
+                          AllocateBuffer(sizeof(value_type) * nonzero_count, pool_));
     value_type* values = reinterpret_cast<value_type*>(values_buffer->mutable_data());
 
     if (ndim <= 1) {
       return Status::NotImplemented("TODO for ndim <= 1");
     } else {
-      RETURN_NOT_OK(AllocateBuffer(pool_, indices_elsize * (nr + 1), &indptr_buffer));
+      ARROW_ASSIGN_OR_RAISE(indptr_buffer,
+                            AllocateBuffer(indices_elsize * (nr + 1), pool_));
       auto* indptr = reinterpret_cast<c_index_value_type*>(indptr_buffer->mutable_data());
 
-      RETURN_NOT_OK(
-          AllocateBuffer(pool_, indices_elsize * nonzero_count, &indices_buffer));
+      ARROW_ASSIGN_OR_RAISE(indices_buffer,
+                            AllocateBuffer(indices_elsize * nonzero_count, pool_));
       auto* indices =
           reinterpret_cast<c_index_value_type*>(indices_buffer->mutable_data());
 
@@ -110,7 +110,7 @@ class SparseCSRMatrixConverter {
         std::make_shared<Tensor>(index_value_type_, indices_buffer, indices_shape);
 
     sparse_index = std::make_shared<SparseCSRIndex>(indptr_tensor, indices_tensor);
-    data = values_buffer;
+    data = std::move(values_buffer);
 
     return Status::OK();
   }
diff --git a/cpp/src/arrow/tensor_test.cc b/cpp/src/arrow/tensor_test.cc
index 5f979c9..52efd82 100644
--- a/cpp/src/arrow/tensor_test.cc
+++ b/cpp/src/arrow/tensor_test.cc
@@ -21,6 +21,7 @@
 #include <cstdint>
 #include <memory>
 #include <string>
+#include <utility>
 #include <vector>
 
 #include <gtest/gtest.h>
@@ -140,8 +141,8 @@ TEST(TestTensor, ZeroDim) {
 
   using T = int64_t;
 
-  std::shared_ptr<Buffer> buffer;
-  ASSERT_OK(AllocateBuffer(values * sizeof(T), &buffer));
+  ASSERT_OK_AND_ASSIGN(std::shared_ptr<Buffer> buffer,
+                       AllocateBuffer(values * sizeof(T)));
 
   Tensor t0(int64(), buffer, shape);
 
@@ -156,8 +157,8 @@ TEST(TestTensor, BasicCtors) {
 
   using T = int64_t;
 
-  std::shared_ptr<Buffer> buffer;
-  ASSERT_OK(AllocateBuffer(values * sizeof(T), &buffer));
+  ASSERT_OK_AND_ASSIGN(std::shared_ptr<Buffer> buffer,
+                       AllocateBuffer(values * sizeof(T)));
 
   Tensor t1(int64(), buffer, shape);
   Tensor t2(int64(), buffer, shape, strides);
@@ -185,8 +186,8 @@ TEST(TestTensor, IsContiguous) {
 
   using T = int64_t;
 
-  std::shared_ptr<Buffer> buffer;
-  ASSERT_OK(AllocateBuffer(values * sizeof(T), &buffer));
+  ASSERT_OK_AND_ASSIGN(std::shared_ptr<Buffer> buffer,
+                       AllocateBuffer(values * sizeof(T)));
 
   std::vector<int64_t> c_strides = {48, 8};
   std::vector<int64_t> f_strides = {8, 32};
@@ -203,8 +204,7 @@ TEST(TestTensor, IsContiguous) {
 TEST(TestTensor, ZeroSizedTensor) {
   std::vector<int64_t> shape = {0};
 
-  std::shared_ptr<Buffer> buffer;
-  ASSERT_OK(AllocateBuffer(0, &buffer));
+  ASSERT_OK_AND_ASSIGN(std::shared_ptr<Buffer> buffer, AllocateBuffer(0));
 
   Tensor t(int64(), buffer, shape);
   ASSERT_EQ(t.strides().size(), 1);
@@ -213,8 +213,7 @@ TEST(TestTensor, ZeroSizedTensor) {
 TEST(TestTensor, CountNonZeroForZeroSizedTensor) {
   std::vector<int64_t> shape = {0};
 
-  std::shared_ptr<Buffer> buffer;
-  ASSERT_OK(AllocateBuffer(0, &buffer));
+  ASSERT_OK_AND_ASSIGN(std::shared_ptr<Buffer> buffer, AllocateBuffer(0));
 
   Tensor t(int64(), buffer, shape);
   AssertCountNonZero(t, 0);
@@ -349,11 +348,10 @@ TEST(TestTensor, EqualsInt64) {
   EXPECT_FALSE(tf3.Equals(tnc));
 
   // zero-size tensor
-  std::shared_ptr<Buffer> empty_buffer1, empty_buffer2;
-  ASSERT_OK(AllocateBuffer(0, &empty_buffer1));
-  ASSERT_OK(AllocateBuffer(0, &empty_buffer2));
-  Tensor empty1(int64(), empty_buffer1, {0});
-  Tensor empty2(int64(), empty_buffer2, {0});
+  ASSERT_OK_AND_ASSIGN(auto empty_buffer1, AllocateBuffer(0));
+  ASSERT_OK_AND_ASSIGN(auto empty_buffer2, AllocateBuffer(0));
+  Tensor empty1(int64(), std::move(empty_buffer1), {0});
+  Tensor empty2(int64(), std::move(empty_buffer2), {0});
   EXPECT_FALSE(empty1.Equals(tc1));
   EXPECT_TRUE(empty1.Equals(empty2));
 }
diff --git a/cpp/src/arrow/testing/gtest_common.h b/cpp/src/arrow/testing/gtest_common.h
index 426bd9e..86ac1de 100644
--- a/cpp/src/arrow/testing/gtest_common.h
+++ b/cpp/src/arrow/testing/gtest_common.h
@@ -21,6 +21,7 @@
 #include <memory>
 #include <random>
 #include <string>
+#include <utility>
 #include <vector>
 
 #include <gtest/gtest.h>
@@ -44,8 +45,7 @@ class TestBase : public ::testing::Test {
   std::shared_ptr<Buffer> MakeRandomNullBitmap(int64_t length, int64_t null_count) {
     const int64_t null_nbytes = BitUtil::BytesForBits(length);
 
-    std::shared_ptr<Buffer> null_bitmap;
-    ARROW_EXPECT_OK(AllocateBuffer(pool_, null_nbytes, &null_bitmap));
+    auto null_bitmap = *AllocateBuffer(null_nbytes, pool_);
     memset(null_bitmap->mutable_data(), 255, null_nbytes);
     for (int64_t i = 0; i < null_count; i++) {
       BitUtil::ClearBit(null_bitmap->mutable_data(), i * (length / null_count));
@@ -64,14 +64,13 @@ class TestBase : public ::testing::Test {
 template <typename ArrayType>
 std::shared_ptr<Array> TestBase::MakeRandomArray(int64_t length, int64_t null_count) {
   const int64_t data_nbytes = length * sizeof(typename ArrayType::value_type);
-  std::shared_ptr<Buffer> data;
-  ARROW_EXPECT_OK(AllocateBuffer(pool_, data_nbytes, &data));
+  auto data = *AllocateBuffer(data_nbytes, pool_);
 
   // Fill with random data
   random_bytes(data_nbytes, random_seed_++, data->mutable_data());
   std::shared_ptr<Buffer> null_bitmap = MakeRandomNullBitmap(length, null_count);
 
-  return std::make_shared<ArrayType>(length, data, null_bitmap, null_count);
+  return std::make_shared<ArrayType>(length, std::move(data), null_bitmap, null_count);
 }
 
 template <>
@@ -85,12 +84,11 @@ inline std::shared_ptr<Array> TestBase::MakeRandomArray<FixedSizeBinaryArray>(
     int64_t length, int64_t null_count) {
   const int byte_width = 10;
   std::shared_ptr<Buffer> null_bitmap = MakeRandomNullBitmap(length, null_count);
-  std::shared_ptr<Buffer> data;
-  ARROW_EXPECT_OK(AllocateBuffer(pool_, byte_width * length, &data));
+  auto data = *AllocateBuffer(byte_width * length, pool_);
 
   ::arrow::random_bytes(data->size(), 0, data->mutable_data());
   return std::make_shared<FixedSizeBinaryArray>(fixed_size_binary(byte_width), length,
-                                                data, null_bitmap, null_count);
+                                                std::move(data), null_bitmap, null_count);
 }
 
 template <>
diff --git a/cpp/src/arrow/testing/gtest_util.h b/cpp/src/arrow/testing/gtest_util.h
index 50fa2ed..6119e0f 100644
--- a/cpp/src/arrow/testing/gtest_util.h
+++ b/cpp/src/arrow/testing/gtest_util.h
@@ -384,8 +384,7 @@ static inline Status GetBitmapFromVector(const std::vector<T>& is_valid,
                                          std::shared_ptr<Buffer>* result) {
   size_t length = is_valid.size();
 
-  std::shared_ptr<Buffer> buffer;
-  RETURN_NOT_OK(AllocateEmptyBitmap(length, &buffer));
+  ARROW_ASSIGN_OR_RAISE(auto buffer, AllocateEmptyBitmap(length));
 
   uint8_t* bitmap = buffer->mutable_data();
   for (size_t i = 0; i < static_cast<size_t>(length); ++i) {
diff --git a/cpp/src/arrow/testing/random.cc b/cpp/src/arrow/testing/random.cc
index 80c813a..140ab45 100644
--- a/cpp/src/arrow/testing/random.cc
+++ b/cpp/src/arrow/testing/random.cc
@@ -88,10 +88,10 @@ std::shared_ptr<Array> RandomArrayGenerator::Boolean(int64_t size, double probab
   GenOpt null_gen(seed(), 0, 1, null_probability);
 
   int64_t null_count = 0;
-  ABORT_NOT_OK(AllocateEmptyBitmap(size, &buffers[0]));
+  buffers[0] = *AllocateEmptyBitmap(size);
   null_gen.GenerateBitmap(buffers[0]->mutable_data(), size, &null_count);
 
-  ABORT_NOT_OK(AllocateEmptyBitmap(size, &buffers[1]));
+  buffers[1] = *AllocateEmptyBitmap(size);
   value_gen.GenerateBitmap(buffers[1]->mutable_data(), size, nullptr);
 
   auto array_data = ArrayData::Make(arrow::boolean(), size, buffers, null_count);
@@ -106,10 +106,10 @@ static std::shared_ptr<NumericArray<ArrowType>> GenerateNumericArray(int64_t siz
   BufferVector buffers{2};
 
   int64_t null_count = 0;
-  ABORT_NOT_OK(AllocateEmptyBitmap(size, &buffers[0]));
+  buffers[0] = *AllocateEmptyBitmap(size);
   options.GenerateBitmap(buffers[0]->mutable_data(), size, &null_count);
 
-  ABORT_NOT_OK(AllocateBuffer(sizeof(CType) * size, &buffers[1]))
+  buffers[1] = *AllocateBuffer(sizeof(CType) * size);
   options.GenerateData(buffers[1]->mutable_data(), size);
 
   auto array_data = ArrayData::Make(type, size, buffers, null_count);
@@ -247,7 +247,7 @@ std::shared_ptr<Array> RandomArrayGenerator::Offsets(int64_t size, int32_t first
 
   BufferVector buffers{2};
 
-  ABORT_NOT_OK(AllocateBuffer(sizeof(int32_t) * size, &buffers[1]));
+  buffers[1] = *AllocateBuffer(sizeof(int32_t) * size);
   auto data = reinterpret_cast<int32_t*>(buffers[1]->mutable_data());
   options.GenerateTypedData(data, size);
   // Ensure offsets are in increasing order
diff --git a/cpp/src/arrow/testing/util.cc b/cpp/src/arrow/testing/util.cc
index 5aa4285..06fc554 100644
--- a/cpp/src/arrow/testing/util.cc
+++ b/cpp/src/arrow/testing/util.cc
@@ -175,10 +175,9 @@ int64_t CountNulls(const std::vector<uint8_t>& valid_bytes) {
 
 Status MakeRandomByteBuffer(int64_t length, MemoryPool* pool,
                             std::shared_ptr<ResizableBuffer>* out, uint32_t seed) {
-  std::shared_ptr<ResizableBuffer> result;
-  RETURN_NOT_OK(AllocateResizableBuffer(pool, length, &result));
+  ARROW_ASSIGN_OR_RAISE(auto result, AllocateResizableBuffer(length, pool));
   random_bytes(length, seed, result->mutable_data());
-  *out = result;
+  *out = std::move(result);
   return Status::OK();
 }
 
diff --git a/cpp/src/arrow/testing/util.h b/cpp/src/arrow/testing/util.h
index 3ddf097..94cd867 100644
--- a/cpp/src/arrow/testing/util.h
+++ b/cpp/src/arrow/testing/util.h
@@ -44,14 +44,13 @@ Status CopyBufferFromVector(const std::vector<T>& values, MemoryPool* pool,
                             std::shared_ptr<Buffer>* result) {
   int64_t nbytes = static_cast<int>(values.size()) * sizeof(T);
 
-  std::shared_ptr<Buffer> buffer;
-  RETURN_NOT_OK(AllocateBuffer(pool, nbytes, &buffer));
+  ARROW_ASSIGN_OR_RAISE(auto buffer, AllocateBuffer(nbytes, pool));
   auto immutable_data = reinterpret_cast<const uint8_t*>(values.data());
   std::copy(immutable_data, immutable_data + nbytes, buffer->mutable_data());
   memset(buffer->mutable_data() + nbytes, 0,
          static_cast<size_t>(buffer->capacity() - nbytes));
 
-  *result = buffer;
+  *result = std::move(buffer);
   return Status::OK();
 }
 
diff --git a/cpp/src/arrow/util/bit_util.cc b/cpp/src/arrow/util/bit_util.cc
index 8fd2472..15bc561 100644
--- a/cpp/src/arrow/util/bit_util.cc
+++ b/cpp/src/arrow/util/bit_util.cc
@@ -61,12 +61,11 @@ Result<std::shared_ptr<Buffer>> BytesToBits(const std::vector<uint8_t>& bytes,
                                             MemoryPool* pool) {
   int64_t bit_length = BytesForBits(bytes.size());
 
-  std::shared_ptr<Buffer> buffer;
-  RETURN_NOT_OK(AllocateBuffer(pool, bit_length, &buffer));
+  ARROW_ASSIGN_OR_RAISE(auto buffer, AllocateBuffer(bit_length, pool));
   uint8_t* out_buf = buffer->mutable_data();
   memset(out_buf, 0, static_cast<size_t>(buffer->capacity()));
   FillBitsFromBytes(bytes, out_buf);
-  return buffer;
+  return std::move(buffer);
 }
 
 }  // namespace BitUtil
@@ -186,8 +185,7 @@ void TransferBitmap(const uint8_t* data, int64_t offset, int64_t length,
 template <bool invert_bits>
 Result<std::shared_ptr<Buffer>> TransferBitmap(MemoryPool* pool, const uint8_t* data,
                                                int64_t offset, int64_t length) {
-  std::shared_ptr<Buffer> buffer;
-  RETURN_NOT_OK(AllocateEmptyBitmap(pool, length, &buffer));
+  ARROW_ASSIGN_OR_RAISE(auto buffer, AllocateEmptyBitmap(length, pool));
   uint8_t* dest = buffer->mutable_data();
 
   TransferBitmap<invert_bits, false>(data, offset, length, 0, dest);
@@ -315,9 +313,8 @@ Result<std::shared_ptr<Buffer>> BitmapOp(MemoryPool* pool, const uint8_t* left,
                                          int64_t left_offset, const uint8_t* right,
                                          int64_t right_offset, int64_t length,
                                          int64_t out_offset) {
-  std::shared_ptr<Buffer> out_buffer;
   const int64_t phys_bits = length + out_offset;
-  RETURN_NOT_OK(AllocateEmptyBitmap(pool, phys_bits, &out_buffer));
+  ARROW_ASSIGN_OR_RAISE(auto out_buffer, AllocateEmptyBitmap(phys_bits, pool));
   BitmapOp<BitOp, LogicalOp>(left, left_offset, right, right_offset, length, out_offset,
                              out_buffer->mutable_data());
   return out_buffer;
@@ -404,13 +401,12 @@ Result<std::shared_ptr<Buffer>> BitmapAllButOne(MemoryPool* pool, int64_t length
     return Status::Invalid("invalid straggler_pos ", straggler_pos);
   }
 
-  std::shared_ptr<Buffer> buffer;
-  RETURN_NOT_OK(AllocateBuffer(pool, BitUtil::BytesForBits(length), &buffer));
+  ARROW_ASSIGN_OR_RAISE(auto buffer, AllocateBuffer(BitUtil::BytesForBits(length), pool));
 
   auto bitmap_data = buffer->mutable_data();
   BitUtil::SetBitsTo(bitmap_data, 0, length, value);
   BitUtil::SetBitTo(bitmap_data, straggler_pos, !value);
-  return buffer;
+  return std::move(buffer);
 }
 
 }  // namespace internal
diff --git a/cpp/src/arrow/util/bit_util_benchmark.cc b/cpp/src/arrow/util/bit_util_benchmark.cc
index 5483401..dc156fb 100644
--- a/cpp/src/arrow/util/bit_util_benchmark.cc
+++ b/cpp/src/arrow/util/bit_util_benchmark.cc
@@ -90,8 +90,7 @@ class NaiveBitmapWriter {
 #endif
 
 static std::shared_ptr<Buffer> CreateRandomBuffer(int64_t nbytes) {
-  std::shared_ptr<Buffer> buffer;
-  ABORT_NOT_OK(AllocateBuffer(nbytes, &buffer));
+  auto buffer = *AllocateBuffer(nbytes);
   memset(buffer->mutable_data(), 0, nbytes);
   random_bytes(nbytes, 0, buffer->mutable_data());
   return buffer;
@@ -335,9 +334,7 @@ static void CopyBitmap(benchmark::State& state) {  // NOLINT non-const reference
   const int64_t offset = Offset;
   const int64_t length = bits_size - offset;
 
-  std::shared_ptr<Buffer> copy;
-  auto pool = default_memory_pool();
-  ABORT_NOT_OK(AllocateEmptyBitmap(pool, length, &copy));
+  auto copy = *AllocateEmptyBitmap(length);
 
   for (auto _ : state) {
     internal::CopyBitmap(src, offset, length, copy->mutable_data(), 0, false);
diff --git a/cpp/src/arrow/util/bit_util_test.cc b/cpp/src/arrow/util/bit_util_test.cc
index a63844b..8b8bea9 100644
--- a/cpp/src/arrow/util/bit_util_test.cc
+++ b/cpp/src/arrow/util/bit_util_test.cc
@@ -63,7 +63,7 @@ void BitmapFromVector(const std::vector<int>& values, int64_t bit_offset,
                       std::shared_ptr<Buffer>* out_buffer, int64_t* out_length) {
   const int64_t length = values.size();
   *out_length = length;
-  ASSERT_OK(AllocateEmptyBitmap(length + bit_offset, out_buffer));
+  ASSERT_OK_AND_ASSIGN(*out_buffer, AllocateEmptyBitmap(length + bit_offset));
   auto writer = internal::BitmapWriter((*out_buffer)->mutable_data(), bit_offset, length);
   WriteVectorToWriter(writer, values);
 }
@@ -734,8 +734,7 @@ TEST(BitUtilTests, TestSetBitsTo) {
 TEST(BitUtilTests, TestCopyBitmap) {
   const int kBufferSize = 1000;
 
-  std::shared_ptr<Buffer> buffer;
-  ASSERT_OK(AllocateBuffer(kBufferSize, &buffer));
+  ASSERT_OK_AND_ASSIGN(auto buffer, AllocateBuffer(kBufferSize));
   memset(buffer->mutable_data(), 0, kBufferSize);
   random_bytes(kBufferSize, 0, buffer->mutable_data());
 
@@ -763,15 +762,13 @@ TEST(BitUtilTests, TestCopyBitmapPreAllocated) {
   std::vector<int64_t> lengths = {kBufferSize * 8 - 4, kBufferSize * 8};
   std::vector<int64_t> offsets = {0, 12, 16, 32, 37, 63, 64, 128};
 
-  std::shared_ptr<Buffer> buffer;
-  ASSERT_OK(AllocateBuffer(kBufferSize, &buffer));
+  ASSERT_OK_AND_ASSIGN(auto buffer, AllocateBuffer(kBufferSize));
   memset(buffer->mutable_data(), 0, kBufferSize);
   random_bytes(kBufferSize, 0, buffer->mutable_data());
   const uint8_t* src = buffer->data();
 
-  std::shared_ptr<Buffer> other_buffer;
   // Add 16 byte padding on both sides
-  ASSERT_OK(AllocateBuffer(kBufferSize + 32, &other_buffer));
+  ASSERT_OK_AND_ASSIGN(auto other_buffer, AllocateBuffer(kBufferSize + 32));
   memset(other_buffer->mutable_data(), 0, kBufferSize + 32);
   random_bytes(kBufferSize + 32, 0, other_buffer->mutable_data());
   const uint8_t* other = other_buffer->data();
@@ -781,8 +778,7 @@ TEST(BitUtilTests, TestCopyBitmapPreAllocated) {
       for (int64_t dest_offset : offsets) {
         const int64_t copy_length = num_bits - offset;
 
-        std::shared_ptr<Buffer> copy;
-        ASSERT_OK(AllocateBuffer(other_buffer->size(), &copy));
+        ASSERT_OK_AND_ASSIGN(auto copy, AllocateBuffer(other_buffer->size()));
         memcpy(copy->mutable_data(), other_buffer->data(), other_buffer->size());
         CopyBitmap(src, offset, copy_length, copy->mutable_data(), dest_offset);
 
@@ -806,15 +802,13 @@ TEST(BitUtilTests, TestCopyAndInvertBitmapPreAllocated) {
   std::vector<int64_t> lengths = {kBufferSize * 8 - 4, kBufferSize * 8};
   std::vector<int64_t> offsets = {0, 12, 16, 32, 37, 63, 64, 128};
 
-  std::shared_ptr<Buffer> buffer;
-  ASSERT_OK(AllocateBuffer(kBufferSize, &buffer));
+  ASSERT_OK_AND_ASSIGN(auto buffer, AllocateBuffer(kBufferSize));
   memset(buffer->mutable_data(), 0, kBufferSize);
   random_bytes(kBufferSize, 0, buffer->mutable_data());
   const uint8_t* src = buffer->data();
 
-  std::shared_ptr<Buffer> other_buffer;
   // Add 16 byte padding on both sides
-  ASSERT_OK(AllocateBuffer(kBufferSize + 32, &other_buffer));
+  ASSERT_OK_AND_ASSIGN(auto other_buffer, AllocateBuffer(kBufferSize + 32));
   memset(other_buffer->mutable_data(), 0, kBufferSize + 32);
   random_bytes(kBufferSize + 32, 0, other_buffer->mutable_data());
   const uint8_t* other = other_buffer->data();
@@ -824,8 +818,7 @@ TEST(BitUtilTests, TestCopyAndInvertBitmapPreAllocated) {
       for (int64_t dest_offset : offsets) {
         const int64_t copy_length = num_bits - offset;
 
-        std::shared_ptr<Buffer> copy;
-        ASSERT_OK(AllocateBuffer(other_buffer->size(), &copy));
+        ASSERT_OK_AND_ASSIGN(auto copy, AllocateBuffer(other_buffer->size()));
         memcpy(copy->mutable_data(), other_buffer->data(), other_buffer->size());
         InvertBitmap(src, offset, copy_length, copy->mutable_data(), dest_offset);
 
@@ -1206,7 +1199,7 @@ TEST(Bitmap, VisitWords) {
   constexpr int64_t nbytes = 1 << 10;
   std::shared_ptr<Buffer> buffer, actual_buffer;
   for (std::shared_ptr<Buffer>* b : {&buffer, &actual_buffer}) {
-    ASSERT_OK(AllocateBuffer(nbytes, b));
+    ASSERT_OK_AND_ASSIGN(*b, AllocateBuffer(nbytes));
     memset((*b)->mutable_data(), 0, nbytes);
   }
   random_bytes(nbytes, 0, buffer->mutable_data());
@@ -1235,8 +1228,7 @@ TEST(Bitmap, VisitPartialWords) {
 
   auto buffer = Buffer::Wrap(words, 2);
   Bitmap bitmap(buffer, 0, nbits);
-  std::shared_ptr<Buffer> storage;
-  ASSERT_OK(AllocateBuffer(nbytes, &storage));
+  ASSERT_OK_AND_ASSIGN(std::shared_ptr<Buffer> storage, AllocateBuffer(nbytes));
 
   // words partially outside the buffer are not accessible, but they are loaded bitwise
   auto first_byte_was_missing = Bitmap(SliceBuffer(buffer, 1), 0, nbits - 8);
@@ -1253,7 +1245,7 @@ TEST(Bitmap, VisitWordsAnd) {
   constexpr int64_t nbytes = 1 << 10;
   std::shared_ptr<Buffer> buffer, actual_buffer, expected_buffer;
   for (std::shared_ptr<Buffer>* b : {&buffer, &actual_buffer, &expected_buffer}) {
-    ASSERT_OK(AllocateBuffer(nbytes, b));
+    ASSERT_OK_AND_ASSIGN(*b, AllocateBuffer(nbytes));
     memset((*b)->mutable_data(), 0, nbytes);
   }
   random_bytes(nbytes, 0, buffer->mutable_data());
diff --git a/cpp/src/arrow/util/io_util.cc b/cpp/src/arrow/util/io_util.cc
index 2210e6b..c7efa15 100644
--- a/cpp/src/arrow/util/io_util.cc
+++ b/cpp/src/arrow/util/io_util.cc
@@ -160,12 +160,11 @@ Result<int64_t> StdinStream::Read(int64_t nbytes, void* out) {
 }
 
 Result<std::shared_ptr<Buffer>> StdinStream::Read(int64_t nbytes) {
-  std::shared_ptr<ResizableBuffer> buffer;
-  ARROW_RETURN_NOT_OK(AllocateResizableBuffer(nbytes, &buffer));
+  ARROW_ASSIGN_OR_RAISE(auto buffer, AllocateResizableBuffer(nbytes));
   ARROW_ASSIGN_OR_RAISE(int64_t bytes_read, Read(nbytes, buffer->mutable_data()));
   ARROW_RETURN_NOT_OK(buffer->Resize(bytes_read, false));
   buffer->ZeroPadding();
-  return buffer;
+  return std::move(buffer);
 }
 
 }  // namespace io
diff --git a/cpp/src/gandiva/annotator_test.cc b/cpp/src/gandiva/annotator_test.cc
index cd829f7..e537943 100644
--- a/cpp/src/gandiva/annotator_test.cc
+++ b/cpp/src/gandiva/annotator_test.cc
@@ -18,6 +18,7 @@
 #include "gandiva/annotator.h"
 
 #include <memory>
+#include <utility>
 
 #include <arrow/memory_pool.h>
 #include <gtest/gtest.h>
@@ -33,16 +34,12 @@ class TestAnnotator : public ::testing::Test {
 ArrayPtr TestAnnotator::MakeInt32Array(int length) {
   arrow::Status status;
 
-  std::shared_ptr<arrow::Buffer> validity;
-  status =
-      arrow::AllocateBuffer(arrow::default_memory_pool(), (length + 63) / 8, &validity);
-  DCHECK_EQ(status.ok(), true);
+  auto validity = *arrow::AllocateBuffer((length + 63) / 8);
 
-  std::shared_ptr<arrow::Buffer> value;
-  status = AllocateBuffer(arrow::default_memory_pool(), length * sizeof(int32_t), &value);
-  DCHECK_EQ(status.ok(), true);
+  auto values = *arrow::AllocateBuffer(length * sizeof(int32_t));
 
-  auto array_data = arrow::ArrayData::Make(arrow::int32(), length, {validity, value});
+  auto array_data = arrow::ArrayData::Make(arrow::int32(), length,
+                                           {std::move(validity), std::move(values)});
   return arrow::MakeArray(array_data);
 }
 
diff --git a/cpp/src/gandiva/projector.cc b/cpp/src/gandiva/projector.cc
index 295a8ed..52be493 100644
--- a/cpp/src/gandiva/projector.cc
+++ b/cpp/src/gandiva/projector.cc
@@ -175,24 +175,21 @@ Status Projector::AllocArrayData(const DataTypePtr& type, int64_t num_records,
   std::vector<std::shared_ptr<arrow::Buffer>> buffers;
 
   // The output vector always has a null bitmap.
-  std::shared_ptr<arrow::Buffer> bitmap_buffer;
   int64_t size = arrow::BitUtil::BytesForBits(num_records);
-  ARROW_RETURN_NOT_OK(arrow::AllocateBuffer(pool, size, &bitmap_buffer));
-  buffers.push_back(bitmap_buffer);
+  ARROW_ASSIGN_OR_RAISE(auto bitmap_buffer, arrow::AllocateBuffer(size, pool));
+  buffers.push_back(std::move(bitmap_buffer));
 
   // String/Binary vectors have an offsets array.
   auto type_id = type->id();
   if (arrow::is_binary_like(type_id)) {
-    std::shared_ptr<arrow::Buffer> offsets_buffer;
     auto offsets_len = arrow::BitUtil::BytesForBits((num_records + 1) * 32);
 
-    ARROW_RETURN_NOT_OK(arrow::AllocateBuffer(pool, offsets_len, &offsets_buffer));
-    buffers.push_back(offsets_buffer);
+    ARROW_ASSIGN_OR_RAISE(auto offsets_buffer, arrow::AllocateBuffer(offsets_len, pool));
+    buffers.push_back(std::move(offsets_buffer));
   }
 
   // The output vector always has a data array.
   int64_t data_len;
-  std::shared_ptr<arrow::ResizableBuffer> data_buffer;
   if (arrow::is_primitive(type_id) || type_id == arrow::Type::DECIMAL) {
     const auto& fw_type = dynamic_cast<const arrow::FixedWidthType&>(*type);
     data_len = arrow::BitUtil::BytesForBits(num_records * fw_type.bit_width());
@@ -202,16 +199,16 @@ Status Projector::AllocArrayData(const DataTypePtr& type, int64_t num_records,
   } else {
     return Status::Invalid("Unsupported output data type " + type->ToString());
   }
-  ARROW_RETURN_NOT_OK(arrow::AllocateResizableBuffer(pool, data_len, &data_buffer));
+  ARROW_ASSIGN_OR_RAISE(auto data_buffer, arrow::AllocateResizableBuffer(data_len, pool));
 
   // This is not strictly required but valgrind gets confused and detects this
   // as uninitialized memory access. See arrow::util::SetBitTo().
   if (type->id() == arrow::Type::BOOL) {
     memset(data_buffer->mutable_data(), 0, data_len);
   }
-  buffers.push_back(data_buffer);
+  buffers.push_back(std::move(data_buffer));
 
-  *array_data = arrow::ArrayData::Make(type, num_records, buffers);
+  *array_data = arrow::ArrayData::Make(type, num_records, std::move(buffers));
   return Status::OK();
 }
 
diff --git a/cpp/src/gandiva/selection_vector.cc b/cpp/src/gandiva/selection_vector.cc
index f62a659..230d603 100644
--- a/cpp/src/gandiva/selection_vector.cc
+++ b/cpp/src/gandiva/selection_vector.cc
@@ -157,7 +157,7 @@ template <typename C_TYPE, typename A_TYPE, SelectionVector::Mode mode>
 Status SelectionVectorImpl<C_TYPE, A_TYPE, mode>::AllocateBuffer(
     int64_t max_slots, arrow::MemoryPool* pool, std::shared_ptr<arrow::Buffer>* buffer) {
   auto buffer_len = max_slots * sizeof(C_TYPE);
-  ARROW_RETURN_NOT_OK(arrow::AllocateBuffer(pool, buffer_len, buffer));
+  ARROW_ASSIGN_OR_RAISE(*buffer, arrow::AllocateBuffer(buffer_len, pool));
 
   return Status::OK();
 }
diff --git a/cpp/src/gandiva/selection_vector_test.cc b/cpp/src/gandiva/selection_vector_test.cc
index 6738927..6868929 100644
--- a/cpp/src/gandiva/selection_vector_test.cc
+++ b/cpp/src/gandiva/selection_vector_test.cc
@@ -18,10 +18,13 @@
 #include "gandiva/selection_vector.h"
 
 #include <memory>
+#include <utility>
 #include <vector>
 
 #include <gtest/gtest.h>
 
+#include "arrow/testing/gtest_util.h"
+
 namespace gandiva {
 
 class TestSelectionVector : public ::testing::Test {
@@ -45,12 +48,10 @@ TEST_F(TestSelectionVector, TestInt16Make) {
 
   // Test with pre-alloced buffer
   std::shared_ptr<SelectionVector> selection2;
-  std::shared_ptr<arrow::Buffer> buffer;
   auto buffer_len = max_slots * sizeof(int16_t);
-  auto astatus = arrow::AllocateBuffer(pool_, buffer_len, &buffer);
-  EXPECT_EQ(astatus.ok(), true);
+  ASSERT_OK_AND_ASSIGN(auto buffer, arrow::AllocateBuffer(buffer_len, pool_));
 
-  status = SelectionVector::MakeInt16(max_slots, buffer, &selection2);
+  status = SelectionVector::MakeInt16(max_slots, std::move(buffer), &selection2);
   EXPECT_EQ(status.ok(), true) << status.message();
   EXPECT_EQ(selection2->GetMaxSlots(), max_slots);
   EXPECT_EQ(selection2->GetNumSlots(), 0);
@@ -60,14 +61,12 @@ TEST_F(TestSelectionVector, TestInt16MakeNegative) {
   int max_slots = 10;
 
   std::shared_ptr<SelectionVector> selection;
-  std::shared_ptr<arrow::Buffer> buffer;
   auto buffer_len = max_slots * sizeof(int16_t);
 
   // alloc a buffer that's insufficient.
-  auto astatus = arrow::AllocateBuffer(pool_, buffer_len - 16, &buffer);
-  EXPECT_EQ(astatus.ok(), true);
+  ASSERT_OK_AND_ASSIGN(auto buffer, arrow::AllocateBuffer(buffer_len - 16, pool_));
 
-  auto status = SelectionVector::MakeInt16(max_slots, buffer, &selection);
+  auto status = SelectionVector::MakeInt16(max_slots, std::move(buffer), &selection);
   EXPECT_EQ(status.IsInvalid(), true);
 }
 
@@ -194,14 +193,12 @@ TEST_F(TestSelectionVector, TestInt32MakeNegative) {
   int max_slots = 10;
 
   std::shared_ptr<SelectionVector> selection;
-  std::shared_ptr<arrow::Buffer> buffer;
   auto buffer_len = max_slots * sizeof(int32_t);
 
   // alloc a buffer that's insufficient.
-  auto astatus = arrow::AllocateBuffer(pool_, buffer_len - 1, &buffer);
-  EXPECT_EQ(astatus.ok(), true);
+  ASSERT_OK_AND_ASSIGN(auto buffer, arrow::AllocateBuffer(buffer_len - 1, pool_));
 
-  auto status = SelectionVector::MakeInt32(max_slots, buffer, &selection);
+  auto status = SelectionVector::MakeInt32(max_slots, std::move(buffer), &selection);
   EXPECT_EQ(status.IsInvalid(), true);
 }
 
@@ -261,14 +258,12 @@ TEST_F(TestSelectionVector, TestInt64MakeNegative) {
   int max_slots = 10;
 
   std::shared_ptr<SelectionVector> selection;
-  std::shared_ptr<arrow::Buffer> buffer;
   auto buffer_len = max_slots * sizeof(int64_t);
 
   // alloc a buffer that's insufficient.
-  auto astatus = arrow::AllocateBuffer(pool_, buffer_len - 1, &buffer);
-  EXPECT_EQ(astatus.ok(), true);
+  ASSERT_OK_AND_ASSIGN(auto buffer, arrow::AllocateBuffer(buffer_len - 1, pool_));
 
-  auto status = SelectionVector::MakeInt64(max_slots, buffer, &selection);
+  auto status = SelectionVector::MakeInt64(max_slots, std::move(buffer), &selection);
   EXPECT_EQ(status.IsInvalid(), true);
 }
 
diff --git a/cpp/src/parquet/arrow/arrow_reader_writer_test.cc b/cpp/src/parquet/arrow/arrow_reader_writer_test.cc
index 0d8cd2d..c62ff86 100644
--- a/cpp/src/parquet/arrow/arrow_reader_writer_test.cc
+++ b/cpp/src/parquet/arrow/arrow_reader_writer_test.cc
@@ -1179,9 +1179,7 @@ TEST_F(TestNullParquetIO, NullListColumn) {
 }
 
 TEST_F(TestNullParquetIO, NullDictionaryColumn) {
-  std::shared_ptr<Buffer> null_bitmap;
-  ASSERT_OK(::arrow::AllocateEmptyBitmap(::arrow::default_memory_pool(), SMALL_SIZE,
-                                         &null_bitmap));
+  ASSERT_OK_AND_ASSIGN(auto null_bitmap, ::arrow::AllocateEmptyBitmap(SMALL_SIZE));
 
   ASSERT_OK_AND_ASSIGN(auto indices, MakeArrayOfNull(::arrow::int8(), SMALL_SIZE));
   std::shared_ptr<::arrow::DictionaryType> dict_type =
diff --git a/cpp/src/parquet/arrow/reader.cc b/cpp/src/parquet/arrow/reader.cc
index abf1ebb..0cc547d 100644
--- a/cpp/src/parquet/arrow/reader.cc
+++ b/cpp/src/parquet/arrow/reader.cc
@@ -551,12 +551,12 @@ class PARQUET_NO_EXPORT StructReader : public ColumnReaderImpl {
 
 Status StructReader::DefLevelsToNullArray(std::shared_ptr<Buffer>* null_bitmap_out,
                                           int64_t* null_count_out) {
-  std::shared_ptr<Buffer> null_bitmap;
   auto null_count = 0;
   const int16_t* def_levels_data;
   int64_t def_levels_length;
   RETURN_NOT_OK(GetDefLevels(&def_levels_data, &def_levels_length));
-  RETURN_NOT_OK(AllocateEmptyBitmap(ctx_->pool, def_levels_length, &null_bitmap));
+  ARROW_ASSIGN_OR_RAISE(auto null_bitmap,
+                        AllocateEmptyBitmap(def_levels_length, ctx_->pool));
   uint8_t* null_bitmap_ptr = null_bitmap->mutable_data();
   for (int64_t i = 0; i < def_levels_length; i++) {
     if (def_levels_data[i] < struct_def_level_) {
@@ -597,7 +597,7 @@ Status StructReader::GetDefLevels(const int16_t** data, int64_t* length) {
     }
     RETURN_NOT_OK(children_[child_index]->GetDefLevels(&child_def_levels, &child_length));
     auto size = child_length * sizeof(int16_t);
-    RETURN_NOT_OK(AllocateResizableBuffer(ctx_->pool, size, &def_levels_buffer_));
+    ARROW_ASSIGN_OR_RAISE(def_levels_buffer_, AllocateResizableBuffer(size, ctx_->pool));
     // Initialize with the minimal def level
     std::memset(def_levels_buffer_->mutable_data(), -1, size);
     result_levels = reinterpret_cast<int16_t*>(def_levels_buffer_->mutable_data());
diff --git a/cpp/src/parquet/arrow/reader_internal.cc b/cpp/src/parquet/arrow/reader_internal.cc
index 2986e60..18cee17 100644
--- a/cpp/src/parquet/arrow/reader_internal.cc
+++ b/cpp/src/parquet/arrow/reader_internal.cc
@@ -798,14 +798,14 @@ Status TransferInt(RecordReader* reader, MemoryPool* pool,
   using ArrowCType = typename ArrowType::c_type;
   using ParquetCType = typename ParquetType::c_type;
   int64_t length = reader->values_written();
-  std::shared_ptr<Buffer> data;
-  RETURN_NOT_OK(::arrow::AllocateBuffer(pool, length * sizeof(ArrowCType), &data));
+  ARROW_ASSIGN_OR_RAISE(auto data,
+                        ::arrow::AllocateBuffer(length * sizeof(ArrowCType), pool));
 
   auto values = reinterpret_cast<const ParquetCType*>(reader->values());
   auto out_ptr = reinterpret_cast<ArrowCType*>(data->mutable_data());
   std::copy(values, values + length, out_ptr);
   *out = std::make_shared<ArrayType<ArrowType>>(
-      type, length, data, reader->ReleaseIsValid(), reader->null_count());
+      type, length, std::move(data), reader->ReleaseIsValid(), reader->null_count());
   return Status::OK();
 }
 
@@ -820,10 +820,9 @@ std::shared_ptr<Array> TransferZeroCopy(RecordReader* reader,
 
 Status TransferBool(RecordReader* reader, MemoryPool* pool, Datum* out) {
   int64_t length = reader->values_written();
-  std::shared_ptr<Buffer> data;
 
   const int64_t buffer_size = BitUtil::BytesForBits(length);
-  RETURN_NOT_OK(::arrow::AllocateBuffer(pool, buffer_size, &data));
+  ARROW_ASSIGN_OR_RAISE(auto data, ::arrow::AllocateBuffer(buffer_size, pool));
 
   // Transfer boolean values to packed bitmap
   auto values = reinterpret_cast<const bool*>(reader->values());
@@ -836,7 +835,7 @@ Status TransferBool(RecordReader* reader, MemoryPool* pool, Datum* out) {
     }
   }
 
-  *out = std::make_shared<BooleanArray>(length, data, reader->ReleaseIsValid(),
+  *out = std::make_shared<BooleanArray>(length, std::move(data), reader->ReleaseIsValid(),
                                         reader->null_count());
   return Status::OK();
 }
@@ -845,8 +844,8 @@ Status TransferInt96(RecordReader* reader, MemoryPool* pool,
                      const std::shared_ptr<DataType>& type, Datum* out) {
   int64_t length = reader->values_written();
   auto values = reinterpret_cast<const Int96*>(reader->values());
-  std::shared_ptr<Buffer> data;
-  RETURN_NOT_OK(::arrow::AllocateBuffer(pool, length * sizeof(int64_t), &data));
+  ARROW_ASSIGN_OR_RAISE(auto data,
+                        ::arrow::AllocateBuffer(length * sizeof(int64_t), pool));
   auto data_ptr = reinterpret_cast<int64_t*>(data->mutable_data());
   for (int64_t i = 0; i < length; i++) {
     if (values[i].value[2] == 0) {
@@ -857,8 +856,8 @@ Status TransferInt96(RecordReader* reader, MemoryPool* pool,
       *data_ptr++ = Int96GetNanoSeconds(values[i]);
     }
   }
-  *out = std::make_shared<TimestampArray>(type, length, data, reader->ReleaseIsValid(),
-                                          reader->null_count());
+  *out = std::make_shared<TimestampArray>(type, length, std::move(data),
+                                          reader->ReleaseIsValid(), reader->null_count());
   return Status::OK();
 }
 
@@ -867,8 +866,8 @@ Status TransferDate64(RecordReader* reader, MemoryPool* pool,
   int64_t length = reader->values_written();
   auto values = reinterpret_cast<const int32_t*>(reader->values());
 
-  std::shared_ptr<Buffer> data;
-  RETURN_NOT_OK(::arrow::AllocateBuffer(pool, length * sizeof(int64_t), &data));
+  ARROW_ASSIGN_OR_RAISE(auto data,
+                        ::arrow::AllocateBuffer(length * sizeof(int64_t), pool));
   auto out_ptr = reinterpret_cast<int64_t*>(data->mutable_data());
 
   for (int64_t i = 0; i < length; i++) {
@@ -876,7 +875,7 @@ Status TransferDate64(RecordReader* reader, MemoryPool* pool,
   }
 
   *out = std::make_shared<::arrow::Date64Array>(
-      type, length, data, reader->ReleaseIsValid(), reader->null_count());
+      type, length, std::move(data), reader->ReleaseIsValid(), reader->null_count());
   return Status::OK();
 }
 
@@ -1066,8 +1065,7 @@ Status ConvertToDecimal128<FLBAType>(const Array& array,
   }
 
   // allocate memory for the decimal array
-  std::shared_ptr<Buffer> data;
-  RETURN_NOT_OK(::arrow::AllocateBuffer(pool, length * type_length, &data));
+  ARROW_ASSIGN_OR_RAISE(auto data, ::arrow::AllocateBuffer(length * type_length, pool));
 
   // raw bytes that we can write to
   uint8_t* out_ptr = data->mutable_data();
@@ -1087,7 +1085,7 @@ Status ConvertToDecimal128<FLBAType>(const Array& array,
   }
 
   *out = std::make_shared<::arrow::Decimal128Array>(
-      type, length, data, fixed_size_binary_array.null_bitmap(), null_count);
+      type, length, std::move(data), fixed_size_binary_array.null_bitmap(), null_count);
 
   return Status::OK();
 }
@@ -1102,8 +1100,7 @@ Status ConvertToDecimal128<ByteArrayType>(const Array& array,
   const auto& decimal_type = static_cast<const ::arrow::Decimal128Type&>(*type);
   const int64_t type_length = decimal_type.byte_width();
 
-  std::shared_ptr<Buffer> data;
-  RETURN_NOT_OK(::arrow::AllocateBuffer(pool, length * type_length, &data));
+  ARROW_ASSIGN_OR_RAISE(auto data, ::arrow::AllocateBuffer(length * type_length, pool));
 
   // raw bytes that we can write to
   uint8_t* out_ptr = data->mutable_data();
@@ -1134,7 +1131,7 @@ Status ConvertToDecimal128<ByteArrayType>(const Array& array,
   }
 
   *out = std::make_shared<::arrow::Decimal128Array>(
-      type, length, data, binary_array.null_bitmap(), null_count);
+      type, length, std::move(data), binary_array.null_bitmap(), null_count);
   return Status::OK();
 }
 
@@ -1162,8 +1159,7 @@ static Status DecimalIntegerTransfer(RecordReader* reader, MemoryPool* pool,
   const auto& decimal_type = static_cast<const ::arrow::Decimal128Type&>(*type);
   const int64_t type_length = decimal_type.byte_width();
 
-  std::shared_ptr<Buffer> data;
-  RETURN_NOT_OK(::arrow::AllocateBuffer(pool, length * type_length, &data));
+  ARROW_ASSIGN_OR_RAISE(auto data, ::arrow::AllocateBuffer(length * type_length, pool));
   uint8_t* out_ptr = data->mutable_data();
 
   using ::arrow::BitUtil::FromLittleEndian;
@@ -1183,10 +1179,10 @@ static Status DecimalIntegerTransfer(RecordReader* reader, MemoryPool* pool,
 
   if (reader->nullable_values()) {
     std::shared_ptr<ResizableBuffer> is_valid = reader->ReleaseIsValid();
-    *out = std::make_shared<::arrow::Decimal128Array>(type, length, data, is_valid,
-                                                      reader->null_count());
+    *out = std::make_shared<::arrow::Decimal128Array>(type, length, std::move(data),
+                                                      is_valid, reader->null_count());
   } else {
-    *out = std::make_shared<::arrow::Decimal128Array>(type, length, data);
+    *out = std::make_shared<::arrow::Decimal128Array>(type, length, std::move(data));
   }
   return Status::OK();
 }
diff --git a/cpp/src/parquet/arrow/test_util.h b/cpp/src/parquet/arrow/test_util.h
index 03bce5c..04517c7 100644
--- a/cpp/src/parquet/arrow/test_util.h
+++ b/cpp/src/parquet/arrow/test_util.h
@@ -149,9 +149,7 @@ NonNullArray(size_t size, std::shared_ptr<Array>* out) {
 
   constexpr int32_t seed = 0;
 
-  std::shared_ptr<Buffer> out_buf;
-  RETURN_NOT_OK(::arrow::AllocateBuffer(::arrow::default_memory_pool(), size * byte_width,
-                                        &out_buf));
+  ARROW_ASSIGN_OR_RAISE(auto out_buf, ::arrow::AllocateBuffer(size * byte_width));
   random_decimals(size, seed, kDecimalPrecision, out_buf->mutable_data());
 
   RETURN_NOT_OK(builder.AppendValues(out_buf->data(), size));
@@ -306,9 +304,7 @@ NullableArray(size_t size, size_t num_nulls, uint32_t seed,
   const int32_t byte_width =
       static_cast<const ::arrow::Decimal128Type&>(*type).byte_width();
 
-  std::shared_ptr<::arrow::Buffer> out_buf;
-  RETURN_NOT_OK(::arrow::AllocateBuffer(::arrow::default_memory_pool(), size * byte_width,
-                                        &out_buf));
+  ARROW_ASSIGN_OR_RAISE(auto out_buf, ::arrow::AllocateBuffer(size * byte_width));
 
   random_decimals(size, seed, precision, out_buf->mutable_data());
 
@@ -382,10 +378,8 @@ Status MakeListArray(const std::shared_ptr<Array>& values, int64_t size,
 // Make an array containing only empty lists, with a null values array
 Status MakeEmptyListsArray(int64_t size, std::shared_ptr<Array>* out_array) {
   // Allocate an offsets buffer containing only zeroes
-  std::shared_ptr<Buffer> offsets_buffer;
   const int64_t offsets_nbytes = (size + 1) * sizeof(int32_t);
-  RETURN_NOT_OK(::arrow::AllocateBuffer(::arrow::default_memory_pool(), offsets_nbytes,
-                                        &offsets_buffer));
+  ARROW_ASSIGN_OR_RAISE(auto offsets_buffer, ::arrow::AllocateBuffer(offsets_nbytes));
   memset(offsets_buffer->mutable_data(), 0, offsets_nbytes);
 
   auto value_field =
@@ -397,7 +391,8 @@ Status MakeEmptyListsArray(int64_t size, std::shared_ptr<Array>* out_array) {
   auto child_data =
       ::arrow::ArrayData::Make(value_field->type(), 0, std::move(child_buffers));
 
-  std::vector<std::shared_ptr<Buffer>> buffers = {nullptr /* bitmap */, offsets_buffer};
+  std::vector<std::shared_ptr<Buffer>> buffers = {nullptr /* bitmap */,
+                                                  std::move(offsets_buffer)};
   auto array_data = ::arrow::ArrayData::Make(list_type, size, std::move(buffers));
   array_data->child_data.push_back(child_data);
 
diff --git a/cpp/src/parquet/bloom_filter.cc b/cpp/src/parquet/bloom_filter.cc
index 21d046f..9d5abea 100644
--- a/cpp/src/parquet/bloom_filter.cc
+++ b/cpp/src/parquet/bloom_filter.cc
@@ -46,7 +46,7 @@ void BlockSplitBloomFilter::Init(uint32_t num_bytes) {
   }
 
   num_bytes_ = num_bytes;
-  PARQUET_THROW_NOT_OK(::arrow::AllocateBuffer(pool_, num_bytes_, &data_));
+  PARQUET_ASSIGN_OR_THROW(data_, ::arrow::AllocateBuffer(num_bytes_, pool_));
   memset(data_->mutable_data(), 0, num_bytes_);
 
   this->hasher_.reset(new MurmurHash3());
@@ -61,7 +61,7 @@ void BlockSplitBloomFilter::Init(const uint8_t* bitset, uint32_t num_bytes) {
   }
 
   num_bytes_ = num_bytes;
-  PARQUET_THROW_NOT_OK(::arrow::AllocateBuffer(pool_, num_bytes_, &data_));
+  PARQUET_ASSIGN_OR_THROW(data_, ::arrow::AllocateBuffer(num_bytes_, pool_));
   memcpy(data_->mutable_data(), bitset, num_bytes_);
 
   this->hasher_.reset(new MurmurHash3());
diff --git a/cpp/src/parquet/column_writer.cc b/cpp/src/parquet/column_writer.cc
index 8fde4eb..dbb7df2 100644
--- a/cpp/src/parquet/column_writer.cc
+++ b/cpp/src/parquet/column_writer.cc
@@ -786,9 +786,9 @@ void ColumnWriterImpl::BuildDataPageV1(int64_t definition_levels_rle_size,
   // Write the page to OutputStream eagerly if there is no dictionary or
   // if dictionary encoding has fallen back to PLAIN
   if (has_dictionary_ && !fallback_) {  // Save pages until end of dictionary encoding
-    std::shared_ptr<Buffer> compressed_data_copy;
-    PARQUET_THROW_NOT_OK(compressed_data->Copy(0, compressed_data->size(), allocator_,
-                                               &compressed_data_copy));
+    PARQUET_ASSIGN_OR_THROW(
+        auto compressed_data_copy,
+        compressed_data->CopySlice(0, compressed_data->size(), allocator_));
     std::unique_ptr<DataPage> page_ptr(new DataPageV1(
         compressed_data_copy, static_cast<int32_t>(num_buffered_values_), encoding_,
         Encoding::RLE, Encoding::RLE, uncompressed_size, page_stats));
@@ -838,8 +838,8 @@ void ColumnWriterImpl::BuildDataPageV2(int64_t definition_levels_rle_size,
   // Write the page to OutputStream eagerly if there is no dictionary or
   // if dictionary encoding has fallen back to PLAIN
   if (has_dictionary_ && !fallback_) {  // Save pages until end of dictionary encoding
-    std::shared_ptr<Buffer> data_copy;
-    PARQUET_THROW_NOT_OK(combined->Copy(0, combined->size(), allocator_, &data_copy));
+    PARQUET_ASSIGN_OR_THROW(auto data_copy,
+                            combined->CopySlice(0, combined->size(), allocator_));
     std::unique_ptr<DataPage> page_ptr(new DataPageV2(
         combined, num_values, null_count, num_values, encoding_, def_levels_byte_length,
         rep_levels_byte_length, uncompressed_size, pager_->has_compressor()));
diff --git a/cpp/src/parquet/column_writer_test.cc b/cpp/src/parquet/column_writer_test.cc
index 68cb78d..a8648b5 100644
--- a/cpp/src/parquet/column_writer_test.cc
+++ b/cpp/src/parquet/column_writer_test.cc
@@ -747,8 +747,7 @@ TEST(TestColumnWriter, RepeatedListsUpdateSpacedBug) {
   std::vector<int32_t> values = {0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12};
 
   // Write the values into uninitialized memory
-  std::shared_ptr<Buffer> values_buffer;
-  ASSERT_OK(::arrow::AllocateBuffer(64, &values_buffer));
+  ASSERT_OK_AND_ASSIGN(auto values_buffer, ::arrow::AllocateBuffer(64));
   memcpy(values_buffer->mutable_data(), values.data(), 13 * sizeof(int32_t));
   auto values_data = reinterpret_cast<const int32_t*>(values_buffer->data());
 
diff --git a/cpp/src/parquet/encoding.cc b/cpp/src/parquet/encoding.cc
index b123c04..1988c36 100644
--- a/cpp/src/parquet/encoding.cc
+++ b/cpp/src/parquet/encoding.cc
@@ -101,9 +101,8 @@ class PlainEncoder : public EncoderImpl, virtual public TypedEncoder<DType> {
 
   void PutSpaced(const T* src, int num_values, const uint8_t* valid_bits,
                  int64_t valid_bits_offset) override {
-    std::shared_ptr<ResizableBuffer> buffer;
-    PARQUET_THROW_NOT_OK(arrow::AllocateResizableBuffer(this->memory_pool(),
-                                                        num_values * sizeof(T), &buffer));
+    PARQUET_ASSIGN_OR_THROW(
+        auto buffer, arrow::AllocateBuffer(num_values * sizeof(T), this->memory_pool()));
     int32_t num_valid_values = 0;
     arrow::internal::BitmapReader valid_bits_reader(valid_bits, valid_bits_offset,
                                                     num_values);
@@ -302,9 +301,8 @@ class PlainEncoder<BooleanType> : public EncoderImpl, virtual public BooleanEnco
 
   void PutSpaced(const bool* src, int num_values, const uint8_t* valid_bits,
                  int64_t valid_bits_offset) override {
-    std::shared_ptr<ResizableBuffer> buffer;
-    PARQUET_THROW_NOT_OK(arrow::AllocateResizableBuffer(this->memory_pool(),
-                                                        num_values * sizeof(T), &buffer));
+    PARQUET_ASSIGN_OR_THROW(
+        auto buffer, arrow::AllocateBuffer(num_values * sizeof(T), this->memory_pool()));
     int32_t num_valid_values = 0;
     arrow::internal::BitmapReader valid_bits_reader(valid_bits, valid_bits_offset,
                                                     num_values);
@@ -900,9 +898,8 @@ template <typename DType>
 void ByteStreamSplitEncoder<DType>::PutSpaced(const T* src, int num_values,
                                               const uint8_t* valid_bits,
                                               int64_t valid_bits_offset) {
-  std::shared_ptr<ResizableBuffer> buffer;
-  PARQUET_THROW_NOT_OK(arrow::AllocateResizableBuffer(this->memory_pool(),
-                                                      num_values * sizeof(T), &buffer));
+  PARQUET_ASSIGN_OR_THROW(
+      auto buffer, arrow::AllocateBuffer(num_values * sizeof(T), this->memory_pool()));
   int32_t num_valid_values = 0;
   arrow::internal::BitmapReader valid_bits_reader(valid_bits, valid_bits_offset,
                                                   num_values);
@@ -2315,7 +2312,7 @@ class ByteStreamSplitDecoder : public DecoderImpl, virtual public TypedDecoder<D
   T* EnsureDecodeBuffer(int64_t min_values) {
     const int64_t size = sizeof(T) * min_values;
     if (!decode_buffer_ || decode_buffer_->size() < size) {
-      PARQUET_THROW_NOT_OK(AllocateBuffer(size, &decode_buffer_));
+      PARQUET_ASSIGN_OR_THROW(decode_buffer_, ::arrow::AllocateBuffer(size));
     }
     return reinterpret_cast<T*>(decode_buffer_->mutable_data());
   }
diff --git a/cpp/src/parquet/platform.cc b/cpp/src/parquet/platform.cc
index a63f4d9..f1eabe0 100644
--- a/cpp/src/parquet/platform.cc
+++ b/cpp/src/parquet/platform.cc
@@ -19,6 +19,7 @@
 
 #include <cstdint>
 #include <memory>
+#include <utility>
 
 #include "arrow/io/memory.h"
 
@@ -33,9 +34,8 @@ std::shared_ptr<::arrow::io::BufferOutputStream> CreateOutputStream(MemoryPool*
 }
 
 std::shared_ptr<ResizableBuffer> AllocateBuffer(MemoryPool* pool, int64_t size) {
-  std::shared_ptr<ResizableBuffer> result;
-  PARQUET_THROW_NOT_OK(arrow::AllocateResizableBuffer(pool, size, &result));
-  return result;
+  PARQUET_ASSIGN_OR_THROW(auto result, arrow::AllocateResizableBuffer(size, pool));
+  return std::move(result);
 }
 
 }  // namespace parquet
diff --git a/python/pyarrow/includes/common.pxd b/python/pyarrow/includes/common.pxd
index 7825168..d8ee866 100644
--- a/python/pyarrow/includes/common.pxd
+++ b/python/pyarrow/includes/common.pxd
@@ -52,6 +52,26 @@ cdef extern from * namespace "cymove" nogil:
     """
     cdef T move" cymove::cymove"[T](T)
 
+cdef extern from * namespace "arrow::py" nogil:
+    """
+    #include <memory>
+    #include <utility>
+
+    namespace arrow {
+    namespace py {
+    template <typename T>
+    std::shared_ptr<T> to_shared(std::unique_ptr<T>& t) {
+        return std::move(t);
+    }
+    template <typename T>
+    std::shared_ptr<T> to_shared(std::unique_ptr<T>&& t) {
+        return std::move(t);
+    }
+    }  // namespace py
+    }  // namespace arrow
+    """
+    cdef shared_ptr[T] to_shared" arrow::py::to_shared"[T](unique_ptr[T])
+
 cdef extern from "arrow/python/platform.h":
     pass
 
diff --git a/python/pyarrow/includes/libarrow.pxd b/python/pyarrow/includes/libarrow.pxd
index ba356ce..572fd41 100644
--- a/python/pyarrow/includes/libarrow.pxd
+++ b/python/pyarrow/includes/libarrow.pxd
@@ -278,11 +278,11 @@ cdef extern from "arrow/api.h" namespace "arrow" nogil:
         CStatus Resize(const int64_t new_size, c_bool shrink_to_fit)
         CStatus Reserve(const int64_t new_size)
 
-    CStatus AllocateBuffer(CMemoryPool* pool, const int64_t size,
-                           shared_ptr[CBuffer]* out)
+    CResult[unique_ptr[CBuffer]] AllocateBuffer(const int64_t size,
+                                                CMemoryPool* pool)
 
-    CStatus AllocateResizableBuffer(CMemoryPool* pool, const int64_t size,
-                                    shared_ptr[CResizableBuffer]* out)
+    CResult[unique_ptr[CResizableBuffer]] AllocateResizableBuffer(
+        const int64_t size, CMemoryPool* pool)
 
     cdef CMemoryPool* c_default_memory_pool" arrow::default_memory_pool"()
 
diff --git a/python/pyarrow/io.pxi b/python/pyarrow/io.pxi
index 11c85b0..c4538db 100644
--- a/python/pyarrow/io.pxi
+++ b/python/pyarrow/io.pxi
@@ -1088,11 +1088,9 @@ cdef class ResizableBuffer(Buffer):
                          .Resize(new_size, c_shrink_to_fit))
 
 
-cdef shared_ptr[CResizableBuffer] _allocate_buffer(CMemoryPool* pool):
-    cdef shared_ptr[CResizableBuffer] result
+cdef shared_ptr[CResizableBuffer] _allocate_buffer(CMemoryPool* pool) except *:
     with nogil:
-        check_status(AllocateResizableBuffer(pool, 0, &result))
-    return result
+        return to_shared(GetResultValue(AllocateResizableBuffer(0, pool)))
 
 
 def allocate_buffer(int64_t size, MemoryPool memory_pool=None,
@@ -1115,18 +1113,19 @@ def allocate_buffer(int64_t size, MemoryPool memory_pool=None,
     buffer : Buffer or ResizableBuffer
     """
     cdef:
-        shared_ptr[CBuffer] buffer
-        shared_ptr[CResizableBuffer] rz_buffer
         CMemoryPool* cpool = maybe_unbox_memory_pool(memory_pool)
+        shared_ptr[CResizableBuffer] c_rz_buffer
+        shared_ptr[CBuffer] c_buffer
 
     if resizable:
         with nogil:
-            check_status(AllocateResizableBuffer(cpool, size, &rz_buffer))
-        return pyarrow_wrap_resizable_buffer(rz_buffer)
+            c_rz_buffer = to_shared(GetResultValue(
+                AllocateResizableBuffer(size, cpool)))
+        return pyarrow_wrap_resizable_buffer(c_rz_buffer)
     else:
         with nogil:
-            check_status(AllocateBuffer(cpool, size, &buffer))
-        return pyarrow_wrap_buffer(buffer)
+            c_buffer = to_shared(GetResultValue(AllocateBuffer(size, cpool)))
+        return pyarrow_wrap_buffer(c_buffer)
 
 
 cdef class BufferOutputStream(NativeFile):
diff --git a/r/src/array_from_vector.cpp b/r/src/array_from_vector.cpp
index e1f44eb..5e3e935 100644
--- a/r/src/array_from_vector.cpp
+++ b/r/src/array_from_vector.cpp
@@ -227,8 +227,8 @@ std::shared_ptr<Array> MakeFactorArrayImpl(Rcpp::IntegerVector_ factor,
   using value_type = typename arrow::TypeTraits<Type>::ArrayType::value_type;
   auto n = factor.size();
 
-  std::shared_ptr<Buffer> indices_buffer;
-  STOP_IF_NOT_OK(AllocateBuffer(n * sizeof(value_type), &indices_buffer));
+  std::shared_ptr<Buffer> indices_buffer =
+      VALUE_OR_STOP(AllocateBuffer(n * sizeof(value_type)));
 
   std::vector<std::shared_ptr<Buffer>> buffers{nullptr, indices_buffer};
 
@@ -243,8 +243,7 @@ std::shared_ptr<Array> MakeFactorArrayImpl(Rcpp::IntegerVector_ factor,
 
   if (i < n) {
     // there are NA's so we need a null buffer
-    std::shared_ptr<Buffer> null_buffer;
-    STOP_IF_NOT_OK(AllocateBuffer(BitUtil::BytesForBits(n), &null_buffer));
+    auto null_buffer = VALUE_OR_STOP(AllocateBuffer(BitUtil::BytesForBits(n)));
     internal::FirstTimeBitmapWriter null_bitmap_writer(null_buffer->mutable_data(), 0, n);
 
     // catch up
@@ -1066,11 +1065,10 @@ std::shared_ptr<Array> MakeSimpleArray(SEXP x) {
                                                std::make_shared<RBuffer<RTYPE>>(vec)};
 
   int null_count = 0;
-  std::shared_ptr<Buffer> null_bitmap;
 
   auto first_na = std::find_if(p_vec_start, p_vec_end, is_na<value_type>);
   if (first_na < p_vec_end) {
-    STOP_IF_NOT_OK(AllocateBuffer(BitUtil::BytesForBits(n), &null_bitmap));
+    auto null_bitmap = VALUE_OR_STOP(AllocateBuffer(BitUtil::BytesForBits(n)));
     internal::FirstTimeBitmapWriter bitmap_writer(null_bitmap->mutable_data(), 0, n);
 
     // first loop to clear all the bits before the first NA