You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@arrow.apache.org by we...@apache.org on 2018/07/25 17:47:45 UTC

[arrow] branch master updated: ARROW-2826: [C++] Remove ArrayBuilder::Init method, clean up Resize, remove PoolBuffer from public API

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

wesm 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 5033f6e  ARROW-2826: [C++] Remove ArrayBuilder::Init method, clean up Resize, remove PoolBuffer from public API
5033f6e is described below

commit 5033f6ea1dc2990cc59ba38566c587368aed17b7
Author: Wes McKinney <we...@apache.org>
AuthorDate: Wed Jul 25 13:47:41 2018 -0400

    ARROW-2826: [C++] Remove ArrayBuilder::Init method, clean up Resize, remove PoolBuffer from public API
    
    This causes an API break as a result of removing ArrayBuilder::Init. I can restore it for 0.10.0 and make it simply call the virtual `ArrayBuilder::Resize` if you believe it appropriate.
    
    This also removes `arrow::PoolBuffer` from the public API as discussed in ARROW-2893 and elsewhere.
    
    Author: Wes McKinney <we...@apache.org>
    Author: Kouhei Sutou <ko...@clear-code.com>
    
    Closes #2315 from wesm/ARROW-2826 and squashes the following commits:
    
    fdd7ac32 <Kouhei Sutou>  Use arrow::ResizableBuffer directory instead of arrow::PoolBuffer
    4f749c7a <Wes McKinney> Fix Python bindings
    eafebaa9 <Wes McKinney> Convert remaining usages of PoolBuffer
    a90fb075 <Wes McKinney> Respond to code review comments, begin excising PoolBuffer
    92fa82b8 <Wes McKinney> Fix up Resize implementations, fix Reserve unit test to work with cleaned up API
    548afaf8 <Wes McKinney> Initial refactor to remove ArrayBuilder::Init and its implementations
---
 c_glib/arrow-glib/buffer.cpp                       |  69 +++----
 c_glib/arrow-glib/buffer.h                         |  93 +--------
 c_glib/arrow-glib/buffer.hpp                       |   3 +-
 c_glib/test/test-buffer-output-stream.rb           |   2 +-
 ...est-pool-buffer.rb => test-resizable-buffer.rb} |   4 +-
 c_glib/test/test-tensor.rb                         |   2 +-
 cpp/src/arrow/array-test.cc                        |  19 +-
 cpp/src/arrow/buffer-test.cc                       |  72 +++----
 cpp/src/arrow/buffer.cc                            |  93 +++++----
 cpp/src/arrow/buffer.h                             |  95 ++++-----
 cpp/src/arrow/builder.cc                           | 224 +++++++--------------
 cpp/src/arrow/builder.h                            |  56 ++----
 cpp/src/arrow/column-benchmark.cc                  |  12 +-
 cpp/src/arrow/compute/kernels/hash.cc              |   4 +-
 cpp/src/arrow/gpu/cuda-benchmark.cc                |   4 +-
 cpp/src/arrow/gpu/cuda-test.cc                     |  25 ++-
 cpp/src/arrow/io/io-memory-benchmark.cc            |  15 +-
 cpp/src/arrow/io/io-memory-test.cc                 |  12 +-
 cpp/src/arrow/ipc/ipc-read-write-benchmark.cc      |   8 +-
 cpp/src/arrow/ipc/ipc-read-write-test.cc           |   8 +-
 cpp/src/arrow/ipc/test-common.h                    |  10 +-
 cpp/src/arrow/python/numpy_to_arrow.cc             |   5 +-
 cpp/src/arrow/table_builder.cc                     |   2 +-
 cpp/src/arrow/test-common.h                        |   5 +-
 cpp/src/arrow/test-util.h                          |  28 ++-
 cpp/src/arrow/util/io-util.h                       |   4 +-
 python/pyarrow/includes/libarrow.pxd               |   4 -
 python/pyarrow/io.pxi                              |   9 +-
 28 files changed, 354 insertions(+), 533 deletions(-)

diff --git a/c_glib/arrow-glib/buffer.cpp b/c_glib/arrow-glib/buffer.cpp
index 4dd9ce3..23ec3b6 100644
--- a/c_glib/arrow-glib/buffer.cpp
+++ b/c_glib/arrow-glib/buffer.cpp
@@ -39,10 +39,7 @@ G_BEGIN_DECLS
  *
  * #GArrowMutableBuffer is mutable.
  *
- * #GArrowResizableBuffer is mutable and
- * resizable. #GArrowResizableBuffer isn't instantiatable.
- *
- * #GArrowPoolBuffer is mutable, resizable and instantiatable.
+ * #GArrowResizableBuffer is mutable and resizable.
  */
 
 typedef struct GArrowBufferPrivate_ {
@@ -502,6 +499,29 @@ garrow_resizable_buffer_class_init(GArrowResizableBufferClass *klass)
 }
 
 /**
+ * garrow_resizable_buffer_new:
+ * @initial_size: The initial buffer size in bytes.
+ * @error: (nullable): Return location for a #GError or %NULL.
+ *
+ * Returns: (nullable): A newly created #GArrowResizableBuffer.
+ *
+ * Since: 0.10.0
+ */
+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]")) {
+    return garrow_resizable_buffer_new_raw(&arrow_buffer);
+  } else {
+    return NULL;
+  }
+}
+
+
+/**
  * garrow_resizable_buffer_resize:
  * @buffer: A #GArrowResizableBuffer.
  * @new_size: The new buffer size in bytes.
@@ -546,36 +566,6 @@ garrow_resizable_buffer_reserve(GArrowResizableBuffer *buffer,
 }
 
 
-G_DEFINE_TYPE(GArrowPoolBuffer,              \
-              garrow_pool_buffer,            \
-              GARROW_TYPE_RESIZABLE_BUFFER)
-
-static void
-garrow_pool_buffer_init(GArrowPoolBuffer *object)
-{
-}
-
-static void
-garrow_pool_buffer_class_init(GArrowPoolBufferClass *klass)
-{
-}
-
-/**
- * garrow_pool_buffer_new:
- *
- * Returns: A newly created #GArrowPoolBuffer.
- *
- * Since: 0.3.0
- */
-GArrowPoolBuffer *
-garrow_pool_buffer_new(void)
-{
-  auto arrow_memory_pool = arrow::default_memory_pool();
-  auto arrow_buffer = std::make_shared<arrow::PoolBuffer>(arrow_memory_pool);
-  return garrow_pool_buffer_new_raw(&arrow_buffer);
-}
-
-
 G_END_DECLS
 
 GArrowBuffer *
@@ -622,11 +612,12 @@ garrow_mutable_buffer_new_raw_bytes(std::shared_ptr<arrow::MutableBuffer> *arrow
   return buffer;
 }
 
-GArrowPoolBuffer *
-garrow_pool_buffer_new_raw(std::shared_ptr<arrow::PoolBuffer> *arrow_buffer)
+GArrowResizableBuffer *
+garrow_resizable_buffer_new_raw(std::shared_ptr<arrow::ResizableBuffer> *arrow_buffer)
 {
-  auto buffer = GARROW_POOL_BUFFER(g_object_new(GARROW_TYPE_POOL_BUFFER,
-                                                "buffer", arrow_buffer,
-                                                NULL));
+  auto buffer =
+    GARROW_RESIZABLE_BUFFER(g_object_new(GARROW_TYPE_RESIZABLE_BUFFER,
+                                         "buffer", arrow_buffer,
+                                         NULL));
   return buffer;
 }
diff --git a/c_glib/arrow-glib/buffer.h b/c_glib/arrow-glib/buffer.h
index 50debcc..c27699a 100644
--- a/c_glib/arrow-glib/buffer.h
+++ b/c_glib/arrow-glib/buffer.h
@@ -77,50 +77,20 @@ GArrowMutableBuffer *garrow_mutable_buffer_slice(GArrowMutableBuffer *buffer,
                                                  gint64 size);
 
 
-#define GARROW_TYPE_RESIZABLE_BUFFER            \
-  (garrow_resizable_buffer_get_type())
-#define GARROW_RESIZABLE_BUFFER(obj)                            \
-  (G_TYPE_CHECK_INSTANCE_CAST((obj),                            \
-                              GARROW_TYPE_RESIZABLE_BUFFER,     \
-                              GArrowResizableBuffer))
-#define GARROW_RESIZABLE_BUFFER_CLASS(klass)                    \
-  (G_TYPE_CHECK_CLASS_CAST((klass),                             \
-                           GARROW_TYPE_RESIZABLE_BUFFER,        \
-                           GArrowResizableBufferClass))
-#define GARROW_IS_RESIZABLE_BUFFER(obj)                                 \
-  (G_TYPE_CHECK_INSTANCE_TYPE((obj), GARROW_TYPE_RESIZABLE_BUFFER))
-#define GARROW_IS_RESIZABLE_BUFFER_CLASS(klass)                         \
-  (G_TYPE_CHECK_CLASS_TYPE((klass), GARROW_TYPE_RESIZABLE_BUFFER))
-#define GARROW_RESIZABLE_BUFFER_GET_CLASS(obj)                  \
-  (G_TYPE_INSTANCE_GET_CLASS((obj),                             \
-                             GARROW_TYPE_RESIZABLE_BUFFER,      \
-                             GArrowResizableBufferClass))
-
-typedef struct _GArrowResizableBuffer         GArrowResizableBuffer;
-#ifndef __GTK_DOC_IGNORE__
-typedef struct _GArrowResizableBufferClass    GArrowResizableBufferClass;
-#endif
-
-/**
- * GArrowResizableBuffer:
- *
- * It wraps `arrow::ResizableBuffer`.
- */
-struct _GArrowResizableBuffer
-{
-  /*< private >*/
-  GArrowMutableBuffer parent_instance;
-};
-
-#ifndef __GTK_DOC_IGNORE__
+#define GARROW_TYPE_RESIZABLE_BUFFER (garrow_resizable_buffer_get_type())
+G_DECLARE_DERIVABLE_TYPE(GArrowResizableBuffer,
+                         garrow_resizable_buffer,
+                         GARROW,
+                         RESIZABLE_BUFFER,
+                         GArrowMutableBuffer)
 struct _GArrowResizableBufferClass
 {
   GArrowMutableBufferClass parent_class;
 };
-#endif
 
-GType garrow_resizable_buffer_get_type(void) G_GNUC_CONST;
 
+GArrowResizableBuffer *garrow_resizable_buffer_new(gint64 initial_size,
+                                                   GError **error);
 gboolean garrow_resizable_buffer_resize(GArrowResizableBuffer *buffer,
                                         gint64 new_size,
                                         GError **error);
@@ -128,51 +98,4 @@ gboolean garrow_resizable_buffer_reserve(GArrowResizableBuffer *buffer,
                                          gint64 new_capacity,
                                          GError **error);
 
-
-#define GARROW_TYPE_POOL_BUFFER                 \
-  (garrow_pool_buffer_get_type())
-#define GARROW_POOL_BUFFER(obj)                         \
-  (G_TYPE_CHECK_INSTANCE_CAST((obj),                    \
-                              GARROW_TYPE_POOL_BUFFER,  \
-                              GArrowPoolBuffer))
-#define GARROW_POOL_BUFFER_CLASS(klass)                 \
-  (G_TYPE_CHECK_CLASS_CAST((klass),                     \
-                           GARROW_TYPE_POOL_BUFFER,     \
-                           GArrowPoolBufferClass))
-#define GARROW_IS_POOL_BUFFER(obj)                              \
-  (G_TYPE_CHECK_INSTANCE_TYPE((obj), GARROW_TYPE_POOL_BUFFER))
-#define GARROW_IS_POOL_BUFFER_CLASS(klass)                      \
-  (G_TYPE_CHECK_CLASS_TYPE((klass), GARROW_TYPE_POOL_BUFFER))
-#define GARROW_POOL_BUFFER_GET_CLASS(obj)               \
-  (G_TYPE_INSTANCE_GET_CLASS((obj),                     \
-                             GARROW_TYPE_POOL_BUFFER,   \
-                             GArrowPoolBufferClass))
-
-typedef struct _GArrowPoolBuffer         GArrowPoolBuffer;
-#ifndef __GTK_DOC_IGNORE__
-typedef struct _GArrowPoolBufferClass    GArrowPoolBufferClass;
-#endif
-
-/**
- * GArrowPoolBuffer:
- *
- * It wraps `arrow::PoolBuffer`.
- */
-struct _GArrowPoolBuffer
-{
-  /*< private >*/
-  GArrowResizableBuffer parent_instance;
-};
-
-#ifndef __GTK_DOC_IGNORE__
-struct _GArrowPoolBufferClass
-{
-  GArrowResizableBufferClass parent_class;
-};
-#endif
-
-GType garrow_pool_buffer_get_type(void) G_GNUC_CONST;
-
-GArrowPoolBuffer *garrow_pool_buffer_new(void);
-
 G_END_DECLS
diff --git a/c_glib/arrow-glib/buffer.hpp b/c_glib/arrow-glib/buffer.hpp
index 3dd3794..5116e84 100644
--- a/c_glib/arrow-glib/buffer.hpp
+++ b/c_glib/arrow-glib/buffer.hpp
@@ -31,4 +31,5 @@ std::shared_ptr<arrow::Buffer> garrow_buffer_get_raw(GArrowBuffer *buffer);
 GArrowMutableBuffer *garrow_mutable_buffer_new_raw(std::shared_ptr<arrow::MutableBuffer> *arrow_buffer);
 GArrowMutableBuffer *garrow_mutable_buffer_new_raw_bytes(std::shared_ptr<arrow::MutableBuffer> *arrow_buffer,
                                                          GBytes *data);
-GArrowPoolBuffer *garrow_pool_buffer_new_raw(std::shared_ptr<arrow::PoolBuffer> *arrow_buffer);
+GArrowResizableBuffer *
+garrow_resizable_buffer_new_raw(std::shared_ptr<arrow::ResizableBuffer> *arrow_buffer);
diff --git a/c_glib/test/test-buffer-output-stream.rb b/c_glib/test/test-buffer-output-stream.rb
index 1b7fae9..e1f2617 100644
--- a/c_glib/test/test-buffer-output-stream.rb
+++ b/c_glib/test/test-buffer-output-stream.rb
@@ -17,7 +17,7 @@
 
 class TestBufferOutputStream < Test::Unit::TestCase
   def test_new
-    buffer = Arrow::PoolBuffer.new
+    buffer = Arrow::ResizableBuffer.new(0)
     output_stream = Arrow::BufferOutputStream.new(buffer)
     output_stream.write("Hello")
     output_stream.close
diff --git a/c_glib/test/test-pool-buffer.rb b/c_glib/test/test-resizable-buffer.rb
similarity index 91%
rename from c_glib/test/test-pool-buffer.rb
rename to c_glib/test/test-resizable-buffer.rb
index 57f3458..84d95de 100644
--- a/c_glib/test/test-pool-buffer.rb
+++ b/c_glib/test/test-resizable-buffer.rb
@@ -15,9 +15,9 @@
 # specific language governing permissions and limitations
 # under the License.
 
-class TestPoolBuffer < Test::Unit::TestCase
+class TestResizableBuffer < Test::Unit::TestCase
   def setup
-    @buffer = Arrow::PoolBuffer.new
+    @buffer = Arrow::ResizableBuffer.new(0)
   end
 
   def test_resize
diff --git a/c_glib/test/test-tensor.rb b/c_glib/test/test-tensor.rb
index 6cead58..e812d5e 100644
--- a/c_glib/test/test-tensor.rb
+++ b/c_glib/test/test-tensor.rb
@@ -116,7 +116,7 @@ class TestTensor < Test::Unit::TestCase
   end
 
   def test_io
-    buffer = Arrow::PoolBuffer.new
+    buffer = Arrow::ResizableBuffer.new(0)
     output = Arrow::BufferOutputStream.new(buffer)
     output.write_tensor(@tensor)
     input = Arrow::BufferInputStream.new(buffer)
diff --git a/cpp/src/arrow/array-test.cc b/cpp/src/arrow/array-test.cc
index df6051e..b7bad67 100644
--- a/cpp/src/arrow/array-test.cc
+++ b/cpp/src/arrow/array-test.cc
@@ -87,8 +87,9 @@ class TestArray : public ::testing::Test {
 };
 
 TEST_F(TestArray, TestNullCount) {
-  auto data = std::make_shared<PoolBuffer>(pool_);
-  auto null_bitmap = std::make_shared<PoolBuffer>(pool_);
+  // These are placeholders
+  auto data = std::make_shared<Buffer>(nullptr, 0);
+  auto null_bitmap = std::make_shared<Buffer>(nullptr, 0);
 
   std::unique_ptr<Int32Array> arr(new Int32Array(100, data, null_bitmap, 10));
   ASSERT_EQ(10, arr->null_count());
@@ -98,7 +99,9 @@ TEST_F(TestArray, TestNullCount) {
 }
 
 TEST_F(TestArray, TestLength) {
-  auto data = std::make_shared<PoolBuffer>(pool_);
+  // Placeholder buffer
+  auto data = std::make_shared<Buffer>(nullptr, 0);
+
   std::unique_ptr<Int32Array> arr(new Int32Array(100, data));
   ASSERT_EQ(arr->length(), 100);
 }
@@ -268,11 +271,13 @@ TEST_F(TestArray, TestCopy) {}
 TEST_F(TestBuilder, TestReserve) {
   UInt8Builder builder(pool_);
 
-  ASSERT_OK(builder.Init(10));
-  ASSERT_EQ(10, builder.capacity());
+  ASSERT_OK(builder.Resize(1000));
+  ASSERT_EQ(1000, builder.capacity());
 
-  ASSERT_OK(builder.Reserve(30));
-  ASSERT_EQ(BitUtil::NextPower2(30), builder.capacity());
+  // Builder only contains 0 elements, but calling Reserve will result in a round
+  // up to next power of 2
+  ASSERT_OK(builder.Reserve(1030));
+  ASSERT_EQ(BitUtil::NextPower2(1030), builder.capacity());
 }
 
 template <typename Attrs>
diff --git a/cpp/src/arrow/buffer-test.cc b/cpp/src/arrow/buffer-test.cc
index 6070820..55b86e9 100644
--- a/cpp/src/arrow/buffer-test.cc
+++ b/cpp/src/arrow/buffer-test.cc
@@ -40,8 +40,9 @@ TEST(TestBuffer, IsMutableFlag) {
   MutableBuffer mbuf(nullptr, 0);
   ASSERT_TRUE(mbuf.is_mutable());
 
-  PoolBuffer pbuf;
-  ASSERT_TRUE(pbuf.is_mutable());
+  std::shared_ptr<ResizableBuffer> pool_buf;
+  ASSERT_OK(AllocateResizableBuffer(0, &pool_buf));
+  ASSERT_TRUE(pool_buf->is_mutable());
 }
 
 TEST(TestBuffer, FromStdString) {
@@ -70,55 +71,58 @@ TEST(TestBuffer, FromStdStringWithMemory) {
 }
 
 TEST(TestBuffer, Resize) {
-  PoolBuffer buf;
+  std::shared_ptr<ResizableBuffer> buf;
+  ASSERT_OK(AllocateResizableBuffer(0, &buf));
 
-  ASSERT_EQ(0, buf.size());
-  ASSERT_OK(buf.Resize(100));
-  ASSERT_EQ(100, buf.size());
-  ASSERT_OK(buf.Resize(200));
-  ASSERT_EQ(200, buf.size());
+  ASSERT_EQ(0, buf->size());
+  ASSERT_OK(buf->Resize(100));
+  ASSERT_EQ(100, buf->size());
+  ASSERT_OK(buf->Resize(200));
+  ASSERT_EQ(200, buf->size());
 
   // Make it smaller, too
-  ASSERT_OK(buf.Resize(50, true));
-  ASSERT_EQ(50, buf.size());
+  ASSERT_OK(buf->Resize(50, true));
+  ASSERT_EQ(50, buf->size());
   // We have actually shrunken in size
   // The spec requires that capacity is a multiple of 64
-  ASSERT_EQ(64, buf.capacity());
+  ASSERT_EQ(64, buf->capacity());
 
   // Resize to a larger capacity again to test shrink_to_fit = false
-  ASSERT_OK(buf.Resize(100));
-  ASSERT_EQ(128, buf.capacity());
-  ASSERT_OK(buf.Resize(50, false));
-  ASSERT_EQ(128, buf.capacity());
+  ASSERT_OK(buf->Resize(100));
+  ASSERT_EQ(128, buf->capacity());
+  ASSERT_OK(buf->Resize(50, false));
+  ASSERT_EQ(128, buf->capacity());
 }
 
 TEST(TestBuffer, TypedResize) {
-  PoolBuffer buf;
-
-  ASSERT_EQ(0, buf.size());
-  ASSERT_OK(buf.TypedResize<double>(100));
-  ASSERT_EQ(800, buf.size());
-  ASSERT_OK(buf.TypedResize<double>(200));
-  ASSERT_EQ(1600, buf.size());
-
-  ASSERT_OK(buf.TypedResize<double>(50, true));
-  ASSERT_EQ(400, buf.size());
-  ASSERT_EQ(448, buf.capacity());
-
-  ASSERT_OK(buf.TypedResize<double>(100));
-  ASSERT_EQ(832, buf.capacity());
-  ASSERT_OK(buf.TypedResize<double>(50, false));
-  ASSERT_EQ(832, buf.capacity());
+  std::shared_ptr<ResizableBuffer> buf;
+  ASSERT_OK(AllocateResizableBuffer(0, &buf));
+
+  ASSERT_EQ(0, buf->size());
+  ASSERT_OK(buf->TypedResize<double>(100));
+  ASSERT_EQ(800, buf->size());
+  ASSERT_OK(buf->TypedResize<double>(200));
+  ASSERT_EQ(1600, buf->size());
+
+  ASSERT_OK(buf->TypedResize<double>(50, true));
+  ASSERT_EQ(400, buf->size());
+  ASSERT_EQ(448, buf->capacity());
+
+  ASSERT_OK(buf->TypedResize<double>(100));
+  ASSERT_EQ(832, buf->capacity());
+  ASSERT_OK(buf->TypedResize<double>(50, false));
+  ASSERT_EQ(832, buf->capacity());
 }
 
 TEST(TestBuffer, ResizeOOM) {
 // This test doesn't play nice with AddressSanitizer
 #ifndef ADDRESS_SANITIZER
   // realloc fails, even though there may be no explicit limit
-  PoolBuffer buf;
-  ASSERT_OK(buf.Resize(100));
+  std::shared_ptr<ResizableBuffer> buf;
+  ASSERT_OK(AllocateResizableBuffer(0, &buf));
+  ASSERT_OK(buf->Resize(100));
   int64_t to_alloc = std::numeric_limits<int64_t>::max();
-  ASSERT_RAISES(OutOfMemory, buf.Resize(to_alloc));
+  ASSERT_RAISES(OutOfMemory, buf->Resize(to_alloc));
 #endif
 }
 
diff --git a/cpp/src/arrow/buffer.cc b/cpp/src/arrow/buffer.cc
index fde20ff..1303534 100644
--- a/cpp/src/arrow/buffer.cc
+++ b/cpp/src/arrow/buffer.cc
@@ -72,60 +72,67 @@ Status Buffer::FromString(const std::string& data, std::shared_ptr<Buffer>* out)
 
 void Buffer::CheckMutable() const { DCHECK(is_mutable()) << "buffer not mutable"; }
 
-PoolBuffer::PoolBuffer(MemoryPool* pool) : ResizableBuffer(nullptr, 0) {
-  if (pool == nullptr) {
-    pool = default_memory_pool();
+/// A Buffer whose lifetime is tied to a particular MemoryPool
+class PoolBuffer : public ResizableBuffer {
+ public:
+  explicit PoolBuffer(MemoryPool* pool) : ResizableBuffer(nullptr, 0) {
+    if (pool == nullptr) {
+      pool = default_memory_pool();
+    }
+    pool_ = pool;
   }
-  pool_ = pool;
-}
 
-PoolBuffer::~PoolBuffer() {
-  if (mutable_data_ != nullptr) {
-    pool_->Free(mutable_data_, capacity_);
+  ~PoolBuffer() override {
+    if (mutable_data_ != nullptr) {
+      pool_->Free(mutable_data_, capacity_);
+    }
   }
-}
 
-Status PoolBuffer::Reserve(const int64_t capacity) {
-  if (!mutable_data_ || capacity > capacity_) {
-    uint8_t* new_data;
-    int64_t new_capacity = BitUtil::RoundUpToMultipleOf64(capacity);
-    if (mutable_data_) {
-      RETURN_NOT_OK(pool_->Reallocate(capacity_, new_capacity, &mutable_data_));
-    } else {
-      RETURN_NOT_OK(pool_->Allocate(new_capacity, &new_data));
-      mutable_data_ = new_data;
+  Status Reserve(const int64_t capacity) override {
+    if (!mutable_data_ || capacity > capacity_) {
+      uint8_t* new_data;
+      int64_t new_capacity = BitUtil::RoundUpToMultipleOf64(capacity);
+      if (mutable_data_) {
+        RETURN_NOT_OK(pool_->Reallocate(capacity_, new_capacity, &mutable_data_));
+      } else {
+        RETURN_NOT_OK(pool_->Allocate(new_capacity, &new_data));
+        mutable_data_ = new_data;
+      }
+      data_ = mutable_data_;
+      capacity_ = new_capacity;
     }
-    data_ = mutable_data_;
-    capacity_ = new_capacity;
+    return Status::OK();
   }
-  return Status::OK();
-}
 
-Status PoolBuffer::Resize(const int64_t new_size, bool shrink_to_fit) {
-  if (!shrink_to_fit || (new_size > size_)) {
-    RETURN_NOT_OK(Reserve(new_size));
-  } else {
-    // Buffer is not growing, so shrink to the requested size without
-    // excess space.
-    int64_t new_capacity = BitUtil::RoundUpToMultipleOf64(new_size);
-    if (capacity_ != new_capacity) {
-      // Buffer hasn't got yet the requested size.
-      if (new_size == 0) {
-        pool_->Free(mutable_data_, capacity_);
-        capacity_ = 0;
-        mutable_data_ = nullptr;
-        data_ = nullptr;
-      } else {
-        RETURN_NOT_OK(pool_->Reallocate(capacity_, new_capacity, &mutable_data_));
-        data_ = mutable_data_;
-        capacity_ = new_capacity;
+  Status Resize(const int64_t new_size, bool shrink_to_fit = true) override {
+    if (!shrink_to_fit || (new_size > size_)) {
+      RETURN_NOT_OK(Reserve(new_size));
+    } else {
+      // Buffer is not growing, so shrink to the requested size without
+      // excess space.
+      int64_t new_capacity = BitUtil::RoundUpToMultipleOf64(new_size);
+      if (capacity_ != new_capacity) {
+        // Buffer hasn't got yet the requested size.
+        if (new_size == 0) {
+          pool_->Free(mutable_data_, capacity_);
+          capacity_ = 0;
+          mutable_data_ = nullptr;
+          data_ = nullptr;
+        } else {
+          RETURN_NOT_OK(pool_->Reallocate(capacity_, new_capacity, &mutable_data_));
+          data_ = mutable_data_;
+          capacity_ = new_capacity;
+        }
       }
     }
+    size_ = new_size;
+
+    return Status::OK();
   }
-  size_ = new_size;
 
-  return Status::OK();
-}
+ private:
+  MemoryPool* pool_;
+};
 
 std::shared_ptr<Buffer> SliceMutableBuffer(const std::shared_ptr<Buffer>& buffer,
                                            const int64_t offset, const int64_t length) {
diff --git a/cpp/src/arrow/buffer.h b/cpp/src/arrow/buffer.h
index 23fbdfa..ea4637c 100644
--- a/cpp/src/arrow/buffer.h
+++ b/cpp/src/arrow/buffer.h
@@ -221,18 +221,44 @@ class ARROW_EXPORT ResizableBuffer : public MutableBuffer {
   ResizableBuffer(uint8_t* data, int64_t size) : MutableBuffer(data, size) {}
 };
 
-/// A Buffer whose lifetime is tied to a particular MemoryPool
-class ARROW_EXPORT PoolBuffer : public ResizableBuffer {
- public:
-  explicit PoolBuffer(MemoryPool* pool = NULLPTR);
-  ~PoolBuffer() override;
+/// \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_EXPORT
+Status AllocateBuffer(MemoryPool* pool, const int64_t size, std::shared_ptr<Buffer>* out);
 
-  Status Resize(const int64_t new_size, bool shrink_to_fit = true) override;
-  Status Reserve(const int64_t new_capacity) override;
+/// \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_EXPORT
+Status AllocateBuffer(const int64_t size, std::shared_ptr<Buffer>* out);
 
- private:
-  MemoryPool* pool_;
-};
+/// \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_EXPORT
+Status AllocateResizableBuffer(MemoryPool* pool, 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_EXPORT
+Status AllocateResizableBuffer(const int64_t size, std::shared_ptr<ResizableBuffer>* out);
 
 /// \class BufferBuilder
 /// \brief A class for incrementally building a contiguous chunk of in-memory data
@@ -254,11 +280,13 @@ class ARROW_EXPORT BufferBuilder {
     if (elements == 0) {
       return Status::OK();
     }
+    int64_t old_capacity = capacity_;
+
     if (buffer_ == NULLPTR) {
-      buffer_ = std::make_shared<PoolBuffer>(pool_);
+      RETURN_NOT_OK(AllocateResizableBuffer(pool_, elements, &buffer_));
+    } else {
+      RETURN_NOT_OK(buffer_->Resize(elements, shrink_to_fit));
     }
-    int64_t old_capacity = capacity_;
-    RETURN_NOT_OK(buffer_->Resize(elements, shrink_to_fit));
     capacity_ = buffer_->capacity();
     data_ = buffer_->mutable_data();
     if (capacity_ > old_capacity) {
@@ -330,7 +358,7 @@ class ARROW_EXPORT BufferBuilder {
   const uint8_t* data() const { return data_; }
 
  protected:
-  std::shared_ptr<PoolBuffer> buffer_;
+  std::shared_ptr<ResizableBuffer> buffer_;
   MemoryPool* pool_;
   uint8_t* data_;
   int64_t capacity_;
@@ -374,45 +402,6 @@ class ARROW_EXPORT TypedBufferBuilder : public BufferBuilder {
   int64_t capacity() const { return capacity_ / sizeof(T); }
 };
 
-/// \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_EXPORT
-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_EXPORT
-Status AllocateBuffer(const int64_t size, std::shared_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
-ARROW_EXPORT
-Status AllocateResizableBuffer(MemoryPool* pool, 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_EXPORT
-Status AllocateResizableBuffer(const int64_t size, std::shared_ptr<ResizableBuffer>* out);
-
 }  // namespace arrow
 
 #endif  // ARROW_BUFFER_H
diff --git a/cpp/src/arrow/builder.cc b/cpp/src/arrow/builder.cc
index b81d914..8bdcb60 100644
--- a/cpp/src/arrow/builder.cc
+++ b/cpp/src/arrow/builder.cc
@@ -81,35 +81,32 @@ Status ArrayBuilder::AppendToBitmap(const uint8_t* valid_bytes, int64_t length)
   return Status::OK();
 }
 
-Status ArrayBuilder::Init(int64_t capacity) {
-  int64_t to_alloc = BitUtil::BytesForBits(capacity);
-  null_bitmap_ = std::make_shared<PoolBuffer>(pool_);
-  RETURN_NOT_OK(null_bitmap_->Resize(to_alloc));
+Status ArrayBuilder::Resize(int64_t capacity) {
+  // Target size of validity (null) bitmap data
+  const int64_t new_bitmap_size = BitUtil::BytesForBits(capacity);
 
-  // Buffers might allocate more then necessary to satisfy padding requirements
-  const int64_t byte_capacity = null_bitmap_->capacity();
-  capacity_ = capacity;
-  null_bitmap_data_ = null_bitmap_->mutable_data();
-  memset(null_bitmap_data_, 0, static_cast<size_t>(byte_capacity));
-
-  return Status::OK();
-}
+  if (capacity_ == 0) {
+    RETURN_NOT_OK(AllocateResizableBuffer(pool_, new_bitmap_size, &null_bitmap_));
+    null_bitmap_data_ = null_bitmap_->mutable_data();
 
-Status ArrayBuilder::Resize(int64_t new_bits) {
-  if (!null_bitmap_) {
-    return Init(new_bits);
-  }
-  int64_t new_bytes = BitUtil::BytesForBits(new_bits);
-  int64_t old_bytes = null_bitmap_->size();
-  RETURN_NOT_OK(null_bitmap_->Resize(new_bytes));
-  null_bitmap_data_ = null_bitmap_->mutable_data();
-  // The buffer might be overpadded to deal with padding according to the spec
-  const int64_t byte_capacity = null_bitmap_->capacity();
-  capacity_ = new_bits;
-  if (old_bytes < new_bytes) {
-    memset(null_bitmap_data_ + old_bytes, 0,
-           static_cast<size_t>(byte_capacity - old_bytes));
+    // Padding is zeroed by AllocateResizableBuffer
+    memset(null_bitmap_data_, 0, static_cast<size_t>(new_bitmap_size));
+  } else {
+    const int64_t old_bitmap_capacity = null_bitmap_->capacity();
+    RETURN_NOT_OK(null_bitmap_->Resize(new_bitmap_size));
+
+    const int64_t new_bitmap_capacity = null_bitmap_->capacity();
+    null_bitmap_data_ = null_bitmap_->mutable_data();
+
+    // Zero the region between the original capacity and the new capacity,
+    // including padding, which has not been zeroed, unlike
+    // AllocateResizableBuffer
+    if (old_bitmap_capacity < new_bitmap_capacity) {
+      memset(null_bitmap_data_ + old_bitmap_capacity, 0,
+             static_cast<size_t>(new_bitmap_capacity - old_bitmap_capacity));
+    }
   }
+  capacity_ = capacity;
   return Status::OK();
 }
 
@@ -128,11 +125,11 @@ Status ArrayBuilder::Finish(std::shared_ptr<Array>* out) {
   return Status::OK();
 }
 
-Status ArrayBuilder::Reserve(int64_t elements) {
-  if (length_ + elements > capacity_) {
+Status ArrayBuilder::Reserve(int64_t additional_elements) {
+  if (length_ + additional_elements > capacity_) {
     // TODO(emkornfield) power of 2 growth is potentially suboptimal
-    int64_t new_capacity = BitUtil::NextPower2(length_ + elements);
-    return Resize(new_capacity);
+    int64_t new_size = BitUtil::NextPower2(length_ + additional_elements);
+    return Resize(new_size);
   }
   return Status::OK();
 }
@@ -198,19 +195,6 @@ Status NullBuilder::FinishInternal(std::shared_ptr<ArrayData>* out) {
 // ----------------------------------------------------------------------
 
 template <typename T>
-Status PrimitiveBuilder<T>::Init(int64_t capacity) {
-  RETURN_NOT_OK(ArrayBuilder::Init(capacity));
-  data_ = std::make_shared<PoolBuffer>(pool_);
-
-  int64_t nbytes = TypeTraits<T>::bytes_required(capacity);
-  RETURN_NOT_OK(data_->Resize(nbytes));
-
-  raw_data_ = reinterpret_cast<value_type*>(data_->mutable_data());
-
-  return Status::OK();
-}
-
-template <typename T>
 void PrimitiveBuilder<T>::Reset() {
   data_.reset();
   raw_data_ = nullptr;
@@ -223,15 +207,15 @@ Status PrimitiveBuilder<T>::Resize(int64_t capacity) {
     capacity = kMinBuilderCapacity;
   }
 
+  int64_t nbytes = TypeTraits<T>::bytes_required(capacity);
   if (capacity_ == 0) {
-    RETURN_NOT_OK(Init(capacity));
+    RETURN_NOT_OK(AllocateResizableBuffer(pool_, nbytes, &data_));
   } else {
-    RETURN_NOT_OK(ArrayBuilder::Resize(capacity));
-    const int64_t new_bytes = TypeTraits<T>::bytes_required(capacity);
-    RETURN_NOT_OK(data_->Resize(new_bytes));
-    raw_data_ = reinterpret_cast<value_type*>(data_->mutable_data());
+    RETURN_NOT_OK(data_->Resize(nbytes));
   }
-  return Status::OK();
+
+  raw_data_ = reinterpret_cast<value_type*>(data_->mutable_data());
+  return ArrayBuilder::Resize(capacity);
 }
 
 template <typename T>
@@ -332,17 +316,6 @@ template class PrimitiveBuilder<DoubleType>;
 AdaptiveIntBuilderBase::AdaptiveIntBuilderBase(MemoryPool* pool)
     : ArrayBuilder(int64(), pool), data_(nullptr), raw_data_(nullptr), int_size_(1) {}
 
-Status AdaptiveIntBuilderBase::Init(int64_t capacity) {
-  RETURN_NOT_OK(ArrayBuilder::Init(capacity));
-  data_ = std::make_shared<PoolBuffer>(pool_);
-
-  int64_t nbytes = capacity * int_size_;
-  RETURN_NOT_OK(data_->Resize(nbytes));
-
-  raw_data_ = reinterpret_cast<uint8_t*>(data_->mutable_data());
-  return Status::OK();
-}
-
 void AdaptiveIntBuilderBase::Reset() {
   ArrayBuilder::Reset();
   data_.reset();
@@ -355,15 +328,15 @@ Status AdaptiveIntBuilderBase::Resize(int64_t capacity) {
     capacity = kMinBuilderCapacity;
   }
 
+  int64_t nbytes = capacity * int_size_;
   if (capacity_ == 0) {
-    RETURN_NOT_OK(Init(capacity));
+    RETURN_NOT_OK(AllocateResizableBuffer(pool_, nbytes, &data_));
   } else {
-    RETURN_NOT_OK(ArrayBuilder::Resize(capacity));
-    const int64_t new_bytes = capacity * int_size_;
-    RETURN_NOT_OK(data_->Resize(new_bytes));
-    raw_data_ = data_->mutable_data();
+    RETURN_NOT_OK(data_->Resize(nbytes));
   }
-  return Status::OK();
+  raw_data_ = reinterpret_cast<uint8_t*>(data_->mutable_data());
+
+  return ArrayBuilder::Resize(capacity);
 }
 
 AdaptiveIntBuilder::AdaptiveIntBuilder(MemoryPool* pool) : AdaptiveIntBuilderBase(pool) {}
@@ -689,26 +662,6 @@ BooleanBuilder::BooleanBuilder(const std::shared_ptr<DataType>& type, MemoryPool
   DCHECK_EQ(Type::BOOL, type->id());
 }
 
-Status BooleanBuilder::Init(int64_t capacity) {
-  RETURN_NOT_OK(ArrayBuilder::Init(capacity));
-  data_ = std::make_shared<PoolBuffer>(pool_);
-
-  int64_t nbytes = BitUtil::BytesForBits(capacity);
-  RETURN_NOT_OK(data_->Resize(nbytes));
-
-  raw_data_ = reinterpret_cast<uint8_t*>(data_->mutable_data());
-
-  // We zero the memory for booleans to keep things simple; for some reason if
-  // we do not, even though we may write every bit (through in-place | or &),
-  // valgrind will still show a warning. If we do not zero the bytes here, we
-  // will have to be careful to zero them in AppendNull and AppendNulls. Also,
-  // zeroing the bits results in deterministic bits when each byte may have a
-  // mix of nulls and not nulls.
-  memset(raw_data_, 0, static_cast<size_t>(nbytes));
-
-  return Status::OK();
-}
-
 void BooleanBuilder::Reset() {
   ArrayBuilder::Reset();
   data_.reset();
@@ -721,23 +674,33 @@ Status BooleanBuilder::Resize(int64_t capacity) {
     capacity = kMinBuilderCapacity;
   }
 
+  const int64_t new_bitmap_size = BitUtil::BytesForBits(capacity);
   if (capacity_ == 0) {
-    RETURN_NOT_OK(Init(capacity));
+    RETURN_NOT_OK(AllocateResizableBuffer(pool_, new_bitmap_size, &data_));
+    raw_data_ = reinterpret_cast<uint8_t*>(data_->mutable_data());
+
+    // We zero the memory for booleans to keep things simple; for some reason if
+    // we do not, even though we may write every bit (through in-place | or &),
+    // valgrind will still show a warning. If we do not zero the bytes here, we
+    // will have to be careful to zero them in AppendNull and AppendNulls. Also,
+    // zeroing the bits results in deterministic bits when each byte may have a
+    // mix of nulls and not nulls.
+    //
+    // We only zero up to new_bitmap_size because the padding was zeroed by
+    // AllocateResizableBuffer
+    memset(raw_data_, 0, static_cast<size_t>(new_bitmap_size));
   } else {
-    RETURN_NOT_OK(ArrayBuilder::Resize(capacity));
-
-    const int64_t old_bytes = data_->size();
-    const int64_t new_bytes = BitUtil::BytesForBits(capacity);
-
-    if (new_bytes > old_bytes) {
-      RETURN_NOT_OK(data_->Resize(new_bytes));
-      raw_data_ = reinterpret_cast<uint8_t*>(data_->mutable_data());
+    const int64_t old_bitmap_capacity = data_->capacity();
+    RETURN_NOT_OK(data_->Resize(new_bitmap_size));
+    const int64_t new_bitmap_capacity = data_->capacity();
+    raw_data_ = reinterpret_cast<uint8_t*>(data_->mutable_data());
 
-      // See comment above about why we zero memory for booleans
-      memset(raw_data_ + old_bytes, 0, static_cast<size_t>(new_bytes - old_bytes));
-    }
+    // See comment above about why we zero memory for booleans
+    memset(raw_data_ + old_bitmap_capacity, 0,
+           static_cast<size_t>(new_bitmap_capacity - old_bitmap_capacity));
   }
-  return Status::OK();
+
+  return ArrayBuilder::Resize(capacity);
 }
 
 Status BooleanBuilder::FinishInternal(std::shared_ptr<ArrayData>* out) {
@@ -1006,27 +969,6 @@ DictionaryBuilder<FixedSizeBinaryType>::DictionaryBuilder(
 }
 
 template <typename T>
-Status DictionaryBuilder<T>::Init(int64_t elements) {
-  RETURN_NOT_OK(ArrayBuilder::Init(elements));
-
-  // Fill the initial hash table
-  RETURN_NOT_OK(internal::NewHashTable(kInitialHashTableSize, pool_, &hash_table_));
-  hash_slots_ = reinterpret_cast<int32_t*>(hash_table_->mutable_data());
-  hash_table_size_ = kInitialHashTableSize;
-  entry_id_offset_ = 0;
-  mod_bitmask_ = kInitialHashTableSize - 1;
-  hash_table_load_threshold_ =
-      static_cast<int64_t>(static_cast<double>(elements) * kMaxHashTableLoad);
-
-  return values_builder_.Init(elements);
-}
-
-Status DictionaryBuilder<NullType>::Init(int64_t elements) {
-  RETURN_NOT_OK(ArrayBuilder::Init(elements));
-  return values_builder_.Init(elements);
-}
-
-template <typename T>
 void DictionaryBuilder<T>::Reset() {
   dict_builder_.Reset();
   overflow_dict_builder_.Reset();
@@ -1040,22 +982,25 @@ Status DictionaryBuilder<T>::Resize(int64_t capacity) {
   }
 
   if (capacity_ == 0) {
-    return Init(capacity);
-  } else {
-    return ArrayBuilder::Resize(capacity);
-  }
+    // Fill the initial hash table
+    RETURN_NOT_OK(internal::NewHashTable(kInitialHashTableSize, pool_, &hash_table_));
+    hash_slots_ = reinterpret_cast<int32_t*>(hash_table_->mutable_data());
+    hash_table_size_ = kInitialHashTableSize;
+    entry_id_offset_ = 0;
+    mod_bitmask_ = kInitialHashTableSize - 1;
+    hash_table_load_threshold_ =
+        static_cast<int64_t>(static_cast<double>(capacity) * kMaxHashTableLoad);
+  }
+  RETURN_NOT_OK(values_builder_.Resize(capacity));
+  return ArrayBuilder::Resize(capacity);
 }
 
 Status DictionaryBuilder<NullType>::Resize(int64_t capacity) {
   if (capacity < kMinBuilderCapacity) {
     capacity = kMinBuilderCapacity;
   }
-
-  if (capacity_ == 0) {
-    return Init(capacity);
-  } else {
-    return ArrayBuilder::Resize(capacity);
-  }
+  RETURN_NOT_OK(values_builder_.Resize(capacity));
+  return ArrayBuilder::Resize(capacity);
 }
 
 template <typename T>
@@ -1313,13 +1258,6 @@ Status ListBuilder::Append(bool is_valid) {
   return AppendNextOffset();
 }
 
-Status ListBuilder::Init(int64_t elements) {
-  DCHECK_LE(elements, kListMaximumElements);
-  RETURN_NOT_OK(ArrayBuilder::Init(elements));
-  // one more then requested for offsets
-  return offsets_builder_.Resize((elements + 1) * sizeof(int32_t));
-}
-
 Status ListBuilder::Resize(int64_t capacity) {
   DCHECK_LE(capacity, kListMaximumElements);
   // one more then requested for offsets
@@ -1371,13 +1309,6 @@ BinaryBuilder::BinaryBuilder(const std::shared_ptr<DataType>& type, MemoryPool*
 
 BinaryBuilder::BinaryBuilder(MemoryPool* pool) : BinaryBuilder(binary(), pool) {}
 
-Status BinaryBuilder::Init(int64_t elements) {
-  DCHECK_LE(elements, kListMaximumElements);
-  RETURN_NOT_OK(ArrayBuilder::Init(elements));
-  // one more then requested for offsets
-  return offsets_builder_.Resize((elements + 1) * sizeof(int32_t));
-}
-
 Status BinaryBuilder::Resize(int64_t capacity) {
   DCHECK_LE(capacity, kListMaximumElements);
   // one more then requested for offsets
@@ -1586,11 +1517,6 @@ Status FixedSizeBinaryBuilder::AppendNull() {
   return byte_builder_.Advance(byte_width_);
 }
 
-Status FixedSizeBinaryBuilder::Init(int64_t elements) {
-  RETURN_NOT_OK(ArrayBuilder::Init(elements));
-  return byte_builder_.Resize(elements * byte_width_);
-}
-
 void FixedSizeBinaryBuilder::Reset() {
   ArrayBuilder::Reset();
   byte_builder_.Reset();
diff --git a/cpp/src/arrow/builder.h b/cpp/src/arrow/builder.h
index 0d60dbd..bbf2570 100644
--- a/cpp/src/arrow/builder.h
+++ b/cpp/src/arrow/builder.h
@@ -91,30 +91,33 @@ class ARROW_EXPORT ArrayBuilder {
   /// Set the next length bits to not null (i.e. valid).
   Status SetNotNull(int64_t length);
 
-  /// Allocates initial capacity requirements for the builder.  In most
-  /// cases subclasses should override and call their parent class's
-  /// method as well.
-  virtual Status Init(int64_t capacity);
+  /// \brief Ensure that enough memory has been allocated to fit the indicated
+  /// number of total elements in the builder, including any that have already
+  /// been appended. Does not account for reallocations that may be due to
+  /// variable size data, like binary values. To make space for incremental
+  /// appends, use Reserve instead.
+  /// \param[in] capacity the minimum number of additional array values
+  /// \return Status
+  virtual Status Resize(int64_t capacity);
 
-  /// Resizes the null_bitmap array.  In most
-  /// cases subclasses should override and call their parent class's
-  /// method as well.
-  virtual Status Resize(int64_t new_bits);
+  /// \brief Ensure that there is enough space allocated to add the indicated
+  /// number of elements without any further calls to Resize. The memory
+  /// allocated is rounded up to the next highest power of 2 similar to memory
+  /// allocations in STL containers like std::vector
+  /// \param[in] additional_capacity the number of additional array values
+  /// \return Status
+  Status Reserve(int64_t additional_capacity);
 
   /// Reset the builder.
   virtual void Reset();
 
-  /// Ensures there is enough space for adding the number of elements by checking
-  /// capacity and calling Resize if necessary.
-  Status Reserve(int64_t elements);
-
   /// For cases where raw data was memcpy'd into the internal buffers, allows us
   /// to advance the length of the builder. It is your responsibility to use
   /// this function responsibly.
   Status Advance(int64_t elements);
 
   ARROW_DEPRECATED("Use Finish instead")
-  std::shared_ptr<PoolBuffer> null_bitmap() const { return null_bitmap_; }
+  std::shared_ptr<ResizableBuffer> null_bitmap() const { return null_bitmap_; }
 
   /// \brief Return result of builder as an internal generic ArrayData
   /// object. Resets builder except for dictionary builder
@@ -183,7 +186,7 @@ class ARROW_EXPORT ArrayBuilder {
   MemoryPool* pool_;
 
   // When null_bitmap are first appended to the builder, the null bitmap is allocated
-  std::shared_ptr<PoolBuffer> null_bitmap_;
+  std::shared_ptr<ResizableBuffer> null_bitmap_;
   int64_t null_count_;
   uint8_t* null_bitmap_data_;
 
@@ -366,15 +369,12 @@ class ARROW_EXPORT PrimitiveBuilder : public ArrayBuilder {
   Status Append(const std::vector<value_type>& values);
 
   Status FinishInternal(std::shared_ptr<ArrayData>* out) override;
-  Status Init(int64_t capacity) override;
   void Reset() override;
 
-  /// Increase the capacity of the builder to accommodate at least the indicated
-  /// number of elements
   Status Resize(int64_t capacity) override;
 
  protected:
-  std::shared_ptr<PoolBuffer> data_;
+  std::shared_ptr<ResizableBuffer> data_;
   value_type* raw_data_;
 };
 
@@ -393,7 +393,6 @@ class ARROW_EXPORT NumericBuilder : public PrimitiveBuilder<T> {
 
   using PrimitiveBuilder<T>::Append;
   using PrimitiveBuilder<T>::AppendValues;
-  using PrimitiveBuilder<T>::Init;
   using PrimitiveBuilder<T>::Resize;
   using PrimitiveBuilder<T>::Reserve;
 
@@ -465,15 +464,11 @@ class ARROW_EXPORT AdaptiveIntBuilderBase : public ArrayBuilder {
   ARROW_DEPRECATED("Use Finish instead")
   std::shared_ptr<Buffer> data() const { return data_; }
 
-  Status Init(int64_t capacity) override;
   void Reset() override;
-
-  /// Increase the capacity of the builder to accommodate at least the indicated
-  /// number of elements
   Status Resize(int64_t capacity) override;
 
  protected:
-  std::shared_ptr<PoolBuffer> data_;
+  std::shared_ptr<ResizableBuffer> data_;
   uint8_t* raw_data_;
 
   uint8_t int_size_;
@@ -829,15 +824,11 @@ class ARROW_EXPORT BooleanBuilder : public ArrayBuilder {
   }
 
   Status FinishInternal(std::shared_ptr<ArrayData>* out) override;
-  Status Init(int64_t capacity) override;
   void Reset() override;
-
-  /// Increase the capacity of the builder to accommodate at least the indicated
-  /// number of elements
   Status Resize(int64_t capacity) override;
 
  protected:
-  std::shared_ptr<PoolBuffer> data_;
+  std::shared_ptr<ResizableBuffer> data_;
   uint8_t* raw_data_;
 };
 
@@ -864,7 +855,6 @@ class ARROW_EXPORT ListBuilder : public ArrayBuilder {
   ListBuilder(MemoryPool* pool, std::shared_ptr<ArrayBuilder> const& value_builder,
               const std::shared_ptr<DataType>& type = NULLPTR);
 
-  Status Init(int64_t elements) override;
   Status Resize(int64_t capacity) override;
   void Reset() override;
   Status FinishInternal(std::shared_ptr<ArrayData>* out) override;
@@ -921,12 +911,13 @@ class ARROW_EXPORT BinaryBuilder : public ArrayBuilder {
 
   Status AppendNull();
 
-  Status Init(int64_t elements) override;
   void Reset() override;
   Status Resize(int64_t capacity) override;
+
   /// \brief Ensures there is enough allocated capacity to append the indicated
   /// number of bytes to the value data buffer without additional allocations
   Status ReserveData(int64_t elements);
+
   Status FinishInternal(std::shared_ptr<ArrayData>* out) override;
 
   /// \return size of values buffer so far
@@ -1019,7 +1010,6 @@ class ARROW_EXPORT FixedSizeBinaryBuilder : public ArrayBuilder {
   Status Append(const std::string& value);
   Status AppendNull();
 
-  Status Init(int64_t elements) override;
   void Reset() override;
   Status Resize(int64_t capacity) override;
   Status FinishInternal(std::shared_ptr<ArrayData>* out) override;
@@ -1169,7 +1159,6 @@ class ARROW_EXPORT DictionaryBuilder : public ArrayBuilder {
   /// \brief Append a whole dense array to the builder
   Status AppendArray(const Array& array);
 
-  Status Init(int64_t elements) override;
   void Reset() override;
   Status Resize(int64_t capacity) override;
   Status FinishInternal(std::shared_ptr<ArrayData>* out) override;
@@ -1228,7 +1217,6 @@ class ARROW_EXPORT DictionaryBuilder<NullType> : public ArrayBuilder {
   /// \brief Append a whole dense array to the builder
   Status AppendArray(const Array& array);
 
-  Status Init(int64_t elements) override;
   Status Resize(int64_t capacity) override;
   Status FinishInternal(std::shared_ptr<ArrayData>* out) override;
 
diff --git a/cpp/src/arrow/column-benchmark.cc b/cpp/src/arrow/column-benchmark.cc
index c37c088..11d97ad 100644
--- a/cpp/src/arrow/column-benchmark.cc
+++ b/cpp/src/arrow/column-benchmark.cc
@@ -26,13 +26,11 @@ namespace arrow {
 namespace {
 template <typename ArrayType>
 Status MakePrimitive(int64_t length, int64_t null_count, std::shared_ptr<Array>* out) {
-  auto pool = default_memory_pool();
-  auto data = std::make_shared<PoolBuffer>(pool);
-  auto null_bitmap = std::make_shared<PoolBuffer>(pool);
-  RETURN_NOT_OK(data->Resize(length * sizeof(typename ArrayType::value_type)));
-  RETURN_NOT_OK(null_bitmap->Resize(BitUtil::BytesForBits(length)));
-  data->ZeroPadding();
-  null_bitmap->ZeroPadding();
+  std::shared_ptr<Buffer> data, null_bitmap;
+
+  RETURN_NOT_OK(AllocateBuffer(length * sizeof(typename ArrayType::value_type), &data));
+  RETURN_NOT_OK(AllocateBuffer(BitUtil::BytesForBits(length), &null_bitmap));
+
   *out = std::make_shared<ArrayType>(length, data, null_bitmap, null_count);
   return Status::OK();
 }
diff --git a/cpp/src/arrow/compute/kernels/hash.cc b/cpp/src/arrow/compute/kernels/hash.cc
index 2341e68..4004f8d 100644
--- a/cpp/src/arrow/compute/kernels/hash.cc
+++ b/cpp/src/arrow/compute/kernels/hash.cc
@@ -176,11 +176,11 @@ template <typename Type>
 struct HashDictionary<Type, enable_if_has_c_type<Type>> {
   using T = typename Type::c_type;
 
-  explicit HashDictionary(MemoryPool* pool)
-      : pool(pool), buffer(std::make_shared<PoolBuffer>(pool)), size(0), capacity(0) {}
+  explicit HashDictionary(MemoryPool* pool) : pool(pool), size(0), capacity(0) {}
 
   Status Init() {
     this->size = 0;
+    RETURN_NOT_OK(AllocateResizableBuffer(this->pool, 0, &this->buffer));
     return Resize(kInitialHashTableSize);
   }
 
diff --git a/cpp/src/arrow/gpu/cuda-benchmark.cc b/cpp/src/arrow/gpu/cuda-benchmark.cc
index 805a044..f791d12 100644
--- a/cpp/src/arrow/gpu/cuda-benchmark.cc
+++ b/cpp/src/arrow/gpu/cuda-benchmark.cc
@@ -48,8 +48,8 @@ static void CudaBufferWriterBenchmark(benchmark::State& state, const int64_t tot
     ABORT_NOT_OK(writer.SetBufferSize(buffer_size));
   }
 
-  std::shared_ptr<PoolBuffer> buffer;
-  ASSERT_OK(test::MakeRandomBytePoolBuffer(total_bytes, default_memory_pool(), &buffer));
+  std::shared_ptr<ResizableBuffer> buffer;
+  ASSERT_OK(test::MakeRandomByteBuffer(total_bytes, default_memory_pool(), &buffer));
 
   const uint8_t* host_data = buffer->data();
   while (state.KeepRunning()) {
diff --git a/cpp/src/arrow/gpu/cuda-test.cc b/cpp/src/arrow/gpu/cuda-test.cc
index 04e1f92..ba37be9 100644
--- a/cpp/src/arrow/gpu/cuda-test.cc
+++ b/cpp/src/arrow/gpu/cuda-test.cc
@@ -71,8 +71,8 @@ TEST_F(TestCudaBuffer, CopyFromHost) {
   std::shared_ptr<CudaBuffer> device_buffer;
   ASSERT_OK(context_->Allocate(kSize, &device_buffer));
 
-  std::shared_ptr<PoolBuffer> host_buffer;
-  ASSERT_OK(test::MakeRandomBytePoolBuffer(kSize, default_memory_pool(), &host_buffer));
+  std::shared_ptr<ResizableBuffer> host_buffer;
+  ASSERT_OK(test::MakeRandomByteBuffer(kSize, default_memory_pool(), &host_buffer));
 
   ASSERT_OK(device_buffer->CopyFromHost(0, host_buffer->data(), 500));
   ASSERT_OK(device_buffer->CopyFromHost(500, host_buffer->data() + 500, kSize - 500));
@@ -83,10 +83,10 @@ TEST_F(TestCudaBuffer, CopyFromHost) {
 TEST_F(TestCudaBuffer, FromBuffer) {
   const int64_t kSize = 1000;
   // Initialize device buffer with random data
-  std::shared_ptr<PoolBuffer> host_buffer;
+  std::shared_ptr<ResizableBuffer> host_buffer;
   std::shared_ptr<CudaBuffer> device_buffer;
   ASSERT_OK(context_->Allocate(kSize, &device_buffer));
-  ASSERT_OK(test::MakeRandomBytePoolBuffer(kSize, default_memory_pool(), &host_buffer));
+  ASSERT_OK(test::MakeRandomByteBuffer(kSize, default_memory_pool(), &host_buffer));
   ASSERT_OK(device_buffer->CopyFromHost(0, host_buffer->data(), 1000));
   // Sanity check
   AssertCudaBufferEquals(*device_buffer, host_buffer->data(), kSize);
@@ -132,8 +132,8 @@ TEST_F(TestCudaBuffer, DISABLED_ExportForIpc) {
   std::shared_ptr<CudaBuffer> device_buffer;
   ASSERT_OK(context_->Allocate(kSize, &device_buffer));
 
-  std::shared_ptr<PoolBuffer> host_buffer;
-  ASSERT_OK(test::MakeRandomBytePoolBuffer(kSize, default_memory_pool(), &host_buffer));
+  std::shared_ptr<ResizableBuffer> host_buffer;
+  ASSERT_OK(test::MakeRandomByteBuffer(kSize, default_memory_pool(), &host_buffer));
   ASSERT_OK(device_buffer->CopyFromHost(0, host_buffer->data(), kSize));
 
   // Export for IPC and serialize
@@ -171,9 +171,8 @@ class TestCudaBufferWriter : public TestCudaBufferBase {
 
   void TestWrites(const int64_t total_bytes, const int64_t chunksize,
                   const int64_t buffer_size = 0) {
-    std::shared_ptr<PoolBuffer> buffer;
-    ASSERT_OK(
-        test::MakeRandomBytePoolBuffer(total_bytes, default_memory_pool(), &buffer));
+    std::shared_ptr<ResizableBuffer> buffer;
+    ASSERT_OK(test::MakeRandomByteBuffer(total_bytes, default_memory_pool(), &buffer));
 
     if (buffer_size > 0) {
       ASSERT_OK(writer_->SetBufferSize(buffer_size));
@@ -223,8 +222,8 @@ TEST_F(TestCudaBufferWriter, BufferedWrites) {
 TEST_F(TestCudaBufferWriter, EdgeCases) {
   Allocate(1000);
 
-  std::shared_ptr<PoolBuffer> buffer;
-  ASSERT_OK(test::MakeRandomBytePoolBuffer(1000, default_memory_pool(), &buffer));
+  std::shared_ptr<ResizableBuffer> buffer;
+  ASSERT_OK(test::MakeRandomByteBuffer(1000, default_memory_pool(), &buffer));
   const uint8_t* host_data = buffer->data();
 
   ASSERT_EQ(0, writer_->buffer_size());
@@ -274,8 +273,8 @@ TEST_F(TestCudaBufferReader, Basics) {
   const int64_t size = 1000;
   ASSERT_OK(context_->Allocate(size, &device_buffer));
 
-  std::shared_ptr<PoolBuffer> buffer;
-  ASSERT_OK(test::MakeRandomBytePoolBuffer(1000, default_memory_pool(), &buffer));
+  std::shared_ptr<ResizableBuffer> buffer;
+  ASSERT_OK(test::MakeRandomByteBuffer(1000, default_memory_pool(), &buffer));
   const uint8_t* host_data = buffer->data();
 
   ASSERT_OK(device_buffer->CopyFromHost(0, host_data, 1000));
diff --git a/cpp/src/arrow/io/io-memory-benchmark.cc b/cpp/src/arrow/io/io-memory-benchmark.cc
index d294b86..b6ffa0e 100644
--- a/cpp/src/arrow/io/io-memory-benchmark.cc
+++ b/cpp/src/arrow/io/io-memory-benchmark.cc
@@ -28,11 +28,9 @@ namespace arrow {
 static void BM_SerialMemcopy(benchmark::State& state) {  // NOLINT non-const reference
   constexpr int64_t kTotalSize = 100 * 1024 * 1024;      // 100MB
 
-  auto buffer1 = std::make_shared<PoolBuffer>(default_memory_pool());
-  ABORT_NOT_OK(buffer1->Resize(kTotalSize));
-
-  auto buffer2 = std::make_shared<PoolBuffer>(default_memory_pool());
-  ABORT_NOT_OK(buffer2->Resize(kTotalSize));
+  std::shared_ptr<Buffer> buffer1, buffer2;
+  ABORT_NOT_OK(AllocateBuffer(kTotalSize, &buffer1));
+  ABORT_NOT_OK(AllocateBuffer(kTotalSize, &buffer2));
   test::random_bytes(kTotalSize, 0, buffer2->mutable_data());
 
   while (state.KeepRunning()) {
@@ -45,11 +43,10 @@ static void BM_SerialMemcopy(benchmark::State& state) {  // NOLINT non-const ref
 static void BM_ParallelMemcopy(benchmark::State& state) {  // NOLINT non-const reference
   constexpr int64_t kTotalSize = 100 * 1024 * 1024;        // 100MB
 
-  auto buffer1 = std::make_shared<PoolBuffer>(default_memory_pool());
-  ABORT_NOT_OK(buffer1->Resize(kTotalSize));
+  std::shared_ptr<Buffer> buffer1, buffer2;
+  ABORT_NOT_OK(AllocateBuffer(kTotalSize, &buffer1));
+  ABORT_NOT_OK(AllocateBuffer(kTotalSize, &buffer2));
 
-  auto buffer2 = std::make_shared<PoolBuffer>(default_memory_pool());
-  ABORT_NOT_OK(buffer2->Resize(kTotalSize));
   test::random_bytes(kTotalSize, 0, buffer2->mutable_data());
 
   while (state.KeepRunning()) {
diff --git a/cpp/src/arrow/io/io-memory-test.cc b/cpp/src/arrow/io/io-memory-test.cc
index 7cb796a..d80aaec 100644
--- a/cpp/src/arrow/io/io-memory-test.cc
+++ b/cpp/src/arrow/io/io-memory-test.cc
@@ -37,12 +37,12 @@ namespace io {
 class TestBufferOutputStream : public ::testing::Test {
  public:
   void SetUp() {
-    buffer_.reset(new PoolBuffer(default_memory_pool()));
+    ASSERT_OK(AllocateResizableBuffer(0, &buffer_));
     stream_.reset(new BufferOutputStream(buffer_));
   }
 
  protected:
-  std::shared_ptr<PoolBuffer> buffer_;
+  std::shared_ptr<ResizableBuffer> buffer_;
   std::unique_ptr<OutputStream> stream_;
 };
 
@@ -135,11 +135,11 @@ TEST(TestMemcopy, ParallelMemcopy) {
     // randomize size so the memcopy alignment is tested
     int64_t total_size = 3 * 1024 * 1024 + std::rand() % 100;
 
-    auto buffer1 = std::make_shared<PoolBuffer>(default_memory_pool());
-    ASSERT_OK(buffer1->Resize(total_size));
+    std::shared_ptr<Buffer> buffer1, buffer2;
+
+    ASSERT_OK(AllocateBuffer(total_size, &buffer1));
+    ASSERT_OK(AllocateBuffer(total_size, &buffer2));
 
-    auto buffer2 = std::make_shared<PoolBuffer>(default_memory_pool());
-    ASSERT_OK(buffer2->Resize(total_size));
     test::random_bytes(total_size, 0, buffer2->mutable_data());
 
     io::FixedSizeBufferWriter writer(buffer1);
diff --git a/cpp/src/arrow/ipc/ipc-read-write-benchmark.cc b/cpp/src/arrow/ipc/ipc-read-write-benchmark.cc
index 8561fb8..2523394 100644
--- a/cpp/src/arrow/ipc/ipc-read-write-benchmark.cc
+++ b/cpp/src/arrow/ipc/ipc-read-write-benchmark.cc
@@ -70,8 +70,8 @@ static void BM_WriteRecordBatch(benchmark::State& state) {  // NOLINT non-const
   // 1MB
   constexpr int64_t kTotalSize = 1 << 20;
 
-  auto buffer = std::make_shared<PoolBuffer>(default_memory_pool());
-  ABORT_NOT_OK(buffer->Resize(kTotalSize & 2));
+  std::shared_ptr<ResizableBuffer> buffer;
+  ABORT_NOT_OK(AllocateResizableBuffer(kTotalSize & 2, &buffer));
   auto record_batch = MakeRecordBatch<Int64Type>(kTotalSize, state.range(0));
 
   while (state.KeepRunning()) {
@@ -91,8 +91,8 @@ static void BM_ReadRecordBatch(benchmark::State& state) {  // NOLINT non-const r
   // 1MB
   constexpr int64_t kTotalSize = 1 << 20;
 
-  auto buffer = std::make_shared<PoolBuffer>(default_memory_pool());
-  ABORT_NOT_OK(buffer->Resize(kTotalSize & 2));
+  std::shared_ptr<ResizableBuffer> buffer;
+  ABORT_NOT_OK(AllocateResizableBuffer(kTotalSize & 2, &buffer));
   auto record_batch = MakeRecordBatch<Int64Type>(kTotalSize, state.range(0));
 
   io::BufferOutputStream stream(buffer);
diff --git a/cpp/src/arrow/ipc/ipc-read-write-test.cc b/cpp/src/arrow/ipc/ipc-read-write-test.cc
index 6c116bc..baf067e 100644
--- a/cpp/src/arrow/ipc/ipc-read-write-test.cc
+++ b/cpp/src/arrow/ipc/ipc-read-write-test.cc
@@ -507,7 +507,7 @@ class TestFileFormat : public ::testing::TestWithParam<MakeRecordBatch*> {
  public:
   void SetUp() {
     pool_ = default_memory_pool();
-    buffer_ = std::make_shared<PoolBuffer>(pool_);
+    ASSERT_OK(AllocateResizableBuffer(pool_, 0, &buffer_));
     sink_.reset(new io::BufferOutputStream(buffer_));
   }
   void TearDown() {}
@@ -549,7 +549,7 @@ class TestFileFormat : public ::testing::TestWithParam<MakeRecordBatch*> {
   MemoryPool* pool_;
 
   std::unique_ptr<io::BufferOutputStream> sink_;
-  std::shared_ptr<PoolBuffer> buffer_;
+  std::shared_ptr<ResizableBuffer> buffer_;
 };
 
 TEST_P(TestFileFormat, RoundTrip) {
@@ -573,7 +573,7 @@ class TestStreamFormat : public ::testing::TestWithParam<MakeRecordBatch*> {
  public:
   void SetUp() {
     pool_ = default_memory_pool();
-    buffer_ = std::make_shared<PoolBuffer>(pool_);
+    ASSERT_OK(AllocateResizableBuffer(pool_, 0, &buffer_));
     sink_.reset(new io::BufferOutputStream(buffer_));
   }
   void TearDown() {}
@@ -611,7 +611,7 @@ class TestStreamFormat : public ::testing::TestWithParam<MakeRecordBatch*> {
   MemoryPool* pool_;
 
   std::unique_ptr<io::BufferOutputStream> sink_;
-  std::shared_ptr<PoolBuffer> buffer_;
+  std::shared_ptr<ResizableBuffer> buffer_;
 };
 
 TEST_P(TestStreamFormat, RoundTrip) {
diff --git a/cpp/src/arrow/ipc/test-common.h b/cpp/src/arrow/ipc/test-common.h
index 4e9fab1..4579615 100644
--- a/cpp/src/arrow/ipc/test-common.h
+++ b/cpp/src/arrow/ipc/test-common.h
@@ -97,13 +97,13 @@ const auto kListListInt32 = list(kListInt32);
 
 Status MakeRandomInt32Array(int64_t length, bool include_nulls, MemoryPool* pool,
                             std::shared_ptr<Array>* out) {
-  std::shared_ptr<PoolBuffer> data;
-  RETURN_NOT_OK(test::MakeRandomInt32PoolBuffer(length, pool, &data));
+  std::shared_ptr<ResizableBuffer> data;
+  RETURN_NOT_OK(test::MakeRandomInt32Buffer(length, pool, &data));
   Int32Builder builder(int32(), pool);
-  RETURN_NOT_OK(builder.Init(length));
+  RETURN_NOT_OK(builder.Resize(length));
   if (include_nulls) {
-    std::shared_ptr<PoolBuffer> valid_bytes;
-    RETURN_NOT_OK(test::MakeRandomBytePoolBuffer(length, pool, &valid_bytes));
+    std::shared_ptr<ResizableBuffer> valid_bytes;
+    RETURN_NOT_OK(test::MakeRandomByteBuffer(length, pool, &valid_bytes));
     RETURN_NOT_OK(builder.AppendValues(reinterpret_cast<const int32_t*>(data->data()),
                                        length, valid_bytes->data()));
     return builder.Finish(out);
diff --git a/cpp/src/arrow/python/numpy_to_arrow.cc b/cpp/src/arrow/python/numpy_to_arrow.cc
index 41bda76..09e6ec7 100644
--- a/cpp/src/arrow/python/numpy_to_arrow.cc
+++ b/cpp/src/arrow/python/numpy_to_arrow.cc
@@ -94,10 +94,9 @@ 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));
 
-  null_bitmap = std::make_shared<PoolBuffer>(pool);
-  RETURN_NOT_OK(null_bitmap->Resize(null_bytes));
-
+  // Padding zeroed by AllocateResizableBuffer
   memset(null_bitmap->mutable_data(), 0, static_cast<size_t>(null_bytes));
   *out = null_bitmap;
   return Status::OK();
diff --git a/cpp/src/arrow/table_builder.cc b/cpp/src/arrow/table_builder.cc
index 8e9babc..ef39d57 100644
--- a/cpp/src/arrow/table_builder.cc
+++ b/cpp/src/arrow/table_builder.cc
@@ -95,7 +95,7 @@ Status RecordBatchBuilder::CreateBuilders() {
 
 Status RecordBatchBuilder::InitBuilders() {
   for (int i = 0; i < this->num_fields(); ++i) {
-    RETURN_NOT_OK(raw_field_builders_[i]->Init(initial_capacity_));
+    RETURN_NOT_OK(raw_field_builders_[i]->Reserve(initial_capacity_));
   }
   return Status::OK();
 }
diff --git a/cpp/src/arrow/test-common.h b/cpp/src/arrow/test-common.h
index f09561e..fd43d53 100644
--- a/cpp/src/arrow/test-common.h
+++ b/cpp/src/arrow/test-common.h
@@ -42,9 +42,10 @@ class TestBase : public ::testing::Test {
   }
 
   std::shared_ptr<Buffer> MakeRandomNullBitmap(int64_t length, int64_t null_count) {
-    auto null_bitmap = std::make_shared<PoolBuffer>(pool_);
     const int64_t null_nbytes = BitUtil::BytesForBits(length);
-    EXPECT_OK(null_bitmap->Resize(null_nbytes));
+
+    std::shared_ptr<Buffer> null_bitmap;
+    EXPECT_OK(AllocateBuffer(pool_, null_nbytes, &null_bitmap));
     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));
diff --git a/cpp/src/arrow/test-util.h b/cpp/src/arrow/test-util.h
index 7109e64..ccd4ad7 100644
--- a/cpp/src/arrow/test-util.h
+++ b/cpp/src/arrow/test-util.h
@@ -284,27 +284,23 @@ static inline int64_t null_count(const std::vector<uint8_t>& valid_bytes) {
   return static_cast<int64_t>(std::count(valid_bytes.cbegin(), valid_bytes.cend(), '\0'));
 }
 
-Status MakeRandomInt32PoolBuffer(int64_t length, MemoryPool* pool,
-                                 std::shared_ptr<PoolBuffer>* pool_buffer,
-                                 uint32_t seed = 0) {
+Status MakeRandomInt32Buffer(int64_t length, MemoryPool* pool,
+                             std::shared_ptr<ResizableBuffer>* out, uint32_t seed = 0) {
   DCHECK(pool);
-  auto data = std::make_shared<PoolBuffer>(pool);
-  RETURN_NOT_OK(data->Resize(sizeof(int32_t) * length));
-  data->ZeroPadding();
+  std::shared_ptr<ResizableBuffer> result;
+  RETURN_NOT_OK(AllocateResizableBuffer(pool, sizeof(int32_t) * length, &result));
   test::rand_uniform_int(length, seed, 0, std::numeric_limits<int32_t>::max(),
-                         reinterpret_cast<int32_t*>(data->mutable_data()));
-  *pool_buffer = data;
+                         reinterpret_cast<int32_t*>(result->mutable_data()));
+  *out = result;
   return Status::OK();
 }
 
-Status MakeRandomBytePoolBuffer(int64_t length, MemoryPool* pool,
-                                std::shared_ptr<PoolBuffer>* pool_buffer,
-                                uint32_t seed = 0) {
-  auto bytes = std::make_shared<PoolBuffer>(pool);
-  RETURN_NOT_OK(bytes->Resize(length));
-  bytes->ZeroPadding();
-  test::random_bytes(length, seed, bytes->mutable_data());
-  *pool_buffer = bytes;
+Status MakeRandomByteBuffer(int64_t length, MemoryPool* pool,
+                            std::shared_ptr<ResizableBuffer>* out, uint32_t seed = 0) {
+  std::shared_ptr<ResizableBuffer> result;
+  RETURN_NOT_OK(AllocateResizableBuffer(pool, length, &result));
+  test::random_bytes(length, seed, result->mutable_data());
+  *out = result;
   return Status::OK();
 }
 
diff --git a/cpp/src/arrow/util/io-util.h b/cpp/src/arrow/util/io-util.h
index 9fbce39..62ae2a6 100644
--- a/cpp/src/arrow/util/io-util.h
+++ b/cpp/src/arrow/util/io-util.h
@@ -104,8 +104,8 @@ class StdinStream : public InputStream {
   }
 
   Status Read(int64_t nbytes, std::shared_ptr<Buffer>* out) override {
-    auto buffer = std::make_shared<PoolBuffer>(NULLPTR);
-    RETURN_NOT_OK(buffer->Resize(nbytes));
+    std::shared_ptr<ResizableBuffer> buffer;
+    RETURN_NOT_OK(AllocateResizableBuffer(nbytes, &buffer));
     int64_t bytes_read;
     RETURN_NOT_OK(Read(nbytes, &bytes_read, buffer->mutable_data()));
     RETURN_NOT_OK(buffer->Resize(bytes_read, false));
diff --git a/python/pyarrow/includes/libarrow.pxd b/python/pyarrow/includes/libarrow.pxd
index da46075..3a2668f 100644
--- a/python/pyarrow/includes/libarrow.pxd
+++ b/python/pyarrow/includes/libarrow.pxd
@@ -221,10 +221,6 @@ cdef extern from "arrow/api.h" namespace "arrow" nogil:
     CStatus AllocateResizableBuffer(CMemoryPool* pool, const int64_t size,
                                     shared_ptr[CResizableBuffer]* out)
 
-    cdef cppclass PoolBuffer(CResizableBuffer):
-        PoolBuffer()
-        PoolBuffer(CMemoryPool*)
-
     cdef CMemoryPool* c_default_memory_pool" arrow::default_memory_pool"()
 
     cdef cppclass CListType" arrow::ListType"(CDataType):
diff --git a/python/pyarrow/io.pxi b/python/pyarrow/io.pxi
index 264d1c3..4566477 100644
--- a/python/pyarrow/io.pxi
+++ b/python/pyarrow/io.pxi
@@ -892,9 +892,10 @@ cdef class ResizableBuffer(Buffer):
                          .Resize(new_size, c_shrink_to_fit))
 
 
-cdef shared_ptr[PoolBuffer] _allocate_buffer(CMemoryPool* pool):
-    cdef shared_ptr[PoolBuffer] result
-    result.reset(new PoolBuffer(pool))
+cdef shared_ptr[CResizableBuffer] _allocate_buffer(CMemoryPool* pool):
+    cdef shared_ptr[CResizableBuffer] result
+    with nogil:
+        check_status(AllocateResizableBuffer(pool, 0, &result))
     return result
 
 
@@ -933,7 +934,7 @@ def allocate_buffer(int64_t size, MemoryPool memory_pool=None,
 cdef class BufferOutputStream(NativeFile):
 
     cdef:
-        shared_ptr[PoolBuffer] buffer
+        shared_ptr[CResizableBuffer] buffer
 
     def __cinit__(self, MemoryPool memory_pool=None):
         self.buffer = _allocate_buffer(maybe_unbox_memory_pool(memory_pool))