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/12/20 14:51:38 UTC

[arrow] branch master updated: ARROW-4084: [C++] Make Status static method support variadic arguments

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 ce9c6e3  ARROW-4084: [C++] Make Status static method support variadic arguments
ce9c6e3 is described below

commit ce9c6e3914274dcaf7806159ea5373e0cb632727
Author: François Saint-Jacques <fs...@gmail.com>
AuthorDate: Thu Dec 20 08:51:31 2018 -0600

    ARROW-4084: [C++] Make Status static method support variadic arguments
    
    - Static constructors like `Status::Invalid` now supports variadic
    arguments à la `Status::Invalid("my", variable, "error message: ", i)`.
    
    - A new macro was added `ARROW_RETURN_IF(cond, status)` which replaces
    the previous `ARROW_RETURN_IF_FALSE` but also adds branch prediction
    hints. Note that only gandiva was refactored with this macro as
    otherwise the code review would have exploded.
    
    - Fixed a bug in memory map implementations not checking the return
    code of `mmap` and `mremap`.
    
    Author: François Saint-Jacques <fs...@gmail.com>
    
    Closes #3228 from fsaintjacques/ARROW-4084-variadic-status-message and squashes the following commits:
    
    a877ab994 <François Saint-Jacques> Travis
    890df68f9 <François Saint-Jacques> Remove gandiva expect string message testing
    71ecbae7d <François Saint-Jacques> Use perfect forwarding.
    774bf9387 <François Saint-Jacques> Add missing string header
    bf5cdfe06 <François Saint-Jacques> Removed code printing in status
    1d1db49c4 <François Saint-Jacques> Reformat
    d9fcad919 <François Saint-Jacques> ARROW-4084: Make Status static method support variadic arguments
---
 cpp/src/arrow/adapters/orc/adapter.cc              |  22 +--
 cpp/src/arrow/array.cc                             |  60 +++-----
 cpp/src/arrow/array/builder_binary.cc              |  21 ++-
 cpp/src/arrow/array/builder_nested.cc              |  12 +-
 cpp/src/arrow/builder.cc                           |   5 +-
 cpp/src/arrow/compute/kernels/cast.cc              |  51 +++----
 cpp/src/arrow/compute/kernels/hash.cc              |   8 +-
 cpp/src/arrow/csv/converter.cc                     |  23 ++-
 cpp/src/arrow/csv/parser.cc                        |   4 +-
 cpp/src/arrow/csv/reader.cc                        |   6 +-
 cpp/src/arrow/dbi/hiveserver2/hiveserver2-test.cc  |   6 +-
 cpp/src/arrow/dbi/hiveserver2/service.cc           |   4 +-
 cpp/src/arrow/dbi/hiveserver2/thrift-internal.cc   |   6 +-
 cpp/src/arrow/flight/internal.cc                   |  11 +-
 cpp/src/arrow/gpu/cuda_arrow_ipc.cc                |   5 +-
 cpp/src/arrow/gpu/cuda_common.h                    |  16 +-
 cpp/src/arrow/io/file-test.cc                      |   4 +-
 cpp/src/arrow/io/file.cc                           |   4 +-
 cpp/src/arrow/io/hdfs-internal.cc                  |   8 +-
 cpp/src/arrow/io/hdfs.cc                           |  34 ++---
 cpp/src/arrow/ipc/dictionary.cc                    |   8 +-
 cpp/src/arrow/ipc/feather.cc                       |   4 +-
 cpp/src/arrow/ipc/json-integration-test.cc         |  10 +-
 cpp/src/arrow/ipc/json-internal.cc                 |  42 ++----
 cpp/src/arrow/ipc/json-internal.h                  |  63 +++-----
 cpp/src/arrow/ipc/json-simple.cc                   |  42 ++----
 cpp/src/arrow/ipc/message.cc                       |  31 ++--
 cpp/src/arrow/ipc/metadata-internal.cc             |   8 +-
 cpp/src/arrow/ipc/reader.cc                        |  29 ++--
 cpp/src/arrow/memory_pool.cc                       |  20 +--
 cpp/src/arrow/python/arrow_to_pandas.cc            | 114 ++++++--------
 cpp/src/arrow/python/common.h                      |   6 +-
 cpp/src/arrow/python/decimal.cc                    |   8 +-
 cpp/src/arrow/python/helpers.cc                    |  24 +--
 cpp/src/arrow/python/inference.cc                  |  17 +--
 cpp/src/arrow/python/numpy-internal.h              |   5 +-
 cpp/src/arrow/python/numpy_convert.cc              |  12 +-
 cpp/src/arrow/python/numpy_to_arrow.cc             |  19 +--
 cpp/src/arrow/python/python_to_arrow.cc            |  15 +-
 cpp/src/arrow/python/serialize.cc                  |   7 +-
 cpp/src/arrow/python/util/datetime.h               |   4 +-
 cpp/src/arrow/record_batch.cc                      |  26 ++--
 cpp/src/arrow/status.cc                            |   1 +
 cpp/src/arrow/status.h                             | 155 +++++++++++--------
 cpp/src/arrow/table.cc                             |  52 +++----
 cpp/src/arrow/util/compression_brotli.cc           |   4 +-
 cpp/src/arrow/util/compression_lz4.cc              |  16 +-
 cpp/src/arrow/util/compression_snappy.cc           |   6 +-
 cpp/src/arrow/util/compression_zlib.cc             |  54 ++-----
 cpp/src/arrow/util/compression_zstd.cc             |   4 +-
 cpp/src/arrow/util/decimal.cc                      |  20 +--
 cpp/src/arrow/util/decimal.h                       |   5 +-
 cpp/src/arrow/util/io-util.cc                      |  54 +++----
 cpp/src/arrow/util/string_builder.h                |  51 +++++++
 cpp/src/gandiva/date_utils.cc                      |  19 +--
 cpp/src/gandiva/engine.cc                          |  29 ++--
 cpp/src/gandiva/expr_validator.cc                  | 168 +++++++++------------
 cpp/src/gandiva/filter.cc                          |  59 +++-----
 cpp/src/gandiva/like_holder.cc                     |  37 ++---
 cpp/src/gandiva/llvm_generator.cc                  |  38 ++---
 cpp/src/gandiva/projector.cc                       | 120 ++++++---------
 cpp/src/gandiva/regex_util.cc                      |  14 +-
 cpp/src/gandiva/selection_vector.cc                |  82 ++++------
 .../tests/projector_build_validation_test.cc       |  13 +-
 cpp/src/parquet/arrow/reader.cc                    |  11 +-
 cpp/src/parquet/arrow/schema.cc                    |  28 ++--
 cpp/src/parquet/arrow/writer.cc                    |  11 +-
 cpp/src/plasma/io.cc                               |  10 +-
 68 files changed, 763 insertions(+), 1122 deletions(-)

diff --git a/cpp/src/arrow/adapters/orc/adapter.cc b/cpp/src/arrow/adapters/orc/adapter.cc
index de803d5..01fc09a 100644
--- a/cpp/src/arrow/adapters/orc/adapter.cc
+++ b/cpp/src/arrow/adapters/orc/adapter.cc
@@ -206,11 +206,7 @@ Status GetArrowType(const liborc::Type* type, std::shared_ptr<DataType>* out) {
       *out = union_(fields, type_codes);
       break;
     }
-    default: {
-      std::stringstream ss;
-      ss << "Unknown Orc type kind: " << kind;
-      return Status::Invalid(ss.str());
-    }
+    default: { return Status::Invalid("Unknown Orc type kind: ", kind); }
   }
   return Status::OK();
 }
@@ -346,11 +342,9 @@ class ORCFileReader::Impl {
   }
 
   Status SelectStripe(liborc::RowReaderOptions* opts, int64_t stripe) {
-    if (stripe < 0 || stripe >= NumberOfStripes()) {
-      std::stringstream ss;
-      ss << "Out of bounds stripe: " << stripe;
-      return Status::Invalid(ss.str());
-    }
+    ARROW_RETURN_IF(stripe < 0 || stripe >= NumberOfStripes(),
+                    Status::Invalid("Out of bounds stripe: ", stripe));
+
     opts->range(stripes_[stripe].offset, stripes_[stripe].length);
     return Status::OK();
   }
@@ -359,9 +353,7 @@ class ORCFileReader::Impl {
                        const std::vector<int>& include_indices) {
     std::list<uint64_t> include_indices_list;
     for (auto it = include_indices.begin(); it != include_indices.end(); ++it) {
-      if (*it < 0) {
-        return Status::Invalid("Negative field index");
-      }
+      ARROW_RETURN_IF(*it < 0, Status::Invalid("Negative field index"));
       include_indices_list.push_back(*it);
     }
     opts->includeTypes(include_indices_list);
@@ -455,9 +447,7 @@ class ORCFileReader::Impl {
       case liborc::DECIMAL:
         return AppendDecimalBatch(type, batch, offset, length, builder);
       default:
-        std::stringstream ss;
-        ss << "Not implemented type kind: " << kind;
-        return Status::NotImplemented(ss.str());
+        return Status::NotImplemented("Not implemented type kind: ", kind);
     }
   }
 
diff --git a/cpp/src/arrow/array.cc b/cpp/src/arrow/array.cc
index d07c27f..66a685b 100644
--- a/cpp/src/arrow/array.cc
+++ b/cpp/src/arrow/array.cc
@@ -638,9 +638,8 @@ Status DictionaryArray::FromArrays(const std::shared_ptr<DataType>& type,
       is_valid = ValidateDictionaryIndices<Int64Type>(indices, upper_bound);
       break;
     default:
-      std::stringstream ss;
-      ss << "Categorical index type not supported: " << indices->type()->ToString();
-      return Status::NotImplemented(ss.str());
+      return Status::NotImplemented("Categorical index type not supported: ",
+                                    indices->type()->ToString());
   }
 
   if (!is_valid.ok()) {
@@ -740,12 +739,11 @@ struct ValidateVisitor {
   Status Visit(const NullArray&) { return Status::OK(); }
 
   Status Visit(const PrimitiveArray& array) {
-    if (array.data()->buffers.size() != 2) {
-      return Status::Invalid("number of buffers was != 2");
-    }
-    if (array.values() == nullptr) {
-      return Status::Invalid("values was null");
-    }
+    ARROW_RETURN_IF(array.data()->buffers.size() != 2,
+                    Status::Invalid("number of buffers was != 2"));
+
+    ARROW_RETURN_IF(array.values() == nullptr, Status::Invalid("values was null"));
+
     return Status::OK();
   }
 
@@ -776,10 +774,8 @@ struct ValidateVisitor {
       return Status::Invalid("value_offsets_ was null");
     }
     if (value_offsets->size() / static_cast<int>(sizeof(int32_t)) < array.length()) {
-      std::stringstream ss;
-      ss << "offset buffer size (bytes): " << value_offsets->size()
-         << " isn't large enough for length: " << array.length();
-      return Status::Invalid(ss.str());
+      return Status::Invalid("offset buffer size (bytes): ", value_offsets->size(),
+                             " isn't large enough for length: ", array.length());
     }
 
     if (!array.values()) {
@@ -788,17 +784,13 @@ struct ValidateVisitor {
 
     const int32_t last_offset = array.value_offset(array.length());
     if (array.values()->length() != last_offset) {
-      std::stringstream ss;
-      ss << "Final offset invariant not equal to values length: " << last_offset
-         << "!=" << array.values()->length();
-      return Status::Invalid(ss.str());
+      return Status::Invalid("Final offset invariant not equal to values length: ",
+                             last_offset, "!=", array.values()->length());
     }
 
     const Status child_valid = ValidateArray(*array.values());
     if (!child_valid.ok()) {
-      std::stringstream ss;
-      ss << "Child array invalid: " << child_valid.ToString();
-      return Status::Invalid(ss.str());
+      return Status::Invalid("Child array invalid: ", child_valid.ToString());
     }
 
     int32_t prev_offset = array.value_offset(0);
@@ -808,18 +800,14 @@ struct ValidateVisitor {
     for (int64_t i = 1; i <= array.length(); ++i) {
       int32_t current_offset = array.value_offset(i);
       if (array.IsNull(i - 1) && current_offset != prev_offset) {
-        std::stringstream ss;
-        ss << "Offset invariant failure at: " << i
-           << " inconsistent value_offsets for null slot" << current_offset
-           << "!=" << prev_offset;
-        return Status::Invalid(ss.str());
+        return Status::Invalid("Offset invariant failure at: ", i,
+                               " inconsistent value_offsets for null slot",
+                               current_offset, "!=", prev_offset);
       }
       if (current_offset < prev_offset) {
-        std::stringstream ss;
-        ss << "Offset invariant failure: " << i
-           << " inconsistent offset for non-null slot: " << current_offset << "<"
-           << prev_offset;
-        return Status::Invalid(ss.str());
+        return Status::Invalid("Offset invariant failure: ", i,
+                               " inconsistent offset for non-null slot: ", current_offset,
+                               "<", prev_offset);
       }
       prev_offset = current_offset;
     }
@@ -842,18 +830,14 @@ struct ValidateVisitor {
       for (int i = 0; i < array.num_fields(); ++i) {
         auto it = array.field(i);
         if (it->length() != array_length) {
-          std::stringstream ss;
-          ss << "Length is not equal from field " << it->type()->ToString()
-             << " at position {" << idx << "}";
-          return Status::Invalid(ss.str());
+          return Status::Invalid("Length is not equal from field ",
+                                 it->type()->ToString(), " at position [", idx, "]");
         }
 
         const Status child_valid = ValidateArray(*it);
         if (!child_valid.ok()) {
-          std::stringstream ss;
-          ss << "Child array invalid: " << child_valid.ToString() << " at position {"
-             << idx << "}";
-          return Status::Invalid(ss.str());
+          return Status::Invalid("Child array invalid: ", child_valid.ToString(),
+                                 " at position [", idx, "}");
         }
         ++idx;
       }
diff --git a/cpp/src/arrow/array/builder_binary.cc b/cpp/src/arrow/array/builder_binary.cc
index ad6ba11..8739859 100644
--- a/cpp/src/arrow/array/builder_binary.cc
+++ b/cpp/src/arrow/array/builder_binary.cc
@@ -59,21 +59,18 @@ Status BinaryBuilder::Resize(int64_t capacity) {
 }
 
 Status BinaryBuilder::ReserveData(int64_t elements) {
-  if (value_data_length() + elements > value_data_capacity()) {
-    if (value_data_length() + elements > kBinaryMemoryLimit) {
-      return Status::CapacityError(
-          "Cannot reserve capacity larger than 2^31 - 1 for binary");
-    }
-    RETURN_NOT_OK(value_data_builder_.Reserve(elements));
-  }
-  return Status::OK();
+  const int64_t size = value_data_length() + elements;
+  ARROW_RETURN_IF(
+      size > kBinaryMemoryLimit,
+      Status::CapacityError("Cannot reserve capacity larger than 2^31 - 1 for binary"));
+
+  return (size > value_data_capacity()) ? value_data_builder_.Reserve(elements)
+                                        : Status::OK();
 }
 
 Status BinaryBuilder::AppendOverflow(int64_t num_bytes) {
-  std::stringstream ss;
-  ss << "BinaryArray cannot contain more than " << kBinaryMemoryLimit << " bytes, have "
-     << num_bytes;
-  return Status::CapacityError(ss.str());
+  return Status::CapacityError("BinaryArray cannot contain more than ",
+                               kBinaryMemoryLimit, " bytes, have ", num_bytes);
 }
 
 Status BinaryBuilder::FinishInternal(std::shared_ptr<ArrayData>* out) {
diff --git a/cpp/src/arrow/array/builder_nested.cc b/cpp/src/arrow/array/builder_nested.cc
index e733243..87c302a 100644
--- a/cpp/src/arrow/array/builder_nested.cc
+++ b/cpp/src/arrow/array/builder_nested.cc
@@ -58,13 +58,11 @@ Status ListBuilder::AppendValues(const int32_t* offsets, int64_t length,
 }
 
 Status ListBuilder::AppendNextOffset() {
-  int64_t num_values = value_builder_->length();
-  if (ARROW_PREDICT_FALSE(num_values > kListMaximumElements)) {
-    std::stringstream ss;
-    ss << "ListArray cannot contain more then INT32_MAX - 1 child elements,"
-       << " have " << num_values;
-    return Status::CapacityError(ss.str());
-  }
+  const int64_t num_values = value_builder_->length();
+  ARROW_RETURN_IF(
+      num_values > kListMaximumElements,
+      Status::CapacityError("ListArray cannot contain more then 2^31 - 1 child elements,",
+                            " have ", num_values));
   return offsets_builder_.Append(static_cast<int32_t>(num_values));
 }
 
diff --git a/cpp/src/arrow/builder.cc b/cpp/src/arrow/builder.cc
index ff2b453..2072edc 100644
--- a/cpp/src/arrow/builder.cc
+++ b/cpp/src/arrow/builder.cc
@@ -93,9 +93,8 @@ Status MakeBuilder(MemoryPool* pool, const std::shared_ptr<DataType>& type,
     }
 
     default: {
-      std::stringstream ss;
-      ss << "MakeBuilder: cannot construct builder for type " << type->ToString();
-      return Status::NotImplemented(ss.str());
+      return Status::NotImplemented("MakeBuilder: cannot construct builder for type ",
+                                    type->ToString());
     }
   }
 }
diff --git a/cpp/src/arrow/compute/kernels/cast.cc b/cpp/src/arrow/compute/kernels/cast.cc
index b148486..2ce0702 100644
--- a/cpp/src/arrow/compute/kernels/cast.cc
+++ b/cpp/src/arrow/compute/kernels/cast.cc
@@ -508,11 +508,9 @@ void ShiftTime(FunctionContext* ctx, const CastOptions& options, const bool is_m
         out_data[i] = static_cast<out_type>(in_data[i] / factor);
       }
     } else {
-#define RAISE_INVALID_CAST(VAL)                                                         \
-  std::stringstream ss;                                                                 \
-  ss << "Casting from " << input.type->ToString() << " to " << output->type->ToString() \
-     << " would lose data: " << VAL;                                                    \
-  ctx->SetStatus(Status::Invalid(ss.str()));
+#define RAISE_INVALID_CAST(VAL)                                                   \
+  ctx->SetStatus(Status::Invalid("Casting from ", input.type->ToString(), " to ", \
+                                 output->type->ToString(), " would lose data: ", VAL));
 
       if (input.null_count != 0) {
         internal::BitmapReader bit_reader(input.buffers[0]->data(), input.offset,
@@ -795,9 +793,8 @@ struct CastFunctor<
         UnpackFixedSizeBinaryDictionary<Int64Type>(ctx, indices, dictionary, output);
         break;
       default:
-        std::stringstream ss;
-        ss << "Invalid index type: " << indices.type()->ToString();
-        ctx->SetStatus(Status::Invalid(ss.str()));
+        ctx->SetStatus(
+            Status::Invalid("Invalid index type: ", indices.type()->ToString()));
         return;
     }
   }
@@ -874,9 +871,8 @@ struct CastFunctor<T, DictionaryType,
             (UnpackBinaryDictionary<Int64Type>(ctx, indices, dictionary, output)));
         break;
       default:
-        std::stringstream ss;
-        ss << "Invalid index type: " << indices.type()->ToString();
-        ctx->SetStatus(Status::Invalid(ss.str()));
+        ctx->SetStatus(
+            Status::Invalid("Invalid index type: ", indices.type()->ToString()));
         return;
     }
   }
@@ -932,9 +928,8 @@ struct CastFunctor<T, DictionaryType,
         UnpackPrimitiveDictionary<Int64Type, c_type>(indices, dictionary, out);
         break;
       default:
-        std::stringstream ss;
-        ss << "Invalid index type: " << indices.type()->ToString();
-        ctx->SetStatus(Status::Invalid(ss.str()));
+        ctx->SetStatus(
+            Status::Invalid("Invalid index type: ", indices.type()->ToString()));
         return;
     }
   }
@@ -960,9 +955,8 @@ struct CastFunctor<O, StringType, enable_if_number<O>> {
 
       auto str = input_array.GetView(i);
       if (!converter(str.data(), str.length(), out_data)) {
-        std::stringstream ss;
-        ss << "Failed to cast String '" << str << "' into " << output->type->ToString();
-        ctx->SetStatus(Status(StatusCode::Invalid, ss.str()));
+        ctx->SetStatus(Status::Invalid("Failed to cast String '", str, "' into ",
+                                       output->type->ToString()));
         return;
       }
     }
@@ -991,10 +985,9 @@ struct CastFunctor<O, StringType,
       bool value;
       auto str = input_array.GetView(i);
       if (!converter(str.data(), str.length(), &value)) {
-        std::stringstream ss;
-        ss << "Failed to cast String '" << input_array.GetString(i) << "' into "
-           << output->type->ToString();
-        ctx->SetStatus(Status(StatusCode::Invalid, ss.str()));
+        ctx->SetStatus(Status::Invalid("Failed to cast String '",
+                                       input_array.GetString(i), "' into ",
+                                       output->type->ToString()));
         return;
       }
 
@@ -1029,9 +1022,8 @@ struct CastFunctor<TimestampType, StringType> {
 
       const auto str = input_array.GetView(i);
       if (!converter(str.data(), str.length(), out_data)) {
-        std::stringstream ss;
-        ss << "Failed to cast String '" << str << "' into " << output->type->ToString();
-        ctx->SetStatus(Status(StatusCode::Invalid, ss.str()));
+        ctx->SetStatus(Status::Invalid("Failed to cast String '", str, "' into ",
+                                       output->type->ToString()));
         return;
       }
     }
@@ -1123,9 +1115,8 @@ static Status AllocateIfNotPreallocated(FunctionContext* ctx, const ArrayData& i
 
     if (!(is_primitive(type_id) || type_id == Type::FIXED_SIZE_BINARY ||
           type_id == Type::DECIMAL)) {
-      std::stringstream ss;
-      ss << "Cannot pre-allocate memory for type: " << out->type->ToString();
-      return Status::NotImplemented(ss.str());
+      return Status::NotImplemented("Cannot pre-allocate memory for type: ",
+                                    out->type->ToString());
     }
 
     if (type_id != Type::NA) {
@@ -1400,10 +1391,8 @@ Status GetCastFunction(const DataType& in_type, const std::shared_ptr<DataType>&
       break;
   }
   if (*kernel == nullptr) {
-    std::stringstream ss;
-    ss << "No cast implemented from " << in_type.ToString() << " to "
-       << out_type->ToString();
-    return Status::NotImplemented(ss.str());
+    return Status::NotImplemented("No cast implemented from ", in_type.ToString(), " to ",
+                                  out_type->ToString());
   }
   return Status::OK();
 }
diff --git a/cpp/src/arrow/compute/kernels/hash.cc b/cpp/src/arrow/compute/kernels/hash.cc
index c057ea5..0513fe1 100644
--- a/cpp/src/arrow/compute/kernels/hash.cc
+++ b/cpp/src/arrow/compute/kernels/hash.cc
@@ -56,11 +56,9 @@ namespace compute {
 
 namespace {
 
-#define CHECK_IMPLEMENTED(KERNEL, FUNCNAME, TYPE)                  \
-  if (!KERNEL) {                                                   \
-    std::stringstream ss;                                          \
-    ss << FUNCNAME << " not implemented for " << type->ToString(); \
-    return Status::NotImplemented(ss.str());                       \
+#define CHECK_IMPLEMENTED(KERNEL, FUNCNAME, TYPE)                                       \
+  if (!KERNEL) {                                                                        \
+    return Status::NotImplemented(FUNCNAME, " not implemented for ", type->ToString()); \
   }
 
 // ----------------------------------------------------------------------
diff --git a/cpp/src/arrow/csv/converter.cc b/cpp/src/arrow/csv/converter.cc
index 8a249a6..1018f85 100644
--- a/cpp/src/arrow/csv/converter.cc
+++ b/cpp/src/arrow/csv/converter.cc
@@ -40,10 +40,9 @@ namespace {
 
 Status GenericConversionError(const std::shared_ptr<DataType>& type, const uint8_t* data,
                               uint32_t size) {
-  std::stringstream ss;
-  ss << "CSV conversion error to " << type->ToString() << ": invalid value '"
-     << std::string(reinterpret_cast<const char*>(data), size) << "'";
-  return Status::Invalid(ss.str());
+  return Status::Invalid("CSV conversion error to ", type->ToString(),
+                         ": invalid value '",
+                         std::string(reinterpret_cast<const char*>(data), size), "'");
 }
 
 inline bool IsWhitespace(uint8_t c) {
@@ -214,9 +213,8 @@ class VarSizeBinaryConverter : public ConcreteConverter {
 
     auto visit = [&](const uint8_t* data, uint32_t size, bool quoted) -> Status {
       if (CheckUTF8 && ARROW_PREDICT_FALSE(!util::ValidateUTF8(data, size))) {
-        std::stringstream ss;
-        ss << "CSV conversion error to " << type_->ToString() << ": invalid UTF8 data";
-        return Status::Invalid(ss.str());
+        return Status::Invalid("CSV conversion error to ", type_->ToString(),
+                               ": invalid UTF8 data");
       }
       builder.UnsafeAppend(data, size);
       return Status::OK();
@@ -256,10 +254,8 @@ Status FixedSizeBinaryConverter::Convert(const BlockParser& parser, int32_t col_
 
   auto visit = [&](const uint8_t* data, uint32_t size, bool quoted) -> Status {
     if (ARROW_PREDICT_FALSE(size != byte_width)) {
-      std::stringstream ss;
-      ss << "CSV conversion error to " << type_->ToString() << ": got a " << size
-         << "-byte long string";
-      return Status::Invalid(ss.str());
+      return Status::Invalid("CSV conversion error to ", type_->ToString(), ": got a ",
+                             size, "-byte long string");
     }
     return builder.Append(data);
   };
@@ -410,9 +406,8 @@ Status Converter::Make(const std::shared_ptr<DataType>& type,
       break;
 
     default: {
-      std::stringstream ss;
-      ss << "CSV conversion to " << type->ToString() << " is not supported";
-      return Status::NotImplemented(ss.str());
+      return Status::NotImplemented("CSV conversion to ", type->ToString(),
+                                    " is not supported");
     }
 
 #undef CONVERTER_CASE
diff --git a/cpp/src/arrow/csv/parser.cc b/cpp/src/arrow/csv/parser.cc
index fe7f841..b1d175a 100644
--- a/cpp/src/arrow/csv/parser.cc
+++ b/cpp/src/arrow/csv/parser.cc
@@ -30,9 +30,7 @@ namespace arrow {
 namespace csv {
 
 static Status ParseError(const char* message) {
-  std::stringstream ss;
-  ss << "CSV parse error: " << message;
-  return Status::Invalid(ss.str());
+  return Status::Invalid("CSV parse error: ", message);
 }
 
 static Status MismatchingColumns(int32_t expected, int32_t actual) {
diff --git a/cpp/src/arrow/csv/reader.cc b/cpp/src/arrow/csv/reader.cc
index b2a6b7b..efd6116 100644
--- a/cpp/src/arrow/csv/reader.cc
+++ b/cpp/src/arrow/csv/reader.cc
@@ -355,10 +355,8 @@ class ThreadedTableReader : public BaseTableReader {
                                       chunk_size, &parsed_size));
           if (parsed_size != chunk_size) {
             DCHECK_EQ(parsed_size, chunk_size);
-            std::stringstream ss;
-            ss << "Chunker and parser disagree on block size: " << chunk_size << " vs "
-               << parsed_size;
-            return Status::Invalid(ss.str());
+            return Status::Invalid("Chunker and parser disagree on block size: ",
+                                   chunk_size, " vs ", parsed_size);
           }
           RETURN_NOT_OK(ProcessData(parser, chunk_index));
           // Keep chunk buffer alive within closure and release it at the end
diff --git a/cpp/src/arrow/dbi/hiveserver2/hiveserver2-test.cc b/cpp/src/arrow/dbi/hiveserver2/hiveserver2-test.cc
index 7022ff0..a774916 100644
--- a/cpp/src/arrow/dbi/hiveserver2/hiveserver2-test.cc
+++ b/cpp/src/arrow/dbi/hiveserver2/hiveserver2-test.cc
@@ -97,10 +97,8 @@ Status Wait(const std::unique_ptr<Operation>& op,
   if (op_state == state) {
     return Status::OK();
   } else {
-    std::stringstream ss;
-    ss << "Failed to reach state '" << OperationStateToString(state) << "' after "
-       << retries << " retries.";
-    return Status::IOError(ss.str());
+    return Status::IOError("Failed to reach state '", OperationStateToString(state),
+                           "' after ", retries, " retries");
   }
 }
 
diff --git a/cpp/src/arrow/dbi/hiveserver2/service.cc b/cpp/src/arrow/dbi/hiveserver2/service.cc
index e2d3f2a..502a8a2 100644
--- a/cpp/src/arrow/dbi/hiveserver2/service.cc
+++ b/cpp/src/arrow/dbi/hiveserver2/service.cc
@@ -92,9 +92,7 @@ Service::Service(const string& host, int port, int conn_timeout,
 
 Status Service::Open() {
   if (impl_->protocol_version < hs2::TProtocolVersion::HIVE_CLI_SERVICE_PROTOCOL_V6) {
-    std::stringstream ss;
-    ss << "Unsupported protocol: " << impl_->protocol_version;
-    return Status::NotImplemented(ss.str());
+    return Status::NotImplemented("Unsupported protocol: ", impl_->protocol_version);
   }
 
   impl_->socket.reset(new TSocket(host_, port_));
diff --git a/cpp/src/arrow/dbi/hiveserver2/thrift-internal.cc b/cpp/src/arrow/dbi/hiveserver2/thrift-internal.cc
index d154e14..171eae3 100644
--- a/cpp/src/arrow/dbi/hiveserver2/thrift-internal.cc
+++ b/cpp/src/arrow/dbi/hiveserver2/thrift-internal.cc
@@ -204,11 +204,7 @@ Status TStatusToStatus(const hs2::TStatus& tstatus) {
       return Status::IOError(tstatus.errorMessage);
     case hs2::TStatusCode::INVALID_HANDLE_STATUS:
       return Status::Invalid("Invalid handle");
-    default: {
-      std::stringstream ss;
-      ss << "Unknown TStatusCode " << tstatus.statusCode;
-      return Status::UnknownError(ss.str());
-    }
+    default: { return Status::UnknownError("Unknown TStatusCode ", tstatus.statusCode); }
   }
 }
 
diff --git a/cpp/src/arrow/flight/internal.cc b/cpp/src/arrow/flight/internal.cc
index 796e609..b4c6b2a 100644
--- a/cpp/src/arrow/flight/internal.cc
+++ b/cpp/src/arrow/flight/internal.cc
@@ -37,16 +37,13 @@ Status FromGrpcStatus(const grpc::Status& grpc_status) {
   if (grpc_status.ok()) {
     return Status::OK();
   }
-  std::stringstream ss;
 
   if (grpc_status.error_code() == grpc::StatusCode::UNIMPLEMENTED) {
-    ss << "gRPC returned unimplemented error, with message: "
-       << grpc_status.error_message();
-    return Status::NotImplemented(ss.str());
+    return Status::NotImplemented("gRPC returned unimplemented error, with message: ",
+                                  grpc_status.error_message());
   } else {
-    ss << "gRPC failed with error code " << grpc_status.error_code()
-       << " and message: " << grpc_status.error_message();
-    return Status::IOError(ss.str());
+    return Status::IOError("gRPC failed with error code ", grpc_status.error_code(),
+                           " and message: ", grpc_status.error_message());
   }
 }
 
diff --git a/cpp/src/arrow/gpu/cuda_arrow_ipc.cc b/cpp/src/arrow/gpu/cuda_arrow_ipc.cc
index 03256a1..b4d8744 100644
--- a/cpp/src/arrow/gpu/cuda_arrow_ipc.cc
+++ b/cpp/src/arrow/gpu/cuda_arrow_ipc.cc
@@ -82,9 +82,8 @@ Status ReadMessage(CudaBufferReader* reader, MemoryPool* pool,
   RETURN_NOT_OK(AllocateBuffer(pool, message_length, &metadata));
   RETURN_NOT_OK(reader->Read(message_length, &bytes_read, metadata->mutable_data()));
   if (bytes_read != message_length) {
-    std::stringstream ss;
-    ss << "Expected " << message_length << " metadata bytes, but only got " << bytes_read;
-    return Status::IOError(ss.str());
+    return Status::IOError("Expected ", message_length, " metadata bytes, but only got ",
+                           bytes_read);
   }
 
   return ipc::Message::ReadFrom(metadata, reader, out);
diff --git a/cpp/src/arrow/gpu/cuda_common.h b/cpp/src/arrow/gpu/cuda_common.h
index a53dd22..2b630c8 100644
--- a/cpp/src/arrow/gpu/cuda_common.h
+++ b/cpp/src/arrow/gpu/cuda_common.h
@@ -34,15 +34,13 @@ namespace cuda {
     (void)ret;            \
   } while (0)
 
-#define CU_RETURN_NOT_OK(STMT)                                                \
-  do {                                                                        \
-    CUresult ret = (STMT);                                                    \
-    if (ret != CUDA_SUCCESS) {                                                \
-      std::stringstream ss;                                                   \
-      ss << "Cuda Driver API call in " << __FILE__ << " at line " << __LINE__ \
-         << " failed with code " << ret << ": " << #STMT;                     \
-      return Status::IOError(ss.str());                                       \
-    }                                                                         \
+#define CU_RETURN_NOT_OK(STMT)                                                  \
+  do {                                                                          \
+    CUresult ret = (STMT);                                                      \
+    if (ret != CUDA_SUCCESS) {                                                  \
+      return Status::IOError("Cuda Driver API call in ", __FILE__, " at line ", \
+                             __LINE__, " failed with code ", ret, ": ", #STMT); \
+    }                                                                           \
   } while (0)
 
 }  // namespace cuda
diff --git a/cpp/src/arrow/io/file-test.cc b/cpp/src/arrow/io/file-test.cc
index 4d710d3..6d780c0 100644
--- a/cpp/src/arrow/io/file-test.cc
+++ b/cpp/src/arrow/io/file-test.cc
@@ -460,9 +460,7 @@ class MyMemoryPool : public MemoryPool {
     *ptr = reinterpret_cast<uint8_t*>(std::realloc(*ptr, new_size));
 
     if (*ptr == NULL) {
-      std::stringstream ss;
-      ss << "realloc of size " << new_size << " failed";
-      return Status::OutOfMemory(ss.str());
+      return Status::OutOfMemory("realloc of size ", new_size, " failed");
     }
 
     return Status::OK();
diff --git a/cpp/src/arrow/io/file.cc b/cpp/src/arrow/io/file.cc
index 869d8e3..0398d5a 100644
--- a/cpp/src/arrow/io/file.cc
+++ b/cpp/src/arrow/io/file.cc
@@ -479,9 +479,7 @@ class MemoryMappedFile::MemoryMap : public MutableBuffer {
     void* result = mmap(nullptr, static_cast<size_t>(initial_size), prot_flags_,
                         map_mode_, file_->fd(), 0);
     if (result == MAP_FAILED) {
-      std::stringstream ss;
-      ss << "Memory mapping file failed: " << std::strerror(errno);
-      return Status::IOError(ss.str());
+      return Status::IOError("Memory mapping file failed: ", std::strerror(errno));
     }
     size_ = capacity_ = initial_size;
     data_ = mutable_data_ = static_cast<uint8_t*>(result);
diff --git a/cpp/src/arrow/io/hdfs-internal.cc b/cpp/src/arrow/io/hdfs-internal.cc
index c8be516..c273ab4 100644
--- a/cpp/src/arrow/io/hdfs-internal.cc
+++ b/cpp/src/arrow/io/hdfs-internal.cc
@@ -218,9 +218,7 @@ static arrow::Status try_dlopen(std::vector<fs::path> potential_paths, const cha
   }
 
   if (out_handle == NULL) {
-    std::stringstream ss;
-    ss << "Unable to load " << name;
-    return arrow::Status::IOError(ss.str());
+    return arrow::Status::IOError("Unable to load ", name);
   }
 
   return arrow::Status::OK();
@@ -243,9 +241,7 @@ static arrow::Status try_dlopen(std::vector<fs::path> potential_paths, const cha
   }
 
   if (out_handle == NULL) {
-    std::stringstream ss;
-    ss << "Unable to load " << name;
-    return arrow::Status::IOError(ss.str());
+    return arrow::Status::IOError("Unable to load ", name);
   }
 
   return arrow::Status::OK();
diff --git a/cpp/src/arrow/io/hdfs.cc b/cpp/src/arrow/io/hdfs.cc
index 030b848..3e9b804 100644
--- a/cpp/src/arrow/io/hdfs.cc
+++ b/cpp/src/arrow/io/hdfs.cc
@@ -57,13 +57,11 @@ std::string TranslateErrno(int error_code) {
 
 }  // namespace
 
-#define CHECK_FAILURE(RETURN_VALUE, WHAT)                                   \
-  do {                                                                      \
-    if (RETURN_VALUE == -1) {                                               \
-      std::stringstream ss;                                                 \
-      ss << "HDFS " << WHAT << " failed, errno: " << TranslateErrno(errno); \
-      return Status::IOError(ss.str());                                     \
-    }                                                                       \
+#define CHECK_FAILURE(RETURN_VALUE, WHAT)                                               \
+  do {                                                                                  \
+    if (RETURN_VALUE == -1) {                                                           \
+      return Status::IOError("HDFS ", WHAT, " failed, errno: ", TranslateErrno(errno)); \
+    }                                                                                   \
   } while (0)
 
 static constexpr int kDefaultHdfsBufferSize = 1 << 16;
@@ -466,10 +464,8 @@ class HadoopFileSystem::HadoopFileSystemImpl {
       if ((errno == 0) || (errno == ENOENT && Exists(path))) {
         num_entries = 0;
       } else {
-        std::stringstream ss;
-        ss << "HDFS list directory of " << path
-           << " failed, errno: " << TranslateErrno(errno);
-        return Status::IOError(ss.str());
+        return Status::IOError("HDFS list directory failed, errno: ",
+                               TranslateErrno(errno));
       }
     }
 
@@ -492,14 +488,9 @@ class HadoopFileSystem::HadoopFileSystemImpl {
     hdfsFile handle = driver_->OpenFile(fs_, path.c_str(), O_RDONLY, buffer_size, 0, 0);
 
     if (handle == nullptr) {
-      std::stringstream ss;
-      if (!Exists(path)) {
-        ss << "HDFS file does not exist: " << path;
-      } else {
-        // TODO(wesm): determine other causes of failure
-        ss << "HDFS path exists, but opening file failed: " << path;
-      }
-      return Status::IOError(ss.str());
+      const char* msg = !Exists(path) ? "HDFS file does not exist: "
+                                      : "HDFS path exists, but opening file failed: ";
+      return Status::IOError(msg, path);
     }
 
     // std::make_shared does not work with private ctors
@@ -521,10 +512,7 @@ class HadoopFileSystem::HadoopFileSystemImpl {
                           static_cast<tSize>(default_block_size));
 
     if (handle == nullptr) {
-      // TODO(wesm): determine cause of failure
-      std::stringstream ss;
-      ss << "Unable to open file " << path;
-      return Status::IOError(ss.str());
+      return Status::IOError("Unable to open file ", path);
     }
 
     // std::make_shared does not work with private ctors
diff --git a/cpp/src/arrow/ipc/dictionary.cc b/cpp/src/arrow/ipc/dictionary.cc
index 488bb75..aa0d908 100644
--- a/cpp/src/arrow/ipc/dictionary.cc
+++ b/cpp/src/arrow/ipc/dictionary.cc
@@ -34,9 +34,7 @@ Status DictionaryMemo::GetDictionary(int64_t id,
                                      std::shared_ptr<Array>* dictionary) const {
   auto it = id_to_dictionary_.find(id);
   if (it == id_to_dictionary_.end()) {
-    std::stringstream ss;
-    ss << "Dictionary with id " << id << " not found";
-    return Status::KeyError(ss.str());
+    return Status::KeyError("Dictionary with id ", id, " not found");
   }
   *dictionary = it->second;
   return Status::OK();
@@ -70,9 +68,7 @@ bool DictionaryMemo::HasDictionaryId(int64_t id) const {
 Status DictionaryMemo::AddDictionary(int64_t id,
                                      const std::shared_ptr<Array>& dictionary) {
   if (HasDictionaryId(id)) {
-    std::stringstream ss;
-    ss << "Dictionary with id " << id << " already exists";
-    return Status::KeyError(ss.str());
+    return Status::KeyError("Dictionary with id ", id, " already exists");
   }
   intptr_t address = reinterpret_cast<intptr_t>(dictionary.get());
   id_to_dictionary_[id] = dictionary;
diff --git a/cpp/src/arrow/ipc/feather.cc b/cpp/src/arrow/ipc/feather.cc
index ebdb335..b0ab62c 100644
--- a/cpp/src/arrow/ipc/feather.cc
+++ b/cpp/src/arrow/ipc/feather.cc
@@ -642,9 +642,7 @@ class TableWriter::TableWriterImpl : public ArrayVisitor {
 
   Status LoadArrayMetadata(const Array& values, ArrayMetadata* meta) {
     if (!(is_primitive(values.type_id()) || is_binary_like(values.type_id()))) {
-      std::stringstream ss;
-      ss << "Array is not primitive type: " << values.type()->ToString();
-      return Status::Invalid(ss.str());
+      return Status::Invalid("Array is not primitive type: ", values.type()->ToString());
     }
 
     meta->type = ToFlatbufferType(values.type_id());
diff --git a/cpp/src/arrow/ipc/json-integration-test.cc b/cpp/src/arrow/ipc/json-integration-test.cc
index 914cdb6..fe69a53 100644
--- a/cpp/src/arrow/ipc/json-integration-test.cc
+++ b/cpp/src/arrow/ipc/json-integration-test.cc
@@ -170,10 +170,8 @@ static Status ValidateArrowVsJson(const std::string& arrow_path,
   const int arrow_nbatches = arrow_reader->num_record_batches();
 
   if (json_nbatches != arrow_nbatches) {
-    std::stringstream ss;
-    ss << "Different number of record batches: " << json_nbatches << " (JSON) vs "
-       << arrow_nbatches << " (Arrow)";
-    return Status::Invalid(ss.str());
+    return Status::Invalid("Different number of record batches: ", json_nbatches,
+                           " (JSON) vs ", arrow_nbatches, " (Arrow)");
   }
 
   std::shared_ptr<RecordBatch> arrow_batch;
@@ -231,9 +229,7 @@ Status RunCommand(const std::string& json_path, const std::string& arrow_path,
 
     return ValidateArrowVsJson(arrow_path, json_path);
   } else {
-    std::stringstream ss;
-    ss << "Unknown command: " << command;
-    return Status::Invalid(ss.str());
+    return Status::Invalid("Unknown command: ", command);
   }
 }
 
diff --git a/cpp/src/arrow/ipc/json-internal.cc b/cpp/src/arrow/ipc/json-internal.cc
index d5a5dd9..05e5475 100644
--- a/cpp/src/arrow/ipc/json-internal.cc
+++ b/cpp/src/arrow/ipc/json-internal.cc
@@ -633,9 +633,7 @@ static Status GetInteger(const rj::Value::ConstObject& json_type,
       *type = is_signed ? int64() : uint64();
       break;
     default:
-      std::stringstream ss;
-      ss << "Invalid bit width: " << bit_width;
-      return Status::Invalid(ss.str());
+      return Status::Invalid("Invalid bit width: ", bit_width);
   }
   return Status::OK();
 }
@@ -654,9 +652,7 @@ static Status GetFloatingPoint(const RjObject& json_type,
   } else if (precision == "HALF") {
     *type = float16();
   } else {
-    std::stringstream ss;
-    ss << "Invalid precision: " << precision;
-    return Status::Invalid(ss.str());
+    return Status::Invalid("Invalid precision: ", precision);
   }
   return Status::OK();
 }
@@ -693,9 +689,7 @@ static Status GetDate(const RjObject& json_type, std::shared_ptr<DataType>* type
   } else if (unit_str == "MILLISECOND") {
     *type = date64();
   } else {
-    std::stringstream ss;
-    ss << "Invalid date unit: " << unit_str;
-    return Status::Invalid(ss.str());
+    return Status::Invalid("Invalid date unit: ", unit_str);
   }
   return Status::OK();
 }
@@ -718,9 +712,7 @@ static Status GetTime(const RjObject& json_type, std::shared_ptr<DataType>* type
   } else if (unit_str == "NANOSECOND") {
     *type = time64(TimeUnit::NANO);
   } else {
-    std::stringstream ss;
-    ss << "Invalid time unit: " << unit_str;
-    return Status::Invalid(ss.str());
+    return Status::Invalid("Invalid time unit: ", unit_str);
   }
 
   const auto& fw_type = checked_cast<const FixedWidthType&>(**type);
@@ -749,9 +741,7 @@ static Status GetTimestamp(const RjObject& json_type, std::shared_ptr<DataType>*
   } else if (unit_str == "NANOSECOND") {
     unit = TimeUnit::NANO;
   } else {
-    std::stringstream ss;
-    ss << "Invalid time unit: " << unit_str;
-    return Status::Invalid(ss.str());
+    return Status::Invalid("Invalid time unit: ", unit_str);
   }
 
   const auto& it_tz = json_type.FindMember("timezone");
@@ -778,9 +768,7 @@ static Status GetUnion(const RjObject& json_type,
   } else if (mode_str == "DENSE") {
     mode = UnionMode::DENSE;
   } else {
-    std::stringstream ss;
-    ss << "Invalid union mode: " << mode_str;
-    return Status::Invalid(ss.str());
+    return Status::Invalid("Invalid union mode: ", mode_str);
   }
 
   const auto& it_type_codes = json_type.FindMember("typeIds");
@@ -838,9 +826,7 @@ static Status GetType(const RjObject& json_type,
   } else if (type_name == "union") {
     return GetUnion(json_type, children, type);
   } else {
-    std::stringstream ss;
-    ss << "Unrecognized type name: " << type_name;
-    return Status::Invalid(ss.str());
+    return Status::Invalid("Unrecognized type name: ", type_name);
   }
   return Status::OK();
 }
@@ -1235,10 +1221,8 @@ class ArrayReader {
     const auto& json_children_arr = json_children->value.GetArray();
 
     if (type.num_children() != static_cast<int>(json_children_arr.Size())) {
-      std::stringstream ss;
-      ss << "Expected " << type.num_children() << " children, but got "
-         << json_children_arr.Size();
-      return Status::Invalid(ss.str());
+      return Status::Invalid("Expected ", type.num_children(), " children, but got ",
+                             json_children_arr.Size());
     }
 
     for (int i = 0; i < static_cast<int>(json_children_arr.Size()); ++i) {
@@ -1342,9 +1326,7 @@ static Status ReadDictionary(const RjObject& obj, const DictionaryTypeMap& id_to
 
   auto it = id_to_field.find(id);
   if (it == id_to_field.end()) {
-    std::stringstream ss;
-    ss << "No dictionary with id " << id;
-    return Status::Invalid(ss.str());
+    return Status::Invalid("No dictionary with id ", id);
   }
   std::vector<std::shared_ptr<Field>> fields = {it->second};
 
@@ -1489,9 +1471,7 @@ Status ReadArray(MemoryPool* pool, const rj::Value& json_array, const Schema& sc
   }
 
   if (result == nullptr) {
-    std::stringstream ss;
-    ss << "Field named " << name << " not found in schema";
-    return Status::KeyError(ss.str());
+    return Status::KeyError("Field named ", name, " not found in schema");
   }
 
   return ReadArray(pool, json_array, result->type(), array);
diff --git a/cpp/src/arrow/ipc/json-internal.h b/cpp/src/arrow/ipc/json-internal.h
index 5516e2d..c8c7249 100644
--- a/cpp/src/arrow/ipc/json-internal.h
+++ b/cpp/src/arrow/ipc/json-internal.h
@@ -49,56 +49,39 @@ using RjWriter = rj::Writer<rj::StringBuffer>;
 using RjArray = rj::Value::ConstArray;
 using RjObject = rj::Value::ConstObject;
 
-#define RETURN_NOT_FOUND(TOK, NAME, PARENT) \
-  if (NAME == (PARENT).MemberEnd()) {       \
-    std::stringstream ss;                   \
-    ss << "field " << TOK << " not found";  \
-    return Status::Invalid(ss.str());       \
+#define RETURN_NOT_FOUND(TOK, NAME, PARENT)              \
+  if (NAME == (PARENT).MemberEnd()) {                    \
+    return Status::Invalid("field ", TOK, " not found"); \
   }
 
-#define RETURN_NOT_STRING(TOK, NAME, PARENT) \
-  RETURN_NOT_FOUND(TOK, NAME, PARENT);       \
-  if (!NAME->value.IsString()) {             \
-    std::stringstream ss;                    \
-    ss << "field was not a string"           \
-       << " line " << __LINE__;              \
-    return Status::Invalid(ss.str());        \
+#define RETURN_NOT_STRING(TOK, NAME, PARENT)                          \
+  RETURN_NOT_FOUND(TOK, NAME, PARENT);                                \
+  if (!NAME->value.IsString()) {                                      \
+    return Status::Invalid("field was not a string line ", __LINE__); \
   }
 
-#define RETURN_NOT_BOOL(TOK, NAME, PARENT) \
-  RETURN_NOT_FOUND(TOK, NAME, PARENT);     \
-  if (!NAME->value.IsBool()) {             \
-    std::stringstream ss;                  \
-    ss << "field was not a boolean"        \
-       << " line " << __LINE__;            \
-    return Status::Invalid(ss.str());      \
+#define RETURN_NOT_BOOL(TOK, NAME, PARENT)                             \
+  RETURN_NOT_FOUND(TOK, NAME, PARENT);                                 \
+  if (!NAME->value.IsBool()) {                                         \
+    return Status::Invalid("field was not a boolean line ", __LINE__); \
   }
 
-#define RETURN_NOT_INT(TOK, NAME, PARENT) \
-  RETURN_NOT_FOUND(TOK, NAME, PARENT);    \
-  if (!NAME->value.IsInt()) {             \
-    std::stringstream ss;                 \
-    ss << "field was not an int"          \
-       << " line " << __LINE__;           \
-    return Status::Invalid(ss.str());     \
+#define RETURN_NOT_INT(TOK, NAME, PARENT)                           \
+  RETURN_NOT_FOUND(TOK, NAME, PARENT);                              \
+  if (!NAME->value.IsInt()) {                                       \
+    return Status::Invalid("field was not an int line ", __LINE__); \
   }
 
-#define RETURN_NOT_ARRAY(TOK, NAME, PARENT) \
-  RETURN_NOT_FOUND(TOK, NAME, PARENT);      \
-  if (!NAME->value.IsArray()) {             \
-    std::stringstream ss;                   \
-    ss << "field was not an array"          \
-       << " line " << __LINE__;             \
-    return Status::Invalid(ss.str());       \
+#define RETURN_NOT_ARRAY(TOK, NAME, PARENT)                           \
+  RETURN_NOT_FOUND(TOK, NAME, PARENT);                                \
+  if (!NAME->value.IsArray()) {                                       \
+    return Status::Invalid("field was not an array line ", __LINE__); \
   }
 
-#define RETURN_NOT_OBJECT(TOK, NAME, PARENT) \
-  RETURN_NOT_FOUND(TOK, NAME, PARENT);       \
-  if (!NAME->value.IsObject()) {             \
-    std::stringstream ss;                    \
-    ss << "field was not an object"          \
-       << " line " << __LINE__;              \
-    return Status::Invalid(ss.str());        \
+#define RETURN_NOT_OBJECT(TOK, NAME, PARENT)                           \
+  RETURN_NOT_FOUND(TOK, NAME, PARENT);                                 \
+  if (!NAME->value.IsObject()) {                                       \
+    return Status::Invalid("field was not an object line ", __LINE__); \
   }
 
 namespace arrow {
diff --git a/cpp/src/arrow/ipc/json-simple.cc b/cpp/src/arrow/ipc/json-simple.cc
index a8d1200..d812f84 100644
--- a/cpp/src/arrow/ipc/json-simple.cc
+++ b/cpp/src/arrow/ipc/json-simple.cc
@@ -41,9 +41,7 @@ using ::arrow::internal::checked_cast;
 static constexpr auto kParseFlags = rj::kParseFullPrecisionFlag | rj::kParseNanAndInfFlag;
 
 static Status JSONTypeError(const char* expected_type, rj::Type json_type) {
-  std::stringstream ss;
-  ss << "Expected " << expected_type << " or null, got type " << json_type;
-  return Status::Invalid(ss.str());
+  return Status::Invalid("Expected ", expected_type, " or null, got type ", json_type);
 }
 
 class Converter {
@@ -184,9 +182,8 @@ class IntegerConverter final : public ConcreteConverter<IntegerConverter<Type>>
       if (v == v64) {
         return builder_->Append(v);
       } else {
-        std::stringstream ss;
-        ss << "Value " << v64 << " out of bounds for " << this->type_->ToString();
-        return Status::Invalid(ss.str());
+        return Status::Invalid("Value ", v64, " out of bounds for ",
+                               this->type_->ToString());
       }
     } else {
       return JSONTypeError("signed int", json_obj.GetType());
@@ -203,9 +200,8 @@ class IntegerConverter final : public ConcreteConverter<IntegerConverter<Type>>
       if (v == v64) {
         return builder_->Append(v);
       } else {
-        std::stringstream ss;
-        ss << "Value " << v64 << " out of bounds for " << this->type_->ToString();
-        return Status::Invalid(ss.str());
+        return Status::Invalid("Value ", v64, " out of bounds for ",
+                               this->type_->ToString());
       }
       return builder_->Append(v);
     } else {
@@ -272,10 +268,8 @@ class DecimalConverter final : public ConcreteConverter<DecimalConverter> {
       auto view = util::string_view(json_obj.GetString(), json_obj.GetStringLength());
       RETURN_NOT_OK(Decimal128::FromString(view, &d, &precision, &scale));
       if (scale != decimal_type_->scale()) {
-        std::stringstream ss;
-        ss << "Invalid scale for decimal: expected " << decimal_type_->scale() << ", got "
-           << scale;
-        return Status::Invalid(ss.str());
+        return Status::Invalid("Invalid scale for decimal: expected ",
+                               decimal_type_->scale(), ", got ", scale);
       }
       return builder_->Append(d);
     }
@@ -390,10 +384,8 @@ class StructConverter final : public ConcreteConverter<StructConverter> {
       auto size = json_obj.Size();
       auto expected_size = static_cast<uint32_t>(type_->num_children());
       if (size != expected_size) {
-        std::stringstream ss;
-        ss << "Expected array of size " << expected_size << ", got array of size "
-           << size;
-        return Status::Invalid(ss.str());
+        return Status::Invalid("Expected array of size ", expected_size,
+                               ", got array of size ", size);
       }
       for (uint32_t i = 0; i < size; ++i) {
         RETURN_NOT_OK(child_converters_[i]->AppendValue(json_obj[i]));
@@ -414,9 +406,8 @@ class StructConverter final : public ConcreteConverter<StructConverter> {
         }
       }
       if (remaining > 0) {
-        std::stringstream ss;
-        ss << "Unexpected members in JSON object for type " << type_->ToString();
-        return Status::Invalid(ss.str());
+        return Status::Invalid("Unexpected members in JSON object for type ",
+                               type_->ToString());
       }
       return builder_->Append();
     }
@@ -460,9 +451,8 @@ Status GetConverter(const std::shared_ptr<DataType>& type,
     SIMPLE_CONVERTER_CASE(Type::STRING, StringConverter)
     SIMPLE_CONVERTER_CASE(Type::DECIMAL, DecimalConverter)
     default: {
-      std::stringstream ss;
-      ss << "JSON conversion to " << type->ToString() << " not implemented";
-      return Status::NotImplemented(ss.str());
+      return Status::NotImplemented("JSON conversion to ", type->ToString(),
+                                    " not implemented");
     }
   }
 
@@ -481,10 +471,8 @@ Status ArrayFromJSON(const std::shared_ptr<DataType>& type,
   rj::Document json_doc;
   json_doc.Parse<kParseFlags>(json_string.data(), json_string.length());
   if (json_doc.HasParseError()) {
-    std::stringstream ss;
-    ss << "JSON parse error at offset " << json_doc.GetErrorOffset() << ": "
-       << GetParseError_En(json_doc.GetParseError());
-    return Status::Invalid(ss.str());
+    return Status::Invalid("JSON parse error at offset ", json_doc.GetErrorOffset(), ": ",
+                           GetParseError_En(json_doc.GetParseError()));
   }
 
   // The JSON document should be an array, append it
diff --git a/cpp/src/arrow/ipc/message.cc b/cpp/src/arrow/ipc/message.cc
index 724e625..8adf4a8 100644
--- a/cpp/src/arrow/ipc/message.cc
+++ b/cpp/src/arrow/ipc/message.cc
@@ -153,10 +153,8 @@ Status Message::ReadFrom(const std::shared_ptr<Buffer>& metadata, io::InputStrea
   std::shared_ptr<Buffer> body;
   RETURN_NOT_OK(stream->Read(body_length, &body));
   if (body->size() < body_length) {
-    std::stringstream ss;
-    ss << "Expected to be able to read " << body_length << " bytes for message body, got "
-       << body->size();
-    return Status::IOError(ss.str());
+    return Status::IOError("Expected to be able to read ", body_length,
+                           " bytes for message body, got ", body->size());
   }
 
   return Message::Open(metadata, body, out);
@@ -171,10 +169,8 @@ Status Message::ReadFrom(const int64_t offset, const std::shared_ptr<Buffer>& me
   std::shared_ptr<Buffer> body;
   RETURN_NOT_OK(file->ReadAt(offset, body_length, &body));
   if (body->size() < body_length) {
-    std::stringstream ss;
-    ss << "Expected to be able to read " << body_length << " bytes for message body, got "
-       << body->size();
-    return Status::IOError(ss.str());
+    return Status::IOError("Expected to be able to read ", body_length,
+                           " bytes for message body, got ", body->size());
   }
 
   return Message::Open(metadata, body, out);
@@ -238,19 +234,16 @@ Status ReadMessage(int64_t offset, int32_t metadata_length, io::RandomAccessFile
   RETURN_NOT_OK(file->ReadAt(offset, metadata_length, &buffer));
 
   if (buffer->size() < metadata_length) {
-    std::stringstream ss;
-    ss << "Expected to read " << metadata_length << " metadata bytes but got "
-       << buffer->size();
-    return Status::Invalid(ss.str());
+    return Status::Invalid("Expected to read ", metadata_length,
+                           " metadata bytes but got ", buffer->size());
   }
 
   int32_t flatbuffer_size = *reinterpret_cast<const int32_t*>(buffer->data());
 
   if (flatbuffer_size + static_cast<int>(sizeof(int32_t)) > metadata_length) {
-    std::stringstream ss;
-    ss << "flatbuffer size " << metadata_length << " invalid. File offset: " << offset
-       << ", metadata length: " << metadata_length;
-    return Status::Invalid(ss.str());
+    return Status::Invalid("flatbuffer size ", metadata_length,
+                           " invalid. File offset: ", offset,
+                           ", metadata length: ", metadata_length);
   }
 
   auto metadata = SliceBuffer(buffer, 4, buffer->size() - 4);
@@ -303,10 +296,8 @@ Status ReadMessage(io::InputStream* file, std::unique_ptr<Message>* message) {
   std::shared_ptr<Buffer> metadata;
   RETURN_NOT_OK(file->Read(message_length, &metadata));
   if (metadata->size() != message_length) {
-    std::stringstream ss;
-    ss << "Expected to read " << message_length << " metadata bytes, but "
-       << "only read " << metadata->size();
-    return Status::Invalid(ss.str());
+    return Status::Invalid("Expected to read ", message_length, " metadata bytes, but ",
+                           "only read ", metadata->size());
   }
 
   return Message::ReadFrom(metadata, file, message);
diff --git a/cpp/src/arrow/ipc/metadata-internal.cc b/cpp/src/arrow/ipc/metadata-internal.cc
index ef189c8..1d4c80c 100644
--- a/cpp/src/arrow/ipc/metadata-internal.cc
+++ b/cpp/src/arrow/ipc/metadata-internal.cc
@@ -443,9 +443,7 @@ static Status TypeToFlatbuffer(FBB& fbb, const DataType& type,
       return UnionToFlatBuffer(fbb, *value_type, children, dictionary_memo, offset);
     default:
       *out_type = flatbuf::Type_NONE;  // Make clang-tidy happy
-      std::stringstream ss;
-      ss << "Unable to convert type: " << type.ToString() << std::endl;
-      return Status::NotImplemented(ss.str());
+      return Status::NotImplemented("Unable to convert type: ", type.ToString());
   }
   return Status::OK();
 }
@@ -483,9 +481,7 @@ static Status TensorTypeToFlatbuffer(FBB& fbb, const DataType& type,
       break;
     default:
       *out_type = flatbuf::Type_NONE;  // Make clang-tidy happy
-      std::stringstream ss;
-      ss << "Unable to convert type: " << type.ToString() << std::endl;
-      return Status::NotImplemented(ss.str());
+      return Status::NotImplemented("Unable to convert type: ", type.ToString());
   }
   return Status::OK();
 }
diff --git a/cpp/src/arrow/ipc/reader.cc b/cpp/src/arrow/ipc/reader.cc
index 65f5d96..b2c2676 100644
--- a/cpp/src/arrow/ipc/reader.cc
+++ b/cpp/src/arrow/ipc/reader.cc
@@ -225,9 +225,7 @@ class ArrayLoader {
 
     const int num_children = type.num_children();
     if (num_children != 1) {
-      std::stringstream ss;
-      ss << "Wrong number of children: " << num_children;
-      return Status::Invalid(ss.str());
+      return Status::Invalid("Wrong number of children: ", num_children);
     }
 
     return LoadChildren(type.children());
@@ -343,9 +341,7 @@ Status ReadDictionary(const Buffer& metadata, const DictionaryTypeMap& dictionar
   int64_t id = *dictionary_id = dictionary_batch->id();
   auto it = dictionary_types.find(id);
   if (it == dictionary_types.end()) {
-    std::stringstream ss;
-    ss << "Do not have type metadata for dictionary with id: " << id;
-    return Status::KeyError(ss.str());
+    return Status::KeyError("Do not have type metadata for dictionary with id: ", id);
   }
 
   std::vector<std::shared_ptr<Field>> fields = {it->second};
@@ -372,10 +368,8 @@ static Status ReadMessageAndValidate(MessageReader* reader, Message::Type expect
   RETURN_NOT_OK(reader->ReadNextMessage(message));
 
   if (!(*message) && !allow_null) {
-    std::stringstream ss;
-    ss << "Expected " << FormatMessageType(expected_type)
-       << " message in stream, was null or length 0";
-    return Status::Invalid(ss.str());
+    return Status::Invalid("Expected ", FormatMessageType(expected_type),
+                           " message in stream, was null or length 0");
   }
 
   if ((*message) == nullptr) {
@@ -383,10 +377,9 @@ static Status ReadMessageAndValidate(MessageReader* reader, Message::Type expect
   }
 
   if ((*message)->type() != expected_type) {
-    std::stringstream ss;
-    ss << "Message not expected type: " << FormatMessageType(expected_type)
-       << ", was: " << (*message)->type();
-    return Status::IOError(ss.str());
+    return Status::IOError(
+        "Message not expected type: ", FormatMessageType(expected_type),
+        ", was: ", (*message)->type());
   }
   return Status::OK();
 }
@@ -512,9 +505,7 @@ class RecordBatchFileReader::RecordBatchFileReaderImpl {
     int magic_size = static_cast<int>(strlen(kArrowMagicBytes));
 
     if (footer_offset_ <= magic_size * 2 + 4) {
-      std::stringstream ss;
-      ss << "File is too small: " << footer_offset_;
-      return Status::Invalid(ss.str());
+      return Status::Invalid("File is too small: ", footer_offset_);
     }
 
     std::shared_ptr<Buffer> buffer;
@@ -523,9 +514,7 @@ class RecordBatchFileReader::RecordBatchFileReaderImpl {
 
     const int64_t expected_footer_size = magic_size + sizeof(int32_t);
     if (buffer->size() < expected_footer_size) {
-      std::stringstream ss;
-      ss << "Unable to read " << expected_footer_size << "from end of file";
-      return Status::Invalid(ss.str());
+      return Status::Invalid("Unable to read ", expected_footer_size, "from end of file");
     }
 
     if (memcmp(buffer->data() + sizeof(int32_t), kArrowMagicBytes, magic_size)) {
diff --git a/cpp/src/arrow/memory_pool.cc b/cpp/src/arrow/memory_pool.cc
index d62db32..fb5beac 100644
--- a/cpp/src/arrow/memory_pool.cc
+++ b/cpp/src/arrow/memory_pool.cc
@@ -55,31 +55,23 @@ Status AllocateAligned(int64_t size, uint8_t** out) {
   *out =
       reinterpret_cast<uint8_t*>(_aligned_malloc(static_cast<size_t>(size), kAlignment));
   if (!*out) {
-    std::stringstream ss;
-    ss << "malloc of size " << size << " failed";
-    return Status::OutOfMemory(ss.str());
+    return Status::OutOfMemory("malloc of size ", size, " failed");
   }
 #elif defined(ARROW_JEMALLOC)
   *out = reinterpret_cast<uint8_t*>(mallocx(
       std::max(static_cast<size_t>(size), kAlignment), MALLOCX_ALIGN(kAlignment)));
   if (*out == NULL) {
-    std::stringstream ss;
-    ss << "malloc of size " << size << " failed";
-    return Status::OutOfMemory(ss.str());
+    return Status::OutOfMemory("malloc of size ", size, " failed");
   }
 #else
   const int result = posix_memalign(reinterpret_cast<void**>(out), kAlignment,
                                     static_cast<size_t>(size));
   if (result == ENOMEM) {
-    std::stringstream ss;
-    ss << "malloc of size " << size << " failed";
-    return Status::OutOfMemory(ss.str());
+    return Status::OutOfMemory("malloc of size ", size, " failed");
   }
 
   if (result == EINVAL) {
-    std::stringstream ss;
-    ss << "invalid alignment parameter: " << kAlignment;
-    return Status::Invalid(ss.str());
+    return Status::Invalid("invalid alignment parameter: ", kAlignment);
   }
 #endif
   return Status::OK();
@@ -118,10 +110,8 @@ class DefaultMemoryPool : public MemoryPool {
     *ptr = reinterpret_cast<uint8_t*>(
         rallocx(*ptr, static_cast<size_t>(new_size), MALLOCX_ALIGN(kAlignment)));
     if (*ptr == NULL) {
-      std::stringstream ss;
-      ss << "realloc of size " << new_size << " failed";
       *ptr = previous_ptr;
-      return Status::OutOfMemory(ss.str());
+      return Status::OutOfMemory("realloc of size ", new_size, " failed");
     }
 #else
     // Note: We cannot use realloc() here as it doesn't guarantee alignment.
diff --git a/cpp/src/arrow/python/arrow_to_pandas.cc b/cpp/src/arrow/python/arrow_to_pandas.cc
index 3e04f27..29d6435 100644
--- a/cpp/src/arrow/python/arrow_to_pandas.cc
+++ b/cpp/src/arrow/python/arrow_to_pandas.cc
@@ -414,9 +414,7 @@ inline Status ConvertBinaryLike(PandasOptions options, const ChunkedArray& data,
         *out_values = WrapBytes<ArrayType>::Wrap(view.data(), view.length());
         if (*out_values == nullptr) {
           PyErr_Clear();
-          std::stringstream ss;
-          ss << "Wrapping " << view << " failed";
-          return Status::UnknownError(ss.str());
+          return Status::UnknownError("Wrapping ", view, " failed");
         }
       }
       ++out_values;
@@ -773,18 +771,16 @@ class ObjectBlock : public PandasBlock {
         CONVERTLISTSLIKE_CASE(ListType, LIST)
         CONVERTLISTSLIKE_CASE(NullType, NA)
         default: {
-          std::stringstream ss;
-          ss << "Not implemented type for conversion from List to Pandas ObjectBlock: "
-             << list_type->value_type()->ToString();
-          return Status::NotImplemented(ss.str());
+          return Status::NotImplemented(
+              "Not implemented type for conversion from List to Pandas ObjectBlock: ",
+              list_type->value_type()->ToString());
         }
       }
     } else if (type == Type::STRUCT) {
       RETURN_NOT_OK(ConvertStruct(options_, data, out_buffer));
     } else {
-      std::stringstream ss;
-      ss << "Unsupported type for object array output: " << col->type()->ToString();
-      return Status::NotImplemented(ss.str());
+      return Status::NotImplemented("Unsupported type for object array output: ",
+                                    col->type()->ToString());
     }
 
     placement_data_[rel_placement] = abs_placement;
@@ -810,10 +806,9 @@ class IntBlock : public PandasBlock {
     const ChunkedArray& data = *col->data().get();
 
     if (type != ARROW_TYPE) {
-      std::stringstream ss;
-      ss << "Cannot write Arrow data of type " << col->type()->ToString();
-      ss << " to a Pandas int" << sizeof(C_TYPE) << " block.";
-      return Status::NotImplemented(ss.str());
+      return Status::NotImplemented("Cannot write Arrow data of type ",
+                                    col->type()->ToString(), " to a Pandas int",
+                                    sizeof(C_TYPE), " block");
     }
 
     ConvertIntegerNoNullsSameType<C_TYPE>(options_, data, out_buffer);
@@ -841,10 +836,9 @@ class Float16Block : public PandasBlock {
     Type::type type = col->type()->id();
 
     if (type != Type::HALF_FLOAT) {
-      std::stringstream ss;
-      ss << "Cannot write Arrow data of type " << col->type()->ToString();
-      ss << " to a Pandas float16 block.";
-      return Status::NotImplemented(ss.str());
+      return Status::NotImplemented("Cannot write Arrow data of type ",
+                                    col->type()->ToString(),
+                                    " to a Pandas float16 block");
     }
 
     npy_half* out_buffer =
@@ -866,10 +860,9 @@ class Float32Block : public PandasBlock {
     Type::type type = col->type()->id();
 
     if (type != Type::FLOAT) {
-      std::stringstream ss;
-      ss << "Cannot write Arrow data of type " << col->type()->ToString();
-      ss << " to a Pandas float32 block.";
-      return Status::NotImplemented(ss.str());
+      return Status::NotImplemented("Cannot write Arrow data of type ",
+                                    col->type()->ToString(),
+                                    " to a Pandas float32 block");
     }
 
     float* out_buffer = reinterpret_cast<float*>(block_data_) + rel_placement * num_rows_;
@@ -922,10 +915,9 @@ class Float64Block : public PandasBlock {
         ConvertNumericNullable<double>(data, NAN, out_buffer);
         break;
       default:
-        std::stringstream ss;
-        ss << "Cannot write Arrow data of type " << col->type()->ToString();
-        ss << " to a Pandas float64 block.";
-        return Status::NotImplemented(ss.str());
+        return Status::NotImplemented("Cannot write Arrow data of type ",
+                                      col->type()->ToString(),
+                                      " to a Pandas float64 block");
     }
 
 #undef INTEGER_CASE
@@ -945,10 +937,9 @@ class BoolBlock : public PandasBlock {
     Type::type type = col->type()->id();
 
     if (type != Type::BOOL) {
-      std::stringstream ss;
-      ss << "Cannot write Arrow data of type " << col->type()->ToString();
-      ss << " to a Pandas boolean block.";
-      return Status::NotImplemented(ss.str());
+      return Status::NotImplemented("Cannot write Arrow data of type ",
+                                    col->type()->ToString(),
+                                    " to a Pandas boolean block");
     }
 
     uint8_t* out_buffer =
@@ -1006,10 +997,9 @@ class DatetimeBlock : public PandasBlock {
         return Status::NotImplemented("Unsupported time unit");
       }
     } else {
-      std::stringstream ss;
-      ss << "Cannot write Arrow data of type " << col->type()->ToString();
-      ss << " to a Pandas datetime block.";
-      return Status::NotImplemented(ss.str());
+      return Status::NotImplemented("Cannot write Arrow data of type ",
+                                    col->type()->ToString(),
+                                    " to a Pandas datetime block.");
     }
 
     placement_data_[rel_placement] = abs_placement;
@@ -1075,9 +1065,8 @@ class CategoricalBlock : public PandasBlock {
       const T* values = arr.raw_values();
       for (int64_t i = 0; i < arr.length(); ++i) {
         if (arr.IsValid(i) && (values[i] < 0 || values[i] >= dict_length)) {
-          std::stringstream ss;
-          ss << "Out of bounds dictionary index: " << static_cast<int64_t>(values[i]);
-          return Status::Invalid(ss.str());
+          return Status::Invalid("Out of bounds dictionary index: ",
+                                 static_cast<int64_t>(values[i]));
         }
       }
       return Status::OK();
@@ -1088,16 +1077,15 @@ class CategoricalBlock : public PandasBlock {
       RETURN_NOT_OK(AllocateNDArrayFromIndices<T>(npy_type, indices_first));
     } else {
       if (options_.zero_copy_only) {
-        std::stringstream ss;
         if (needs_copy_) {
-          ss << "Need to allocate categorical memory, "
-             << "but only zero-copy conversions allowed.";
-        } else {
-          ss << "Needed to copy " << data.num_chunks() << " chunks with "
-             << indices_first->null_count()
-             << " indices nulls, but zero_copy_only was True";
+          return Status::Invalid("Need to allocate categorical memory, but ",
+                                 "only zero-copy conversions "
+                                 "allowed");
         }
-        return Status::Invalid(ss.str());
+
+        return Status::Invalid("Needed to copy ", data.num_chunks(), " chunks with ",
+                               indices_first->null_count(),
+                               " indices nulls, but zero_copy_only was True");
       }
       RETURN_NOT_OK(AllocateNDArray(npy_type, 1));
 
@@ -1155,10 +1143,8 @@ class CategoricalBlock : public PandasBlock {
         RETURN_NOT_OK(WriteIndices<Int64Type>(converted_col));
         break;
       default: {
-        std::stringstream ss;
-        ss << "Categorical index type not supported: "
-           << dict_type.index_type()->ToString();
-        return Status::NotImplemented(ss.str());
+        return Status::NotImplemented("Categorical index type not supported: ",
+                                      dict_type.index_type()->ToString());
       }
     }
 
@@ -1349,10 +1335,8 @@ static Status GetPandasBlockType(const Column& col, const PandasOptions& options
     case Type::LIST: {
       auto list_type = std::static_pointer_cast<ListType>(col.type());
       if (!ListTypeSupported(*list_type->value_type())) {
-        std::stringstream ss;
-        ss << "Not implemented type for list in DataFrameBlock: "
-           << list_type->value_type()->ToString();
-        return Status::NotImplemented(ss.str());
+        return Status::NotImplemented("Not implemented type for list in DataFrameBlock: ",
+                                      list_type->value_type()->ToString());
       }
       *output_type = PandasBlock::OBJECT;
     } break;
@@ -1360,10 +1344,9 @@ static Status GetPandasBlockType(const Column& col, const PandasOptions& options
       *output_type = PandasBlock::CATEGORICAL;
       break;
     default:
-      std::stringstream ss;
-      ss << "No known equivalent Pandas block for Arrow data of type ";
-      ss << col.type()->ToString() << " is known.";
-      return Status::NotImplemented(ss.str());
+      return Status::NotImplemented(
+          "No known equivalent Pandas block for Arrow data of type ",
+          col.type()->ToString(), " is known.");
   }
   return Status::OK();
 }
@@ -1657,10 +1640,8 @@ class ArrowDeserializer {
     if (data_.num_chunks() == 1 && data_.null_count() == 0) {
       return ConvertValuesZeroCopy<TYPE>(options_, npy_type, data_.chunk(0));
     } else if (options_.zero_copy_only) {
-      std::stringstream ss;
-      ss << "Needed to copy " << data_.num_chunks() << " chunks with "
-         << data_.null_count() << " nulls, but zero_copy_only was True";
-      return Status::Invalid(ss.str());
+      return Status::Invalid("Needed to copy ", data_.num_chunks(), " chunks with ",
+                             data_.null_count(), " nulls, but zero_copy_only was True");
     }
 
     RETURN_NOT_OK(AllocateOutput(npy_type));
@@ -1751,10 +1732,8 @@ class ArrowDeserializer {
     if (data_.num_chunks() == 1 && data_.null_count() == 0) {
       return ConvertValuesZeroCopy<TYPE>(options_, traits::npy_type, data_.chunk(0));
     } else if (options_.zero_copy_only) {
-      std::stringstream ss;
-      ss << "Needed to copy " << data_.num_chunks() << " chunks with "
-         << data_.null_count() << " nulls, but zero_copy_only was True";
-      return Status::Invalid(ss.str());
+      return Status::Invalid("Needed to copy ", data_.num_chunks(), " chunks with ",
+                             data_.null_count(), " nulls, but zero_copy_only was True");
     }
 
     if (data_.null_count() > 0) {
@@ -1854,9 +1833,8 @@ class ArrowDeserializer {
       CONVERTVALUES_LISTSLIKE_CASE(Decimal128Type, DECIMAL)
       CONVERTVALUES_LISTSLIKE_CASE(ListType, LIST)
       default: {
-        std::stringstream ss;
-        ss << "Not implemented type for lists: " << list_type->value_type()->ToString();
-        return Status::NotImplemented(ss.str());
+        return Status::NotImplemented("Not implemented type for lists: ",
+                                      list_type->value_type()->ToString());
       }
     }
 #undef CONVERTVALUES_LISTSLIKE_CASE
diff --git a/cpp/src/arrow/python/common.h b/cpp/src/arrow/python/common.h
index 6587bd3..6e41bed 100644
--- a/cpp/src/arrow/python/common.h
+++ b/cpp/src/arrow/python/common.h
@@ -215,10 +215,8 @@ struct PyBytesView {
       this->ref.reset();
       return Status::OK();
     } else {
-      std::stringstream ss;
-      ss << "Expected " << expected_msg << ", got a '" << Py_TYPE(obj)->tp_name
-         << "' object";
-      return Status::TypeError(ss.str());
+      return Status::TypeError("Expected ", expected_msg, ", got a '",
+                               Py_TYPE(obj)->tp_name, "' object");
     }
   }
 
diff --git a/cpp/src/arrow/python/decimal.cc b/cpp/src/arrow/python/decimal.cc
index 051f31f..8db7c01 100644
--- a/cpp/src/arrow/python/decimal.cc
+++ b/cpp/src/arrow/python/decimal.cc
@@ -125,11 +125,9 @@ Status DecimalFromPythonDecimal(PyObject* python_decimal, const DecimalType& arr
   const int32_t scale = arrow_type.scale();
 
   if (ARROW_PREDICT_FALSE(inferred_precision > precision)) {
-    std::stringstream buf;
-    buf << "Decimal type with precision " << inferred_precision
-        << " does not fit into precision inferred from first array element: "
-        << precision;
-    return Status::Invalid(buf.str());
+    return Status::Invalid(
+        "Decimal type with precision ", inferred_precision,
+        " does not fit into precision inferred from first array element: ", precision);
   }
 
   if (scale != inferred_scale) {
diff --git a/cpp/src/arrow/python/helpers.cc b/cpp/src/arrow/python/helpers.cc
index 2f43db6..28ed1a6 100644
--- a/cpp/src/arrow/python/helpers.cc
+++ b/cpp/src/arrow/python/helpers.cc
@@ -164,11 +164,10 @@ namespace {
 
 Status IntegerOverflowStatus(PyObject* obj, const std::string& overflow_message) {
   if (overflow_message.empty()) {
-    std::stringstream ss;
     std::string obj_as_stdstring;
     RETURN_NOT_OK(PyObject_StdStringStr(obj, &obj_as_stdstring));
-    ss << "Value " << obj_as_stdstring << " too large to fit in C integer type";
-    return Status::Invalid(ss.str());
+    return Status::Invalid("Value ", obj_as_stdstring,
+                           " too large to fit in C integer type");
   } else {
     return Status::Invalid(overflow_message);
   }
@@ -299,13 +298,10 @@ bool PandasObjectIsNull(PyObject* obj) {
 }
 
 Status InvalidValue(PyObject* obj, const std::string& why) {
-  std::stringstream ss;
-
   std::string obj_as_str;
   RETURN_NOT_OK(internal::PyObject_StdStringStr(obj, &obj_as_str));
-  ss << "Could not convert " << obj_as_str << " with type " << Py_TYPE(obj)->tp_name
-     << ": " << why;
-  return Status::Invalid(ss.str());
+  return Status::Invalid("Could not convert ", obj_as_str, " with type ",
+                         Py_TYPE(obj)->tp_name, ": ", why);
 }
 
 Status UnboxIntegerAsInt64(PyObject* obj, int64_t* out) {
@@ -355,10 +351,8 @@ Status IntegerScalarToDoubleSafe(PyObject* obj, double* out) {
   constexpr int64_t kDoubleMin = -(1LL << 53);
 
   if (value < kDoubleMin || value > kDoubleMax) {
-    std::stringstream ss;
-    ss << "Integer value " << value << " is outside of the range exactly"
-       << " representable by a IEEE 754 double precision value";
-    return Status::Invalid(ss.str());
+    return Status::Invalid("Integer value ", value, " is outside of the range exactly",
+                           " representable by a IEEE 754 double precision value");
   }
   *out = static_cast<double>(value);
   return Status::OK();
@@ -372,10 +366,8 @@ Status IntegerScalarToFloat32Safe(PyObject* obj, float* out) {
   constexpr int64_t kFloatMin = -(1LL << 24);
 
   if (value < kFloatMin || value > kFloatMax) {
-    std::stringstream ss;
-    ss << "Integer value " << value << " is outside of the range exactly"
-       << " representable by a IEEE 754 single precision value";
-    return Status::Invalid(ss.str());
+    return Status::Invalid("Integer value ", value, " is outside of the range exactly",
+                           " representable by a IEEE 754 single precision value");
   }
   *out = static_cast<float>(value);
   return Status::OK();
diff --git a/cpp/src/arrow/python/inference.cc b/cpp/src/arrow/python/inference.cc
index 0f1d85e..c9db5f4 100644
--- a/cpp/src/arrow/python/inference.cc
+++ b/cpp/src/arrow/python/inference.cc
@@ -58,10 +58,9 @@ class NumPyDtypeUnifier {
   NumPyDtypeUnifier() : current_type_num_(-1), current_dtype_(NULLPTR) {}
 
   Status InvalidMix(int new_dtype) {
-    std::stringstream ss;
-    ss << "Cannot mix NumPy dtypes " << GetNumPyTypeName(current_type_num_) << " and "
-       << GetNumPyTypeName(new_dtype);
-    return Status::Invalid(ss.str());
+    return Status::Invalid("Cannot mix NumPy dtypes ",
+                           GetNumPyTypeName(current_type_num_), " and ",
+                           GetNumPyTypeName(new_dtype));
   }
 
   int Observe_BOOL(PyArray_Descr* descr, int dtype) { return INVALID; }
@@ -250,9 +249,7 @@ class NumPyDtypeUnifier {
         action = Observe_DATETIME(descr);
         break;
       default:
-        std::stringstream ss;
-        ss << "Unsupported numpy type " << GetNumPyTypeName(dtype) << std::endl;
-        return Status::NotImplemented(ss.str());
+        return Status::NotImplemented("Unsupported numpy type ", GetNumPyTypeName(dtype));
     }
 
     if (action == INVALID) {
@@ -480,10 +477,8 @@ class TypeInferrer {
       } else if (PyBytes_Check(key_obj)) {
         key = internal::PyBytes_AsStdString(key_obj);
       } else {
-        std::stringstream ss;
-        ss << "Expected dict key of type str or bytes, got '" << Py_TYPE(key_obj)->tp_name
-           << "'";
-        return Status::TypeError(ss.str());
+        return Status::TypeError("Expected dict key of type str or bytes, got '",
+                                 Py_TYPE(key_obj)->tp_name, "'");
       }
       // Get or create visitor for this key
       auto it = struct_inferrers_.find(key);
diff --git a/cpp/src/arrow/python/numpy-internal.h b/cpp/src/arrow/python/numpy-internal.h
index 463795a..6954e35 100644
--- a/cpp/src/arrow/python/numpy-internal.h
+++ b/cpp/src/arrow/python/numpy-internal.h
@@ -143,9 +143,8 @@ inline Status VisitNumpyArrayInline(PyArrayObject* arr, VISITOR* visitor) {
     TYPE_VISIT_INLINE(DATETIME);
     TYPE_VISIT_INLINE(OBJECT);
   }
-  std::stringstream ss;
-  ss << "NumPy type not implemented: " << GetNumPyTypeName(PyArray_TYPE(arr));
-  return Status::NotImplemented(ss.str());
+  return Status::NotImplemented("NumPy type not implemented: ",
+                                GetNumPyTypeName(PyArray_TYPE(arr)));
 }
 
 #undef TYPE_VISIT_INLINE
diff --git a/cpp/src/arrow/python/numpy_convert.cc b/cpp/src/arrow/python/numpy_convert.cc
index d95e337..c73e0bc 100644
--- a/cpp/src/arrow/python/numpy_convert.cc
+++ b/cpp/src/arrow/python/numpy_convert.cc
@@ -92,9 +92,7 @@ Status GetTensorType(PyObject* dtype, std::shared_ptr<DataType>* out) {
     TO_ARROW_TYPE_CASE(FLOAT32, float32);
     TO_ARROW_TYPE_CASE(FLOAT64, float64);
     default: {
-      std::stringstream ss;
-      ss << "Unsupported numpy type " << descr->type_num << std::endl;
-      return Status::NotImplemented(ss.str());
+      return Status::NotImplemented("Unsupported numpy type ", descr->type_num);
     }
   }
   return Status::OK();
@@ -119,9 +117,7 @@ Status GetNumPyType(const DataType& type, int* type_num) {
     NUMPY_TYPE_CASE(FLOAT, FLOAT32);
     NUMPY_TYPE_CASE(DOUBLE, FLOAT64);
     default: {
-      std::stringstream ss;
-      ss << "Unsupported tensor type: " << type.ToString() << std::endl;
-      return Status::NotImplemented(ss.str());
+      return Status::NotImplemented("Unsupported tensor type: ", type.ToString());
     }
   }
 #undef NUMPY_TYPE_CASE
@@ -181,9 +177,7 @@ Status NumPyDtypeToArrow(PyArray_Descr* descr, std::shared_ptr<DataType>* out) {
       }
     } break;
     default: {
-      std::stringstream ss;
-      ss << "Unsupported numpy type " << descr->type_num << std::endl;
-      return Status::NotImplemented(ss.str());
+      return Status::NotImplemented("Unsupported numpy type ", descr->type_num);
     }
   }
 
diff --git a/cpp/src/arrow/python/numpy_to_arrow.cc b/cpp/src/arrow/python/numpy_to_arrow.cc
index da288d3..461a085 100644
--- a/cpp/src/arrow/python/numpy_to_arrow.cc
+++ b/cpp/src/arrow/python/numpy_to_arrow.cc
@@ -283,9 +283,8 @@ class NumPyConverter {
   }
 
   Status TypeNotImplemented(std::string type_name) {
-    std::stringstream ss;
-    ss << "NumPyConverter doesn't implement <" << type_name << "> conversion. ";
-    return Status::NotImplemented(ss.str());
+    return Status::NotImplemented("NumPyConverter doesn't implement <", type_name,
+                                  "> conversion. ");
   }
 
   MemoryPool* pool_;
@@ -574,9 +573,8 @@ Status NumPyConverter::Visit(const FixedSizeBinaryType& type) {
   auto byte_width = type.byte_width();
 
   if (itemsize_ != byte_width) {
-    std::stringstream ss;
-    ss << "Got bytestring of length " << itemsize_ << " (expected " << byte_width << ")";
-    return Status::Invalid(ss.str());
+    return Status::Invalid("Got bytestring of length ", itemsize_, " (expected ",
+                           byte_width, ")");
   }
 
   FixedSizeBinaryBuilder builder(::arrow::fixed_size_binary(byte_width), pool_);
@@ -651,9 +649,8 @@ Status NumPyConverter::Visit(const StringType& type) {
       if (ARROW_PREDICT_TRUE(util::ValidateUTF8(data, itemsize_))) {
         return builder.Append(data, itemsize_);
       } else {
-        std::stringstream ss;
-        ss << "Encountered non-UTF8 binary value: " << HexEncode(data, itemsize_);
-        return Status::Invalid(ss.str());
+        return Status::Invalid("Encountered non-UTF8 binary value: ",
+                               HexEncode(data, itemsize_));
       }
     } else {
       return AppendUTF32(reinterpret_cast<const char*>(data), itemsize_, byteorder,
@@ -697,9 +694,7 @@ Status NumPyConverter::Visit(const StructType& type) {
     for (auto field : type.children()) {
       PyObject* tup = PyDict_GetItemString(dtype_->fields, field->name().c_str());
       if (tup == NULL) {
-        std::stringstream ss;
-        ss << "Missing field '" << field->name() << "' in struct array";
-        return Status::TypeError(ss.str());
+        return Status::TypeError("Missing field '", field->name(), "' in struct array");
       }
       PyArray_Descr* sub_dtype =
           reinterpret_cast<PyArray_Descr*>(PyTuple_GET_ITEM(tup, 0));
diff --git a/cpp/src/arrow/python/python_to_arrow.cc b/cpp/src/arrow/python/python_to_arrow.cc
index a77cebc..f5e6a57 100644
--- a/cpp/src/arrow/python/python_to_arrow.cc
+++ b/cpp/src/arrow/python/python_to_arrow.cc
@@ -402,10 +402,7 @@ class TimestampConverter : public TypedConverter<TimestampType, TimestampConvert
       std::shared_ptr<DataType> type;
       RETURN_NOT_OK(NumPyDtypeToArrow(PyArray_DescrFromScalar(obj), &type));
       if (type->id() != Type::TIMESTAMP) {
-        std::ostringstream ss;
-        ss << "Expected np.datetime64 but got: ";
-        ss << type->ToString();
-        return Status::Invalid(ss.str());
+        return Status::Invalid("Expected np.datetime64 but got: ", type->ToString());
       }
       const TimestampType& ttype = checked_cast<const TimestampType&>(*type);
       if (unit_ != ttype.unit()) {
@@ -705,10 +702,7 @@ Status ListConverter::AppendNdarrayItem(PyObject* obj) {
       return value_converter_->AppendSingleVirtual(obj);
     }
     default: {
-      std::stringstream ss;
-      ss << "Unknown list item type: ";
-      ss << value_type_->ToString();
-      return Status::TypeError(ss.str());
+      return Status::TypeError("Unknown list item type: ", value_type_->ToString());
     }
   }
 }
@@ -911,9 +905,8 @@ Status GetConverter(const std::shared_ptr<DataType>& type, bool from_pandas,
           new StructConverter(from_pandas, strict_conversions));
       break;
     default:
-      std::stringstream ss;
-      ss << "Sequence converter for type " << type->ToString() << " not implemented";
-      return Status::NotImplemented(ss.str());
+      return Status::NotImplemented("Sequence converter for type ", type->ToString(),
+                                    " not implemented");
   }
   return Status::OK();
 }
diff --git a/cpp/src/arrow/python/serialize.cc b/cpp/src/arrow/python/serialize.cc
index 7911557..ca94369 100644
--- a/cpp/src/arrow/python/serialize.cc
+++ b/cpp/src/arrow/python/serialize.cc
@@ -407,10 +407,9 @@ Status CallCustomCallback(PyObject* context, PyObject* method_name, PyObject* el
                           PyObject** result) {
   *result = NULL;
   if (context == Py_None) {
-    std::stringstream ss;
-    ss << "error while calling callback on " << internal::PyObject_StdStringRepr(elem)
-       << ": handler not registered";
-    return Status::SerializationError(ss.str());
+    return Status::SerializationError("error while calling callback on ",
+                                      internal::PyObject_StdStringRepr(elem),
+                                      ": handler not registered");
   } else {
     *result = PyObject_CallMethodObjArgs(context, method_name, elem, NULL);
     return PassPyError();
diff --git a/cpp/src/arrow/python/util/datetime.h b/cpp/src/arrow/python/util/datetime.h
index 7350dea..dc46297 100644
--- a/cpp/src/arrow/python/util/datetime.h
+++ b/cpp/src/arrow/python/util/datetime.h
@@ -199,9 +199,7 @@ static inline Status PyTime_convert_int(int64_t val, const TimeUnit::type unit,
   switch (unit) {
     case TimeUnit::NANO:
       if (val % 1000 != 0) {
-        std::stringstream ss;
-        ss << "Value " << val << " has non-zero nanoseconds";
-        return Status::Invalid(ss.str());
+        return Status::Invalid("Value ", val, " has non-zero nanoseconds");
       }
       val /= 1000;
     // fall through
diff --git a/cpp/src/arrow/record_batch.cc b/cpp/src/arrow/record_batch.cc
index 33287c1..baaf5cb 100644
--- a/cpp/src/arrow/record_batch.cc
+++ b/cpp/src/arrow/record_batch.cc
@@ -95,16 +95,13 @@ class SimpleRecordBatch : public RecordBatch {
     DCHECK(column != nullptr);
 
     if (!field->type()->Equals(column->type())) {
-      std::stringstream ss;
-      ss << "Column data type " << field->type()->name()
-         << " does not match field data type " << column->type()->name();
-      return Status::Invalid(ss.str());
+      return Status::Invalid("Column data type ", field->type()->name(),
+                             " does not match field data type ", column->type()->name());
     }
     if (column->length() != num_rows_) {
-      std::stringstream ss;
-      ss << "Added column's length must match record batch's length. Expected length "
-         << num_rows_ << " but got length " << column->length();
-      return Status::Invalid(ss.str());
+      return Status::Invalid(
+          "Added column's length must match record batch's length. Expected length ",
+          num_rows_, " but got length ", column->length());
     }
 
     std::shared_ptr<Schema> new_schema;
@@ -229,17 +226,14 @@ Status RecordBatch::Validate() const {
     auto arr_shared = this->column_data(i);
     const ArrayData& arr = *arr_shared;
     if (arr.length != num_rows_) {
-      std::stringstream ss;
-      ss << "Number of rows in column " << i << " did not match batch: " << arr.length
-         << " vs " << num_rows_;
-      return Status::Invalid(ss.str());
+      return Status::Invalid("Number of rows in column ", i,
+                             " did not match batch: ", arr.length, " vs ", num_rows_);
     }
     const auto& schema_type = *schema_->field(i)->type();
     if (!arr.type->Equals(schema_type)) {
-      std::stringstream ss;
-      ss << "Column " << i << " type not match schema: " << arr.type->ToString() << " vs "
-         << schema_type.ToString();
-      return Status::Invalid(ss.str());
+      return Status::Invalid("Column ", i,
+                             " type not match schema: ", arr.type->ToString(), " vs ",
+                             schema_type.ToString());
     }
   }
   return Status::OK();
diff --git a/cpp/src/arrow/status.cc b/cpp/src/arrow/status.cc
index 8be8b36..db7f087 100644
--- a/cpp/src/arrow/status.cc
+++ b/cpp/src/arrow/status.cc
@@ -13,6 +13,7 @@
 #include "arrow/status.h"
 
 #include <assert.h>
+#include <sstream>
 
 namespace arrow {
 
diff --git a/cpp/src/arrow/status.h b/cpp/src/arrow/status.h
index e3632a6..12975af 100644
--- a/cpp/src/arrow/status.h
+++ b/cpp/src/arrow/status.h
@@ -25,34 +25,41 @@
 #endif
 
 #include "arrow/util/macros.h"
+#include "arrow/util/string_builder.h"
 #include "arrow/util/visibility.h"
 
 #ifdef ARROW_EXTRA_ERROR_CONTEXT
 
-/// \brief Propagate any non-successful Status to the caller
-#define ARROW_RETURN_NOT_OK(s)                                                      \
-  do {                                                                              \
-    ::arrow::Status _s = (s);                                                       \
-    if (ARROW_PREDICT_FALSE(!_s.ok())) {                                            \
-      std::stringstream ss;                                                         \
-      ss << __FILE__ << ":" << __LINE__ << " code: " << #s << "\n" << _s.message(); \
-      return ::arrow::Status(_s.code(), ss.str());                                  \
-    }                                                                               \
+/// \brief Return with given status if condition is met.
+#define ARROW_RETURN_IF(condition, status)                        \
+  do {                                                            \
+    if (ARROW_PREDICT_FALSE(condition)) {                         \
+      ::arrow::Status _s = (status);                              \
+      std::stringstream ss;                                       \
+      ss << __FILE__ << ":" << __LINE__ << " : " << _s.message(); \
+      return ::arrow::Status(_s.code(), ss.str());                \
+    }                                                             \
   } while (0)
 
 #else
 
-/// \brief Propagate any non-successful Status to the caller
-#define ARROW_RETURN_NOT_OK(s)           \
-  do {                                   \
-    ::arrow::Status _s = (s);            \
-    if (ARROW_PREDICT_FALSE(!_s.ok())) { \
-      return _s;                         \
-    }                                    \
-  } while (false)
+#define ARROW_RETURN_IF(condition, status) \
+  do {                                     \
+    if (ARROW_PREDICT_FALSE(condition)) {  \
+      return (status);                     \
+    }                                      \
+  } while (0)
 
 #endif  // ARROW_EXTRA_ERROR_CONTEXT
 
+/// \brief Propagate any non-successful Status to the caller
+#define ARROW_RETURN_NOT_OK(status)  \
+  do {                               \
+    ::arrow::Status __s = (status);  \
+    ARROW_RETURN_IF(!__s.ok(), __s); \
+                                     \
+  } while (false)
+
 #define RETURN_NOT_OK_ELSE(s, else_) \
   do {                               \
     ::arrow::Status _s = (s);        \
@@ -62,17 +69,6 @@
     }                                \
   } while (false)
 
-#define ARROW_RETURN_FAILURE_IF_FALSE(condition, status)                                 \
-  do {                                                                                   \
-    if (!(condition)) {                                                                  \
-      Status _status = (status);                                                         \
-      std::stringstream ss;                                                              \
-      ss << __FILE__ << ":" << __LINE__ << " code: " << _status.CodeAsString() << " \n " \
-         << _status.message();                                                           \
-      return ::arrow::Status(_status.code(), ss.str());                                  \
-    }                                                                                    \
-  } while (0)
-
 // This is an internal-use macro and should not be used in public headers.
 #ifndef RETURN_NOT_OK
 #define RETURN_NOT_OK(s) ARROW_RETURN_NOT_OK(s)
@@ -149,84 +145,119 @@ class ARROW_EXPORT Status {
   static Status OK() { return Status(); }
 
   /// Return a success status with a specific message
-  static Status OK(const std::string& msg) { return Status(StatusCode::OK, msg); }
+  template <typename... Args>
+  static Status OK(Args&&... args) {
+    return Status(StatusCode::OK, util::StringBuilder(std::forward<Args>(args)...));
+  }
 
   /// Return an error status for out-of-memory conditions
-  static Status OutOfMemory(const std::string& msg) {
-    return Status(StatusCode::OutOfMemory, msg);
+  template <typename... Args>
+  static Status OutOfMemory(Args&&... args) {
+    return Status(StatusCode::OutOfMemory,
+                  util::StringBuilder(std::forward<Args>(args)...));
   }
 
   /// Return an error status for failed key lookups (e.g. column name in a table)
-  static Status KeyError(const std::string& msg) {
-    return Status(StatusCode::KeyError, msg);
+  template <typename... Args>
+  static Status KeyError(Args&&... args) {
+    return Status(StatusCode::KeyError, util::StringBuilder(std::forward<Args>(args)...));
   }
 
   /// Return an error status for type errors (such as mismatching data types)
-  static Status TypeError(const std::string& msg) {
-    return Status(StatusCode::TypeError, msg);
+  template <typename... Args>
+  static Status TypeError(Args&&... args) {
+    return Status(StatusCode::TypeError,
+                  util::StringBuilder(std::forward<Args>(args)...));
   }
 
   /// Return an error status for unknown errors
-  static Status UnknownError(const std::string& msg) {
-    return Status(StatusCode::UnknownError, msg);
+  template <typename... Args>
+  static Status UnknownError(Args&&... args) {
+    return Status(StatusCode::UnknownError,
+                  util::StringBuilder(std::forward<Args>(args)...));
   }
 
   /// Return an error status when an operation or a combination of operation and
   /// data types is unimplemented
-  static Status NotImplemented(const std::string& msg) {
-    return Status(StatusCode::NotImplemented, msg);
+  template <typename... Args>
+  static Status NotImplemented(Args&&... args) {
+    return Status(StatusCode::NotImplemented,
+                  util::StringBuilder(std::forward<Args>(args)...));
   }
 
   /// Return an error status for invalid data (for example a string that fails parsing)
-  static Status Invalid(const std::string& msg) {
-    return Status(StatusCode::Invalid, msg);
+  template <typename... Args>
+  static Status Invalid(Args&&... args) {
+    return Status(StatusCode::Invalid, util::StringBuilder(std::forward<Args>(args)...));
   }
 
   /// Return an error status when a container's capacity would exceed its limits
-  static Status CapacityError(const std::string& msg) {
-    return Status(StatusCode::CapacityError, msg);
+  template <typename... Args>
+  static Status CapacityError(Args&&... args) {
+    return Status(StatusCode::CapacityError,
+                  util::StringBuilder(std::forward<Args>(args)...));
   }
 
   /// Return an error status when some IO-related operation failed
-  static Status IOError(const std::string& msg) {
-    return Status(StatusCode::IOError, msg);
+  template <typename... Args>
+  static Status IOError(Args&&... args) {
+    return Status(StatusCode::IOError, util::StringBuilder(std::forward<Args>(args)...));
   }
 
   /// Return an error status when some (de)serialization operation failed
-  static Status SerializationError(const std::string& msg) {
-    return Status(StatusCode::SerializationError, msg);
+  template <typename... Args>
+  static Status SerializationError(Args&&... args) {
+    return Status(StatusCode::SerializationError,
+                  util::StringBuilder(std::forward<Args>(args)...));
   }
 
-  static Status RError(const std::string& msg) { return Status(StatusCode::RError, msg); }
+  template <typename... Args>
+  static Status RError(Args&&... args) {
+    return Status(StatusCode::RError, util::StringBuilder(std::forward<Args>(args)...));
+  }
 
-  static Status PlasmaObjectExists(const std::string& msg) {
-    return Status(StatusCode::PlasmaObjectExists, msg);
+  template <typename... Args>
+  static Status PlasmaObjectExists(Args&&... args) {
+    return Status(StatusCode::PlasmaObjectExists,
+                  util::StringBuilder(std::forward<Args>(args)...));
   }
 
-  static Status PlasmaObjectNonexistent(const std::string& msg) {
-    return Status(StatusCode::PlasmaObjectNonexistent, msg);
+  template <typename... Args>
+  static Status PlasmaObjectNonexistent(Args&&... args) {
+    return Status(StatusCode::PlasmaObjectNonexistent,
+                  util::StringBuilder(std::forward<Args>(args)...));
   }
 
-  static Status PlasmaObjectAlreadySealed(const std::string& msg) {
-    return Status(StatusCode::PlasmaObjectAlreadySealed, msg);
+  template <typename... Args>
+  static Status PlasmaObjectAlreadySealed(Args&&... args) {
+    return Status(StatusCode::PlasmaObjectAlreadySealed,
+                  util::StringBuilder(std::forward<Args>(args)...));
   }
 
-  static Status PlasmaStoreFull(const std::string& msg) {
-    return Status(StatusCode::PlasmaStoreFull, msg);
+  template <typename... Args>
+  static Status PlasmaStoreFull(Args&&... args) {
+    return Status(StatusCode::PlasmaStoreFull,
+                  util::StringBuilder(std::forward<Args>(args)...));
   }
 
   static Status StillExecuting() { return Status(StatusCode::StillExecuting, ""); }
 
-  static Status CodeGenError(const std::string& msg) {
-    return Status(StatusCode::CodeGenError, msg);
+  template <typename... Args>
+  static Status CodeGenError(Args&&... args) {
+    return Status(StatusCode::CodeGenError,
+                  util::StringBuilder(std::forward<Args>(args)...));
   }
 
-  static Status ExpressionValidationError(const std::string& msg) {
-    return Status(StatusCode::ExpressionValidationError, msg);
+  template <typename... Args>
+  static Status ExpressionValidationError(Args&&... args) {
+    return Status(StatusCode::ExpressionValidationError,
+                  util::StringBuilder(std::forward<Args>(args)...));
   }
 
-  static Status ExecutionError(const std::string& msg) {
-    return Status(StatusCode::ExecutionError, msg);
+  template <typename... Args>
+  static Status ExecutionError(Args&&... args) {
+    return Status(StatusCode::ExecutionError,
+                  util::StringBuilder(std::forward<Args>(args)...));
   }
 
   /// Return true iff the status indicates success.
diff --git a/cpp/src/arrow/table.cc b/cpp/src/arrow/table.cc
index 1f3d927..d232ac3 100644
--- a/cpp/src/arrow/table.cc
+++ b/cpp/src/arrow/table.cc
@@ -234,10 +234,8 @@ Status Column::ValidateData() {
   for (int i = 0; i < data_->num_chunks(); ++i) {
     std::shared_ptr<DataType> type = data_->chunk(i)->type();
     if (!this->type()->Equals(type)) {
-      std::stringstream ss;
-      ss << "In chunk " << i << " expected type " << this->type()->ToString()
-         << " but saw " << type->ToString();
-      return Status::Invalid(ss.str());
+      return Status::Invalid("In chunk ", i, " expected type ", this->type()->ToString(),
+                             " but saw ", type->ToString());
     }
   }
   return Status::OK();
@@ -301,10 +299,9 @@ class SimpleTable : public Table {
     DCHECK(col != nullptr);
 
     if (col->length() != num_rows_) {
-      std::stringstream ss;
-      ss << "Added column's length must match table's length. Expected length "
-         << num_rows_ << " but got length " << col->length();
-      return Status::Invalid(ss.str());
+      return Status::Invalid(
+          "Added column's length must match table's length. Expected length ", num_rows_,
+          " but got length ", col->length());
     }
 
     std::shared_ptr<Schema> new_schema;
@@ -319,10 +316,9 @@ class SimpleTable : public Table {
     DCHECK(col != nullptr);
 
     if (col->length() != num_rows_) {
-      std::stringstream ss;
-      ss << "Added column's length must match table's length. Expected length "
-         << num_rows_ << " but got length " << col->length();
-      return Status::Invalid(ss.str());
+      return Status::Invalid(
+          "Added column's length must match table's length. Expected length ", num_rows_,
+          " but got length ", col->length());
     }
 
     std::shared_ptr<Schema> new_schema;
@@ -363,15 +359,11 @@ class SimpleTable : public Table {
     for (int i = 0; i < num_columns(); ++i) {
       const Column* col = columns_[i].get();
       if (col == nullptr) {
-        std::stringstream ss;
-        ss << "Column " << i << " was null";
-        return Status::Invalid(ss.str());
+        return Status::Invalid("Column ", i, " was null");
       }
       if (!col->field()->Equals(*schema_->field(i))) {
-        std::stringstream ss;
-        ss << "Column field " << i << " named " << col->name()
-           << " is inconsistent with schema";
-        return Status::Invalid(ss.str());
+        return Status::Invalid("Column field ", i, " named ", col->name(),
+                               " is inconsistent with schema");
       }
     }
 
@@ -379,10 +371,8 @@ class SimpleTable : public Table {
     for (int i = 0; i < num_columns(); ++i) {
       const Column* col = columns_[i].get();
       if (col->length() != num_rows_) {
-        std::stringstream ss;
-        ss << "Column " << i << " named " << col->name() << " expected length "
-           << num_rows_ << " but got length " << col->length();
-        return Status::Invalid(ss.str());
+        return Status::Invalid("Column ", i, " named ", col->name(), " expected length ",
+                               num_rows_, " but got length ", col->length());
       }
     }
     return Status::OK();
@@ -414,11 +404,9 @@ Status Table::FromRecordBatches(const std::shared_ptr<Schema>& schema,
 
   for (int i = 0; i < nbatches; ++i) {
     if (!batches[i]->schema()->Equals(*schema, false)) {
-      std::stringstream ss;
-      ss << "Schema at index " << static_cast<int>(i) << " was different: \n"
-         << schema->ToString() << "\nvs\n"
-         << batches[i]->schema()->ToString();
-      return Status::Invalid(ss.str());
+      return Status::Invalid("Schema at index ", static_cast<int>(i),
+                             " was different: \n", schema->ToString(), "\nvs\n",
+                             batches[i]->schema()->ToString());
     }
   }
 
@@ -458,11 +446,9 @@ Status ConcatenateTables(const std::vector<std::shared_ptr<Table>>& tables,
 
   for (int i = 1; i < ntables; ++i) {
     if (!tables[i]->schema()->Equals(*schema, false)) {
-      std::stringstream ss;
-      ss << "Schema at index " << static_cast<int>(i) << " was different: \n"
-         << schema->ToString() << "\nvs\n"
-         << tables[i]->schema()->ToString();
-      return Status::Invalid(ss.str());
+      return Status::Invalid("Schema at index ", static_cast<int>(i),
+                             " was different: \n", schema->ToString(), "\nvs\n",
+                             tables[i]->schema()->ToString());
     }
   }
 
diff --git a/cpp/src/arrow/util/compression_brotli.cc b/cpp/src/arrow/util/compression_brotli.cc
index 89d099d..3d75253 100644
--- a/cpp/src/arrow/util/compression_brotli.cc
+++ b/cpp/src/arrow/util/compression_brotli.cc
@@ -81,9 +81,7 @@ class BrotliDecompressor : public Decompressor {
   Status BrotliError(const char* msg) { return Status::IOError(msg); }
 
   Status BrotliError(BrotliDecoderErrorCode code, const char* prefix_msg) {
-    std::stringstream ss;
-    ss << prefix_msg << BrotliDecoderErrorString(code);
-    return Status::IOError(ss.str());
+    return Status::IOError(prefix_msg, BrotliDecoderErrorString(code));
   }
 
   BrotliDecoderState* state_ = nullptr;
diff --git a/cpp/src/arrow/util/compression_lz4.cc b/cpp/src/arrow/util/compression_lz4.cc
index 97fd46a..d157ba6 100644
--- a/cpp/src/arrow/util/compression_lz4.cc
+++ b/cpp/src/arrow/util/compression_lz4.cc
@@ -31,6 +31,10 @@
 namespace arrow {
 namespace util {
 
+static Status LZ4Error(LZ4F_errorCode_t ret, const char* prefix_msg) {
+  return Status::IOError(prefix_msg, LZ4F_getErrorName(ret));
+}
+
 // ----------------------------------------------------------------------
 // Lz4 decompressor implementation
 
@@ -79,12 +83,6 @@ class LZ4Decompressor : public Decompressor {
   bool IsFinished() override { return finished_; }
 
  protected:
-  Status LZ4Error(LZ4F_errorCode_t ret, const char* prefix_msg) {
-    std::stringstream ss;
-    ss << prefix_msg << LZ4F_getErrorName(ret);
-    return Status::IOError(ss.str());
-  }
-
   LZ4F_dctx* ctx_ = nullptr;
   bool finished_;
 };
@@ -125,12 +123,6 @@ class LZ4Compressor : public Compressor {
              bool* should_retry) override;
 
  protected:
-  Status LZ4Error(LZ4F_errorCode_t ret, const char* prefix_msg) {
-    std::stringstream ss;
-    ss << prefix_msg << LZ4F_getErrorName(ret);
-    return Status::IOError(ss.str());
-  }
-
   LZ4F_cctx* ctx_ = nullptr;
   LZ4F_preferences_t prefs_;
   bool first_time_;
diff --git a/cpp/src/arrow/util/compression_snappy.cc b/cpp/src/arrow/util/compression_snappy.cc
index 1b483e5..058593f 100644
--- a/cpp/src/arrow/util/compression_snappy.cc
+++ b/cpp/src/arrow/util/compression_snappy.cc
@@ -57,10 +57,8 @@ Status SnappyCodec::Decompress(int64_t input_len, const uint8_t* input,
     return Status::IOError("Corrupt snappy compressed data.");
   }
   if (output_buffer_len < static_cast<int64_t>(decompressed_size)) {
-    std::stringstream ss;
-    ss << "Output buffer size (" << output_buffer_len << ") must be " << decompressed_size
-       << " or larger.";
-    return Status::Invalid(ss.str());
+    return Status::Invalid("Output buffer size (", output_buffer_len, ") must be ",
+                           decompressed_size, " or larger.");
   }
   if (output_len) {
     *output_len = static_cast<int64_t>(decompressed_size);
diff --git a/cpp/src/arrow/util/compression_zlib.cc b/cpp/src/arrow/util/compression_zlib.cc
index 686dffa..dfda317 100644
--- a/cpp/src/arrow/util/compression_zlib.cc
+++ b/cpp/src/arrow/util/compression_zlib.cc
@@ -76,6 +76,10 @@ static int DecompressionWindowBitsForFormat(GZipCodec::Format format) {
   }
 }
 
+static Status ZlibErrorPrefix(const char* prefix_msg, const char* msg) {
+  return Status::IOError(prefix_msg, (msg) ? msg : "(unknown error)");
+}
+
 // ----------------------------------------------------------------------
 // gzip decompressor implementation
 
@@ -142,14 +146,7 @@ class GZipDecompressor : public Decompressor {
 
  protected:
   Status ZlibError(const char* prefix_msg) {
-    std::stringstream ss;
-    ss << prefix_msg;
-    if (stream_.msg && *stream_.msg) {
-      ss << stream_.msg;
-    } else {
-      ss << "(unknown error)";
-    }
-    return Status::IOError(ss.str());
+    return ZlibErrorPrefix(prefix_msg, stream_.msg);
   }
 
   z_stream stream_;
@@ -197,14 +194,7 @@ class GZipCompressor : public Compressor {
 
  protected:
   Status ZlibError(const char* prefix_msg) {
-    std::stringstream ss;
-    ss << prefix_msg;
-    if (stream_.msg && *stream_.msg) {
-      ss << stream_.msg;
-    } else {
-      ss << "(unknown error)";
-    }
-    return Status::IOError(ss.str());
+    return ZlibErrorPrefix(prefix_msg, stream_.msg);
   }
 
   z_stream stream_;
@@ -344,9 +334,7 @@ class GZipCodec::GZipCodecImpl {
     int window_bits = CompressionWindowBitsForFormat(format_);
     if ((ret = deflateInit2(&stream_, Z_DEFAULT_COMPRESSION, Z_DEFLATED, window_bits,
                             kGZipDefaultCompressionLevel, Z_DEFAULT_STRATEGY)) != Z_OK) {
-      std::stringstream ss;
-      ss << "zlib deflateInit failed: " << std::string(stream_.msg);
-      return Status::IOError(ss.str());
+      return ZlibErrorPrefix("zlib deflateInit failed: ", stream_.msg);
     }
     compressor_initialized_ = true;
     return Status::OK();
@@ -367,9 +355,7 @@ class GZipCodec::GZipCodecImpl {
     // Initialize to run either deflate or zlib/gzip format
     int window_bits = DecompressionWindowBitsForFormat(format_);
     if ((ret = inflateInit2(&stream_, window_bits)) != Z_OK) {
-      std::stringstream ss;
-      ss << "zlib inflateInit failed: " << std::string(stream_.msg);
-      return Status::IOError(ss.str());
+      return ZlibErrorPrefix("zlib inflateInit failed: ", stream_.msg);
     }
     decompressor_initialized_ = true;
     return Status::OK();
@@ -401,9 +387,7 @@ class GZipCodec::GZipCodecImpl {
 
     // Reset the stream for this block
     if (inflateReset(&stream_) != Z_OK) {
-      std::stringstream ss;
-      ss << "zlib inflateReset failed: " << std::string(stream_.msg);
-      return Status::IOError(ss.str());
+      return ZlibErrorPrefix("zlib inflateReset failed: ", stream_.msg);
     }
 
     int ret = 0;
@@ -425,18 +409,13 @@ class GZipCodec::GZipCodecImpl {
       if (ret == Z_STREAM_END || ret != Z_OK) break;
 
       // Failure, buffer was too small
-      std::stringstream ss;
-      ss << "Too small a buffer passed to GZipCodec. InputLength=" << input_length
-         << " OutputLength=" << output_buffer_length;
-      return Status::IOError(ss.str());
+      return Status::IOError("Too small a buffer passed to GZipCodec. InputLength=",
+                             input_length, " OutputLength=", output_buffer_length);
     }
 
     // Failure for some other reason
     if (ret != Z_STREAM_END) {
-      std::stringstream ss;
-      ss << "GZipCodec failed: ";
-      if (stream_.msg != NULL) ss << stream_.msg;
-      return Status::IOError(ss.str());
+      return ZlibErrorPrefix("GZipCodec failed: ", stream_.msg);
     }
 
     if (output_length) {
@@ -475,15 +454,12 @@ class GZipCodec::GZipCodecImpl {
         // small
         return Status::IOError("zlib deflate failed, output buffer too small");
       }
-      std::stringstream ss;
-      ss << "zlib deflate failed: " << stream_.msg;
-      return Status::IOError(ss.str());
+
+      return ZlibErrorPrefix("zlib deflate failed: ", stream_.msg);
     }
 
     if (deflateReset(&stream_) != Z_OK) {
-      std::stringstream ss;
-      ss << "zlib deflateReset failed: " << std::string(stream_.msg);
-      return Status::IOError(ss.str());
+      return ZlibErrorPrefix("zlib deflateReset failed: ", stream_.msg);
     }
 
     // Actual output length
diff --git a/cpp/src/arrow/util/compression_zstd.cc b/cpp/src/arrow/util/compression_zstd.cc
index 083cae9..de9df8f 100644
--- a/cpp/src/arrow/util/compression_zstd.cc
+++ b/cpp/src/arrow/util/compression_zstd.cc
@@ -36,9 +36,7 @@ namespace util {
 constexpr int kZSTDDefaultCompressionLevel = 1;
 
 static Status ZSTDError(size_t ret, const char* prefix_msg) {
-  std::stringstream ss;
-  ss << prefix_msg << ZSTD_getErrorName(ret);
-  return Status::IOError(ss.str());
+  return Status::IOError(prefix_msg, ZSTD_getErrorName(ret));
 }
 
 // ----------------------------------------------------------------------
diff --git a/cpp/src/arrow/util/decimal.cc b/cpp/src/arrow/util/decimal.cc
index c47ac82..f6e1105 100644
--- a/cpp/src/arrow/util/decimal.cc
+++ b/cpp/src/arrow/util/decimal.cc
@@ -345,9 +345,7 @@ Status Decimal128::FromString(const util::string_view& s, Decimal128* out,
 
   DecimalComponents dec;
   if (!ParseDecimalComponents(s.data(), s.size(), &dec)) {
-    std::stringstream ss;
-    ss << "The string '" << s << "' is not a valid decimal number";
-    return Status::Invalid(ss.str());
+    return Status::Invalid("The string '", s, "' is not a valid decimal number");
   }
   std::string exponent_value = dec.exponent_sign + dec.exponent_digits;
 
@@ -878,11 +876,9 @@ Status Decimal128::Rescale(int32_t original_scale, int32_t new_scale,
 
   // Fail if we overflow or truncate
   if (ARROW_PREDICT_FALSE(rescale_would_cause_data_loss)) {
-    std::stringstream buf;
-    buf << "Rescaling decimal value " << ToString(original_scale)
-        << " from original scale of " << original_scale << " to new scale of "
-        << new_scale << " would cause data loss";
-    return Status::Invalid(buf.str());
+    return Status::Invalid("Rescaling decimal value ", ToString(original_scale),
+                           " from original scale of ", original_scale,
+                           " to new scale of ", new_scale, " would cause data loss");
   }
 
   return Status::OK();
@@ -909,11 +905,9 @@ Status Decimal128::FromBigEndian(const uint8_t* bytes, int32_t length, Decimal12
   int64_t high, low;
 
   if (length < kMinDecimalBytes || length > kMaxDecimalBytes) {
-    std::ostringstream stream;
-    stream << "Length of byte array passed to Decimal128::FromBigEndian ";
-    stream << "was " << length << ", but must be between ";
-    stream << kMinDecimalBytes << " and " << kMaxDecimalBytes;
-    return Status::Invalid(stream.str());
+    return Status::Invalid("Length of byte array passed to Decimal128::FromBigEndian ",
+                           "was ", length, ", but must be between ", kMinDecimalBytes,
+                           " and ", kMaxDecimalBytes);
   }
 
   // Bytes are coming in big-endian, so the first byte is the MSB and therefore holds the
diff --git a/cpp/src/arrow/util/decimal.h b/cpp/src/arrow/util/decimal.h
index fe76d25..f59a4a4 100644
--- a/cpp/src/arrow/util/decimal.h
+++ b/cpp/src/arrow/util/decimal.h
@@ -149,9 +149,8 @@ class ARROW_EXPORT Decimal128 {
     constexpr auto max_value = std::numeric_limits<T>::max();
     const auto& self = *this;
     if (self < min_value || self > max_value) {
-      std::stringstream buf;
-      buf << "Invalid cast from Decimal128 to " << sizeof(T) << " byte integer";
-      return Status::Invalid(buf.str());
+      return Status::Invalid("Invalid cast from Decimal128 to ", sizeof(T),
+                             " byte integer");
     }
     *out = static_cast<T>(low_bits_);
     return Status::OK();
diff --git a/cpp/src/arrow/util/io-util.cc b/cpp/src/arrow/util/io-util.cc
index 74ad806..5d67fe8 100644
--- a/cpp/src/arrow/util/io-util.cc
+++ b/cpp/src/arrow/util/io-util.cc
@@ -113,10 +113,8 @@ static inline Status CheckFileOpResult(int ret, int errno_actual,
                                        const PlatformFilename& file_name,
                                        const char* opname) {
   if (ret == -1) {
-    std::stringstream ss;
-    ss << "Failed to " << opname << " file: " << file_name.string();
-    ss << " , error: " << std::strerror(errno_actual);
-    return Status::IOError(ss.str());
+    return Status::IOError("Failed to ", opname, " file: ", file_name.string(),
+                           " , error: ", std::strerror(errno_actual));
   }
   return Status::OK();
 }
@@ -232,12 +230,18 @@ Status CreatePipe(int fd[2]) {
 #endif
 
   if (ret == -1) {
-    return Status::IOError(std::string("Error creating pipe: ") +
-                           std::string(strerror(errno)));
+    return Status::IOError("Error creating pipe: ", std::strerror(errno));
   }
   return Status::OK();
 }
 
+static Status StatusFromErrno(const char* prefix) {
+#ifdef _WIN32
+  errno = __map_mman_error(GetLastError(), EPERM);
+#endif
+  return Status::IOError(prefix, std::strerror(errno));
+}
+
 //
 // Compatible way to remap a memory map
 //
@@ -251,18 +255,12 @@ Status MemoryMapRemap(void* addr, size_t old_size, size_t new_size, int fildes,
   HANDLE fm, h;
 
   if (!UnmapViewOfFile(addr)) {
-    errno = __map_mman_error(GetLastError(), EPERM);
-    std::stringstream ss;
-    ss << "UnmapViewOfFile failed: " << std::strerror(errno);
-    return Status::IOError(ss.str());
+    return StatusFromErrno("UnmapViewOfFile failed: ");
   }
 
   h = reinterpret_cast<HANDLE>(_get_osfhandle(fildes));
   if (h == INVALID_HANDLE_VALUE) {
-    errno = __map_mman_error(GetLastError(), EPERM);
-    std::stringstream ss;
-    ss << "cannot get file handle: " << std::strerror(errno);
-    return Status::IOError(ss.str());
+    return StatusFromErrno("Cannot get file handle: ");
   }
 
   LONG new_size_low = static_cast<LONG>(new_size & 0xFFFFFFFFL);
@@ -272,18 +270,12 @@ Status MemoryMapRemap(void* addr, size_t old_size, size_t new_size, int fildes,
   SetEndOfFile(h);
   fm = CreateFileMapping(h, NULL, PAGE_READWRITE, 0, 0, "");
   if (fm == NULL) {
-    errno = __map_mman_error(GetLastError(), EPERM);
-    std::stringstream ss;
-    ss << "mremap failed: " << std::strerror(errno);
-    return Status::IOError(ss.str());
+    return StatusFromErrno("CreateFileMapping failed: ");
   }
   *new_addr = MapViewOfFile(fm, FILE_MAP_WRITE, 0, 0, new_size);
   CloseHandle(fm);
   if (new_addr == NULL) {
-    errno = __map_mman_error(GetLastError(), EPERM);
-    std::stringstream ss;
-    ss << "mremap failed: " << std::strerror(errno);
-    return Status::IOError(ss.str());
+    return StatusFromErrno("MapViewOfFile failed: ");
   }
   return Status::OK();
 #else
@@ -291,26 +283,26 @@ Status MemoryMapRemap(void* addr, size_t old_size, size_t new_size, int fildes,
   // we have to close the mmap first, truncate the file to the new size
   // and recreate the mmap
   if (munmap(addr, old_size) == -1) {
-    std::stringstream ss;
-    ss << "munmap failed: " << std::strerror(errno);
-    return Status::IOError(ss.str());
+    return StatusFromErrno("munmap failed: ");
   }
   if (ftruncate(fildes, new_size) == -1) {
-    std::stringstream ss;
-    ss << "cannot truncate file: " << std::strerror(errno);
-    return Status::IOError(ss.str());
+    return StatusFromErrno("ftruncate failed: ");
   }
   // we set READ / WRITE flags on the new map, since we could only have
   // unlarged a RW map in the first place
   *new_addr = mmap(NULL, new_size, PROT_READ | PROT_WRITE, MAP_SHARED, fildes, 0);
+  if (*new_addr == MAP_FAILED) {
+    return StatusFromErrno("mmap failed: ");
+  }
   return Status::OK();
 #else
   if (ftruncate(fildes, new_size) == -1) {
-    std::stringstream ss;
-    ss << "file truncate failed: " << std::strerror(errno);
-    return Status::IOError(ss.str());
+    return StatusFromErrno("ftruncate failed: ");
   }
   *new_addr = mremap(addr, old_size, new_size, MREMAP_MAYMOVE);
+  if (*new_addr == MAP_FAILED) {
+    return StatusFromErrno("mremap failed: ");
+  }
   return Status::OK();
 #endif
 #endif
diff --git a/cpp/src/arrow/util/string_builder.h b/cpp/src/arrow/util/string_builder.h
new file mode 100644
index 0000000..7b3e107
--- /dev/null
+++ b/cpp/src/arrow/util/string_builder.h
@@ -0,0 +1,51 @@
+// 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. template <typename T>
+
+#ifndef ARROW_UTIL_STRING_BUILDER_H
+#define ARROW_UTIL_STRING_BUILDER_H
+
+#include <sstream>
+#include <string>
+#include <utility>
+
+namespace arrow {
+namespace util {
+
+template <typename Head>
+void StringBuilderRecursive(std::stringstream& stream, Head&& head) {
+  stream << head;
+}
+
+template <typename Head, typename... Tail>
+void StringBuilderRecursive(std::stringstream& stream, Head&& head, Tail&&... tail) {
+  StringBuilderRecursive(stream, std::forward<Head>(head));
+  StringBuilderRecursive(stream, std::forward<Tail>(tail)...);
+}
+
+template <typename... Args>
+std::string StringBuilder(Args&&... args) {
+  std::stringstream stream;
+
+  StringBuilderRecursive(stream, std::forward<Args>(args)...);
+
+  return stream.str();
+}
+
+}  // namespace util
+}  // namespace arrow
+
+#endif  // ARROW_UTIL_STRING_BUILDER_H
diff --git a/cpp/src/gandiva/date_utils.cc b/cpp/src/gandiva/date_utils.cc
index 2686b19..8a7e1f0 100644
--- a/cpp/src/gandiva/date_utils.cc
+++ b/cpp/src/gandiva/date_utils.cc
@@ -75,11 +75,8 @@ Status DateUtils::ToInternalFormat(const std::string& format,
         buffer.str("");
         continue;
       } else {
-        if (buffer.str().length() > 0) {
-          std::stringstream err_msg;
-          err_msg << "Invalid date format string '" << format << "' at position " << i;
-          return Status::Invalid(err_msg.str());
-        }
+        ARROW_RETURN_IF(buffer.str().length() > 0,
+                        Status::Invalid("Invalid date format string '", format, "'"));
 
         is_in_quoted_text = true;
         continue;
@@ -156,10 +153,7 @@ Status DateUtils::ToInternalFormat(const std::string& format,
         }
       }
     } else {
-      // no potential matches found
-      std::stringstream err_msg;
-      err_msg << "Invalid date format string '" << format << "' at position " << i;
-      return Status::Invalid(err_msg.str());
+      return Status::Invalid("Invalid date format string '", format, "'");
     }
   }
 
@@ -170,11 +164,10 @@ Status DateUtils::ToInternalFormat(const std::string& format,
     if (exactMatches.size() == 1 && exactMatches[0].length() == buffer.str().length()) {
       builder << sql_date_format_to_boost_map_[exactMatches[0]];
     } else {
-      // we didn't successfully parse the entire string
+      // Format partially parsed
       int64_t pos = format.length() - buffer.str().length();
-      std::stringstream err_msg;
-      err_msg << "Invalid date format string '" << format << "' at position " << pos;
-      return Status::Invalid(err_msg.str());
+      return Status::Invalid("Invalid date format string '", format, "' at position ",
+                             pos);
     }
   }
   std::string final_pattern = builder.str();
diff --git a/cpp/src/gandiva/engine.cc b/cpp/src/gandiva/engine.cc
index 59884c5..da7a6d8 100644
--- a/cpp/src/gandiva/engine.cc
+++ b/cpp/src/gandiva/engine.cc
@@ -103,12 +103,11 @@ Status Engine::LoadPreCompiledIRFiles(const std::string& byte_code_file_path) {
   /// Read from file into memory buffer.
   llvm::ErrorOr<std::unique_ptr<llvm::MemoryBuffer>> buffer_or_error =
       llvm::MemoryBuffer::getFile(byte_code_file_path);
-  if (!buffer_or_error) {
-    std::stringstream ss;
-    ss << "Could not load module from IR " << byte_code_file_path << ": "
-       << buffer_or_error.getError().message();
-    return Status::CodeGenError(ss.str());
-  }
+  ARROW_RETURN_IF(
+      !buffer_or_error,
+      Status::CodeGenError("Could not load module from IR ", byte_code_file_path, ": ",
+                           buffer_or_error.getError().message()));
+
   std::unique_ptr<llvm::MemoryBuffer> buffer = move(buffer_or_error.get());
 
   /// Parse the IR module.
@@ -123,15 +122,11 @@ Status Engine::LoadPreCompiledIRFiles(const std::string& byte_code_file_path) {
   }
   std::unique_ptr<llvm::Module> ir_module = move(module_or_error.get());
 
-  /// Verify the IR module
-  if (llvm::verifyModule(*ir_module, &llvm::errs())) {
-    return Status::CodeGenError("verify of IR Module failed");
-  }
+  ARROW_RETURN_IF(llvm::verifyModule(*ir_module, &llvm::errs()),
+                  Status::CodeGenError("verify of IR Module failed"));
+  ARROW_RETURN_IF(llvm::Linker::linkModules(*module_, move(ir_module)),
+                  Status::CodeGenError("failed to link IR Modules"));
 
-  // Link this to the primary module.
-  if (llvm::Linker::linkModules(*module_, move(ir_module))) {
-    return Status::CodeGenError("failed to link IR Modules");
-  }
   return Status::OK();
 }
 
@@ -197,13 +192,13 @@ Status Engine::FinalizeModule(bool optimise_ir, bool dump_ir) {
     }
   }
 
-  if (llvm::verifyModule(*module_, &llvm::errs())) {
-    return Status::CodeGenError("verify of module failed after optimisation passes");
-  }
+  ARROW_RETURN_IF(llvm::verifyModule(*module_, &llvm::errs()),
+                  Status::CodeGenError("Module verification failed after optimizer"));
 
   // do the compilation
   execution_engine_->finalizeObject();
   module_finalized_ = true;
+
   return Status::OK();
 }
 
diff --git a/cpp/src/gandiva/expr_validator.cc b/cpp/src/gandiva/expr_validator.cc
index 3f5d637..43de9d7 100644
--- a/cpp/src/gandiva/expr_validator.cc
+++ b/cpp/src/gandiva/expr_validator.cc
@@ -24,133 +24,114 @@
 namespace gandiva {
 
 Status ExprValidator::Validate(const ExpressionPtr& expr) {
-  if (expr == nullptr) {
-    return Status::ExpressionValidationError("Expression cannot be null.");
-  }
+  ARROW_RETURN_IF(expr == nullptr,
+                  Status::ExpressionValidationError("Expression cannot be null"));
+
   Node& root = *expr->root();
-  Status status = root.Accept(*this);
-  if (!status.ok()) {
-    return status;
-  }
-  // validate return type matches
-  // no need to check if type is supported
-  // since root type has been validated.
-  if (!root.return_type()->Equals(*expr->result()->type())) {
-    std::stringstream ss;
-    ss << "Return type of root node " << root.return_type()->name()
-       << " does not match that of expression " << *expr->result()->type();
-    return Status::ExpressionValidationError(ss.str());
-  }
+  ARROW_RETURN_NOT_OK(root.Accept(*this));
+
+  // Ensure root's return type match the expression return type. Type
+  // support validation is not required because root type is already supported.
+  ARROW_RETURN_IF(!root.return_type()->Equals(*expr->result()->type()),
+                  Status::ExpressionValidationError("Return type of root node ",
+                                                    root.return_type()->name(),
+                                                    " does not match that of expression ",
+                                                    expr->result()->type()->name()));
+
   return Status::OK();
 }
 
 Status ExprValidator::Visit(const FieldNode& node) {
   auto llvm_type = types_->IRType(node.return_type()->id());
-  if (llvm_type == nullptr) {
-    std::stringstream ss;
-    ss << "Field " << node.field()->name() << " has unsupported data type "
-       << node.return_type()->name();
-    return Status::ExpressionValidationError(ss.str());
-  }
+  ARROW_RETURN_IF(llvm_type == nullptr,
+                  Status::ExpressionValidationError("Field ", node.field()->name(),
+                                                    " has unsupported data type ",
+                                                    node.return_type()->name()));
 
+  // Ensure that field is found in schema
   auto field_in_schema_entry = field_map_.find(node.field()->name());
+  ARROW_RETURN_IF(field_in_schema_entry == field_map_.end(),
+                  Status::ExpressionValidationError("Field ", node.field()->name(),
+                                                    " not in schema."));
 
-  // validate that field is in schema.
-  if (field_in_schema_entry == field_map_.end()) {
-    std::stringstream ss;
-    ss << "Field " << node.field()->name() << " not in schema.";
-    return Status::ExpressionValidationError(ss.str());
-  }
-
+  // Ensure that that the found field match.
   FieldPtr field_in_schema = field_in_schema_entry->second;
-  // validate that field matches the definition in schema.
-  if (!field_in_schema->Equals(node.field())) {
-    std::stringstream ss;
-    ss << "Field definition in schema " << field_in_schema->ToString()
-       << " different from field in expression " << node.field()->ToString();
-    return Status::ExpressionValidationError(ss.str());
-  }
+  ARROW_RETURN_IF(!field_in_schema->Equals(node.field()),
+                  Status::ExpressionValidationError(
+                      "Field definition in schema ", field_in_schema->ToString(),
+                      " different from field in expression ", node.field()->ToString()));
+
   return Status::OK();
 }
 
 Status ExprValidator::Visit(const FunctionNode& node) {
   auto desc = node.descriptor();
   FunctionSignature signature(desc->name(), desc->params(), desc->return_type());
+
   const NativeFunction* native_function = registry_.LookupSignature(signature);
-  if (native_function == nullptr) {
-    std::stringstream ss;
-    ss << "Function " << signature.ToString() << " not supported yet. ";
-    return Status::ExpressionValidationError(ss.str());
-  }
+  ARROW_RETURN_IF(native_function == nullptr,
+                  Status::ExpressionValidationError("Function ", signature.ToString(),
+                                                    " not supported yet. "));
 
   for (auto& child : node.children()) {
-    Status status = child->Accept(*this);
-    ARROW_RETURN_NOT_OK(status);
+    ARROW_RETURN_NOT_OK(child->Accept(*this));
   }
+
   return Status::OK();
 }
 
 Status ExprValidator::Visit(const IfNode& node) {
-  Status status = node.condition()->Accept(*this);
-  ARROW_RETURN_NOT_OK(status);
-  status = node.then_node()->Accept(*this);
-  ARROW_RETURN_NOT_OK(status);
-  status = node.else_node()->Accept(*this);
-  ARROW_RETURN_NOT_OK(status);
+  ARROW_RETURN_NOT_OK(node.condition()->Accept(*this));
+  ARROW_RETURN_NOT_OK(node.then_node()->Accept(*this));
+  ARROW_RETURN_NOT_OK(node.else_node()->Accept(*this));
 
   auto if_node_ret_type = node.return_type();
   auto then_node_ret_type = node.then_node()->return_type();
   auto else_node_ret_type = node.else_node()->return_type();
 
-  if (!if_node_ret_type->Equals(*then_node_ret_type)) {
-    std::stringstream ss;
-    ss << "Return type of if " << *if_node_ret_type << " and then " << *then_node_ret_type
-       << " not matching.";
-    return Status::ExpressionValidationError(ss.str());
-  }
+  // Then-branch return type must match.
+  ARROW_RETURN_IF(!if_node_ret_type->Equals(*then_node_ret_type),
+                  Status::ExpressionValidationError(
+                      "Return type of if ", if_node_ret_type->ToString(), " and then ",
+                      then_node_ret_type->ToString(), " not matching."));
 
-  if (!if_node_ret_type->Equals(*else_node_ret_type)) {
-    std::stringstream ss;
-    ss << "Return type of if " << *if_node_ret_type << " and else " << *else_node_ret_type
-       << " not matching.";
-    return Status::ExpressionValidationError(ss.str());
-  }
+  // Else-branch return type must match.
+  ARROW_RETURN_IF(!if_node_ret_type->Equals(*else_node_ret_type),
+                  Status::ExpressionValidationError(
+                      "Return type of if ", if_node_ret_type->ToString(), " and else ",
+                      else_node_ret_type->ToString(), " not matching."));
 
   return Status::OK();
 }
 
 Status ExprValidator::Visit(const LiteralNode& node) {
   auto llvm_type = types_->IRType(node.return_type()->id());
-  if (llvm_type == nullptr) {
-    std::stringstream ss;
-    ss << "Value " << node.holder() << " has unsupported data type "
-       << node.return_type()->name();
-    return Status::ExpressionValidationError(ss.str());
-  }
+  ARROW_RETURN_IF(llvm_type == nullptr,
+                  Status::ExpressionValidationError("Value ", node.holder(),
+                                                    " has unsupported data type ",
+                                                    node.return_type()->name()));
+
   return Status::OK();
 }
 
 Status ExprValidator::Visit(const BooleanNode& node) {
-  Status status;
-
-  if (node.children().size() < 2) {
-    std::stringstream ss;
-    ss << "Boolean expression has " << node.children().size()
-       << " children, expected atleast two";
-    return Status::ExpressionValidationError(ss.str());
-  }
+  ARROW_RETURN_IF(
+      node.children().size() < 2,
+      Status::ExpressionValidationError("Boolean expression has ", node.children().size(),
+                                        " children, expected atleast two"));
 
   for (auto& child : node.children()) {
-    if (!child->return_type()->Equals(arrow::boolean())) {
-      std::stringstream ss;
-      ss << "Boolean expression has a child with return type "
-         << child->return_type()->name() << ", expected return type boolean";
-      return Status::ExpressionValidationError(ss.str());
-    }
-
-    status = child->Accept(*this);
-    ARROW_RETURN_NOT_OK(status);
+    const auto bool_type = arrow::boolean();
+    const auto ret_type = child->return_type();
+
+    ARROW_RETURN_IF(!ret_type->Equals(bool_type),
+                    Status::ExpressionValidationError(
+                        "Boolean expression has a child with return type ",
+                        ret_type->ToString(), ", expected return type boolean"));
+
+    ARROW_RETURN_NOT_OK(child->Accept(*this));
   }
+
   return Status::OK();
 }
 
@@ -178,18 +159,13 @@ Status ExprValidator::Visit(const InExpressionNode<std::string>& node) {
 Status ExprValidator::ValidateInExpression(size_t number_of_values,
                                            DataTypePtr in_expr_return_type,
                                            DataTypePtr type_of_values) {
-  if (static_cast<int32_t>(number_of_values) == 0) {
-    std::stringstream ss;
-    ss << "IN Expression needs a non-empty constant list to match.";
-    return Status::ExpressionValidationError(ss.str());
-  }
-
-  if (!in_expr_return_type->Equals(type_of_values)) {
-    std::stringstream ss;
-    ss << "Evaluation expression for IN clause returns " << in_expr_return_type
-       << " values are of type" << type_of_values;
-    return Status::ExpressionValidationError(ss.str());
-  }
+  ARROW_RETURN_IF(number_of_values == 0,
+                  Status::ExpressionValidationError(
+                      "IN Expression needs a non-empty constant list to match."));
+  ARROW_RETURN_IF(!in_expr_return_type->Equals(type_of_values),
+                  Status::ExpressionValidationError(
+                      "Evaluation expression for IN clause returns ", in_expr_return_type,
+                      " values are of type", type_of_values));
 
   return Status::OK();
 }
diff --git a/cpp/src/gandiva/filter.cc b/cpp/src/gandiva/filter.cc
index 7a24d95..6075e25 100644
--- a/cpp/src/gandiva/filter.cc
+++ b/cpp/src/gandiva/filter.cc
@@ -40,32 +40,28 @@ Filter::Filter(std::unique_ptr<LLVMGenerator> llvm_generator, SchemaPtr schema,
 Status Filter::Make(SchemaPtr schema, ConditionPtr condition,
                     std::shared_ptr<Configuration> configuration,
                     std::shared_ptr<Filter>* filter) {
-  ARROW_RETURN_FAILURE_IF_FALSE(schema != nullptr,
-                                Status::Invalid("schema cannot be null"));
-  ARROW_RETURN_FAILURE_IF_FALSE(condition != nullptr,
-                                Status::Invalid("condition cannot be null"));
-  ARROW_RETURN_FAILURE_IF_FALSE(configuration != nullptr,
-                                Status::Invalid("configuration cannot be null"));
+  ARROW_RETURN_IF(schema == nullptr, Status::Invalid("Schema cannot be null"));
+  ARROW_RETURN_IF(condition == nullptr, Status::Invalid("Condition cannot be null"));
+  ARROW_RETURN_IF(configuration == nullptr,
+                  Status::Invalid("Configuration cannot be null"));
+
   static Cache<FilterCacheKey, std::shared_ptr<Filter>> cache;
   FilterCacheKey cache_key(schema, configuration, *(condition.get()));
-  std::shared_ptr<Filter> cachedFilter = cache.GetModule(cache_key);
+  auto cachedFilter = cache.GetModule(cache_key);
   if (cachedFilter != nullptr) {
     *filter = cachedFilter;
     return Status::OK();
   }
+
   // Build LLVM generator, and generate code for the specified expression
   std::unique_ptr<LLVMGenerator> llvm_gen;
-  Status status = LLVMGenerator::Make(configuration, &llvm_gen);
-  ARROW_RETURN_NOT_OK(status);
+  ARROW_RETURN_NOT_OK(LLVMGenerator::Make(configuration, &llvm_gen));
 
   // Run the validation on the expression.
   // Return if the expression is invalid since we will not be able to process further.
   ExprValidator expr_validator(llvm_gen->types(), schema);
-  status = expr_validator.Validate(condition);
-  ARROW_RETURN_NOT_OK(status);
-
-  status = llvm_gen->Build({condition});
-  ARROW_RETURN_NOT_OK(status);
+  ARROW_RETURN_NOT_OK(expr_validator.Validate(condition));
+  ARROW_RETURN_NOT_OK(llvm_gen->Build({condition}));
 
   // Instantiate the filter with the completely built llvm generator
   *filter = std::make_shared<Filter>(std::move(llvm_gen), schema, configuration);
@@ -76,42 +72,33 @@ Status Filter::Make(SchemaPtr schema, ConditionPtr condition,
 
 Status Filter::Evaluate(const arrow::RecordBatch& batch,
                         std::shared_ptr<SelectionVector> out_selection) {
-  if (!batch.schema()->Equals(*schema_)) {
-    return Status::Invalid("Schema in RecordBatch must match the schema in Make()");
-  }
-  if (batch.num_rows() == 0) {
-    return Status::Invalid("RecordBatch must be non-empty.");
-  }
-  if (out_selection == nullptr) {
-    return Status::Invalid("out_selection must be non-null.");
-  }
-  if (out_selection->GetMaxSlots() < batch.num_rows()) {
-    std::stringstream ss;
-    ss << "out_selection has " << out_selection->GetMaxSlots()
-       << " slots, which is less than the batch size " << batch.num_rows();
-    return Status::Invalid(ss.str());
-  }
+  const auto num_rows = batch.num_rows();
+  ARROW_RETURN_IF(!batch.schema()->Equals(*schema_),
+                  Status::Invalid("RecordBatch schema must expected filter schema"));
+  ARROW_RETURN_IF(num_rows == 0, Status::Invalid("RecordBatch must be non-empty."));
+  ARROW_RETURN_IF(out_selection == nullptr,
+                  Status::Invalid("out_selection must be non-null."));
+  ARROW_RETURN_IF(out_selection->GetMaxSlots() < num_rows,
+                  Status::Invalid("Output selection vector capacity too small"));
 
   // Allocate three local_bitmaps (one for output, one for validity, one to compute the
   // intersection).
-  LocalBitMapsHolder bitmaps(batch.num_rows(), 3 /*local_bitmaps*/);
+  LocalBitMapsHolder bitmaps(num_rows, 3 /*local_bitmaps*/);
   int64_t bitmap_size = bitmaps.GetLocalBitMapSize();
 
   auto validity = std::make_shared<arrow::Buffer>(bitmaps.GetLocalBitMap(0), bitmap_size);
   auto value = std::make_shared<arrow::Buffer>(bitmaps.GetLocalBitMap(1), bitmap_size);
-  auto array_data =
-      arrow::ArrayData::Make(arrow::boolean(), batch.num_rows(), {validity, value});
+  auto array_data = arrow::ArrayData::Make(arrow::boolean(), num_rows, {validity, value});
 
   // Execute the expression(s).
-  auto status = llvm_generator_->Execute(batch, {array_data});
-  ARROW_RETURN_NOT_OK(status);
+  ARROW_RETURN_NOT_OK(llvm_generator_->Execute(batch, {array_data}));
 
   // Compute the intersection of the value and validity.
   auto result = bitmaps.GetLocalBitMap(2);
   BitMapAccumulator::IntersectBitMaps(
-      result, {bitmaps.GetLocalBitMap(0), bitmaps.GetLocalBitMap((1))}, batch.num_rows());
+      result, {bitmaps.GetLocalBitMap(0), bitmaps.GetLocalBitMap((1))}, num_rows);
 
-  return out_selection->PopulateFromBitMap(result, bitmap_size, batch.num_rows() - 1);
+  return out_selection->PopulateFromBitMap(result, bitmap_size, num_rows - 1);
 }
 
 }  // namespace gandiva
diff --git a/cpp/src/gandiva/like_holder.cc b/cpp/src/gandiva/like_holder.cc
index d659b22..051b75b 100644
--- a/cpp/src/gandiva/like_holder.cc
+++ b/cpp/src/gandiva/like_holder.cc
@@ -50,39 +50,40 @@ const FunctionNode LikeHolder::TryOptimize(const FunctionNode& node) {
     }
   }
 
-  // didn't hit any of the optimisation paths. return original.
+  // Could not optimize, return original node.
   return node;
 }
 
+static bool IsArrowStringLiteral(arrow::Type::type type) {
+  return type == arrow::Type::STRING || type == arrow::Type::BINARY;
+}
+
 Status LikeHolder::Make(const FunctionNode& node, std::shared_ptr<LikeHolder>* holder) {
-  if (node.children().size() != 2) {
-    return Status::Invalid("'like' function requires two parameters");
-  }
+  ARROW_RETURN_IF(node.children().size() != 2,
+                  Status::Invalid("'like' function requires two parameters"));
 
   auto literal = dynamic_cast<LiteralNode*>(node.children().at(1).get());
-  if (literal == nullptr) {
-    return Status::Invalid("'like' function requires a literal as the second parameter");
-  }
+  ARROW_RETURN_IF(
+      literal == nullptr,
+      Status::Invalid("'like' function requires a literal as the second parameter"));
 
   auto literal_type = literal->return_type()->id();
-  if (literal_type != arrow::Type::STRING && literal_type != arrow::Type::BINARY) {
-    return Status::Invalid(
-        "'like' function requires a string literal as the second parameter");
-  }
-  auto pattern = boost::get<std::string>(literal->holder());
-  return Make(pattern, holder);
+  ARROW_RETURN_IF(
+      !IsArrowStringLiteral(literal_type),
+      Status::Invalid(
+          "'like' function requires a string literal as the second parameter"));
+
+  return Make(boost::get<std::string>(literal->holder()), holder);
 }
 
 Status LikeHolder::Make(const std::string& sql_pattern,
                         std::shared_ptr<LikeHolder>* holder) {
   std::string pcre_pattern;
-  auto status = RegexUtil::SqlLikePatternToPcre(sql_pattern, pcre_pattern);
-  ARROW_RETURN_NOT_OK(status);
+  ARROW_RETURN_NOT_OK(RegexUtil::SqlLikePatternToPcre(sql_pattern, pcre_pattern));
 
   auto lholder = std::shared_ptr<LikeHolder>(new LikeHolder(pcre_pattern));
-  if (!lholder->regex_.ok()) {
-    return Status::Invalid("building re2 regex failed for pattern " + pcre_pattern);
-  }
+  ARROW_RETURN_IF(!lholder->regex_.ok(),
+                  Status::Invalid("Building RE2 pattern '", pcre_pattern, "' failed"));
 
   *holder = lholder;
   return Status::OK();
diff --git a/cpp/src/gandiva/llvm_generator.cc b/cpp/src/gandiva/llvm_generator.cc
index 82d0386..50f147b 100644
--- a/cpp/src/gandiva/llvm_generator.cc
+++ b/cpp/src/gandiva/llvm_generator.cc
@@ -44,10 +44,10 @@ LLVMGenerator::LLVMGenerator()
 Status LLVMGenerator::Make(std::shared_ptr<Configuration> config,
                            std::unique_ptr<LLVMGenerator>* llvm_generator) {
   std::unique_ptr<LLVMGenerator> llvmgen_obj(new LLVMGenerator());
-  Status status = Engine::Make(config, &(llvmgen_obj->engine_));
-  ARROW_RETURN_NOT_OK(status);
 
+  ARROW_RETURN_NOT_OK(Engine::Make(config, &(llvmgen_obj->engine_)));
   *llvm_generator = std::move(llvmgen_obj);
+
   return Status::OK();
 }
 
@@ -57,33 +57,29 @@ Status LLVMGenerator::Add(const ExpressionPtr expr, const FieldDescriptorPtr out
   // decompose the expression to separate out value and validities.
   ExprDecomposer decomposer(function_registry_, annotator_);
   ValueValidityPairPtr value_validity;
-  auto status = decomposer.Decompose(*expr->root(), &value_validity);
-  ARROW_RETURN_NOT_OK(status);
+  ARROW_RETURN_NOT_OK(decomposer.Decompose(*expr->root(), &value_validity));
 
   // Generate the IR function for the decomposed expression.
   llvm::Function* ir_function = nullptr;
-  status = CodeGenExprValue(value_validity->value_expr(), output, idx, &ir_function);
-  ARROW_RETURN_NOT_OK(status);
+  ARROW_RETURN_NOT_OK(
+      CodeGenExprValue(value_validity->value_expr(), output, idx, &ir_function));
 
   std::unique_ptr<CompiledExpr> compiled_expr(
       new CompiledExpr(value_validity, output, ir_function));
   compiled_exprs_.push_back(std::move(compiled_expr));
+
   return Status::OK();
 }
 
 /// Build and optimise module for projection expression.
 Status LLVMGenerator::Build(const ExpressionVector& exprs) {
-  Status status;
-
   for (auto& expr : exprs) {
     auto output = annotator_.AddOutputFieldDescriptor(expr->result());
-    status = Add(expr, output);
-    ARROW_RETURN_NOT_OK(status);
+    ARROW_RETURN_NOT_OK(Add(expr, output));
   }
 
-  // optimise, compile and finalize the module
-  status = engine_->FinalizeModule(optimise_ir_, dump_ir_);
-  ARROW_RETURN_NOT_OK(status);
+  // Optimize, compile and finalize the module
+  ARROW_RETURN_NOT_OK(engine_->FinalizeModule(optimise_ir_, dump_ir_));
 
   // setup the jit functions for each expression.
   for (auto& compiled_expr : compiled_exprs_) {
@@ -91,6 +87,7 @@ Status LLVMGenerator::Build(const ExpressionVector& exprs) {
     EvalFunc fn = reinterpret_cast<EvalFunc>(engine_->CompiledFunction(ir_func));
     compiled_expr->set_jit_function(fn);
   }
+
   return Status::OK();
 }
 
@@ -107,13 +104,15 @@ Status LLVMGenerator::Execute(const arrow::RecordBatch& record_batch,
     EvalFunc jit_function = compiled_expr->jit_function();
     jit_function(eval_batch->GetBufferArray(), eval_batch->GetLocalBitMapArray(),
                  (int64_t)eval_batch->GetExecutionContext(), record_batch.num_rows());
-    // check for execution errors
-    if (eval_batch->GetExecutionContext()->has_error()) {
-      return Status::ExecutionError(eval_batch->GetExecutionContext()->get_error());
-    }
+
+    ARROW_RETURN_IF(
+        eval_batch->GetExecutionContext()->has_error(),
+        Status::ExecutionError(eval_batch->GetExecutionContext()->get_error()));
+
     // generate validity vectors.
     ComputeBitMapsForExpr(*compiled_expr, *eval_batch);
   }
+
   return Status::OK();
 }
 
@@ -233,8 +232,8 @@ Status LLVMGenerator::CodeGenExprValue(DexPtr value_expr, FieldDescriptorPtr out
   engine_->AddFunctionToCompile(func_name);
   *fn = llvm::Function::Create(prototype, llvm::GlobalValue::ExternalLinkage, func_name,
                                module());
-  ARROW_RETURN_FAILURE_IF_FALSE((*fn != nullptr),
-                                Status::CodeGenError("Error creating function."));
+  ARROW_RETURN_IF((*fn == nullptr), Status::CodeGenError("Error creating function."));
+
   // Name the arguments
   llvm::Function::arg_iterator args = (*fn)->arg_begin();
   llvm::Value* arg_addrs = &*args;
@@ -396,6 +395,7 @@ llvm::Value* LLVMGenerator::AddFunctionCall(const std::string& full_name,
     value = ir_builder()->CreateCall(fn, args, full_name);
     DCHECK(value->getType() == ret_type);
   }
+
   return value;
 }
 
diff --git a/cpp/src/gandiva/projector.cc b/cpp/src/gandiva/projector.cc
index 40fdc20..d5902fc 100644
--- a/cpp/src/gandiva/projector.cc
+++ b/cpp/src/gandiva/projector.cc
@@ -45,12 +45,10 @@ Status Projector::Make(SchemaPtr schema, const ExpressionVector& exprs,
 Status Projector::Make(SchemaPtr schema, const ExpressionVector& exprs,
                        std::shared_ptr<Configuration> configuration,
                        std::shared_ptr<Projector>* projector) {
-  ARROW_RETURN_FAILURE_IF_FALSE(schema != nullptr,
-                                Status::Invalid("schema cannot be null"));
-  ARROW_RETURN_FAILURE_IF_FALSE(!exprs.empty(),
-                                Status::Invalid("expressions need to be non-empty"));
-  ARROW_RETURN_FAILURE_IF_FALSE(configuration != nullptr,
-                                Status::Invalid("configuration cannot be null"));
+  ARROW_RETURN_IF(schema == nullptr, Status::Invalid("Schema cannot be null"));
+  ARROW_RETURN_IF(exprs.empty(), Status::Invalid("Expressions cannot be empty"));
+  ARROW_RETURN_IF(configuration == nullptr,
+                  Status::Invalid("Configuration cannot be null"));
 
   // see if equivalent projector was already built
   static Cache<ProjectorCacheKey, std::shared_ptr<Projector>> cache;
@@ -63,23 +61,21 @@ Status Projector::Make(SchemaPtr schema, const ExpressionVector& exprs,
 
   // Build LLVM generator, and generate code for the specified expressions
   std::unique_ptr<LLVMGenerator> llvm_gen;
-  Status status = LLVMGenerator::Make(configuration, &llvm_gen);
-  ARROW_RETURN_NOT_OK(status);
+  ARROW_RETURN_NOT_OK(LLVMGenerator::Make(configuration, &llvm_gen));
 
   // Run the validation on the expressions.
   // Return if any of the expression is invalid since
   // we will not be able to process further.
   ExprValidator expr_validator(llvm_gen->types(), schema);
   for (auto& expr : exprs) {
-    status = expr_validator.Validate(expr);
-    ARROW_RETURN_NOT_OK(status);
+    ARROW_RETURN_NOT_OK(expr_validator.Validate(expr));
   }
 
-  status = llvm_gen->Build(exprs);
-  ARROW_RETURN_NOT_OK(status);
+  ARROW_RETURN_NOT_OK(llvm_gen->Build(exprs));
 
   // save the output field types. Used for validation at Evaluate() time.
   std::vector<FieldPtr> output_fields;
+  output_fields.reserve(exprs.size());
   for (auto& expr : exprs) {
     output_fields.push_back(expr->result());
   }
@@ -94,86 +90,70 @@ Status Projector::Make(SchemaPtr schema, const ExpressionVector& exprs,
 
 Status Projector::Evaluate(const arrow::RecordBatch& batch,
                            const ArrayDataVector& output_data_vecs) {
-  Status status = ValidateEvaluateArgsCommon(batch);
-  ARROW_RETURN_NOT_OK(status);
-
-  if (output_data_vecs.size() != output_fields_.size()) {
-    std::stringstream ss;
-    ss << "number of buffers for output_data_vecs is " << output_data_vecs.size()
-       << ", expected " << output_fields_.size();
-    return Status::Invalid(ss.str());
-  }
+  ARROW_RETURN_NOT_OK(ValidateEvaluateArgsCommon(batch));
+  ARROW_RETURN_IF(
+      output_data_vecs.size() != output_fields_.size(),
+      Status::Invalid("Number of output buffers must match number of fields"));
 
   int idx = 0;
   for (auto& array_data : output_data_vecs) {
+    const auto output_field = output_fields_[idx];
     if (array_data == nullptr) {
-      std::stringstream ss;
-      ss << "array for output field " << output_fields_[idx]->name() << "is null.";
-      return Status::Invalid(ss.str());
+      return Status::Invalid("Output array for field ", output_field->name(),
+                             " should not be null");
     }
 
-    Status status =
-        ValidateArrayDataCapacity(*array_data, *(output_fields_[idx]), batch.num_rows());
-    ARROW_RETURN_NOT_OK(status);
+    ARROW_RETURN_NOT_OK(
+        ValidateArrayDataCapacity(*array_data, *output_field, batch.num_rows()));
     ++idx;
   }
+
   return llvm_generator_->Execute(batch, output_data_vecs);
 }
 
 Status Projector::Evaluate(const arrow::RecordBatch& batch, arrow::MemoryPool* pool,
                            arrow::ArrayVector* output) {
-  Status status = ValidateEvaluateArgsCommon(batch);
-  ARROW_RETURN_NOT_OK(status);
-
-  if (output == nullptr) {
-    return Status::Invalid("output must be non-null.");
-  }
-
-  if (pool == nullptr) {
-    return Status::Invalid("memory pool must be non-null.");
-  }
+  ARROW_RETURN_NOT_OK(ValidateEvaluateArgsCommon(batch));
+  ARROW_RETURN_IF(output == nullptr, Status::Invalid("Output must be non-null."));
+  ARROW_RETURN_IF(pool == nullptr, Status::Invalid("Memory pool must be non-null."));
 
   // Allocate the output data vecs.
   ArrayDataVector output_data_vecs;
+  output_data_vecs.reserve(output_fields_.size());
   for (auto& field : output_fields_) {
     ArrayDataPtr output_data;
 
-    status = AllocArrayData(field->type(), batch.num_rows(), pool, &output_data);
-    ARROW_RETURN_NOT_OK(status);
-
+    ARROW_RETURN_NOT_OK(
+        AllocArrayData(field->type(), batch.num_rows(), pool, &output_data));
     output_data_vecs.push_back(output_data);
   }
 
   // Execute the expression(s).
-  status = llvm_generator_->Execute(batch, output_data_vecs);
-  ARROW_RETURN_NOT_OK(status);
+  ARROW_RETURN_NOT_OK(llvm_generator_->Execute(batch, output_data_vecs));
 
   // Create and return array arrays.
   output->clear();
   for (auto& array_data : output_data_vecs) {
     output->push_back(arrow::MakeArray(array_data));
   }
+
   return Status::OK();
 }
 
 // TODO : handle variable-len vectors
 Status Projector::AllocArrayData(const DataTypePtr& type, int64_t num_records,
                                  arrow::MemoryPool* pool, ArrayDataPtr* array_data) {
-  if (!arrow::is_primitive(type->id())) {
-    return Status::Invalid("Unsupported output data type " + type->ToString());
-  }
+  ARROW_RETURN_IF(!arrow::is_primitive(type->id()),
+                  Status::Invalid("Unsupported output data type ", type));
 
-  arrow::Status astatus;
   std::shared_ptr<arrow::Buffer> null_bitmap;
-  int64_t size = arrow::BitUtil::BytesForBits(num_records);
-  astatus = arrow::AllocateBuffer(pool, size, &null_bitmap);
-  ARROW_RETURN_NOT_OK(astatus);
+  int64_t bitmap_bytes = arrow::BitUtil::BytesForBits(num_records);
+  ARROW_RETURN_NOT_OK(arrow::AllocateBuffer(pool, bitmap_bytes, &null_bitmap));
 
   std::shared_ptr<arrow::Buffer> data;
   const auto& fw_type = dynamic_cast<const arrow::FixedWidthType&>(*type);
   int64_t data_len = arrow::BitUtil::BytesForBits(num_records * fw_type.bit_width());
-  astatus = arrow::AllocateBuffer(pool, data_len, &data);
-  ARROW_RETURN_NOT_OK(astatus);
+  ARROW_RETURN_NOT_OK(arrow::AllocateBuffer(pool, data_len, &data));
 
   // Valgrind detects unitialized memory at byte level. Boolean types use bits
   // and can leave buffer memory uninitialized in the last byte.
@@ -186,47 +166,33 @@ Status Projector::AllocArrayData(const DataTypePtr& type, int64_t num_records,
 }
 
 Status Projector::ValidateEvaluateArgsCommon(const arrow::RecordBatch& batch) {
-  if (!batch.schema()->Equals(*schema_)) {
-    return Status::Invalid("Schema in RecordBatch must match the schema in Make()");
-  }
-  if (batch.num_rows() == 0) {
-    return Status::Invalid("RecordBatch must be non-empty.");
-  }
+  ARROW_RETURN_IF(!batch.schema()->Equals(*schema_),
+                  Status::Invalid("Schema in RecordBatch must match schema in Make()"));
+  ARROW_RETURN_IF(batch.num_rows() == 0,
+                  Status::Invalid("RecordBatch must be non-empty."));
+
   return Status::OK();
 }
 
 Status Projector::ValidateArrayDataCapacity(const arrow::ArrayData& array_data,
                                             const arrow::Field& field,
                                             int64_t num_records) {
-  // verify that there are atleast two buffers (validity and data).
-  if (array_data.buffers.size() < 2) {
-    std::stringstream ss;
-    ss << "number of buffers for output field " << field.name() << "is "
-       << array_data.buffers.size() << ", must have minimum 2.";
-    return Status::Invalid(ss.str());
-  }
+  ARROW_RETURN_IF(array_data.buffers.size() < 2,
+                  Status::Invalid("ArrayData must have at least 2 buffers"));
 
-  // verify size of bitmap buffer.
   int64_t min_bitmap_len = arrow::BitUtil::BytesForBits(num_records);
   int64_t bitmap_len = array_data.buffers[0]->capacity();
-  if (bitmap_len < min_bitmap_len) {
-    std::stringstream ss;
-    ss << "bitmap buffer for output field " << field.name() << "has size " << bitmap_len
-       << ", must have minimum size " << min_bitmap_len;
-    return Status::Invalid(ss.str());
-  }
+  ARROW_RETURN_IF(bitmap_len < min_bitmap_len,
+                  Status::Invalid("Bitmap buffer too small for ", field.name()));
 
   // verify size of data buffer.
   // TODO : handle variable-len vectors
   const auto& fw_type = dynamic_cast<const arrow::FixedWidthType&>(*field.type());
   int64_t min_data_len = arrow::BitUtil::BytesForBits(num_records * fw_type.bit_width());
   int64_t data_len = array_data.buffers[1]->capacity();
-  if (data_len < min_data_len) {
-    std::stringstream ss;
-    ss << "data buffer for output field " << field.name() << " has size " << data_len
-       << ", must have minimum size " << min_data_len;
-    return Status::Invalid(ss.str());
-  }
+  ARROW_RETURN_IF(data_len < min_data_len,
+                  Status::Invalid("Data buffer too small for ", field.name()));
+
   return Status::OK();
 }
 
diff --git a/cpp/src/gandiva/regex_util.cc b/cpp/src/gandiva/regex_util.cc
index 893af09..1d38606 100644
--- a/cpp/src/gandiva/regex_util.cc
+++ b/cpp/src/gandiva/regex_util.cc
@@ -38,20 +38,16 @@ Status RegexUtil::SqlLikePatternToPcre(const std::string& sql_pattern, char esca
     if (cur == escape_char) {
       // escape char must be followed by '_', '%' or the escape char itself.
       ++idx;
-      if (idx == sql_pattern.size()) {
-        std::stringstream msg;
-        msg << "unexpected escape char at the end of pattern " << sql_pattern;
-        return Status::Invalid(msg.str());
-      }
+      ARROW_RETURN_IF(
+          idx == sql_pattern.size(),
+          Status::Invalid("Unexpected escape char at the end of pattern ", sql_pattern));
 
       cur = sql_pattern.at(idx);
       if (cur == '_' || cur == '%' || cur == escape_char) {
         pcre_pattern += cur;
       } else {
-        std::stringstream msg;
-        msg << "invalid escape sequence in pattern " << sql_pattern << " at offset "
-            << idx;
-        return Status::Invalid(msg.str());
+        return Status::Invalid("Invalid escape sequence in pattern ", sql_pattern,
+                               " at offset ", idx);
       }
     } else if (cur == '_') {
       pcre_pattern += '.';
diff --git a/cpp/src/gandiva/selection_vector.cc b/cpp/src/gandiva/selection_vector.cc
index 9266ca7..f89b80c 100644
--- a/cpp/src/gandiva/selection_vector.cc
+++ b/cpp/src/gandiva/selection_vector.cc
@@ -28,22 +28,15 @@ namespace gandiva {
 
 Status SelectionVector::PopulateFromBitMap(const uint8_t* bitmap, int64_t bitmap_size,
                                            int64_t max_bitmap_index) {
-  if (bitmap_size % 8 != 0) {
-    std::stringstream ss;
-    ss << "bitmap size " << bitmap_size << " must be padded to 64-bit size";
-    return Status::Invalid(ss.str());
-  }
-  if (max_bitmap_index < 0) {
-    std::stringstream ss;
-    ss << "max bitmap index " << max_bitmap_index << " must be positive";
-    return Status::Invalid(ss.str());
-  }
-  if (static_cast<uint64_t>(max_bitmap_index) > GetMaxSupportedValue()) {
-    std::stringstream ss;
-    ss << "max_bitmap_index " << max_bitmap_index << " must be <= maxSupportedValue "
-       << GetMaxSupportedValue() << " in selection vector";
-    return Status::Invalid(ss.str());
-  }
+  const uint64_t max_idx = static_cast<uint64_t>(max_bitmap_index);
+  ARROW_RETURN_IF(bitmap_size % 8, Status::Invalid("Bitmap size ", bitmap_size,
+                                                   " must be aligned to 64-bit size"));
+  ARROW_RETURN_IF(max_bitmap_index < 0,
+                  Status::Invalid("Max bitmap index must be positive"));
+  ARROW_RETURN_IF(
+      max_idx > GetMaxSupportedValue(),
+      Status::Invalid("max_bitmap_index ", max_idx, " must be <= maxSupportedValue ",
+                      GetMaxSupportedValue(), " in selection vector"));
 
   int64_t max_slots = GetMaxSlots();
 
@@ -64,9 +57,9 @@ Status SelectionVector::PopulateFromBitMap(const uint8_t* bitmap, int64_t bitmap
         break;
       }
 
-      if (selection_idx >= max_slots) {
-        return Status::Invalid("selection vector has no remaining slots");
-      }
+      ARROW_RETURN_IF(selection_idx >= max_slots,
+                      Status::Invalid("selection vector has no remaining slots"));
+
       SetIndex(selection_idx, pos_in_bitmap);
       ++selection_idx;
 
@@ -81,60 +74,54 @@ Status SelectionVector::PopulateFromBitMap(const uint8_t* bitmap, int64_t bitmap
 Status SelectionVector::MakeInt16(int64_t max_slots,
                                   std::shared_ptr<arrow::Buffer> buffer,
                                   std::shared_ptr<SelectionVector>* selection_vector) {
-  auto status = SelectionVectorInt16::ValidateBuffer(max_slots, buffer);
-  ARROW_RETURN_NOT_OK(status);
-
+  ARROW_RETURN_NOT_OK(SelectionVectorInt16::ValidateBuffer(max_slots, buffer));
   *selection_vector = std::make_shared<SelectionVectorInt16>(max_slots, buffer);
+
   return Status::OK();
 }
 
 Status SelectionVector::MakeInt16(int64_t max_slots, arrow::MemoryPool* pool,
                                   std::shared_ptr<SelectionVector>* selection_vector) {
   std::shared_ptr<arrow::Buffer> buffer;
-  auto status = SelectionVectorInt16::AllocateBuffer(max_slots, pool, &buffer);
-  ARROW_RETURN_NOT_OK(status);
-
+  ARROW_RETURN_NOT_OK(SelectionVectorInt16::AllocateBuffer(max_slots, pool, &buffer));
   *selection_vector = std::make_shared<SelectionVectorInt16>(max_slots, buffer);
+
   return Status::OK();
 }
 
 Status SelectionVector::MakeInt32(int64_t max_slots,
                                   std::shared_ptr<arrow::Buffer> buffer,
                                   std::shared_ptr<SelectionVector>* selection_vector) {
-  auto status = SelectionVectorInt32::ValidateBuffer(max_slots, buffer);
-  ARROW_RETURN_NOT_OK(status);
-
+  ARROW_RETURN_NOT_OK(SelectionVectorInt32::ValidateBuffer(max_slots, buffer));
   *selection_vector = std::make_shared<SelectionVectorInt32>(max_slots, buffer);
+
   return Status::OK();
 }
 
 Status SelectionVector::MakeInt32(int64_t max_slots, arrow::MemoryPool* pool,
                                   std::shared_ptr<SelectionVector>* selection_vector) {
   std::shared_ptr<arrow::Buffer> buffer;
-  auto status = SelectionVectorInt32::AllocateBuffer(max_slots, pool, &buffer);
-  ARROW_RETURN_NOT_OK(status);
-
+  ARROW_RETURN_NOT_OK(SelectionVectorInt32::AllocateBuffer(max_slots, pool, &buffer));
   *selection_vector = std::make_shared<SelectionVectorInt32>(max_slots, buffer);
+
   return Status::OK();
 }
 
 Status SelectionVector::MakeInt64(int64_t max_slots,
                                   std::shared_ptr<arrow::Buffer> buffer,
                                   std::shared_ptr<SelectionVector>* selection_vector) {
-  auto status = SelectionVectorInt64::ValidateBuffer(max_slots, buffer);
-  ARROW_RETURN_NOT_OK(status);
-
+  ARROW_RETURN_NOT_OK(SelectionVectorInt64::ValidateBuffer(max_slots, buffer));
   *selection_vector = std::make_shared<SelectionVectorInt64>(max_slots, buffer);
+
   return Status::OK();
 }
 
 Status SelectionVector::MakeInt64(int64_t max_slots, arrow::MemoryPool* pool,
                                   std::shared_ptr<SelectionVector>* selection_vector) {
   std::shared_ptr<arrow::Buffer> buffer;
-  auto status = SelectionVectorInt64::AllocateBuffer(max_slots, pool, &buffer);
-  ARROW_RETURN_NOT_OK(status);
-
+  ARROW_RETURN_NOT_OK(SelectionVectorInt64::AllocateBuffer(max_slots, pool, &buffer));
   *selection_vector = std::make_shared<SelectionVectorInt64>(max_slots, buffer);
+
   return Status::OK();
 }
 
@@ -142,8 +129,7 @@ template <typename C_TYPE, typename A_TYPE>
 Status SelectionVectorImpl<C_TYPE, A_TYPE>::AllocateBuffer(
     int64_t max_slots, arrow::MemoryPool* pool, std::shared_ptr<arrow::Buffer>* buffer) {
   auto buffer_len = max_slots * sizeof(C_TYPE);
-  auto astatus = arrow::AllocateBuffer(pool, buffer_len, buffer);
-  ARROW_RETURN_NOT_OK(astatus);
+  ARROW_RETURN_NOT_OK(arrow::AllocateBuffer(pool, buffer_len, buffer));
 
   return Status::OK();
 }
@@ -151,19 +137,13 @@ Status SelectionVectorImpl<C_TYPE, A_TYPE>::AllocateBuffer(
 template <typename C_TYPE, typename A_TYPE>
 Status SelectionVectorImpl<C_TYPE, A_TYPE>::ValidateBuffer(
     int64_t max_slots, std::shared_ptr<arrow::Buffer> buffer) {
-  // verify buffer is mutable
-  if (!buffer->is_mutable()) {
-    return Status::Invalid("buffer for selection vector must be mutable");
-  }
+  ARROW_RETURN_IF(!buffer->is_mutable(),
+                  Status::Invalid("buffer for selection vector must be mutable"));
+
+  const int64_t min_len = max_slots * sizeof(C_TYPE);
+  ARROW_RETURN_IF(buffer->size() < min_len,
+                  Status::Invalid("Buffer for selection vector is too small"));
 
-  // verify size of buffer.
-  int64_t min_len = max_slots * sizeof(C_TYPE);
-  if (buffer->size() < min_len) {
-    std::stringstream ss;
-    ss << "buffer for selection_data has size " << buffer->size()
-       << ", must have minimum size " << min_len;
-    return Status::Invalid(ss.str());
-  }
   return Status::OK();
 }
 
diff --git a/cpp/src/gandiva/tests/projector_build_validation_test.cc b/cpp/src/gandiva/tests/projector_build_validation_test.cc
index ddcb729..18f0295 100644
--- a/cpp/src/gandiva/tests/projector_build_validation_test.cc
+++ b/cpp/src/gandiva/tests/projector_build_validation_test.cc
@@ -191,8 +191,6 @@ TEST_F(TestProjector, TestIfNotMatchingReturnType) {
   std::shared_ptr<Projector> projector;
   Status status = Projector::Make(schema, {expr}, &projector);
   EXPECT_TRUE(status.IsExpressionValidationError());
-  std::string expected_error = "Return type of if bool and then int32 not matching.";
-  EXPECT_TRUE(status.message().find(expected_error) != std::string::npos);
 }
 
 TEST_F(TestProjector, TestElseNotMatchingReturnType) {
@@ -218,8 +216,6 @@ TEST_F(TestProjector, TestElseNotMatchingReturnType) {
   std::shared_ptr<Projector> projector;
   Status status = Projector::Make(schema, {expr}, &projector);
   EXPECT_TRUE(status.IsExpressionValidationError());
-  std::string expected_error = "Return type of if int32 and else bool not matching.";
-  EXPECT_TRUE(status.message().find(expected_error) != std::string::npos);
 }
 
 TEST_F(TestProjector, TestElseNotSupportedType) {
@@ -245,8 +241,7 @@ TEST_F(TestProjector, TestElseNotSupportedType) {
   std::shared_ptr<Projector> projector;
   Status status = Projector::Make(schema, {expr}, &projector);
   EXPECT_TRUE(status.IsExpressionValidationError());
-  std::string expected_error = "Field c has unsupported data type list";
-  EXPECT_TRUE(status.message().find(expected_error) != std::string::npos);
+  EXPECT_EQ(status.code(), StatusCode::ExpressionValidationError);
 }
 
 TEST_F(TestProjector, TestAndMinChildren) {
@@ -266,8 +261,6 @@ TEST_F(TestProjector, TestAndMinChildren) {
   std::shared_ptr<Projector> projector;
   Status status = Projector::Make(schema, {expr}, &projector);
   EXPECT_TRUE(status.IsExpressionValidationError());
-  std::string expected_error = "Boolean expression has 1 children, expected atleast two";
-  EXPECT_TRUE(status.message().find(expected_error) != std::string::npos);
 }
 
 TEST_F(TestProjector, TestAndBooleanArgType) {
@@ -289,10 +282,6 @@ TEST_F(TestProjector, TestAndBooleanArgType) {
   std::shared_ptr<Projector> projector;
   Status status = Projector::Make(schema, {expr}, &projector);
   EXPECT_TRUE(status.IsExpressionValidationError());
-  std::string expected_error =
-      "Boolean expression has a child with return type int32, expected return type "
-      "boolean";
-  EXPECT_TRUE(status.message().find(expected_error) != std::string::npos);
 }
 
 }  // namespace gandiva
diff --git a/cpp/src/parquet/arrow/reader.cc b/cpp/src/parquet/arrow/reader.cc
index 7830b6a..b5905fd 100644
--- a/cpp/src/parquet/arrow/reader.cc
+++ b/cpp/src/parquet/arrow/reader.cc
@@ -690,10 +690,8 @@ Status FileReader::GetRecordBatchReader(const std::vector<int>& row_group_indice
   int max_num = num_row_groups();
   for (auto row_group_index : row_group_indices) {
     if (row_group_index < 0 || row_group_index >= max_num) {
-      std::ostringstream ss;
-      ss << "Some index in row_group_indices is " << row_group_index
-         << ", which is either < 0 or >= num_row_groups(" << max_num << ")";
-      return Status::Invalid(ss.str());
+      return Status::Invalid("Some index in row_group_indices is ", row_group_index,
+                             ", which is either < 0 or >= num_row_groups(", max_num, ")");
     }
   }
 
@@ -1495,9 +1493,8 @@ Status PrimitiveImpl::NextBatch(int64_t records_to_read,
       TRANSFER_CASE(TIME32, ::arrow::Time32Type, Int32Type)
       TRANSFER_CASE(TIME64, ::arrow::Time64Type, Int64Type)
     default:
-      std::stringstream ss;
-      ss << "No support for reading columns of type " << field_->type()->ToString();
-      return Status::NotImplemented(ss.str());
+      return Status::NotImplemented("No support for reading columns of type ",
+                                    field_->type()->ToString());
   }
 
   DCHECK_NE(result.kind(), Datum::NONE);
diff --git a/cpp/src/parquet/arrow/schema.cc b/cpp/src/parquet/arrow/schema.cc
index af9fbc9..fed0e59 100644
--- a/cpp/src/parquet/arrow/schema.cc
+++ b/cpp/src/parquet/arrow/schema.cc
@@ -80,10 +80,9 @@ static Status FromFLBA(const PrimitiveNode& node, std::shared_ptr<ArrowType>* ou
       *out = MakeDecimal128Type(node);
       break;
     default:
-      std::stringstream ss;
-      ss << "Unhandled logical type " << LogicalTypeToString(node.logical_type())
-         << " for fixed-length binary array";
-      return Status::NotImplemented(ss.str());
+      return Status::NotImplemented("Unhandled logical type ",
+                                    LogicalTypeToString(node.logical_type()),
+                                    " for fixed-length binary array");
   }
 
   return Status::OK();
@@ -122,10 +121,9 @@ static Status FromInt32(const PrimitiveNode& node, std::shared_ptr<ArrowType>* o
       *out = MakeDecimal128Type(node);
       break;
     default:
-      std::stringstream ss;
-      ss << "Unhandled logical type " << LogicalTypeToString(node.logical_type())
-         << " for INT32";
-      return Status::NotImplemented(ss.str());
+      return Status::NotImplemented("Unhandled logical type ",
+                                    LogicalTypeToString(node.logical_type()),
+                                    " for INT32");
   }
   return Status::OK();
 }
@@ -154,10 +152,9 @@ static Status FromInt64(const PrimitiveNode& node, std::shared_ptr<ArrowType>* o
       *out = ::arrow::time64(::arrow::TimeUnit::MICRO);
       break;
     default:
-      std::stringstream ss;
-      ss << "Unhandled logical type " << LogicalTypeToString(node.logical_type())
-         << " for INT64";
-      return Status::NotImplemented(ss.str());
+      return Status::NotImplemented("Unhandled logical type ",
+                                    LogicalTypeToString(node.logical_type()),
+                                    " for INT64");
   }
   return Status::OK();
 }
@@ -613,10 +610,9 @@ Status FieldToNode(const std::shared_ptr<Field>& field,
     }
     default: {
       // TODO: DENSE_UNION, SPARE_UNION, JSON_SCALAR, DECIMAL_TEXT, VARCHAR
-      std::stringstream ss;
-      ss << "Unhandled type for Arrow to Parquet schema conversion: ";
-      ss << field->type()->ToString();
-      return Status::NotImplemented(ss.str());
+      return Status::NotImplemented(
+          "Unhandled type for Arrow to Parquet schema conversion: ",
+          field->type()->ToString());
     }
   }
   PARQUET_CATCH_NOT_OK(*out =
diff --git a/cpp/src/parquet/arrow/writer.cc b/cpp/src/parquet/arrow/writer.cc
index bce9f37..a8153ca 100644
--- a/cpp/src/parquet/arrow/writer.cc
+++ b/cpp/src/parquet/arrow/writer.cc
@@ -676,10 +676,8 @@ Status ArrowColumnWriter::WriteTimestampsCoerce(const bool truncated_timestamps_
   auto DivideBy = [&](const int64_t factor) {
     for (int64_t i = 0; i < array.length(); i++) {
       if (!truncated_timestamps_allowed && !data.IsNull(i) && (values[i] % factor != 0)) {
-        std::stringstream ss;
-        ss << "Casting from " << type.ToString() << " to " << target_type->ToString()
-           << " would lose data: " << values[i];
-        return Status::Invalid(ss.str());
+        return Status::Invalid("Casting from ", type.ToString(), " to ",
+                               target_type->ToString(), " would lose data: ", values[i]);
       }
       buffer[i] = values[i] / factor;
     }
@@ -950,9 +948,8 @@ Status ArrowColumnWriter::Write(const Array& data) {
     default:
       break;
   }
-  std::stringstream ss;
-  ss << "Data type not supported as list value: " << values_array->type()->ToString();
-  return Status::NotImplemented(ss.str());
+  return Status::NotImplemented("Data type not supported as list value: ",
+                                values_array->type()->ToString());
 }
 
 }  // namespace
diff --git a/cpp/src/plasma/io.cc b/cpp/src/plasma/io.cc
index d63ceb6..d2794e8 100644
--- a/cpp/src/plasma/io.cc
+++ b/cpp/src/plasma/io.cc
@@ -49,7 +49,7 @@ Status WriteBytes(int fd, uint8_t* cursor, size_t length) {
       if (errno == EAGAIN || errno == EWOULDBLOCK || errno == EINTR) {
         continue;
       }
-      return Status::IOError(std::string(strerror(errno)));
+      return Status::IOError(strerror(errno));
     } else if (nbytes == 0) {
       return Status::IOError("Encountered unexpected EOF");
     }
@@ -80,7 +80,7 @@ Status ReadBytes(int fd, uint8_t* cursor, size_t length) {
       if (errno == EAGAIN || errno == EWOULDBLOCK || errno == EINTR) {
         continue;
       }
-      return Status::IOError(std::string(strerror(errno)));
+      return Status::IOError(strerror(errno));
     } else if (0 == nbytes) {
       return Status::IOError("Encountered unexpected EOF");
     }
@@ -171,12 +171,12 @@ Status ConnectIpcSocketRetry(const std::string& pathname, int num_retries,
     *fd = ConnectIpcSock(pathname);
     --num_retries;
   }
+
   // If we could not connect to the socket, exit.
   if (*fd == -1) {
-    std::stringstream ss;
-    ss << "Could not connect to socket " << pathname;
-    return Status::IOError(ss.str());
+    return Status::IOError("Could not connect to socket ", pathname);
   }
+
   return Status::OK();
 }