This is an automated email from the ASF dual-hosted git repository.
apitrou pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/arrow.git
The following commit(s) were added to refs/heads/main by this push:
new a32730c488 GH-48251: [C++][CI] Add CSV fuzzing seed corpus generator
(#48252)
a32730c488 is described below
commit a32730c4884171fb9659bc1c0b4d86347a77c529
Author: Antoine Pitrou <[email protected]>
AuthorDate: Mon Dec 1 10:14:37 2025 +0100
GH-48251: [C++][CI] Add CSV fuzzing seed corpus generator (#48252)
### Rationale for this change
The CSV seed corpus for fuzzing currently consists of sample data files
from the Pandas project and our own testing repository. This PR adds an
executable that generates custom seed files with well-defined characteristics
designed to exercise the various data types that the CSV reader is able to
infer automatically.
This PR also switches the `RandomArrayGenerator` facility to the non-"fast"
PCG random generators, which give better output especially relative to the
seed. This requires some minor changes in the tests to workaround some issues
that changing the random generator surfaced.
### Are these changes tested?
By existing tests.
### Are there any user-facing changes?
No.
* GitHub Issue: #48251
Authored-by: Antoine Pitrou <[email protected]>
Signed-off-by: Antoine Pitrou <[email protected]>
---
cpp/build-support/fuzzing/generate_corpuses.sh | 2 +-
cpp/src/arrow/compute/kernels/aggregate_test.cc | 21 +--
cpp/src/arrow/csv/CMakeLists.txt | 6 +
cpp/src/arrow/csv/fuzz.cc | 8 +-
cpp/src/arrow/csv/generate_fuzz_corpus.cc | 204 ++++++++++++++++++++++++
cpp/src/arrow/ipc/read_write_test.cc | 18 +--
cpp/src/arrow/testing/random.cc | 20 +--
cpp/src/arrow/util/rle_encoding_internal.h | 4 +
cpp/src/arrow/util/rle_encoding_test.cc | 11 +-
9 files changed, 259 insertions(+), 35 deletions(-)
diff --git a/cpp/build-support/fuzzing/generate_corpuses.sh
b/cpp/build-support/fuzzing/generate_corpuses.sh
index 233c9be0eb..6ebc86ffad 100755
--- a/cpp/build-support/fuzzing/generate_corpuses.sh
+++ b/cpp/build-support/fuzzing/generate_corpuses.sh
@@ -71,7 +71,7 @@ rm -rf ${PANDAS_DIR}
git clone --depth=1 https://github.com/pandas-dev/pandas ${PANDAS_DIR}
rm -rf ${CORPUS_DIR}
-mkdir -p ${CORPUS_DIR}
+${OUT}/arrow-csv-generate-fuzz-corpus ${CORPUS_DIR}
# Add examples from arrow-testing repo
cp ${ARROW_ROOT}/testing/data/csv/*.csv ${CORPUS_DIR}
# Add examples from Pandas test suite
diff --git a/cpp/src/arrow/compute/kernels/aggregate_test.cc
b/cpp/src/arrow/compute/kernels/aggregate_test.cc
index 171aa17cc8..a2bf0b97fd 100644
--- a/cpp/src/arrow/compute/kernels/aggregate_test.cc
+++ b/cpp/src/arrow/compute/kernels/aggregate_test.cc
@@ -3275,15 +3275,8 @@ void CheckVarStd(const Datum& array, const
VarianceOptions& options,
auto var = checked_cast<const DoubleScalar*>(out_var.scalar().get());
auto std = checked_cast<const DoubleScalar*>(out_std.scalar().get());
ASSERT_TRUE(var->is_valid && std->is_valid);
- // Near zero these macros don't work as well
- // (and MinGW can give results slightly off from zero)
- if (std::abs(expected_var) < 1e-20) {
- ASSERT_NEAR(std->value * std->value, var->value, 1e-20);
- ASSERT_NEAR(var->value, expected_var, 1e-20);
- } else {
- ASSERT_DOUBLE_EQ(std->value * std->value, var->value);
- ASSERT_DOUBLE_EQ(var->value, expected_var); // < 4ULP
- }
+ AssertWithinUlp(std->value * std->value, var->value, /*n_ulps=*/2);
+ AssertWithinUlp(var->value, expected_var, /*n_ulps=*/5);
}
template <typename ArrowType>
@@ -4159,6 +4152,14 @@ class TestRandomQuantileKernel : public
TestPrimitiveQuantileKernel<ArrowType> {
void VerifyTDigest(const std::shared_ptr<ChunkedArray>& chunked,
std::vector<double>& quantiles) {
+ // For some reason, TDigest computations with libc++ seem much less
accurate.
+ // A possible explanation is that libc++ has less precise implementations
+ // of std::sin and std::asin, used in the TDigest implementation.
+# ifdef _LIBCPP_VERSION
+ constexpr double kRelativeTolerance = 0.09;
+# else
+ constexpr double kRelativeTolerance = 0.05;
+# endif
TDigestOptions options(quantiles);
ASSERT_OK_AND_ASSIGN(Datum out, TDigest(chunked, options));
const auto& out_array = out.make_array();
@@ -4173,7 +4174,7 @@ class TestRandomQuantileKernel : public
TestPrimitiveQuantileKernel<ArrowType> {
const double* approx = out_array->data()->GetValues<double>(1);
for (size_t i = 0; i < quantiles.size(); ++i) {
const auto& exact_scalar =
checked_pointer_cast<DoubleScalar>(exact[i][0].scalar());
- const double tolerance = std::fabs(exact_scalar->value) * 0.05;
+ const double tolerance = std::fabs(exact_scalar->value) *
kRelativeTolerance;
EXPECT_NEAR(approx[i], exact_scalar->value, tolerance) << quantiles[i];
}
}
diff --git a/cpp/src/arrow/csv/CMakeLists.txt b/cpp/src/arrow/csv/CMakeLists.txt
index 55047ca204..524a780b47 100644
--- a/cpp/src/arrow/csv/CMakeLists.txt
+++ b/cpp/src/arrow/csv/CMakeLists.txt
@@ -30,6 +30,12 @@ add_arrow_benchmark(converter_benchmark PREFIX "arrow-csv")
add_arrow_benchmark(parser_benchmark PREFIX "arrow-csv")
add_arrow_benchmark(writer_benchmark PREFIX "arrow-csv")
+if(ARROW_BUILD_FUZZING_UTILITIES)
+ add_executable(arrow-csv-generate-fuzz-corpus generate_fuzz_corpus.cc)
+ target_link_libraries(arrow-csv-generate-fuzz-corpus ${ARROW_UTIL_LIB}
+ ${ARROW_TEST_LINK_LIBS})
+endif()
+
add_arrow_fuzz_target(fuzz PREFIX "arrow-csv")
arrow_install_all_headers("arrow/csv")
diff --git a/cpp/src/arrow/csv/fuzz.cc b/cpp/src/arrow/csv/fuzz.cc
index e745c2c0bd..721673ae0d 100644
--- a/cpp/src/arrow/csv/fuzz.cc
+++ b/cpp/src/arrow/csv/fuzz.cc
@@ -41,11 +41,15 @@ Status FuzzCsvReader(const uint8_t* data, int64_t size) {
auto io_context = arrow::io::default_io_context();
auto read_options = ReadOptions::Defaults();
- // Make chunking more likely
- read_options.block_size = 4096;
+ // Make chunking more likely to exercise chunked reading and optional
parallelization.
+ // Most files in the seed corpus are currently in the 4-10 kB range.
+ read_options.block_size = 1000;
auto parse_options = ParseOptions::Defaults();
auto convert_options = ConvertOptions::Defaults();
convert_options.auto_dict_encode = true;
+ // This is the default value, but we might want to turn this knob to have a
better
+ // mix of dict-encoded and non-dict-encoded columns when reading.
+ convert_options.auto_dict_max_cardinality = 50;
auto input_stream =
std::make_shared<::arrow::io::BufferReader>(std::make_shared<Buffer>(data,
size));
diff --git a/cpp/src/arrow/csv/generate_fuzz_corpus.cc
b/cpp/src/arrow/csv/generate_fuzz_corpus.cc
new file mode 100644
index 0000000000..eeba49e87f
--- /dev/null
+++ b/cpp/src/arrow/csv/generate_fuzz_corpus.cc
@@ -0,0 +1,204 @@
+// 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.
+
+// A command line executable that generates a bunch of valid IPC files
+// containing example record batches. Those are used as fuzzing seeds
+// to make fuzzing more efficient.
+
+#include <cstdlib>
+#include <functional>
+#include <iostream>
+#include <memory>
+#include <string>
+#include <vector>
+
+#include "arrow/array.h"
+#include "arrow/array/util.h"
+#include "arrow/compute/cast.h"
+#include "arrow/csv/options.h"
+#include "arrow/csv/writer.h"
+#include "arrow/io/file.h"
+#include "arrow/io/memory.h"
+#include "arrow/ipc/writer.h"
+#include "arrow/json/from_string.h"
+#include "arrow/record_batch.h"
+#include "arrow/result.h"
+#include "arrow/testing/random.h"
+#include "arrow/util/io_util.h"
+
+namespace arrow::csv {
+
+using ::arrow::internal::CreateDir;
+using ::arrow::internal::PlatformFilename;
+using ::arrow::json::ArrayFromJSONString;
+
+Result<std::shared_ptr<Buffer>> WriteRecordBatch(
+ const std::shared_ptr<RecordBatch>& batch, const WriteOptions& options) {
+ ARROW_ASSIGN_OR_RAISE(auto sink, io::BufferOutputStream::Create(1024));
+ ARROW_ASSIGN_OR_RAISE(auto writer, MakeCSVWriter(sink.get(),
batch->schema(), options));
+ RETURN_NOT_OK(writer->WriteRecordBatch(*batch));
+ RETURN_NOT_OK(writer->Close());
+ return sink->Finish();
+}
+
+Result<std::shared_ptr<RecordBatch>> MakeBatch(
+ std::function<Result<std::shared_ptr<Array>>(int64_t length, double
null_probability)>
+ array_factory,
+ int64_t length) {
+ ArrayVector columns;
+ FieldVector fields;
+
+ struct ColumnSpec {
+ std::string name;
+ double null_probability;
+ };
+ for (auto spec : {ColumnSpec{"with_nulls", 0.2}, ColumnSpec{"without_nulls",
0.0}}) {
+ ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Array> column,
+ array_factory(length, spec.null_probability));
+ columns.push_back(column);
+ fields.push_back(field(spec.name, column->type()));
+ }
+ return RecordBatch::Make(schema(std::move(fields)), length,
std::move(columns));
+}
+
+Result<RecordBatchVector> Batches() {
+ ::arrow::random::RandomArrayGenerator gen(/*seed=*/42);
+ RecordBatchVector batches;
+
+ auto append_batch = [&](auto array_factory, int64_t length) -> Status {
+ ARROW_ASSIGN_OR_RAISE(auto batch, MakeBatch(array_factory, length));
+ batches.push_back(batch);
+ return Status::OK();
+ };
+
+ // Ideally, we should exercise all possible inference kinds (see
inference_internal.h)
+ auto make_nulls = [&](int64_t length, double null_probability) {
+ return MakeArrayOfNull(null(), length);
+ };
+ auto make_ints = [&](int64_t length, double null_probability) {
+ return gen.Int64(length, /*min=*/-1'000'000, /*max=*/1'000'000,
null_probability);
+ };
+ auto make_floats = [&](int64_t length, double null_probability) {
+ return gen.Float64(length, /*min=*/-100.0, /*max=*/100.0,
null_probability);
+ };
+ auto make_booleans = [&](int64_t length, double null_probability) {
+ return gen.Boolean(length, /*true_probability=*/0.8, null_probability);
+ };
+ auto make_dates = [&](int64_t length, double null_probability) {
+ return gen.Date64(length, /*min=*/1, /*max=*/365 * 60, null_probability);
+ };
+ auto make_times = [&](int64_t length, double null_probability) {
+ return gen.Int32(length, /*min=*/0, /*max=*/86399, null_probability)
+ ->View(time32(TimeUnit::SECOND));
+ };
+
+ std::string timezone;
+ auto make_timestamps = [&](int64_t length, double null_probability) {
+ return gen.Int64(length, /*min=*/1, /*max=*/1764079190, null_probability)
+ ->View(timestamp(TimeUnit::SECOND, timezone));
+ };
+ auto make_timestamps_ns = [&](int64_t length, double null_probability) {
+ return gen
+ .Int64(length, /*min=*/1, /*max=*/1764079190LL * 1'000'000'000,
null_probability)
+ ->View(timestamp(TimeUnit::NANO, timezone));
+ };
+
+ auto make_strings = [&](int64_t length, double null_probability) {
+ return gen.String(length, /*min_length=*/3, /*max_length=*/15,
null_probability);
+ };
+ auto make_string_with_repeats = [&](int64_t length, double null_probability)
{
+ // `unique` should be less than `auto_dict_max_cardinality` in fuzz target
+ return gen.StringWithRepeats(length, /*unique=*/10, /*min_length=*/3,
+ /*max_length=*/15, null_probability);
+ };
+
+ RETURN_NOT_OK(append_batch(make_nulls, /*length=*/2000));
+ RETURN_NOT_OK(append_batch(make_ints, /*length=*/500));
+ RETURN_NOT_OK(append_batch(make_floats, /*length=*/150));
+ RETURN_NOT_OK(append_batch(make_booleans, /*length=*/500));
+
+ RETURN_NOT_OK(append_batch(make_dates, /*length=*/200));
+ RETURN_NOT_OK(append_batch(make_times, /*length=*/400));
+ timezone = "";
+ RETURN_NOT_OK(append_batch(make_timestamps, /*length=*/200));
+ RETURN_NOT_OK(append_batch(make_timestamps_ns, /*length=*/100));
+ // Will generate timestamps with a "Z" suffix
+ timezone = "UTC";
+ RETURN_NOT_OK(append_batch(make_timestamps, /*length=*/200));
+ RETURN_NOT_OK(append_batch(make_timestamps_ns, /*length=*/100));
+ // Will generate timestamps with a "+0100" or "+0200" suffix
+ timezone = "Europe/Paris";
+ RETURN_NOT_OK(append_batch(make_timestamps, /*length=*/200));
+ RETURN_NOT_OK(append_batch(make_timestamps_ns, /*length=*/100));
+
+ RETURN_NOT_OK(append_batch(make_strings, /*length=*/300));
+ RETURN_NOT_OK(append_batch(make_string_with_repeats, /*length=*/300));
+ // XXX Cannot add non-UTF8 binary as the CSV writer doesn't support writing
it
+
+ return batches;
+}
+
+Status DoMain(const std::string& out_dir) {
+ ARROW_ASSIGN_OR_RAISE(auto dir_fn, PlatformFilename::FromString(out_dir));
+ RETURN_NOT_OK(CreateDir(dir_fn));
+
+ int sample_num = 1;
+ auto sample_name = [&]() -> std::string {
+ return "csv-file-" + std::to_string(sample_num++);
+ };
+
+ ARROW_ASSIGN_OR_RAISE(auto batches, Batches());
+
+ auto options = WriteOptions::Defaults();
+ RETURN_NOT_OK(options.Validate());
+
+ for (const auto& batch : batches) {
+ RETURN_NOT_OK(batch->ValidateFull());
+ ARROW_ASSIGN_OR_RAISE(auto buffer, WriteRecordBatch(batch, options));
+
+ ARROW_ASSIGN_OR_RAISE(auto sample_fn, dir_fn.Join(sample_name()));
+ std::cerr << sample_fn.ToString() << std::endl;
+ ARROW_ASSIGN_OR_RAISE(auto file,
io::FileOutputStream::Open(sample_fn.ToString()));
+ RETURN_NOT_OK(file->Write(buffer));
+ RETURN_NOT_OK(file->Close());
+ }
+ return Status::OK();
+}
+
+ARROW_NORETURN void Usage() {
+ std::cerr << "Usage: arrow-csv-generate-fuzz-corpus "
+ << "<output directory>" << std::endl;
+ std::exit(2);
+}
+
+int Main(int argc, char** argv) {
+ if (argc != 2) {
+ Usage();
+ }
+ auto out_dir = std::string(argv[1]);
+
+ Status st = DoMain(out_dir);
+ if (!st.ok()) {
+ std::cerr << st.ToString() << std::endl;
+ return 1;
+ }
+ return 0;
+}
+
+} // namespace arrow::csv
+
+int main(int argc, char** argv) { return arrow::csv::Main(argc, argv); }
diff --git a/cpp/src/arrow/ipc/read_write_test.cc
b/cpp/src/arrow/ipc/read_write_test.cc
index 84ec923ce8..315d8bd07d 100644
--- a/cpp/src/arrow/ipc/read_write_test.cc
+++ b/cpp/src/arrow/ipc/read_write_test.cc
@@ -952,23 +952,23 @@ TEST_F(TestWriteRecordBatch, SliceTruncatesBuffers) {
}
TEST_F(TestWriteRecordBatch, RoundtripPreservesBufferSizes) {
- // ARROW-7975
+ // ARROW-7975: deserialized buffers should have logically exact size (no
padding)
random::RandomArrayGenerator rg(/*seed=*/0);
+ constexpr int64_t kLength = 30;
- int64_t length = 15;
- auto arr = rg.String(length, 0, 10, 0.1);
- auto batch = RecordBatch::Make(::arrow::schema({field("f0", utf8())}),
length, {arr});
+ auto arr =
+ rg.String(kLength, /*min_length=*/0, /*max_length=*/10,
/*null_probability=*/0.3);
+ ASSERT_NE(arr->null_count(), 0); // required for validity bitmap size
assertion below
+
+ auto batch = RecordBatch::Make(::arrow::schema({field("f0", utf8())}),
kLength, {arr});
- ASSERT_OK_AND_ASSIGN(
- mmap_, io::MemoryMapFixture::InitMemoryMap(
- /*buffer_size=*/1 << 20,
TempFile("test-roundtrip-buffer-sizes")));
DictionaryMemo dictionary_memo;
ASSERT_OK_AND_ASSIGN(
auto result,
DoStandardRoundTrip(*batch, IpcWriteOptions::Defaults(),
&dictionary_memo));
- // Make sure that the validity bitmap is size 2 as expected
- ASSERT_EQ(2, arr->data()->buffers[0]->size());
+ // Make sure that the validity bitmap has expected size
+ ASSERT_EQ(bit_util::BytesForBits(kLength), arr->data()->buffers[0]->size());
for (size_t i = 0; i < arr->data()->buffers.size(); ++i) {
ASSERT_EQ(arr->data()->buffers[i]->size(),
diff --git a/cpp/src/arrow/testing/random.cc b/cpp/src/arrow/testing/random.cc
index 5f95638b7d..c50387e490 100644
--- a/cpp/src/arrow/testing/random.cc
+++ b/cpp/src/arrow/testing/random.cc
@@ -65,7 +65,7 @@ template <typename ValueType, typename DistributionType>
struct GeneratorFactory {
GeneratorFactory(ValueType min, ValueType max) : min_(min), max_(max) {}
- auto operator()(pcg32_fast* rng) const {
+ auto operator()(pcg32* rng) const {
return [dist = DistributionType(min_, max_), rng]() mutable {
return static_cast<ValueType>(dist(*rng));
};
@@ -80,7 +80,7 @@ template <typename DistributionType>
struct GeneratorFactory<Float16, DistributionType> {
GeneratorFactory(Float16 min, Float16 max) : min_(min.ToFloat()),
max_(max.ToFloat()) {}
- auto operator()(pcg32_fast* rng) const {
+ auto operator()(pcg32* rng) const {
return [dist = DistributionType(min_, max_), rng]() mutable {
return Float16(dist(*rng)).bits();
};
@@ -121,7 +121,7 @@ struct GenerateOptions {
GenerateTypedDataNoNan(data, n);
return;
}
- pcg32_fast rng(seed_++);
+ pcg32 rng(seed_++);
auto gen = generator_factory_(&rng);
::arrow::random::bernoulli_distribution nan_dist(nan_probability_);
const PhysicalType nan_value = get_nan();
@@ -130,7 +130,7 @@ struct GenerateOptions {
}
void GenerateTypedDataNoNan(PhysicalType* data, size_t n) {
- pcg32_fast rng(seed_++);
+ pcg32 rng(seed_++);
auto gen = generator_factory_(&rng);
std::generate(data, data + n, [&] { return gen(); });
@@ -138,7 +138,7 @@ struct GenerateOptions {
void GenerateBitmap(uint8_t* buffer, size_t n, int64_t* null_count) {
int64_t count = 0;
- pcg32_fast rng(seed_++);
+ pcg32 rng(seed_++);
::arrow::random::bernoulli_distribution dist(1.0 - probability_);
for (size_t i = 0; i < n; i++) {
@@ -749,7 +749,7 @@ void ShuffleListViewDataInPlace(SeedType seed, ArrayData*
data) {
auto* offsets = data->GetMutableValues<offset_type>(1);
auto* sizes = data->GetMutableValues<offset_type>(2);
- pcg32_fast rng(seed);
+ pcg32 rng(seed);
using UniformDist = std::uniform_int_distribution<int64_t>;
UniformDist dist;
for (int64_t i = data->length - 1; i > 0; --i) {
@@ -888,7 +888,7 @@ Result<std::shared_ptr<Array>> RandomListView(RAG& self,
const Array& values,
auto sizes = buffers[1]->mutable_data_as<offset_type>();
// Derive sizes from offsets taking coverage into account
- pcg32_fast rng(self.seed());
+ pcg32 rng(self.seed());
using NormalDist = std::normal_distribution<double>;
NormalDist size_dist;
for (int64_t i = 0; i < length; ++i) {
@@ -977,7 +977,7 @@ std::shared_ptr<Array> RandomArrayGenerator::Map(const
std::shared_ptr<Array>& k
std::shared_ptr<Array> RandomArrayGenerator::RunEndEncoded(
std::shared_ptr<DataType> value_type, int64_t logical_size, double
null_probability) {
Int32Builder run_ends_builder;
- pcg32_fast rng(seed());
+ pcg32 rng(seed());
DCHECK_LE(logical_size, std::numeric_limits<int32_t>::max());
@@ -1447,7 +1447,7 @@ std::shared_ptr<arrow::RecordBatch> GenerateBatch(const
FieldVector& fields,
void rand_day_millis(int64_t N,
std::vector<DayTimeIntervalType::DayMilliseconds>* out) {
const int random_seed = 0;
- arrow::random::pcg32_fast gen(random_seed);
+ arrow::random::pcg32 gen(random_seed);
std::uniform_int_distribution<int32_t> d(std::numeric_limits<int32_t>::min(),
std::numeric_limits<int32_t>::max());
out->resize(N, {});
@@ -1462,7 +1462,7 @@ void rand_day_millis(int64_t N,
std::vector<DayTimeIntervalType::DayMilliseconds
void rand_month_day_nanos(int64_t N,
std::vector<MonthDayNanoIntervalType::MonthDayNanos>* out) {
const int random_seed = 0;
- arrow::random::pcg32_fast gen(random_seed);
+ arrow::random::pcg32 gen(random_seed);
std::uniform_int_distribution<int64_t> d(std::numeric_limits<int64_t>::min(),
std::numeric_limits<int64_t>::max());
out->resize(N, {});
diff --git a/cpp/src/arrow/util/rle_encoding_internal.h
b/cpp/src/arrow/util/rle_encoding_internal.h
index 46991fc717..2c084e0b4c 100644
--- a/cpp/src/arrow/util/rle_encoding_internal.h
+++ b/cpp/src/arrow/util/rle_encoding_internal.h
@@ -532,6 +532,10 @@ class RleBitPackedEncoder {
}
/// Returns the maximum byte size it could take to encode 'num_values'.
+ ///
+ /// Note: because of the way CheckBufferFull() is called, you have to
+ /// reserve an extra "RleEncoder::MinBufferSize" bytes. These extra bytes
+ /// won't be used but not reserving them can cause the encoder to fail.
static int64_t MaxBufferSize(int bit_width, int64_t num_values) {
// For a bit_width > 1, the worst case is the repetition of "literal run
of length 8
// and then a repeated run of length 8".
diff --git a/cpp/src/arrow/util/rle_encoding_test.cc
b/cpp/src/arrow/util/rle_encoding_test.cc
index c709095a6c..b2d4f7df6f 100644
--- a/cpp/src/arrow/util/rle_encoding_test.cc
+++ b/cpp/src/arrow/util/rle_encoding_test.cc
@@ -993,11 +993,15 @@ void CheckRoundTrip(const Array& data, int bit_width,
bool spaced, int32_t parts
const int data_size = static_cast<int>(data.length());
const int data_values_count =
static_cast<int>(data.length() - spaced * data.null_count());
- const int buffer_size =
- static_cast<int>(RleBitPackedEncoder::MaxBufferSize(bit_width,
data_size));
+ const int buffer_size = static_cast<int>(
+ ::arrow::util::RleBitPackedEncoder::MaxBufferSize(bit_width,
data_values_count) +
+ ::arrow::util::RleBitPackedEncoder::MinBufferSize(bit_width));
+
ASSERT_GE(parts, 1);
ASSERT_LE(parts, data_size);
+ ARROW_SCOPED_TRACE("bit_width = ", bit_width, ", spaced = ", spaced,
+ ", data_size = ", data_size, ", buffer_size = ",
buffer_size);
const value_type* data_values = static_cast<const
ArrayType&>(data).raw_values();
// Encode the data into `buffer` using the encoder.
@@ -1008,7 +1012,8 @@ void CheckRoundTrip(const Array& data, int bit_width,
bool spaced, int32_t parts
// Depending on `spaced` we treat nulls as regular values.
if (data.IsValid(i) || !spaced) {
bool success = encoder.Put(static_cast<uint64_t>(data_values[i]));
- ASSERT_TRUE(success) << "Encoding failed in pos " << i;
+ ASSERT_TRUE(success) << "Encoding failed in pos " << i
+ << ", current encoder len: " << encoder.len();
++encoded_values_size;
}
}