You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@arrow.apache.org by ap...@apache.org on 2021/08/26 12:02:58 UTC

[arrow] branch master updated: ARROW-8022: [C++] Add static and small vector implementations

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

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


The following commit(s) were added to refs/heads/master by this push:
     new 257d0aa  ARROW-8022: [C++] Add static and small vector implementations
257d0aa is described below

commit 257d0aa936786c095b5560adb27bffaaccaed589
Author: Antoine Pitrou <an...@python.org>
AuthorDate: Thu Aug 26 14:01:25 2021 +0200

    ARROW-8022: [C++] Add static and small vector implementations
    
    Closes #10915 from pitrou/ARROW-8022-small-vector
    
    Authored-by: Antoine Pitrou <an...@python.org>
    Signed-off-by: Antoine Pitrou <an...@python.org>
---
 cpp/src/arrow/c/bridge.cc                    |  25 +-
 cpp/src/arrow/result.h                       |  52 +-
 cpp/src/arrow/testing/gtest_util.h           |  16 +-
 cpp/src/arrow/util/CMakeLists.txt            |   2 +
 cpp/src/arrow/util/aligned_storage.h         | 127 +++++
 cpp/src/arrow/util/launder.h                 |  35 ++
 cpp/src/arrow/util/macros.h                  |   6 +
 cpp/src/arrow/util/small_vector.h            | 519 ++++++++++++++++++
 cpp/src/arrow/util/small_vector_benchmark.cc | 344 ++++++++++++
 cpp/src/arrow/util/small_vector_test.cc      | 786 +++++++++++++++++++++++++++
 10 files changed, 1868 insertions(+), 44 deletions(-)

diff --git a/cpp/src/arrow/c/bridge.cc b/cpp/src/arrow/c/bridge.cc
index a43bf81..8b81534 100644
--- a/cpp/src/arrow/c/bridge.cc
+++ b/cpp/src/arrow/c/bridge.cc
@@ -38,6 +38,7 @@
 #include "arrow/util/key_value_metadata.h"
 #include "arrow/util/logging.h"
 #include "arrow/util/macros.h"
+#include "arrow/util/small_vector.h"
 #include "arrow/util/string_view.h"
 #include "arrow/util/value_parsing.h"
 #include "arrow/visitor_inline.h"
@@ -47,6 +48,9 @@ namespace arrow {
 using internal::checked_cast;
 using internal::checked_pointer_cast;
 
+using internal::SmallVector;
+using internal::StaticVector;
+
 using internal::ArrayExportGuard;
 using internal::ArrayExportTraits;
 using internal::SchemaExportGuard;
@@ -65,9 +69,6 @@ Status ExportingNotImplemented(const DataType& type) {
 
 // XXX use Gandiva's SimpleArena?
 
-template <typename T>
-using PoolVector = std::vector<T, ::arrow::stl::allocator<T>>;
-
 template <typename Derived>
 struct PoolAllocationMixin {
   static void* operator new(size_t size) {
@@ -90,8 +91,8 @@ struct ExportedSchemaPrivateData : PoolAllocationMixin<ExportedSchemaPrivateData
   std::string name_;
   std::string metadata_;
   struct ArrowSchema dictionary_;
-  PoolVector<struct ArrowSchema> children_;
-  PoolVector<struct ArrowSchema*> child_pointers_;
+  SmallVector<struct ArrowSchema, 1> children_;
+  SmallVector<struct ArrowSchema*, 4> child_pointers_;
 
   ExportedSchemaPrivateData() = default;
   ARROW_DEFAULT_MOVE_AND_ASSIGN(ExportedSchemaPrivateData);
@@ -225,7 +226,7 @@ struct SchemaExporter {
     c_struct->flags = flags_;
 
     c_struct->n_children = static_cast<int64_t>(child_exporters_.size());
-    c_struct->children = pdata->child_pointers_.data();
+    c_struct->children = c_struct->n_children ? pdata->child_pointers_.data() : nullptr;
     c_struct->dictionary = dict_exporter_ ? &pdata->dictionary_ : nullptr;
     c_struct->private_data = pdata;
     c_struct->release = ReleaseExportedSchema;
@@ -475,10 +476,10 @@ namespace {
 
 struct ExportedArrayPrivateData : PoolAllocationMixin<ExportedArrayPrivateData> {
   // The buffers are owned by the ArrayData member
-  PoolVector<const void*> buffers_;
+  StaticVector<const void*, 3> buffers_;
   struct ArrowArray dictionary_;
-  PoolVector<struct ArrowArray> children_;
-  PoolVector<struct ArrowArray*> child_pointers_;
+  SmallVector<struct ArrowArray, 1> children_;
+  SmallVector<struct ArrowArray*, 4> child_pointers_;
 
   std::shared_ptr<ArrayData> data_;
 
@@ -574,7 +575,7 @@ struct ArrayExporter {
     c_struct_->n_buffers = static_cast<int64_t>(pdata->buffers_.size());
     c_struct_->n_children = static_cast<int64_t>(pdata->child_pointers_.size());
     c_struct_->buffers = pdata->buffers_.data();
-    c_struct_->children = pdata->child_pointers_.data();
+    c_struct_->children = c_struct_->n_children ? pdata->child_pointers_.data() : nullptr;
     c_struct_->dictionary = dict_exporter_ ? &pdata->dictionary_ : nullptr;
     c_struct_->private_data = pdata;
     c_struct_->release = ReleaseExportedArray;
@@ -687,8 +688,8 @@ class FormatStringParser {
     }
   }
 
-  std::vector<util::string_view> Split(util::string_view v, char delim = ',') {
-    std::vector<util::string_view> parts;
+  SmallVector<util::string_view, 2> Split(util::string_view v, char delim = ',') {
+    SmallVector<util::string_view, 2> parts;
     size_t start = 0, end;
     while (true) {
       end = v.find_first_of(delim, start);
diff --git a/cpp/src/arrow/result.h b/cpp/src/arrow/result.h
index 3ef4f8c..7fdbeea 100644
--- a/cpp/src/arrow/result.h
+++ b/cpp/src/arrow/result.h
@@ -25,6 +25,7 @@
 #include <utility>
 
 #include "arrow/status.h"
+#include "arrow/util/aligned_storage.h"
 #include "arrow/util/compare.h"
 
 namespace arrow {
@@ -34,15 +35,6 @@ struct EnsureResult;
 
 namespace internal {
 
-#if __cplusplus >= 201703L
-using std::launder;
-#else
-template <class T>
-constexpr T* launder(T* p) noexcept {
-  return p;
-}
-#endif
-
 ARROW_EXPORT void DieWithMessage(const std::string& msg);
 
 ARROW_EXPORT void InvalidValueOrDie(const Status& st);
@@ -119,7 +111,7 @@ class ARROW_MUST_USE_TYPE Result : public util::EqualityComparable<Result<T>> {
   /// `Result<std::vector<int>>`. While `return {}` seems like it would return
   /// an empty vector, it will actually invoke the default constructor of
   /// Result.
-  explicit Result()  // NOLINT(runtime/explicit)
+  explicit Result() noexcept  // NOLINT(runtime/explicit)
       : status_(Status::UnknownError("Uninitialized Result<T>")) {}
 
   ~Result() noexcept { Destroy(); }
@@ -134,7 +126,7 @@ class ARROW_MUST_USE_TYPE Result : public util::EqualityComparable<Result<T>> {
   /// convenience.
   ///
   /// \param status The non-OK Status object to initialize to.
-  Result(const Status& status)  // NOLINT(runtime/explicit)
+  Result(const Status& status) noexcept  // NOLINT(runtime/explicit)
       : status_(status) {
     if (ARROW_PREDICT_FALSE(status.ok())) {
       internal::DieWithMessage(std::string("Constructed with a non-error status: ") +
@@ -196,7 +188,7 @@ class ARROW_MUST_USE_TYPE Result : public util::EqualityComparable<Result<T>> {
   /// object results in a compilation error.
   ///
   /// \param other The value to copy from.
-  Result(const Result& other) : status_(other.status_) {
+  Result(const Result& other) noexcept : status_(other.status_) {
     if (ARROW_PREDICT_TRUE(status_.ok())) {
       ConstructValue(other.ValueUnsafe());
     }
@@ -211,7 +203,7 @@ class ARROW_MUST_USE_TYPE Result : public util::EqualityComparable<Result<T>> {
   template <typename U, typename E = typename std::enable_if<
                             std::is_constructible<T, const U&>::value &&
                             std::is_convertible<U, T>::value>::type>
-  Result(const Result<U>& other) : status_(other.status_) {
+  Result(const Result<U>& other) noexcept : status_(other.status_) {
     if (ARROW_PREDICT_TRUE(status_.ok())) {
       ConstructValue(other.ValueUnsafe());
     }
@@ -220,9 +212,9 @@ class ARROW_MUST_USE_TYPE Result : public util::EqualityComparable<Result<T>> {
   /// Copy-assignment operator.
   ///
   /// \param other The Result object to copy.
-  Result& operator=(const Result& other) {
+  Result& operator=(const Result& other) noexcept {
     // Check for self-assignment.
-    if (this == &other) {
+    if (ARROW_PREDICT_FALSE(this == &other)) {
       return *this;
     }
     Destroy();
@@ -263,7 +255,7 @@ class ARROW_MUST_USE_TYPE Result : public util::EqualityComparable<Result<T>> {
   /// status.
   Result& operator=(Result&& other) noexcept {
     // Check for self-assignment.
-    if (this == &other) {
+    if (ARROW_PREDICT_FALSE(this == &other)) {
       return *this;
     }
     Destroy();
@@ -293,7 +285,7 @@ class ARROW_MUST_USE_TYPE Result : public util::EqualityComparable<Result<T>> {
   /// \return True if this Result object's status is OK (i.e. a call to ok()
   /// returns true). If this function returns true, then it is safe to access
   /// the wrapped element through a call to ValueOrDie().
-  bool ok() const { return status_.ok(); }
+  constexpr bool ok() const { return status_.ok(); }
 
   /// \brief Equivalent to ok().
   // operator bool() const { return ok(); }
@@ -302,7 +294,7 @@ class ARROW_MUST_USE_TYPE Result : public util::EqualityComparable<Result<T>> {
   ///
   /// \return The stored non-OK status object, or an OK status if this object
   ///         has a value.
-  const Status& status() const { return status_; }
+  constexpr const Status& status() const { return status_; }
 
   /// Gets the stored `T` value.
   ///
@@ -426,32 +418,32 @@ class ARROW_MUST_USE_TYPE Result : public util::EqualityComparable<Result<T>> {
     return U(ValueUnsafe());
   }
 
-  const T& ValueUnsafe() const& {
-    return *internal::launder(reinterpret_cast<const T*>(&data_));
-  }
+  constexpr const T& ValueUnsafe() const& { return *storage_.get(); }
 
-  T& ValueUnsafe() & { return *internal::launder(reinterpret_cast<T*>(&data_)); }
+#if __cpp_constexpr >= 201304L  // non-const constexpr
+  constexpr T& ValueUnsafe() & { return *storage_.get(); }
+#else
+  T& ValueUnsafe() & { return *storage_.get(); }
+#endif
 
   T ValueUnsafe() && { return MoveValueUnsafe(); }
 
-  T MoveValueUnsafe() {
-    return std::move(*internal::launder(reinterpret_cast<T*>(&data_)));
-  }
+  T MoveValueUnsafe() { return std::move(*storage_.get()); }
 
  private:
   Status status_;  // pointer-sized
-  typename std::aligned_storage<sizeof(T), alignof(T)>::type data_;
+  internal::AlignedStorage<T> storage_;
 
   template <typename U>
-  void ConstructValue(U&& u) {
-    new (&data_) T(std::forward<U>(u));
+  void ConstructValue(U&& u) noexcept {
+    storage_.construct(std::forward<U>(u));
   }
 
-  void Destroy() {
+  void Destroy() noexcept {
     if (ARROW_PREDICT_TRUE(status_.ok())) {
       static_assert(offsetof(Result<T>, status_) == 0,
                     "Status is guaranteed to be at the start of Result<>");
-      internal::launder(reinterpret_cast<const T*>(&data_))->~T();
+      storage_.destroy();
     }
   }
 };
diff --git a/cpp/src/arrow/testing/gtest_util.h b/cpp/src/arrow/testing/gtest_util.h
index a58064e..55734d2 100644
--- a/cpp/src/arrow/testing/gtest_util.h
+++ b/cpp/src/arrow/testing/gtest_util.h
@@ -559,7 +559,7 @@ void PrintTo(const Result<T>& result, std::ostream* os) {
   }
 }
 
-// A data type with only move constructors.
+// A data type with only move constructors (no copy, no default).
 struct MoveOnlyDataType {
   explicit MoveOnlyDataType(int x) : data(new int(x)) {}
 
@@ -572,6 +572,14 @@ struct MoveOnlyDataType {
     return *this;
   }
 
+  MoveOnlyDataType& operator=(int x) {
+    if (data != nullptr) {
+      delete data;
+    }
+    data = new int(x);
+    return *this;
+  }
+
   ~MoveOnlyDataType() { Destroy(); }
 
   void Destroy() {
@@ -591,10 +599,14 @@ struct MoveOnlyDataType {
 
   int ToInt() const { return data == nullptr ? -42 : *data; }
 
-  bool operator==(int other) const { return data != nullptr && *data == other; }
   bool operator==(const MoveOnlyDataType& other) const {
     return data != nullptr && other.data != nullptr && *data == *other.data;
   }
+  bool operator<(const MoveOnlyDataType& other) const {
+    return data == nullptr || (other.data != nullptr && *data < *other.data);
+  }
+
+  bool operator==(int other) const { return data != nullptr && *data == other; }
   friend bool operator==(int left, const MoveOnlyDataType& right) {
     return right == left;
   }
diff --git a/cpp/src/arrow/util/CMakeLists.txt b/cpp/src/arrow/util/CMakeLists.txt
index 660fb26..aa875ab 100644
--- a/cpp/src/arrow/util/CMakeLists.txt
+++ b/cpp/src/arrow/util/CMakeLists.txt
@@ -59,6 +59,7 @@ add_arrow_test(utility-test
                range_test.cc
                reflection_test.cc
                rle_encoding_test.cc
+               small_vector_test.cc
                stl_util_test.cc
                string_test.cc
                tdigest_test.cc
@@ -88,6 +89,7 @@ add_arrow_benchmark(int_util_benchmark)
 add_arrow_benchmark(machine_benchmark)
 add_arrow_benchmark(queue_benchmark)
 add_arrow_benchmark(range_benchmark)
+add_arrow_benchmark(small_vector_benchmark)
 add_arrow_benchmark(tdigest_benchmark)
 add_arrow_benchmark(thread_pool_benchmark)
 add_arrow_benchmark(trie_benchmark)
diff --git a/cpp/src/arrow/util/aligned_storage.h b/cpp/src/arrow/util/aligned_storage.h
new file mode 100644
index 0000000..f6acb36
--- /dev/null
+++ b/cpp/src/arrow/util/aligned_storage.h
@@ -0,0 +1,127 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#pragma once
+
+#include <cstring>
+#include <type_traits>
+#include <utility>
+
+#include "arrow/util/launder.h"
+#include "arrow/util/macros.h"
+
+namespace arrow {
+namespace internal {
+
+template <typename T>
+class AlignedStorage {
+ public:
+  static constexpr bool can_memcpy = std::is_trivial<T>::value;
+
+#if __cpp_constexpr >= 201304L  // non-const constexpr
+  constexpr T* get() noexcept { return launder(reinterpret_cast<T*>(&data_)); }
+#else
+  T* get() noexcept { return launder(reinterpret_cast<T*>(&data_)); }
+#endif
+
+  constexpr const T* get() const noexcept {
+    return launder(reinterpret_cast<const T*>(&data_));
+  }
+
+  void destroy() noexcept {
+    if (!std::is_trivially_destructible<T>::value) {
+      get()->~T();
+    }
+  }
+
+  template <typename... A>
+  void construct(A&&... args) noexcept {
+    new (&data_) T(std::forward<A>(args)...);
+  }
+
+  template <typename V>
+  void assign(V&& v) noexcept {
+    *get() = std::forward<V>(v);
+  }
+
+  void move_construct(AlignedStorage* other) noexcept {
+    new (&data_) T(std::move(*other->get()));
+  }
+
+  void move_assign(AlignedStorage* other) noexcept { *get() = std::move(*other->get()); }
+
+  template <bool CanMemcpy = can_memcpy>
+  static typename std::enable_if<CanMemcpy>::type move_construct_several(
+      AlignedStorage* ARROW_RESTRICT src, AlignedStorage* ARROW_RESTRICT dest, size_t n,
+      size_t memcpy_length) noexcept {
+    memcpy(dest->get(), src->get(), memcpy_length * sizeof(T));
+  }
+
+  template <bool CanMemcpy = can_memcpy>
+  static typename std::enable_if<CanMemcpy>::type
+  move_construct_several_and_destroy_source(AlignedStorage* ARROW_RESTRICT src,
+                                            AlignedStorage* ARROW_RESTRICT dest, size_t n,
+                                            size_t memcpy_length) noexcept {
+    memcpy(dest->get(), src->get(), memcpy_length * sizeof(T));
+  }
+
+  template <bool CanMemcpy = can_memcpy>
+  static typename std::enable_if<!CanMemcpy>::type move_construct_several(
+      AlignedStorage* ARROW_RESTRICT src, AlignedStorage* ARROW_RESTRICT dest, size_t n,
+      size_t memcpy_length) noexcept {
+    for (size_t i = 0; i < n; ++i) {
+      new (dest[i].get()) T(std::move(*src[i].get()));
+    }
+  }
+
+  template <bool CanMemcpy = can_memcpy>
+  static typename std::enable_if<!CanMemcpy>::type
+  move_construct_several_and_destroy_source(AlignedStorage* ARROW_RESTRICT src,
+                                            AlignedStorage* ARROW_RESTRICT dest, size_t n,
+                                            size_t memcpy_length) noexcept {
+    for (size_t i = 0; i < n; ++i) {
+      new (dest[i].get()) T(std::move(*src[i].get()));
+      src[i].destroy();
+    }
+  }
+
+  static void move_construct_several(AlignedStorage* ARROW_RESTRICT src,
+                                     AlignedStorage* ARROW_RESTRICT dest,
+                                     size_t n) noexcept {
+    move_construct_several(src, dest, n, n);
+  }
+
+  static void move_construct_several_and_destroy_source(
+      AlignedStorage* ARROW_RESTRICT src, AlignedStorage* ARROW_RESTRICT dest,
+      size_t n) noexcept {
+    move_construct_several_and_destroy_source(src, dest, n, n);
+  }
+
+  static void destroy_several(AlignedStorage* p, size_t n) noexcept {
+    if (!std::is_trivially_destructible<T>::value) {
+      for (size_t i = 0; i < n; ++i) {
+        p[i].destroy();
+      }
+    }
+  }
+
+ private:
+  typename std::aligned_storage<sizeof(T), alignof(T)>::type data_;
+};
+
+}  // namespace internal
+}  // namespace arrow
diff --git a/cpp/src/arrow/util/launder.h b/cpp/src/arrow/util/launder.h
new file mode 100644
index 0000000..37e2a71
--- /dev/null
+++ b/cpp/src/arrow/util/launder.h
@@ -0,0 +1,35 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#pragma once
+
+#include <new>
+
+namespace arrow {
+namespace internal {
+
+#if __cplusplus >= 201703L
+using std::launder;
+#else
+template <class T>
+constexpr T* launder(T* p) noexcept {
+  return p;
+}
+#endif
+
+}  // namespace internal
+}  // namespace arrow
diff --git a/cpp/src/arrow/util/macros.h b/cpp/src/arrow/util/macros.h
index afa7201..5858f9e 100644
--- a/cpp/src/arrow/util/macros.h
+++ b/cpp/src/arrow/util/macros.h
@@ -78,6 +78,12 @@
 #define ARROW_MUST_USE_TYPE
 #endif
 
+#if defined(__GNUC__) || defined(__clang__) || defined(_MSC_VER)
+#define ARROW_RESTRICT __restrict
+#else
+#define ARROW_RESTRICT
+#endif
+
 // ----------------------------------------------------------------------
 // C++/CLI support macros (see ARROW-1134)
 
diff --git a/cpp/src/arrow/util/small_vector.h b/cpp/src/arrow/util/small_vector.h
new file mode 100644
index 0000000..0712952
--- /dev/null
+++ b/cpp/src/arrow/util/small_vector.h
@@ -0,0 +1,519 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#pragma once
+
+#include <algorithm>
+#include <cassert>
+#include <cstddef>
+#include <initializer_list>
+#include <iterator>
+#include <limits>
+#include <new>
+#include <type_traits>
+#include <utility>
+
+#include "arrow/util/aligned_storage.h"
+#include "arrow/util/macros.h"
+
+namespace arrow {
+namespace internal {
+
+template <typename T, size_t N, bool NonTrivialDestructor>
+struct StaticVectorStorageBase {
+  using storage_type = AlignedStorage<T>;
+
+  storage_type static_data_[N];
+  size_t size_ = 0;
+
+  void destroy() noexcept {}
+};
+
+template <typename T, size_t N>
+struct StaticVectorStorageBase<T, N, true> {
+  using storage_type = AlignedStorage<T>;
+
+  storage_type static_data_[N];
+  size_t size_ = 0;
+
+  ~StaticVectorStorageBase() noexcept { destroy(); }
+
+  void destroy() noexcept { storage_type::destroy_several(static_data_, size_); }
+};
+
+template <typename T, size_t N, bool D = !std::is_trivially_destructible<T>::value>
+struct StaticVectorStorage : public StaticVectorStorageBase<T, N, D> {
+  using Base = StaticVectorStorageBase<T, N, D>;
+  using typename Base::storage_type;
+
+  using Base::size_;
+  using Base::static_data_;
+
+  StaticVectorStorage() noexcept = default;
+
+#if __cpp_constexpr >= 201304L  // non-const constexpr
+  constexpr storage_type* storage_ptr() { return static_data_; }
+#else
+  storage_type* storage_ptr() { return static_data_; }
+#endif
+
+  constexpr const storage_type* const_storage_ptr() const { return static_data_; }
+
+  // Adjust storage size, but don't initialize any objects
+  void bump_size(size_t addend) {
+    assert(size_ + addend <= N);
+    size_ += addend;
+  }
+
+  void ensure_capacity(size_t min_capacity) { assert(min_capacity <= N); }
+
+  // Adjust storage size, but don't destroy any objects
+  void reduce_size(size_t reduce_by) {
+    assert(reduce_by <= size_);
+    size_ -= reduce_by;
+  }
+
+  // Move objects from another storage, but don't destroy any objects currently
+  // stored in *this.
+  // You need to call destroy() first if necessary (e.g. in a
+  // move assignment operator).
+  void move_construct(StaticVectorStorage&& other) noexcept {
+    size_ = other.size_;
+    if (size_ != 0) {
+      // Use a compile-time memcpy size (N) for trivial types
+      storage_type::move_construct_several(other.static_data_, static_data_, size_, N);
+    }
+  }
+
+  constexpr size_t capacity() const { return N; }
+
+  constexpr size_t max_size() const { return N; }
+
+  void reserve(size_t n) {}
+
+  void clear() {
+    storage_type::destroy_several(static_data_, size_);
+    size_ = 0;
+  }
+};
+
+template <typename T, size_t N>
+struct SmallVectorStorage {
+  using storage_type = AlignedStorage<T>;
+
+  storage_type static_data_[N];
+  size_t size_ = 0;
+  storage_type* data_ = static_data_;
+  size_t dynamic_capacity_ = 0;
+
+  SmallVectorStorage() noexcept = default;
+
+  ~SmallVectorStorage() { destroy(); }
+
+#if __cpp_constexpr >= 201304L  // non-const constexpr
+  constexpr storage_type* storage_ptr() { return data_; }
+#else
+  storage_type* storage_ptr() { return data_; }
+#endif
+
+  constexpr const storage_type* const_storage_ptr() const { return data_; }
+
+  void bump_size(size_t addend) {
+    const size_t new_size = size_ + addend;
+    ensure_capacity(new_size);
+    size_ = new_size;
+  }
+
+  void ensure_capacity(size_t min_capacity) {
+    if (dynamic_capacity_) {
+      // Grow dynamic storage if necessary
+      if (min_capacity > dynamic_capacity_) {
+        size_t new_capacity = std::max(dynamic_capacity_ * 2, min_capacity);
+        reallocate_dynamic(new_capacity);
+      }
+    } else if (min_capacity > N) {
+      switch_to_dynamic(min_capacity);
+    }
+  }
+
+  void reduce_size(size_t reduce_by) {
+    assert(reduce_by <= size_);
+    size_ -= reduce_by;
+  }
+
+  void destroy() noexcept {
+    storage_type::destroy_several(data_, size_);
+    if (dynamic_capacity_) {
+      delete[] data_;
+    }
+  }
+
+  void move_construct(SmallVectorStorage&& other) noexcept {
+    size_ = other.size_;
+    dynamic_capacity_ = other.dynamic_capacity_;
+    if (dynamic_capacity_) {
+      data_ = other.data_;
+      other.data_ = other.static_data_;
+      other.dynamic_capacity_ = 0;
+      other.size_ = 0;
+    } else if (size_ != 0) {
+      // Use a compile-time memcpy size (N) for trivial types
+      storage_type::move_construct_several(other.static_data_, static_data_, size_, N);
+    }
+  }
+
+  constexpr size_t capacity() const { return dynamic_capacity_ ? dynamic_capacity_ : N; }
+
+  constexpr size_t max_size() const { return std::numeric_limits<size_t>::max(); }
+
+  void reserve(size_t n) {
+    if (dynamic_capacity_) {
+      if (n > dynamic_capacity_) {
+        reallocate_dynamic(n);
+      }
+    } else if (n > N) {
+      switch_to_dynamic(n);
+    }
+  }
+
+  void clear() {
+    storage_type::destroy_several(data_, size_);
+    size_ = 0;
+  }
+
+ private:
+  void switch_to_dynamic(size_t new_capacity) {
+    dynamic_capacity_ = new_capacity;
+    data_ = new storage_type[new_capacity];
+    storage_type::move_construct_several_and_destroy_source(static_data_, data_, size_);
+  }
+
+  void reallocate_dynamic(size_t new_capacity) {
+    assert(new_capacity >= size_);
+    auto new_data = new storage_type[new_capacity];
+    storage_type::move_construct_several_and_destroy_source(data_, new_data, size_);
+    delete[] data_;
+    dynamic_capacity_ = new_capacity;
+    data_ = new_data;
+  }
+};
+
+template <typename T, size_t N, typename Storage>
+class StaticVectorImpl {
+ private:
+  Storage storage_;
+
+  T* data_ptr() { return storage_.storage_ptr()->get(); }
+
+  constexpr const T* const_data_ptr() const {
+    return storage_.const_storage_ptr()->get();
+  }
+
+ public:
+  using size_type = size_t;
+  using difference_type = ptrdiff_t;
+  using value_type = T;
+  using pointer = T*;
+  using const_pointer = const T*;
+  using reference = T&;
+  using const_reference = const T&;
+  using iterator = T*;
+  using const_iterator = const T*;
+  using reverse_iterator = std::reverse_iterator<iterator>;
+  using const_reverse_iterator = std::reverse_iterator<const_iterator>;
+
+  constexpr StaticVectorImpl() noexcept = default;
+
+  // Move and copy constructors
+  StaticVectorImpl(StaticVectorImpl&& other) noexcept {
+    storage_.move_construct(std::move(other.storage_));
+  }
+
+  StaticVectorImpl& operator=(StaticVectorImpl&& other) noexcept {
+    if (ARROW_PREDICT_TRUE(&other != this)) {
+      // TODO move_assign?
+      storage_.destroy();
+      storage_.move_construct(std::move(other.storage_));
+    }
+    return *this;
+  }
+
+  StaticVectorImpl(const StaticVectorImpl& other) {
+    init_by_copying(other.storage_.size_, other.const_data_ptr());
+  }
+
+  StaticVectorImpl& operator=(const StaticVectorImpl& other) noexcept {
+    if (ARROW_PREDICT_TRUE(&other != this)) {
+      assign_by_copying(other.storage_.size_, other.data());
+    }
+    return *this;
+  }
+
+  // Automatic conversion from std::vector<T>, for convenience
+  StaticVectorImpl(const std::vector<T>& other) {  // NOLINT: explicit
+    init_by_copying(other.size(), other.data());
+  }
+
+  StaticVectorImpl(std::vector<T>&& other) noexcept {  // NOLINT: explicit
+    init_by_moving(other.size(), other.data());
+  }
+
+  StaticVectorImpl& operator=(const std::vector<T>& other) {
+    assign_by_copying(other.size(), other.data());
+    return *this;
+  }
+
+  StaticVectorImpl& operator=(std::vector<T>&& other) noexcept {
+    assign_by_moving(other.size(), other.data());
+    return *this;
+  }
+
+  // Constructing from count and optional initialization value
+  explicit StaticVectorImpl(size_t count) {
+    storage_.bump_size(count);
+    auto* p = storage_.storage_ptr();
+    for (size_t i = 0; i < count; ++i) {
+      p[i].construct();
+    }
+  }
+
+  StaticVectorImpl(size_t count, const T& value) {
+    storage_.bump_size(count);
+    auto* p = storage_.storage_ptr();
+    for (size_t i = 0; i < count; ++i) {
+      p[i].construct(value);
+    }
+  }
+
+  StaticVectorImpl(std::initializer_list<T> values) {
+    storage_.bump_size(values.size());
+    auto* p = storage_.storage_ptr();
+    for (auto&& v : values) {
+      // Unfortunately, cannot move initializer values
+      p++->construct(v);
+    }
+  }
+
+  // Size inspection
+
+  constexpr bool empty() const { return storage_.size_ == 0; }
+
+  constexpr size_t size() const { return storage_.size_; }
+
+  constexpr size_t capacity() const { return storage_.capacity(); }
+
+  constexpr size_t max_size() const { return storage_.max_size(); }
+
+  // Data access
+
+  T& operator[](size_t i) { return data_ptr()[i]; }
+
+  constexpr const T& operator[](size_t i) const { return const_data_ptr()[i]; }
+
+  T& front() { return data_ptr()[0]; }
+
+  constexpr const T& front() const { return const_data_ptr()[0]; }
+
+  T& back() { return data_ptr()[storage_.size_ - 1]; }
+
+  constexpr const T& back() const { return const_data_ptr()[storage_.size_ - 1]; }
+
+  T* data() { return data_ptr(); }
+
+  constexpr const T* data() const { return const_data_ptr(); }
+
+  // Iterators
+
+  iterator begin() { return iterator(data_ptr()); }
+
+  constexpr const_iterator begin() const { return const_iterator(const_data_ptr()); }
+
+  constexpr const_iterator cbegin() const { return const_iterator(const_data_ptr()); }
+
+  iterator end() { return iterator(data_ptr() + storage_.size_); }
+
+  constexpr const_iterator end() const {
+    return const_iterator(const_data_ptr() + storage_.size_);
+  }
+
+  constexpr const_iterator cend() const {
+    return const_iterator(const_data_ptr() + storage_.size_);
+  }
+
+  reverse_iterator rbegin() { return reverse_iterator(end()); }
+
+  constexpr const_reverse_iterator rbegin() const {
+    return const_reverse_iterator(end());
+  }
+
+  constexpr const_reverse_iterator crbegin() const {
+    return const_reverse_iterator(end());
+  }
+
+  reverse_iterator rend() { return reverse_iterator(begin()); }
+
+  constexpr const_reverse_iterator rend() const {
+    return const_reverse_iterator(begin());
+  }
+
+  constexpr const_reverse_iterator crend() const {
+    return const_reverse_iterator(begin());
+  }
+
+  // Mutations
+
+  void reserve(size_t n) { storage_.reserve(n); }
+
+  void clear() { storage_.clear(); }
+
+  void push_back(const T& value) {
+    storage_.bump_size(1);
+    storage_.storage_ptr()[storage_.size_ - 1].construct(value);
+  }
+
+  void push_back(T&& value) {
+    storage_.bump_size(1);
+    storage_.storage_ptr()[storage_.size_ - 1].construct(std::move(value));
+  }
+
+  template <typename... Args>
+  void emplace_back(Args&&... args) {
+    storage_.bump_size(1);
+    storage_.storage_ptr()[storage_.size_ - 1].construct(std::forward<Args>(args)...);
+  }
+
+  template <typename InputIt>
+  iterator insert(const_iterator insert_at, InputIt first, InputIt last) {
+    const size_t n = storage_.size_;
+    const size_t it_size = static_cast<size_t>(last - first);  // XXX might be O(n)?
+    const size_t pos = static_cast<size_t>(insert_at - const_data_ptr());
+    storage_.bump_size(it_size);
+    auto* p = storage_.storage_ptr();
+    if (it_size == 0) {
+      return p[pos].get();
+    }
+    const size_t end_pos = pos + it_size;
+
+    // Move [pos; n) to [end_pos; end_pos + n - pos)
+    size_t i = n;
+    size_t j = end_pos + n - pos;
+    while (j > std::max(n, end_pos)) {
+      p[--j].move_construct(&p[--i]);
+    }
+    while (j > end_pos) {
+      p[--j].move_assign(&p[--i]);
+    }
+    assert(j == end_pos);
+    // Copy [first; last) to [pos; end_pos)
+    j = pos;
+    while (j < std::min(n, end_pos)) {
+      p[j++].assign(*first++);
+    }
+    while (j < end_pos) {
+      p[j++].construct(*first++);
+    }
+    assert(first == last);
+    return p[pos].get();
+  }
+
+  void resize(size_t n) {
+    const size_t old_size = storage_.size_;
+    if (n > storage_.size_) {
+      storage_.bump_size(n - old_size);
+      auto* p = storage_.storage_ptr();
+      for (size_t i = old_size; i < n; ++i) {
+        p[i].construct(T{});
+      }
+    } else {
+      auto* p = storage_.storage_ptr();
+      for (size_t i = n; i < old_size; ++i) {
+        p[i].destroy();
+      }
+      storage_.reduce_size(old_size - n);
+    }
+  }
+
+  void resize(size_t n, const T& value) {
+    const size_t old_size = storage_.size_;
+    if (n > storage_.size_) {
+      storage_.bump_size(n - old_size);
+      auto* p = storage_.storage_ptr();
+      for (size_t i = old_size; i < n; ++i) {
+        p[i].construct(value);
+      }
+    } else {
+      auto* p = storage_.storage_ptr();
+      for (size_t i = n; i < old_size; ++i) {
+        p[i].destroy();
+      }
+      storage_.reduce_size(old_size - n);
+    }
+  }
+
+ private:
+  template <typename InputIt>
+  void init_by_copying(size_t n, InputIt src) {
+    storage_.bump_size(n);
+    auto* dest = storage_.storage_ptr();
+    for (size_t i = 0; i < n; ++i, ++src) {
+      dest[i].construct(*src);
+    }
+  }
+
+  template <typename InputIt>
+  void init_by_moving(size_t n, InputIt src) {
+    init_by_copying(n, std::make_move_iterator(src));
+  }
+
+  template <typename InputIt>
+  void assign_by_copying(size_t n, InputIt src) {
+    const size_t old_size = storage_.size_;
+    if (n > old_size) {
+      storage_.bump_size(n - old_size);
+      auto* dest = storage_.storage_ptr();
+      for (size_t i = 0; i < old_size; ++i, ++src) {
+        dest[i].assign(*src);
+      }
+      for (size_t i = old_size; i < n; ++i, ++src) {
+        dest[i].construct(*src);
+      }
+    } else {
+      auto* dest = storage_.storage_ptr();
+      for (size_t i = 0; i < n; ++i, ++src) {
+        dest[i].assign(*src);
+      }
+      for (size_t i = n; i < old_size; ++i) {
+        dest[i].destroy();
+      }
+      storage_.reduce_size(old_size - n);
+    }
+  }
+
+  template <typename InputIt>
+  void assign_by_moving(size_t n, InputIt src) {
+    assign_by_copying(n, std::make_move_iterator(src));
+  }
+};
+
+template <typename T, size_t N>
+using StaticVector = StaticVectorImpl<T, N, StaticVectorStorage<T, N>>;
+
+template <typename T, size_t N>
+using SmallVector = StaticVectorImpl<T, N, SmallVectorStorage<T, N>>;
+
+}  // namespace internal
+}  // namespace arrow
diff --git a/cpp/src/arrow/util/small_vector_benchmark.cc b/cpp/src/arrow/util/small_vector_benchmark.cc
new file mode 100644
index 0000000..96f94c3
--- /dev/null
+++ b/cpp/src/arrow/util/small_vector_benchmark.cc
@@ -0,0 +1,344 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <algorithm>
+#include <cstdint>
+#include <iterator>
+#include <memory>
+#include <numeric>
+#include <string>
+#include <vector>
+
+#include <benchmark/benchmark.h>
+
+#include "arrow/testing/util.h"
+#include "arrow/util/logging.h"
+#include "arrow/util/macros.h"
+#include "arrow/util/small_vector.h"
+
+namespace arrow {
+namespace internal {
+
+template <typename T>
+T ValueInitializer();
+template <typename T>
+T ValueInitializer(int seed);
+
+template <>
+int ValueInitializer<int>() {
+  return 42;
+}
+template <>
+int ValueInitializer<int>(int seed) {
+  return 42;
+}
+
+template <>
+std::string ValueInitializer<std::string>() {
+  return "42";
+}
+template <>
+std::string ValueInitializer<std::string>(int seed) {
+  return std::string("x", seed & 0x3f);  // avoid making string too long
+}
+
+template <>
+std::shared_ptr<int> ValueInitializer<std::shared_ptr<int>>() {
+  return std::make_shared<int>(42);
+}
+template <>
+std::shared_ptr<int> ValueInitializer<std::shared_ptr<int>>(int seed) {
+  return std::make_shared<int>(seed);
+}
+
+template <typename Vector>
+ARROW_NOINLINE int64_t ConsumeVector(Vector v) {
+  return reinterpret_cast<intptr_t>(v.data());
+}
+
+template <typename Vector>
+ARROW_NOINLINE int64_t IngestVector(const Vector& v) {
+  return reinterpret_cast<intptr_t>(v.data());
+}
+
+// With ARROW_NOINLINE, try to make sure the number of items is not constant-propagated
+template <typename Vector>
+ARROW_NOINLINE void BenchmarkMoveVector(benchmark::State& state, Vector vec) {
+  constexpr int kNumIters = 1000;
+
+  for (auto _ : state) {
+    int64_t dummy = 0;
+    for (int i = 0; i < kNumIters; ++i) {
+      Vector tmp(std::move(vec));
+      dummy += IngestVector(tmp);
+      vec = std::move(tmp);
+    }
+    benchmark::DoNotOptimize(dummy);
+  }
+
+  state.SetItemsProcessed(state.iterations() * kNumIters * 2);
+}
+
+template <typename Vector>
+void MoveEmptyVector(benchmark::State& state) {
+  BenchmarkMoveVector(state, Vector{});
+}
+
+template <typename Vector>
+void MoveShortVector(benchmark::State& state) {
+  using T = typename Vector::value_type;
+  constexpr int kSize = 3;
+  const auto initializer = ValueInitializer<T>();
+
+  BenchmarkMoveVector(state, Vector(kSize, initializer));
+}
+
+template <typename Vector>
+void CopyEmptyVector(benchmark::State& state) {
+  constexpr int kNumIters = 1000;
+
+  const Vector vec{};
+
+  for (auto _ : state) {
+    int64_t dummy = 0;
+    for (int i = 0; i < kNumIters; ++i) {
+      dummy += ConsumeVector(vec);
+    }
+    benchmark::DoNotOptimize(dummy);
+  }
+
+  state.SetItemsProcessed(state.iterations() * kNumIters);
+}
+
+template <typename Vector>
+void CopyShortVector(benchmark::State& state) {
+  constexpr int kSize = 3;
+  constexpr int kNumIters = 1000;
+
+  const Vector vec(kSize);
+
+  for (auto _ : state) {
+    int64_t dummy = 0;
+    for (int i = 0; i < kNumIters; ++i) {
+      dummy += ConsumeVector(vec);
+    }
+    benchmark::DoNotOptimize(dummy);
+  }
+
+  state.SetItemsProcessed(state.iterations() * kNumIters);
+}
+
+// With ARROW_NOINLINE, try to make sure the number of items is not constant-propagated
+template <typename Vector>
+ARROW_NOINLINE void BenchmarkConstructFromStdVector(benchmark::State& state,
+                                                    const int nitems) {
+  using T = typename Vector::value_type;
+  constexpr int kNumIters = 1000;
+  const std::vector<T> src(nitems, ValueInitializer<T>());
+
+  for (auto _ : state) {
+    int64_t dummy = 0;
+    for (int i = 0; i < kNumIters; ++i) {
+      Vector vec(src);
+      dummy += IngestVector(vec);
+    }
+    benchmark::DoNotOptimize(dummy);
+  }
+
+  state.SetItemsProcessed(state.iterations() * kNumIters);
+}
+
+template <typename Vector>
+void ConstructFromEmptyStdVector(benchmark::State& state) {
+  BenchmarkConstructFromStdVector<Vector>(state, 0);
+}
+
+template <typename Vector>
+void ConstructFromShortStdVector(benchmark::State& state) {
+  BenchmarkConstructFromStdVector<Vector>(state, 3);
+}
+
+// With ARROW_NOINLINE, try to make sure the number of items is not constant-propagated
+template <typename Vector>
+ARROW_NOINLINE void BenchmarkVectorPushBack(benchmark::State& state, const int nitems) {
+  using T = typename Vector::value_type;
+  constexpr int kNumIters = 1000;
+
+  ARROW_CHECK_LE(static_cast<size_t>(nitems), Vector{}.max_size());
+
+  for (auto _ : state) {
+    int64_t dummy = 0;
+    for (int i = 0; i < kNumIters; ++i) {
+      Vector vec;
+      vec.reserve(nitems);
+      for (int j = 0; j < nitems; ++j) {
+        vec.push_back(ValueInitializer<T>(j));
+      }
+      dummy += reinterpret_cast<intptr_t>(vec.data());
+      benchmark::ClobberMemory();
+    }
+    benchmark::DoNotOptimize(dummy);
+  }
+
+  state.SetItemsProcessed(state.iterations() * kNumIters * nitems);
+}
+
+template <typename Vector>
+void ShortVectorPushBack(benchmark::State& state) {
+  BenchmarkVectorPushBack<Vector>(state, 3);
+}
+
+template <typename Vector>
+void LongVectorPushBack(benchmark::State& state) {
+  BenchmarkVectorPushBack<Vector>(state, 100);
+}
+
+// With ARROW_NOINLINE, try to make sure the source data is not constant-propagated
+// (we could also use random data)
+template <typename Vector, typename T = typename Vector::value_type>
+ARROW_NOINLINE void BenchmarkShortVectorInsert(benchmark::State& state,
+                                               const std::vector<T>& src) {
+  constexpr int kNumIters = 1000;
+
+  for (auto _ : state) {
+    int64_t dummy = 0;
+    for (int i = 0; i < kNumIters; ++i) {
+      Vector vec;
+      vec.reserve(4);
+      vec.insert(vec.begin(), src.begin(), src.begin() + 2);
+      vec.insert(vec.begin(), src.begin() + 2, src.end());
+      dummy += reinterpret_cast<intptr_t>(vec.data());
+      benchmark::ClobberMemory();
+    }
+    benchmark::DoNotOptimize(dummy);
+  }
+
+  state.SetItemsProcessed(state.iterations() * kNumIters * 4);
+}
+
+template <typename Vector>
+void ShortVectorInsert(benchmark::State& state) {
+  using T = typename Vector::value_type;
+  const std::vector<T> src(4, ValueInitializer<T>());
+  BenchmarkShortVectorInsert<Vector>(state, src);
+}
+
+template <typename Vector>
+ARROW_NOINLINE void BenchmarkVectorInsertAtEnd(benchmark::State& state,
+                                               const int nitems) {
+  using T = typename Vector::value_type;
+  constexpr int kNumIters = 1000;
+
+  ARROW_CHECK_LE(static_cast<size_t>(nitems), Vector{}.max_size());
+  ARROW_CHECK_EQ(nitems % 2, 0);
+
+  std::vector<T> src;
+  for (int j = 0; j < nitems / 2; ++j) {
+    src.push_back(ValueInitializer<T>(j));
+  }
+
+  for (auto _ : state) {
+    int64_t dummy = 0;
+    for (int i = 0; i < kNumIters; ++i) {
+      Vector vec;
+      vec.reserve(nitems);
+      vec.insert(vec.end(), src.begin(), src.end());
+      vec.insert(vec.end(), src.begin(), src.end());
+      dummy += reinterpret_cast<intptr_t>(vec.data());
+      benchmark::ClobberMemory();
+    }
+    benchmark::DoNotOptimize(dummy);
+  }
+
+  state.SetItemsProcessed(state.iterations() * kNumIters * nitems);
+}
+
+template <typename Vector>
+void ShortVectorInsertAtEnd(benchmark::State& state) {
+  BenchmarkVectorInsertAtEnd<Vector>(state, 4);
+}
+
+template <typename Vector>
+void LongVectorInsertAtEnd(benchmark::State& state) {
+  BenchmarkVectorInsertAtEnd<Vector>(state, 100);
+}
+
+#define SHORT_VECTOR_BENCHMARKS(VEC_TYPE_FACTORY)                                  \
+  BENCHMARK_TEMPLATE(MoveEmptyVector, VEC_TYPE_FACTORY(int));                      \
+  BENCHMARK_TEMPLATE(MoveEmptyVector, VEC_TYPE_FACTORY(std::string));              \
+  BENCHMARK_TEMPLATE(MoveEmptyVector, VEC_TYPE_FACTORY(std::shared_ptr<int>));     \
+  BENCHMARK_TEMPLATE(MoveShortVector, VEC_TYPE_FACTORY(int));                      \
+  BENCHMARK_TEMPLATE(MoveShortVector, VEC_TYPE_FACTORY(std::string));              \
+  BENCHMARK_TEMPLATE(MoveShortVector, VEC_TYPE_FACTORY(std::shared_ptr<int>));     \
+  BENCHMARK_TEMPLATE(CopyEmptyVector, VEC_TYPE_FACTORY(int));                      \
+  BENCHMARK_TEMPLATE(CopyEmptyVector, VEC_TYPE_FACTORY(std::string));              \
+  BENCHMARK_TEMPLATE(CopyEmptyVector, VEC_TYPE_FACTORY(std::shared_ptr<int>));     \
+  BENCHMARK_TEMPLATE(CopyShortVector, VEC_TYPE_FACTORY(int));                      \
+  BENCHMARK_TEMPLATE(CopyShortVector, VEC_TYPE_FACTORY(std::string));              \
+  BENCHMARK_TEMPLATE(CopyShortVector, VEC_TYPE_FACTORY(std::shared_ptr<int>));     \
+  BENCHMARK_TEMPLATE(ConstructFromEmptyStdVector, VEC_TYPE_FACTORY(int));          \
+  BENCHMARK_TEMPLATE(ConstructFromEmptyStdVector, VEC_TYPE_FACTORY(std::string));  \
+  BENCHMARK_TEMPLATE(ConstructFromEmptyStdVector,                                  \
+                     VEC_TYPE_FACTORY(std::shared_ptr<int>));                      \
+  BENCHMARK_TEMPLATE(ConstructFromShortStdVector, VEC_TYPE_FACTORY(int));          \
+  BENCHMARK_TEMPLATE(ConstructFromShortStdVector, VEC_TYPE_FACTORY(std::string));  \
+  BENCHMARK_TEMPLATE(ConstructFromShortStdVector,                                  \
+                     VEC_TYPE_FACTORY(std::shared_ptr<int>));                      \
+  BENCHMARK_TEMPLATE(ShortVectorPushBack, VEC_TYPE_FACTORY(int));                  \
+  BENCHMARK_TEMPLATE(ShortVectorPushBack, VEC_TYPE_FACTORY(std::string));          \
+  BENCHMARK_TEMPLATE(ShortVectorPushBack, VEC_TYPE_FACTORY(std::shared_ptr<int>)); \
+  BENCHMARK_TEMPLATE(ShortVectorInsert, VEC_TYPE_FACTORY(int));                    \
+  BENCHMARK_TEMPLATE(ShortVectorInsert, VEC_TYPE_FACTORY(std::string));            \
+  BENCHMARK_TEMPLATE(ShortVectorInsert, VEC_TYPE_FACTORY(std::shared_ptr<int>));   \
+  BENCHMARK_TEMPLATE(ShortVectorInsertAtEnd, VEC_TYPE_FACTORY(int));               \
+  BENCHMARK_TEMPLATE(ShortVectorInsertAtEnd, VEC_TYPE_FACTORY(std::string));       \
+  BENCHMARK_TEMPLATE(ShortVectorInsertAtEnd, VEC_TYPE_FACTORY(std::shared_ptr<int>));
+
+#define LONG_VECTOR_BENCHMARKS(VEC_TYPE_FACTORY)                                  \
+  BENCHMARK_TEMPLATE(LongVectorPushBack, VEC_TYPE_FACTORY(int));                  \
+  BENCHMARK_TEMPLATE(LongVectorPushBack, VEC_TYPE_FACTORY(std::string));          \
+  BENCHMARK_TEMPLATE(LongVectorPushBack, VEC_TYPE_FACTORY(std::shared_ptr<int>)); \
+  BENCHMARK_TEMPLATE(LongVectorInsertAtEnd, VEC_TYPE_FACTORY(int));               \
+  BENCHMARK_TEMPLATE(LongVectorInsertAtEnd, VEC_TYPE_FACTORY(std::string));       \
+  BENCHMARK_TEMPLATE(LongVectorInsertAtEnd, VEC_TYPE_FACTORY(std::shared_ptr<int>));
+
+// NOTE: the macro name below (STD_VECTOR etc.) is reflected in the
+// benchmark name, so use descriptive names.
+
+#ifdef ARROW_WITH_BENCHMARKS_REFERENCE
+
+#define STD_VECTOR(T) std::vector<T>
+SHORT_VECTOR_BENCHMARKS(STD_VECTOR);
+LONG_VECTOR_BENCHMARKS(STD_VECTOR);
+#undef STD_VECTOR
+
+#endif
+
+#define STATIC_VECTOR(T) StaticVector<T, 4>
+SHORT_VECTOR_BENCHMARKS(STATIC_VECTOR);
+#undef STATIC_VECTOR
+
+#define SMALL_VECTOR(T) SmallVector<T, 4>
+SHORT_VECTOR_BENCHMARKS(SMALL_VECTOR);
+LONG_VECTOR_BENCHMARKS(SMALL_VECTOR);
+#undef SMALL_VECTOR
+
+#undef SHORT_VECTOR_BENCHMARKS
+#undef LONG_VECTOR_BENCHMARKS
+
+}  // namespace internal
+}  // namespace arrow
diff --git a/cpp/src/arrow/util/small_vector_test.cc b/cpp/src/arrow/util/small_vector_test.cc
new file mode 100644
index 0000000..f9ec5fe
--- /dev/null
+++ b/cpp/src/arrow/util/small_vector_test.cc
@@ -0,0 +1,786 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <algorithm>
+#include <cstddef>
+#include <iterator>
+#include <limits>
+#include <memory>
+#include <string>
+#include <type_traits>
+
+#include <gtest/gtest.h>
+
+#include "arrow/testing/gtest_util.h"
+#include "arrow/testing/matchers.h"
+#include "arrow/util/small_vector.h"
+
+using testing::ElementsAre;
+using testing::ElementsAreArray;
+
+namespace arrow {
+namespace internal {
+
+struct HeapInt {
+  HeapInt() : HeapInt(0) {}
+
+  explicit HeapInt(int x) : ptr(new int(x)) {}
+
+  HeapInt& operator=(int x) {
+    ptr.reset(new int(x));
+    return *this;
+  }
+
+  HeapInt(const HeapInt& other) : HeapInt(other.ToInt()) {}
+
+  HeapInt& operator=(const HeapInt& other) {
+    *this = other.ToInt();
+    return *this;
+  }
+
+  int ToInt() const { return ptr == nullptr ? -98 : *ptr; }
+
+  bool operator==(const HeapInt& other) const {
+    return ptr != nullptr && other.ptr != nullptr && *ptr == *other.ptr;
+  }
+  bool operator<(const HeapInt& other) const {
+    return ptr == nullptr || (other.ptr != nullptr && *ptr < *other.ptr);
+  }
+
+  bool operator==(int other) const { return ptr != nullptr && *ptr == other; }
+  friend bool operator==(int left, const HeapInt& right) { return right == left; }
+
+  std::unique_ptr<int> ptr;
+};
+
+template <typename Vector>
+bool UsesStaticStorage(const Vector& v) {
+  const uint8_t* p = reinterpret_cast<const uint8_t*>(v.data());
+  if (p == nullptr) {
+    return true;
+  }
+  const uint8_t* v_start = reinterpret_cast<const uint8_t*>(&v);
+  return (p >= v_start && p < v_start + sizeof(v));
+}
+
+struct StaticVectorTraits {
+  template <typename T, size_t N>
+  using VectorType = StaticVector<T, N>;
+
+  static bool CanOverflow() { return false; }
+
+  static constexpr size_t MaxSizeFor(size_t n) { return n; }
+
+  static constexpr size_t TestSizeFor(size_t max_size) { return max_size; }
+};
+
+struct SmallVectorTraits {
+  template <typename T, size_t N>
+  using VectorType = SmallVector<T, N>;
+
+  static bool CanOverflow() { return true; }
+
+  static constexpr size_t MaxSizeFor(size_t n) {
+    return std::numeric_limits<size_t>::max();
+  }
+
+  static constexpr size_t TestSizeFor(size_t max_size) {
+    return max_size > 6 ? max_size / 3 : 2;
+  }
+};
+
+using VectorTraits = ::testing::Types<StaticVectorTraits, SmallVectorTraits>;
+
+template <typename T, typename I>
+struct VectorIntLikeParam {
+  using Traits = T;
+  using IntLike = I;
+
+  static constexpr bool IsMoveOnly() { return !std::is_copy_constructible<I>::value; }
+};
+
+using VectorIntLikeParams =
+    ::testing::Types<VectorIntLikeParam<StaticVectorTraits, int>,
+                     VectorIntLikeParam<SmallVectorTraits, int>,
+                     VectorIntLikeParam<StaticVectorTraits, HeapInt>,
+                     VectorIntLikeParam<SmallVectorTraits, HeapInt>,
+                     VectorIntLikeParam<StaticVectorTraits, MoveOnlyDataType>,
+                     VectorIntLikeParam<SmallVectorTraits, MoveOnlyDataType>>;
+
+template <typename Param>
+class TestSmallStaticVector : public ::testing::Test {
+  template <bool B, typename T = void>
+  using enable_if_t = typename std::enable_if<B, T>::type;
+
+  template <typename P>
+  using enable_if_move_only = enable_if_t<P::IsMoveOnly(), int>;
+
+  template <typename P>
+  using enable_if_not_move_only = enable_if_t<!P::IsMoveOnly(), int>;
+
+ public:
+  using Traits = typename Param::Traits;
+  using IntLike = typename Param::IntLike;
+
+  template <typename T, size_t N>
+  using VectorType = typename Traits::template VectorType<T, N>;
+
+  template <size_t N>
+  using IntVectorType = VectorType<IntLike, N>;
+
+  template <size_t N>
+  IntVectorType<N> MakeVector(const std::vector<int>& init_values) {
+    IntVectorType<N> ints;
+    for (auto v : init_values) {
+      ints.emplace_back(v);
+    }
+    return ints;
+  }
+
+  template <size_t N>
+  IntVectorType<N> CheckFourValues() {
+    IntVectorType<N> ints;
+    EXPECT_EQ(ints.size(), 0);
+    EXPECT_EQ(ints.capacity(), N);
+    EXPECT_EQ(ints.max_size(), Traits::MaxSizeFor(N));
+    EXPECT_TRUE(UsesStaticStorage(ints));
+
+    ints.emplace_back(3);
+    ints.emplace_back(42);
+    EXPECT_EQ(ints.size(), 2);
+    EXPECT_EQ(ints.capacity(), N);
+    EXPECT_EQ(ints[0], 3);
+    EXPECT_EQ(ints[1], 42);
+    EXPECT_TRUE(UsesStaticStorage(ints));
+
+    ints.push_back(IntLike(5));
+    ints.emplace_back(false);
+    EXPECT_EQ(ints.size(), 4);
+    EXPECT_EQ(ints[2], 5);
+    EXPECT_EQ(ints[3], 0);
+
+    ints[3] = IntLike(8);
+    EXPECT_EQ(ints[3], 8);
+    EXPECT_EQ(ints.back(), 8);
+    ints.front() = IntLike(-1);
+    EXPECT_EQ(ints[0], -1);
+    EXPECT_EQ(ints.front(), -1);
+
+    return ints;
+  }
+
+  void TestBasics() {
+    constexpr size_t N = Traits::TestSizeFor(4);
+    const auto ints = CheckFourValues<N>();
+    EXPECT_EQ(UsesStaticStorage(ints), !Traits::CanOverflow());
+  }
+
+  void TestAlwaysStatic() {
+    const auto ints = CheckFourValues<4>();
+    EXPECT_TRUE(UsesStaticStorage(ints));
+  }
+
+  template <size_t N>
+  void CheckReserve(size_t max_size, bool expect_overflow) {
+    IntVectorType<N> ints;
+    ints.emplace_back(123);
+
+    size_t orig_capacity = ints.capacity();
+
+    ints.reserve(max_size / 3);
+    ASSERT_EQ(ints.capacity(), std::max(max_size / 3, orig_capacity));
+    ASSERT_EQ(ints.size(), 1);
+    ASSERT_EQ(ints[0], 123);
+
+    ints.reserve(4 * max_size / 5);
+    ASSERT_EQ(ints.capacity(), std::max(4 * max_size / 5, orig_capacity));
+    ASSERT_EQ(ints.size(), 1);
+    ASSERT_EQ(ints[0], 123);
+    ASSERT_EQ(UsesStaticStorage(ints), !expect_overflow);
+
+    size_t old_capacity = ints.capacity();
+    ints.reserve(max_size / 5);  // no-op
+    ASSERT_EQ(ints.capacity(), old_capacity);
+    ASSERT_EQ(ints.size(), 1);
+    ASSERT_EQ(ints[0], 123);
+
+    ints.reserve(1);  // no-op
+    ASSERT_EQ(ints.capacity(), old_capacity);
+    ASSERT_EQ(ints.size(), 1);
+    ASSERT_EQ(ints[0], 123);
+  }
+
+  void TestReserve() {
+    CheckReserve<Traits::TestSizeFor(12)>(12, /*expect_overflow=*/Traits::CanOverflow());
+    CheckReserve<12>(12, /*expect_overflow=*/false);
+  }
+
+  template <size_t N>
+  void CheckClear(bool expect_overflow) {
+    IntVectorType<N> ints = MakeVector<N>({5, 6, 7, 8, 9});
+    ASSERT_EQ(ints.size(), 5);
+    size_t capacity = ints.capacity();
+
+    ints.clear();
+    ASSERT_EQ(ints.size(), 0);
+    ASSERT_EQ(ints.capacity(), capacity);
+    ASSERT_EQ(UsesStaticStorage(ints), !expect_overflow);
+  }
+
+  void TestClear() {
+    CheckClear<Traits::TestSizeFor(5)>(/*expect_overflow=*/Traits::CanOverflow());
+    CheckClear<6>(/*expect_overflow=*/false);
+  }
+
+  template <bool IsMoveOnly = Param::IsMoveOnly()>
+  void TestConstructFromCount(enable_if_t<!IsMoveOnly>* = 0) {
+    constexpr size_t N = Traits::TestSizeFor(4);
+    {
+      const IntVectorType<N> ints(3);
+      ASSERT_EQ(ints.size(), 3);
+      ASSERT_EQ(ints.capacity(), std::max<size_t>(N, 3));
+      for (int i = 0; i < 3; ++i) {
+        ASSERT_EQ(ints[i], 0);
+      }
+      EXPECT_THAT(ints, ElementsAre(0, 0, 0));
+    }
+  }
+
+  template <bool IsMoveOnly = Param::IsMoveOnly()>
+  void TestConstructFromCount(enable_if_t<IsMoveOnly>* = 0) {
+    GTEST_SKIP() << "Cannot construct vector of move-only type with value count";
+  }
+
+  template <size_t N>
+  void CheckConstructFromValues() {
+    {
+      const IntVectorType<N> ints{};
+      ASSERT_EQ(ints.size(), 0);
+      ASSERT_EQ(ints.capacity(), N);
+    }
+    {
+      const IntVectorType<N> ints{IntLike(4), IntLike(5), IntLike(6)};
+      ASSERT_EQ(ints.size(), 3);
+      ASSERT_EQ(ints.capacity(), std::max<size_t>(N, 3));
+      ASSERT_EQ(ints[0], 4);
+      ASSERT_EQ(ints[1], 5);
+      ASSERT_EQ(ints[2], 6);
+      ASSERT_EQ(ints.front(), 4);
+      ASSERT_EQ(ints.back(), 6);
+    }
+  }
+
+  template <bool IsMoveOnly = Param::IsMoveOnly()>
+  void TestConstructFromValues(enable_if_t<!IsMoveOnly>* = 0) {
+    CheckConstructFromValues<Traits::TestSizeFor(4)>();
+    CheckConstructFromValues<5>();
+  }
+
+  template <bool IsMoveOnly = Param::IsMoveOnly()>
+  void TestConstructFromValues(enable_if_t<IsMoveOnly>* = 0) {
+    GTEST_SKIP() << "Cannot construct vector of move-only type with explicit values";
+  }
+
+  void CheckConstructFromMovedStdVector() {
+    constexpr size_t N = Traits::TestSizeFor(6);
+    {
+      std::vector<IntLike> src;
+      const IntVectorType<N> ints(std::move(src));
+      ASSERT_EQ(ints.size(), 0);
+      ASSERT_EQ(ints.capacity(), N);
+    }
+    {
+      std::vector<IntLike> src;
+      for (int i = 0; i < 6; ++i) {
+        src.emplace_back(i + 4);
+      }
+      const IntVectorType<N> ints(std::move(src));
+      ASSERT_EQ(ints.size(), 6);
+      ASSERT_EQ(ints.capacity(), std::max<size_t>(N, 6));
+      EXPECT_THAT(ints, ElementsAre(4, 5, 6, 7, 8, 9));
+    }
+  }
+
+  template <bool IsMoveOnly = Param::IsMoveOnly()>
+  void CheckConstructFromCopiedStdVector(enable_if_t<!IsMoveOnly>* = 0) {
+    constexpr size_t N = Traits::TestSizeFor(6);
+    {
+      const std::vector<IntLike> src;
+      const IntVectorType<N> ints(src);
+      ASSERT_EQ(ints.size(), 0);
+      ASSERT_EQ(ints.capacity(), N);
+    }
+    {
+      std::vector<IntLike> values;
+      for (int i = 0; i < 6; ++i) {
+        values.emplace_back(i + 4);
+      }
+      const auto& src = values;
+      const IntVectorType<N> ints(src);
+      ASSERT_EQ(ints.size(), 6);
+      ASSERT_EQ(ints.capacity(), std::max<size_t>(N, 6));
+      EXPECT_THAT(ints, ElementsAre(4, 5, 6, 7, 8, 9));
+    }
+  }
+
+  template <bool IsMoveOnly = Param::IsMoveOnly()>
+  void CheckConstructFromCopiedStdVector(enable_if_t<IsMoveOnly>* = 0) {}
+
+  void TestConstructFromStdVector() {
+    CheckConstructFromMovedStdVector();
+    CheckConstructFromCopiedStdVector();
+  }
+
+  void CheckAssignFromMovedStdVector() {
+    constexpr size_t N = Traits::TestSizeFor(6);
+    {
+      std::vector<IntLike> src;
+      IntVectorType<N> ints = MakeVector<N>({42});
+      ints = std::move(src);
+      ASSERT_EQ(ints.size(), 0);
+      ASSERT_EQ(ints.capacity(), N);
+    }
+    {
+      std::vector<IntLike> src;
+      for (int i = 0; i < 6; ++i) {
+        src.emplace_back(i + 4);
+      }
+      IntVectorType<N> ints = MakeVector<N>({42});
+      ints = std::move(src);
+      ASSERT_EQ(ints.size(), 6);
+      ASSERT_EQ(ints.capacity(), std::max<size_t>(N, 6));
+      EXPECT_THAT(ints, ElementsAre(4, 5, 6, 7, 8, 9));
+    }
+  }
+
+  template <bool IsMoveOnly = Param::IsMoveOnly()>
+  void CheckAssignFromCopiedStdVector(enable_if_t<!IsMoveOnly>* = 0) {
+    constexpr size_t N = Traits::TestSizeFor(6);
+    {
+      const std::vector<IntLike> src;
+      IntVectorType<N> ints = MakeVector<N>({42});
+      ints = src;
+      ASSERT_EQ(ints.size(), 0);
+      ASSERT_EQ(ints.capacity(), N);
+    }
+    {
+      std::vector<IntLike> values;
+      for (int i = 0; i < 6; ++i) {
+        values.emplace_back(i + 4);
+      }
+      const auto& src = values;
+      IntVectorType<N> ints = MakeVector<N>({42});
+      ints = src;
+      ASSERT_EQ(ints.size(), 6);
+      ASSERT_EQ(ints.capacity(), std::max<size_t>(N, 6));
+      EXPECT_THAT(ints, ElementsAre(4, 5, 6, 7, 8, 9));
+    }
+  }
+
+  template <bool IsMoveOnly = Param::IsMoveOnly()>
+  void CheckAssignFromCopiedStdVector(enable_if_t<IsMoveOnly>* = 0) {}
+
+  void TestAssignFromStdVector() {
+    CheckAssignFromMovedStdVector();
+    CheckAssignFromCopiedStdVector();
+  }
+
+  template <size_t N>
+  void CheckMove(bool expect_overflow) {
+    IntVectorType<N> ints = MakeVector<N>({4, 5, 6, 7, 8});
+
+    IntVectorType<N> moved_ints(std::move(ints));
+    ASSERT_EQ(moved_ints.size(), 5);
+    EXPECT_THAT(moved_ints, ElementsAre(4, 5, 6, 7, 8));
+    ASSERT_EQ(UsesStaticStorage(moved_ints), !expect_overflow);
+    ASSERT_TRUE(UsesStaticStorage(ints));
+
+    IntVectorType<N> moved_moved_ints = std::move(moved_ints);
+    ASSERT_EQ(moved_moved_ints.size(), 5);
+    EXPECT_THAT(moved_moved_ints, ElementsAre(4, 5, 6, 7, 8));
+
+    // Move into itself
+    moved_moved_ints = std::move(moved_moved_ints);
+    ASSERT_EQ(moved_moved_ints.size(), 5);
+    EXPECT_THAT(moved_moved_ints, ElementsAre(4, 5, 6, 7, 8));
+  }
+
+  void TestMove() {
+    CheckMove<Traits::TestSizeFor(5)>(/*expect_overflow=*/Traits::CanOverflow());
+    CheckMove<5>(/*expect_overflow=*/false);
+  }
+
+  template <size_t N>
+  void CheckCopy(bool expect_overflow) {
+    IntVectorType<N> ints = MakeVector<N>({4, 5, 6, 7, 8});
+
+    IntVectorType<N> copied_ints(ints);
+    ASSERT_EQ(copied_ints.size(), 5);
+    ASSERT_EQ(ints.size(), 5);
+    EXPECT_THAT(copied_ints, ElementsAre(4, 5, 6, 7, 8));
+    EXPECT_THAT(ints, ElementsAre(4, 5, 6, 7, 8));
+    ASSERT_EQ(UsesStaticStorage(copied_ints), !expect_overflow);
+
+    IntVectorType<N> copied_copied_ints = copied_ints;
+    ASSERT_EQ(copied_copied_ints.size(), 5);
+    ASSERT_EQ(copied_ints.size(), 5);
+    EXPECT_THAT(copied_copied_ints, ElementsAre(4, 5, 6, 7, 8));
+    EXPECT_THAT(copied_ints, ElementsAre(4, 5, 6, 7, 8));
+
+    auto copy_into = [](const IntVectorType<N>& src, IntVectorType<N>* dest) {
+      *dest = src;
+    };
+
+    // Copy into itself
+    // (avoiding the trivial form `copied_copied_ints = copied_copied_ints`
+    //  that would produce a clang warning)
+    copy_into(copied_copied_ints, &copied_copied_ints);
+    ASSERT_EQ(copied_copied_ints.size(), 5);
+    EXPECT_THAT(copied_copied_ints, ElementsAre(4, 5, 6, 7, 8));
+  }
+
+  template <bool IsMoveOnly = Param::IsMoveOnly()>
+  void TestCopy(enable_if_t<!IsMoveOnly>* = 0) {
+    CheckCopy<Traits::TestSizeFor(5)>(/*expect_overflow=*/Traits::CanOverflow());
+    CheckCopy<5>(/*expect_overflow=*/false);
+  }
+
+  template <bool IsMoveOnly = Param::IsMoveOnly()>
+  void TestCopy(enable_if_t<IsMoveOnly>* = 0) {
+    GTEST_SKIP() << "Cannot copy vector of move-only type";
+  }
+
+  template <bool IsMoveOnly = Param::IsMoveOnly()>
+  void TestResize(enable_if_t<!IsMoveOnly>* = 0) {
+    constexpr size_t N = Traits::TestSizeFor(8);
+    {
+      IntVectorType<N> ints;
+      ints.resize(2);
+      ASSERT_GE(ints.capacity(), 2);
+      EXPECT_THAT(ints, ElementsAreArray(std::vector<int>(2, 0)));
+      ints.resize(3);
+      ASSERT_GE(ints.capacity(), 3);
+      EXPECT_THAT(ints, ElementsAreArray(std::vector<int>(3, 0)));
+      ints.resize(8);
+      ASSERT_GE(ints.capacity(), 8);
+      EXPECT_THAT(ints, ElementsAreArray(std::vector<int>(8, 0)));
+      ints.resize(6);
+      ints.resize(6);  // no-op
+      ASSERT_GE(ints.capacity(), 8);
+      EXPECT_THAT(ints, ElementsAreArray(std::vector<int>(6, 0)));
+      ints.resize(0);
+      ASSERT_GE(ints.capacity(), 8);
+      EXPECT_THAT(ints, ElementsAreArray(std::vector<int>(0, 0)));
+      ints.resize(5);
+      ASSERT_GE(ints.capacity(), 8);
+      EXPECT_THAT(ints, ElementsAreArray(std::vector<int>(5, 0)));
+      ints.resize(7);
+      ASSERT_GE(ints.capacity(), 8);
+      EXPECT_THAT(ints, ElementsAreArray(std::vector<int>(7, 0)));
+    }
+    {
+      IntVectorType<N> ints;
+      ints.resize(2, IntLike(2));
+      ASSERT_GE(ints.capacity(), 2);
+      EXPECT_THAT(ints, ElementsAre(2, 2));
+      ints.resize(3, IntLike(3));
+      ASSERT_GE(ints.capacity(), 3);
+      EXPECT_THAT(ints, ElementsAre(2, 2, 3));
+      ints.resize(8, IntLike(8));
+      ASSERT_GE(ints.capacity(), 8);
+      EXPECT_THAT(ints, ElementsAre(2, 2, 3, 8, 8, 8, 8, 8));
+      ints.resize(6, IntLike(6));
+      ints.resize(6, IntLike(6));  // no-op
+      ASSERT_GE(ints.capacity(), 8);
+      EXPECT_THAT(ints, ElementsAre(2, 2, 3, 8, 8, 8));
+      ints.resize(0, IntLike(0));
+      ASSERT_GE(ints.capacity(), 8);
+      EXPECT_THAT(ints, ElementsAre());
+      ints.resize(5, IntLike(5));
+      ASSERT_GE(ints.capacity(), 8);
+      EXPECT_THAT(ints, ElementsAre(5, 5, 5, 5, 5));
+      ints.resize(7, IntLike(7));
+      ASSERT_GE(ints.capacity(), 8);
+      EXPECT_THAT(ints, ElementsAre(5, 5, 5, 5, 5, 7, 7));
+    }
+  }
+
+  template <bool IsMoveOnly = Param::IsMoveOnly()>
+  void TestResize(enable_if_t<IsMoveOnly>* = 0) {
+    GTEST_SKIP() << "Cannot resize vector of move-only type";
+  }
+
+  template <size_t N>
+  void CheckSort() {
+    IntVectorType<N> ints;
+    for (int v : {42, 2, 123, -5, 6, 12, 8, 13}) {
+      ints.emplace_back(v);
+    }
+    std::sort(ints.begin(), ints.end());
+    EXPECT_THAT(ints, ElementsAre(-5, 2, 6, 8, 12, 13, 42, 123));
+  }
+
+  void TestSort() {
+    CheckSort<Traits::TestSizeFor(8)>();
+    CheckSort<8>();
+  }
+
+  void TestIterators() {
+    constexpr size_t N = Traits::TestSizeFor(5);
+    {
+      // Forward iterators
+      IntVectorType<N> ints;
+      ASSERT_EQ(ints.begin(), ints.end());
+
+      for (int v : {5, 6, 7, 8, 42}) {
+        ints.emplace_back(v);
+      }
+
+      auto it = ints.begin();
+      ASSERT_NE(it, ints.end());
+      ASSERT_EQ(*it++, 5);
+      ASSERT_EQ(ints.end() - it, 4);
+
+      auto it2 = ++it;
+      ASSERT_EQ(*it, 7);
+      ASSERT_EQ(*it2, 7);
+      ASSERT_EQ(it, it2);
+
+      ASSERT_EQ(ints.end() - it, 3);
+      ASSERT_EQ(*it--, 7);
+      ASSERT_NE(it, it2);
+
+      ASSERT_EQ(ints.end() - it, 4);
+      ASSERT_NE(it, ints.end());
+      ASSERT_EQ(*--it, 5);
+      ASSERT_EQ(*it, 5);
+      ASSERT_EQ(ints.end() - it, 5);
+      it += 4;
+      ASSERT_EQ(*it, 42);
+      ASSERT_EQ(ints.end() - it, 1);
+      ASSERT_NE(it, ints.end());
+      ASSERT_EQ(*(it - 3), 6);
+      ASSERT_EQ(++it, ints.end());
+    }
+    {
+      // Reverse iterators
+      IntVectorType<N> ints;
+      ASSERT_EQ(ints.rbegin(), ints.rend());
+
+      for (int v : {42, 8, 7, 6, 5}) {
+        ints.emplace_back(v);
+      }
+
+      auto it = ints.rbegin();
+      ASSERT_NE(it, ints.rend());
+      ASSERT_EQ(*it++, 5);
+      ASSERT_EQ(ints.rend() - it, 4);
+
+      auto it2 = ++it;
+      ASSERT_EQ(*it, 7);
+      ASSERT_EQ(*it2, 7);
+      ASSERT_EQ(it, it2);
+
+      ASSERT_EQ(ints.rend() - it, 3);
+      ASSERT_EQ(*it--, 7);
+      ASSERT_NE(it, it2);
+
+      ASSERT_EQ(ints.rend() - it, 4);
+      ASSERT_NE(it, ints.rend());
+      ASSERT_EQ(*--it, 5);
+      ASSERT_EQ(*it, 5);
+      ASSERT_EQ(ints.rend() - it, 5);
+      it += 4;
+      ASSERT_EQ(*it, 42);
+      ASSERT_EQ(ints.rend() - it, 1);
+      ASSERT_NE(it, ints.rend());
+      ASSERT_EQ(*(it - 3), 6);
+      ASSERT_EQ(++it, ints.rend());
+    }
+  }
+
+  void TestConstIterators() {
+    constexpr size_t N = Traits::TestSizeFor(5);
+    {
+      const IntVectorType<N> ints{};
+      ASSERT_EQ(ints.begin(), ints.end());
+      ASSERT_EQ(ints.rbegin(), ints.rend());
+    }
+    {
+      // Forward iterators
+      IntVectorType<N> underlying_ints = MakeVector<N>({5, 6, 7, 8, 42});
+      const IntVectorType<N>& ints = underlying_ints;
+
+      auto it = ints.begin();
+      ASSERT_NE(it, ints.end());
+      ASSERT_EQ(*it++, 5);
+      auto it2 = it++;
+      ASSERT_EQ(*it2, 6);
+      ASSERT_EQ(*it, 7);
+      ASSERT_NE(it, it2);
+      ASSERT_EQ(*++it2, 7);
+      ASSERT_EQ(it, it2);
+
+      // Conversion from non-const iterator
+      it = underlying_ints.begin() + 1;
+      ASSERT_NE(it, underlying_ints.end());
+      ASSERT_EQ(*it, 6);
+      it += underlying_ints.end() - it;
+      ASSERT_EQ(it, underlying_ints.end());
+    }
+    {
+      // Reverse iterators
+      IntVectorType<N> underlying_ints = MakeVector<N>({42, 8, 7, 6, 5});
+      const IntVectorType<N>& ints = underlying_ints;
+
+      auto it = ints.rbegin();
+      ASSERT_NE(it, ints.rend());
+      ASSERT_EQ(*it++, 5);
+      auto it2 = it++;
+      ASSERT_EQ(*it2, 6);
+      ASSERT_EQ(*it, 7);
+      ASSERT_NE(it, it2);
+      ASSERT_EQ(*++it2, 7);
+      ASSERT_EQ(it, it2);
+
+      // Conversion from non-const iterator
+      it = underlying_ints.rbegin() + 1;
+      ASSERT_NE(it, underlying_ints.rend());
+      ASSERT_EQ(*it, 6);
+      it += underlying_ints.rend() - it;
+      ASSERT_EQ(it, underlying_ints.rend());
+    }
+  }
+
+  void TestInsertIteratorPair() {
+    // insert(const_iterator, InputIt first, InputIt last)
+    constexpr size_t N = Traits::TestSizeFor(10);
+    {
+      // empty source and destination
+      const std::vector<int> src{};
+      IntVectorType<N> ints;
+      ints.insert(ints.begin(), src.begin(), src.end());
+      ASSERT_EQ(ints.size(), 0);
+
+      ints.emplace_back(42);
+      ints.insert(ints.begin(), src.begin(), src.end());
+      ints.insert(ints.end(), src.begin(), src.end());
+      EXPECT_THAT(ints, ElementsAre(42));
+    }
+    const std::vector<int> src{0, 1, 2, 3, 4, 5, 6, 7, 8, 9};
+    {
+      // insert at start
+      IntVectorType<N> ints;
+      ints.insert(ints.begin(), src.begin() + 4, src.begin() + 7);
+      EXPECT_THAT(ints, ElementsAre(4, 5, 6));
+      ints.insert(ints.begin(), src.begin() + 1, src.begin() + 4);
+      EXPECT_THAT(ints, ElementsAre(1, 2, 3, 4, 5, 6));
+      ints.insert(ints.begin(), src.begin(), src.begin() + 1);
+      EXPECT_THAT(ints, ElementsAre(0, 1, 2, 3, 4, 5, 6));
+      ints.insert(ints.begin(), src.begin() + 7, src.begin() + 10);
+      EXPECT_THAT(ints, ElementsAre(7, 8, 9, 0, 1, 2, 3, 4, 5, 6));
+    }
+    {
+      // insert at end
+      IntVectorType<N> ints;
+      ints.insert(ints.end(), src.begin() + 4, src.begin() + 7);
+      EXPECT_THAT(ints, ElementsAre(4, 5, 6));
+      ints.insert(ints.end(), src.begin() + 1, src.begin() + 4);
+      EXPECT_THAT(ints, ElementsAre(4, 5, 6, 1, 2, 3));
+      ints.insert(ints.end(), src.begin(), src.begin() + 1);
+      EXPECT_THAT(ints, ElementsAre(4, 5, 6, 1, 2, 3, 0));
+      ints.insert(ints.end(), src.begin() + 7, src.begin() + 10);
+      EXPECT_THAT(ints, ElementsAre(4, 5, 6, 1, 2, 3, 0, 7, 8, 9));
+    }
+    {
+      // insert at some point inside
+      IntVectorType<N> ints;
+      ints.insert(ints.begin(), src.begin() + 4, src.begin() + 7);
+      EXPECT_THAT(ints, ElementsAre(4, 5, 6));
+      ints.insert(ints.begin() + 2, src.begin() + 1, src.begin() + 4);
+      EXPECT_THAT(ints, ElementsAre(4, 5, 1, 2, 3, 6));
+      ints.insert(ints.begin() + 2, src.begin(), src.begin() + 1);
+      EXPECT_THAT(ints, ElementsAre(4, 5, 0, 1, 2, 3, 6));
+      ints.insert(ints.begin() + 2, src.begin() + 7, src.begin() + 10);
+      EXPECT_THAT(ints, ElementsAre(4, 5, 7, 8, 9, 0, 1, 2, 3, 6));
+    }
+    {
+      // insert from a std::move_iterator (potentially move-only)
+      IntVectorType<N> src = MakeVector<N>({0, 1, 2, 3, 4, 5, 6, 7, 8, 9});
+      IntVectorType<N> ints;
+      auto move_it = [&](size_t i) { return std::make_move_iterator(src.begin() + i); };
+      ints.insert(ints.begin(), move_it(4), move_it(7));
+      EXPECT_THAT(ints, ElementsAre(4, 5, 6));
+      ints.insert(ints.begin() + 2, move_it(1), move_it(4));
+      EXPECT_THAT(ints, ElementsAre(4, 5, 1, 2, 3, 6));
+      ints.insert(ints.begin() + 2, move_it(0), move_it(1));
+      EXPECT_THAT(ints, ElementsAre(4, 5, 0, 1, 2, 3, 6));
+      ints.insert(ints.begin() + 2, move_it(7), move_it(10));
+      EXPECT_THAT(ints, ElementsAre(4, 5, 7, 8, 9, 0, 1, 2, 3, 6));
+    }
+  }
+};
+
+TYPED_TEST_SUITE(TestSmallStaticVector, VectorIntLikeParams);
+
+TYPED_TEST(TestSmallStaticVector, Basics) { this->TestBasics(); }
+
+TYPED_TEST(TestSmallStaticVector, AlwaysStatic) { this->TestAlwaysStatic(); }
+
+TYPED_TEST(TestSmallStaticVector, Reserve) { this->TestReserve(); }
+
+TYPED_TEST(TestSmallStaticVector, Clear) { this->TestClear(); }
+
+TYPED_TEST(TestSmallStaticVector, ConstructFromCount) { this->TestConstructFromCount(); }
+
+TYPED_TEST(TestSmallStaticVector, ConstructFromValues) {
+  this->TestConstructFromValues();
+}
+
+TYPED_TEST(TestSmallStaticVector, ConstructFromStdVector) {
+  this->TestConstructFromStdVector();
+}
+
+TYPED_TEST(TestSmallStaticVector, AssignFromStdVector) {
+  this->TestAssignFromStdVector();
+}
+
+TYPED_TEST(TestSmallStaticVector, Move) { this->TestMove(); }
+
+TYPED_TEST(TestSmallStaticVector, Copy) { this->TestCopy(); }
+
+TYPED_TEST(TestSmallStaticVector, Resize) { this->TestResize(); }
+
+TYPED_TEST(TestSmallStaticVector, Sort) { this->TestSort(); }
+
+TYPED_TEST(TestSmallStaticVector, Iterators) { this->TestIterators(); }
+
+TYPED_TEST(TestSmallStaticVector, ConstIterators) { this->TestConstIterators(); }
+
+TYPED_TEST(TestSmallStaticVector, InsertIteratorPair) { this->TestInsertIteratorPair(); }
+
+TEST(StaticVector, Traits) {
+  ASSERT_TRUE((std::is_trivially_destructible<StaticVector<int, 4>>::value));
+  ASSERT_FALSE((std::is_trivially_destructible<StaticVector<std::string, 4>>::value));
+}
+
+TEST(SmallVector, Traits) {
+  ASSERT_FALSE((std::is_trivially_destructible<SmallVector<int, 4>>::value));
+}
+
+}  // namespace internal
+}  // namespace arrow