bkietz commented on a change in pull request #9715:
URL: https://github.com/apache/arrow/pull/9715#discussion_r596029757



##########
File path: cpp/src/arrow/testing/random.h
##########
@@ -358,6 +362,10 @@ class ARROW_TESTING_EXPORT RandomArrayGenerator {
   std::default_random_engine seed_rng_;
 };
 
+ARROW_TESTING_EXPORT
+std::shared_ptr<arrow::RecordBatch> Generate(const FieldVector& fields, 
int64_t size,
+                                             SeedType seed);

Review comment:
       What do you think about replacing explicit seed arguments (which are 
usually repeated within a suite) with
   ```c++
   /// The seed used for random generation. Non const to support customizing 
the seed for a suite
   static SeedType kSeed = 0xDEADBEEF;
   ```
   
   or
   
   ```c++
   /// Sets the seed used for random generation while it is in scope.
   /// May be constructed at static scope to set the seed for an entire
   /// test/benchmark suite. If no SeedGuard is in scope, the seed will
   /// be kDefaultSeed
   struct SeedGuard {
     explicit SeedGuard(SeedType seed) { PushSeed(seed); }
     ~SeedGuard() { PopSeed(); }
   
     static constexpr SeedType kDefaultSeed = 0xDEADBEEF;
   
     static void PopSeed();
     static void PushSeed(SeedType);
   };
   ```
   ?

##########
File path: cpp/src/arrow/testing/random.cc
##########
@@ -369,12 +371,16 @@ std::shared_ptr<Array> 
RandomArrayGenerator::FixedSizeBinary(int64_t size,
                                                 std::move(null_bitmap), 
null_count);
 }
 
-std::shared_ptr<Array> RandomArrayGenerator::Offsets(int64_t size, int32_t 
first_offset,
-                                                     int32_t last_offset,
-                                                     double null_probability,
-                                                     bool force_empty_nulls) {
-  using GenOpt = GenerateOptions<int32_t, 
std::uniform_int_distribution<int32_t>>;
-  GenOpt options(seed(), first_offset, last_offset, null_probability);
+namespace {
+template <typename ArrayType>

Review comment:
       When I read this first, I was looking for `typename 
ArrayType::TypeClass::offset_type` because I assumed this parameter referred to 
`StringArray` or so
   ```suggestion
   template <typename OffsetArrayType>
   ```

##########
File path: cpp/src/arrow/testing/random_test.cc
##########
@@ -0,0 +1,195 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <gtest/gtest.h>
+
+#include "arrow/array.h"
+#include "arrow/record_batch.h"
+#include "arrow/testing/gtest_util.h"
+#include "arrow/testing/random.h"
+#include "arrow/type.h"
+#include "arrow/util/key_value_metadata.h"
+
+namespace arrow {
+namespace random {
+
+class RandomArrayTest : public 
::testing::TestWithParam<std::shared_ptr<Field>> {
+ protected:
+  std::shared_ptr<Field> GetField() { return GetParam(); }
+};
+
+template <typename T>
+class RandomNumericArrayTest : public ::testing::Test {
+ protected:
+  std::shared_ptr<Field> GetField() { return field("field0", 
std::make_shared<T>()); }
+
+  std::shared_ptr<NumericArray<T>> Downcast(std::shared_ptr<Array> array) {
+    return internal::checked_pointer_cast<NumericArray<T>>(array);
+  }
+};
+
+TEST_P(RandomArrayTest, GenerateArray) {
+  auto field = GetField();
+  auto batch = Generate({field}, 128, 0xDEADBEEF);
+  AssertSchemaEqual(schema({field}), batch->schema());
+  auto array = batch->column(0);
+  ASSERT_EQ(128, array->length());
+  ASSERT_OK(array->ValidateFull());
+}
+
+TEST_P(RandomArrayTest, GenerateNonNullArray) {
+  auto field =
+      GetField()->WithMetadata(key_value_metadata({{"null_probability", 
"0.0"}}));
+  if (field->type()->id() == Type::type::NA) {
+    GTEST_SKIP() << "Cannot generate non-null null arrays";
+  }
+  auto batch = Generate({field}, 128, 0xDEADBEEF);
+  AssertSchemaEqual(schema({field}), batch->schema());
+  auto array = batch->column(0);
+  ASSERT_OK(array->ValidateFull());
+  ASSERT_EQ(0, array->null_count());
+}
+
+TEST_P(RandomArrayTest, GenerateNonNullableArray) {
+  auto field = GetField()->WithNullable(false);
+  if (field->type()->id() == Type::type::NA) {
+    GTEST_SKIP() << "Cannot generate non-null null arrays";
+  }
+  auto batch = Generate({field}, 128, 0xDEADBEEF);
+  AssertSchemaEqual(schema({field}), batch->schema());
+  auto array = batch->column(0);
+  ASSERT_OK(array->ValidateFull());
+  ASSERT_EQ(0, array->null_count());
+}
+
+struct FieldParamName {
+  template <class ParamType>
+  std::string operator()(const ::testing::TestParamInfo<ParamType>& info) 
const {
+    return std::to_string(info.index) + info.param->name();
+  }
+};
+
+auto values = ::testing::Values(
+    field("null", null()), field("bool", boolean()), field("uint8", uint8()),
+    field("int8", int8()), field("uint16", uint16()), field("int16", int16()),
+    field("uint32", uint32()), field("int32", int32()), field("uint64", 
uint64()),
+    field("int64", int64()), field("float16", float16()), field("float32", 
float32()),
+    field("float64", float64()), field("string", utf8()), field("binary", 
binary()),
+    field("fixed_size_binary", fixed_size_binary(8)),
+    field("decimal128", decimal128(8, 3)), field("decimal256", decimal256(16, 
4)),
+    field("date32", date32()), field("date64", date64()),
+    field("timestampns", timestamp(TimeUnit::NANO)),
+    field("timestamps", timestamp(TimeUnit::SECOND, "America/Phoenix")),
+    field("time32ms", time32(TimeUnit::MILLI)), field("time64ns", 
time64(TimeUnit::NANO)),
+    field("time32s", time32(TimeUnit::SECOND)),
+    field("time64us", time64(TimeUnit::MICRO)), field("month_interval", 
month_interval()),
+    field("daytime_interval", day_time_interval()), field("listint8", 
list(int8())),
+    field("listlistint8", list(list(int8()))),
+    field("listint8emptynulls", list(int8()), true,
+          key_value_metadata({{"force_empty_nulls", "true"}})),
+    field("listint81024values", list(int8()), true,
+          key_value_metadata({{"values", "1024"}})),
+    field("structints", struct_({
+                            field("int8", int8()),
+                            field("int16", int16()),
+                            field("int32", int32()),
+                        })),
+    field("structnested", struct_({
+                              field("string", utf8()),
+                              field("list", list(int64())),
+                              field("timestamp", timestamp(TimeUnit::MILLI)),
+                          })),
+    field("sparseunion", sparse_union({
+                             field("int8", int8()),
+                             field("int16", int16()),
+                             field("int32", int32()),
+                         })),
+    field("denseunion", dense_union({
+                            field("int8", int8()),
+                            field("int16", int16()),
+                            field("int32", int32()),
+                        })),
+    field("dictionary", dictionary(int8(), utf8())), field("map", map(int8(), 
utf8())),
+    field("fixedsizelist", fixed_size_list(int8(), 4)),
+    field("durationns", duration(TimeUnit::NANO)), field("largestring", 
large_utf8()),
+    field("largebinary", large_binary()),
+    field("largelistlistint8", large_list(list(int8()))));
+
+INSTANTIATE_TEST_SUITE_P(
+    TestRandomArrayGeneration, RandomArrayTest, values,
+    [](const ::testing::TestParamInfo<RandomArrayTest::ParamType>& info) {
+      return std::to_string(info.index) + info.param->name();
+    });
+
+using NumericTypes =
+    ::testing::Types<UInt8Type, Int8Type, UInt16Type, Int16Type, UInt32Type, 
Int32Type,
+                     HalfFloatType, FloatType, DoubleType>;
+TYPED_TEST_SUITE(RandomNumericArrayTest, NumericTypes);
+
+TYPED_TEST(RandomNumericArrayTest, GenerateMinMax) {
+  auto field = this->GetField()->WithMetadata(
+      key_value_metadata({{"min", "0"}, {"max", "127"}, {"nan_probability", 
"0.0"}}));
+  auto batch = Generate({field}, 128, 0xDEADBEEF);
+  AssertSchemaEqual(schema({field}), batch->schema());
+  auto array = this->Downcast(batch->column(0));
+  auto it = array->begin();
+  while (it != array->end()) {
+    if ((*it).has_value()) {
+      ASSERT_GE(**it, typename TypeParam::c_type(0));
+      ASSERT_LE(**it, typename TypeParam::c_type(127));
+    }
+    it++;
+  }
+}
+
+TEST(TypeSpecificTests, FloatNan) {
+  auto field = arrow::field("float32", float32())
+                   ->WithMetadata(key_value_metadata({{"nan_probability", 
"1.0"}}));
+  auto batch = Generate({field}, 128, 0xDEADBEEF);
+  AssertSchemaEqual(schema({field}), batch->schema());
+  auto array = 
internal::checked_pointer_cast<NumericArray<FloatType>>(batch->column(0));
+  auto it = array->begin();
+  while (it != array->end()) {
+    if ((*it).has_value()) {
+      ASSERT_TRUE(std::isnan(**it));
+    }
+    it++;
+  }
+}
+
+TEST(TypeSpecificTests, RepeatedStrings) {
+  auto field =
+      arrow::field("string", 
utf8())->WithMetadata(key_value_metadata({{"unique", "1"}}));
+  auto batch = Generate({field}, 128, 0xDEADBEEF);
+  AssertSchemaEqual(schema({field}), batch->schema());
+  auto array = internal::checked_pointer_cast<StringArray>(batch->column(0));
+  auto it = array->begin();
+  util::optional<util::string_view> singular_value;
+  while (it != array->end()) {
+    if ((*it).has_value()) {
+      if (!singular_value.has_value()) {
+        singular_value = *it;
+      } else {
+        ASSERT_EQ(*singular_value, **it);
+      }
+    }
+    it++;
+  }

Review comment:
       ```suggestion
     util::string_view singular_value = array->GetView(0);
     for (auto slot : *array) {
       if (!slot.has_value()) continue;
       ASSERT_EQ(slot, singular_value);
     }
   ```

##########
File path: cpp/src/arrow/testing/random.cc
##########
@@ -558,5 +584,248 @@ std::shared_ptr<Array> 
RandomArrayGenerator::ArrayOf(std::shared_ptr<DataType> t
   return RandomArrayGeneratorOfImpl{this, type, size, null_probability, 
nullptr}.Finish();
 }
 
+namespace {
+template <typename T>
+typename T::c_type GetMetadata(const KeyValueMetadata* metadata, const 
std::string& key,
+                               typename T::c_type default_value) {
+  if (!metadata) return default_value;
+  const auto index = metadata->FindKey(key);
+  if (index < 0) return default_value;
+  const auto& value = metadata->value(index);
+  typename T::c_type output{};
+  auto type = checked_pointer_cast<T>(TypeTraits<T>::type_singleton());
+  if (!internal::ParseValue(*type, value.data(), value.length(), &output)) {
+    ABORT_NOT_OK(Status::Invalid("Could not parse ", key, " = ", value));
+  }
+  return output;
+}
+
+Result<std::shared_ptr<Array>> GenerateArray(const Field& field, int64_t 
length,
+                                             RandomArrayGenerator* generator) {
+#define GENERATE_INTEGRAL_CASE_VIEW(BASE_TYPE, VIEW_TYPE)                      
          \
+  case VIEW_TYPE::type_id: {                                                   
          \
+    const BASE_TYPE::c_type min_value = GetMetadata<BASE_TYPE>(                
          \
+        field.metadata().get(), "min", 
std::numeric_limits<BASE_TYPE::c_type>::min());   \
+    const BASE_TYPE::c_type max_value = GetMetadata<BASE_TYPE>(                
          \
+        field.metadata().get(), "max", 
std::numeric_limits<BASE_TYPE::c_type>::max());   \
+    return generator->Numeric<BASE_TYPE>(length, min_value, max_value, 
null_probability) \
+        ->View(field.type());                                                  
          \
+  }
+#define GENERATE_INTEGRAL_CASE(ARROW_TYPE) \
+  GENERATE_INTEGRAL_CASE_VIEW(ARROW_TYPE, ARROW_TYPE)
+#define GENERATE_FLOATING_CASE(ARROW_TYPE, GENERATOR_FUNC)                     
         \
+  case ARROW_TYPE::type_id: {                                                  
         \
+    const ARROW_TYPE::c_type min_value = GetMetadata<ARROW_TYPE>(              
         \
+        field.metadata().get(), "min", 
std::numeric_limits<ARROW_TYPE::c_type>::min()); \
+    const ARROW_TYPE::c_type max_value = GetMetadata<ARROW_TYPE>(              
         \
+        field.metadata().get(), "max", 
std::numeric_limits<ARROW_TYPE::c_type>::max()); \
+    const double nan_probability =                                             
         \
+        GetMetadata<DoubleType>(field.metadata().get(), "nan_probability", 0); 
         \
+    return generator->GENERATOR_FUNC(length, min_value, max_value, 
null_probability,    \
+                                     nan_probability);                         
         \
+  }
+
+  const double null_probability =
+      field.nullable()
+          ? GetMetadata<DoubleType>(field.metadata().get(), 
"null_probability", 0.01)
+          : 0.0;
+  switch (field.type()->id()) {
+    case Type::type::NA:
+      return std::make_shared<NullArray>(length);
+
+    case Type::type::BOOL: {
+      const double true_probability =
+          GetMetadata<DoubleType>(field.metadata().get(), "true_probability", 
0.5);
+      return generator->Boolean(length, true_probability, null_probability);
+    }
+
+      GENERATE_INTEGRAL_CASE(UInt8Type);
+      GENERATE_INTEGRAL_CASE(Int8Type);
+      GENERATE_INTEGRAL_CASE(UInt16Type);
+      GENERATE_INTEGRAL_CASE(Int16Type);
+      GENERATE_INTEGRAL_CASE(UInt32Type);
+      GENERATE_INTEGRAL_CASE(Int32Type);
+      GENERATE_INTEGRAL_CASE(UInt64Type);
+      GENERATE_INTEGRAL_CASE(Int64Type);
+      GENERATE_INTEGRAL_CASE_VIEW(Int16Type, HalfFloatType);
+      GENERATE_FLOATING_CASE(FloatType, Float32);
+      GENERATE_FLOATING_CASE(DoubleType, Float64);
+
+    case Type::type::STRING:
+    case Type::type::BINARY: {
+      const int32_t min_length = 
GetMetadata<Int32Type>(field.metadata().get(), "min", 0);
+      const int32_t max_length =
+          GetMetadata<Int32Type>(field.metadata().get(), "max", 1024);
+      const int32_t unique_values =
+          GetMetadata<Int32Type>(field.metadata().get(), "unique", -1);
+      if (unique_values > 0) {
+        return generator
+            ->StringWithRepeats(length, unique_values, min_length, max_length,
+                                null_probability)
+            ->View(field.type());
+      }
+      return generator->String(length, min_length, max_length, 
null_probability)
+          ->View(field.type());
+    }
+
+    case Type::type::DECIMAL128:
+    case Type::type::DECIMAL256:
+    case Type::type::FIXED_SIZE_BINARY: {
+      auto byte_width =
+          
internal::checked_pointer_cast<FixedSizeBinaryType>(field.type())->byte_width();
+      return generator->FixedSizeBinary(length, byte_width, null_probability)
+          ->View(field.type());
+    }
+
+      GENERATE_INTEGRAL_CASE_VIEW(Int32Type, Date32Type);
+      GENERATE_INTEGRAL_CASE_VIEW(Int64Type, Date64Type);
+      GENERATE_INTEGRAL_CASE_VIEW(Int64Type, TimestampType);
+      GENERATE_INTEGRAL_CASE_VIEW(Int32Type, Time32Type);
+      GENERATE_INTEGRAL_CASE_VIEW(Int64Type, Time64Type);
+      GENERATE_INTEGRAL_CASE_VIEW(Int32Type, MonthIntervalType);
+
+      // This isn't as flexible as it could be, but the array-of-structs 
layout of this
+      // type means it's not a (useful) composition of other generators
+      GENERATE_INTEGRAL_CASE_VIEW(Int64Type, DayTimeIntervalType);
+
+    case Type::type::LIST: {
+      const int32_t values_length = GetMetadata<Int32Type>(
+          field.metadata().get(), "values", static_cast<int32_t>(length));

Review comment:
       It's slightly odd that list's options aren't more similar to string's: 
here we specify the total length of the child whereas for strings we specify 
min/max length.

##########
File path: cpp/src/arrow/testing/random.h
##########
@@ -358,6 +362,10 @@ class ARROW_TESTING_EXPORT RandomArrayGenerator {
   std::default_random_engine seed_rng_;
 };
 
+ARROW_TESTING_EXPORT
+std::shared_ptr<arrow::RecordBatch> Generate(const FieldVector& fields, 
int64_t size,

Review comment:
       It'd probably be better to name this GenerateBatch in case we add 
similar methods later (GenerateTable(fields, ...), GenerateArray(field, ...), 
...)
   ```suggestion
   std::shared_ptr<arrow::RecordBatch> GenerateBatch(const FieldVector& fields, 
int64_t size,
   ```
   GenerateArray would probably be worth making public now.
   
   Additionally, please provide a (default null) schema-level metadata argument.

##########
File path: cpp/src/arrow/testing/random.cc
##########
@@ -558,5 +584,248 @@ std::shared_ptr<Array> 
RandomArrayGenerator::ArrayOf(std::shared_ptr<DataType> t
   return RandomArrayGeneratorOfImpl{this, type, size, null_probability, 
nullptr}.Finish();
 }
 
+namespace {
+template <typename T>
+typename T::c_type GetMetadata(const KeyValueMetadata* metadata, const 
std::string& key,
+                               typename T::c_type default_value) {
+  if (!metadata) return default_value;
+  const auto index = metadata->FindKey(key);
+  if (index < 0) return default_value;
+  const auto& value = metadata->value(index);
+  typename T::c_type output{};
+  auto type = checked_pointer_cast<T>(TypeTraits<T>::type_singleton());
+  if (!internal::ParseValue(*type, value.data(), value.length(), &output)) {
+    ABORT_NOT_OK(Status::Invalid("Could not parse ", key, " = ", value));
+  }
+  return output;
+}
+
+Result<std::shared_ptr<Array>> GenerateArray(const Field& field, int64_t 
length,
+                                             RandomArrayGenerator* generator) {
+#define GENERATE_INTEGRAL_CASE_VIEW(BASE_TYPE, VIEW_TYPE)                      
          \
+  case VIEW_TYPE::type_id: {                                                   
          \
+    const BASE_TYPE::c_type min_value = GetMetadata<BASE_TYPE>(                
          \
+        field.metadata().get(), "min", 
std::numeric_limits<BASE_TYPE::c_type>::min());   \
+    const BASE_TYPE::c_type max_value = GetMetadata<BASE_TYPE>(                
          \
+        field.metadata().get(), "max", 
std::numeric_limits<BASE_TYPE::c_type>::max());   \
+    return generator->Numeric<BASE_TYPE>(length, min_value, max_value, 
null_probability) \
+        ->View(field.type());                                                  
          \
+  }
+#define GENERATE_INTEGRAL_CASE(ARROW_TYPE) \
+  GENERATE_INTEGRAL_CASE_VIEW(ARROW_TYPE, ARROW_TYPE)
+#define GENERATE_FLOATING_CASE(ARROW_TYPE, GENERATOR_FUNC)                     
         \
+  case ARROW_TYPE::type_id: {                                                  
         \
+    const ARROW_TYPE::c_type min_value = GetMetadata<ARROW_TYPE>(              
         \
+        field.metadata().get(), "min", 
std::numeric_limits<ARROW_TYPE::c_type>::min()); \
+    const ARROW_TYPE::c_type max_value = GetMetadata<ARROW_TYPE>(              
         \
+        field.metadata().get(), "max", 
std::numeric_limits<ARROW_TYPE::c_type>::max()); \
+    const double nan_probability =                                             
         \
+        GetMetadata<DoubleType>(field.metadata().get(), "nan_probability", 0); 
         \
+    return generator->GENERATOR_FUNC(length, min_value, max_value, 
null_probability,    \
+                                     nan_probability);                         
         \
+  }
+
+  const double null_probability =
+      field.nullable()
+          ? GetMetadata<DoubleType>(field.metadata().get(), 
"null_probability", 0.01)
+          : 0.0;
+  switch (field.type()->id()) {
+    case Type::type::NA:

Review comment:
       should we check that `null_probability` is nonzero? Or 
`field.nullable()`?

##########
File path: cpp/src/arrow/testing/random.cc
##########
@@ -558,5 +584,248 @@ std::shared_ptr<Array> 
RandomArrayGenerator::ArrayOf(std::shared_ptr<DataType> t
   return RandomArrayGeneratorOfImpl{this, type, size, null_probability, 
nullptr}.Finish();
 }
 
+namespace {
+template <typename T>
+typename T::c_type GetMetadata(const KeyValueMetadata* metadata, const 
std::string& key,
+                               typename T::c_type default_value) {

Review comment:
       This doesn't need an explicit template argument; it can infer from 
`default_value`
   ```suggestion
   template <typename T, typename ArrowType = typename 
CTypeTraits<T>::ArrowType>
   enable_if_parameter_free<ArrowType, T> GetMetadata(const KeyValueMetadata* 
metadata, const std::string& key, T default_value) {
   ```
   (not necessary if you prefer to be explicit)

##########
File path: cpp/src/arrow/filesystem/s3fs_benchmark.cc
##########
@@ -146,32 +147,25 @@ class MinioFixture : public benchmark::Fixture {
   /// Appends integer columns to the beginning (to act as indices).
   Status MakeParquetObject(const std::string& path, int num_columns, int 
num_rows) {
     std::vector<std::shared_ptr<ChunkedArray>> columns;
-    std::vector<std::shared_ptr<Field>> fields;
-
-    {
-      arrow::random::RandomArrayGenerator generator(0);
-      std::shared_ptr<Array> values = generator.Int64(num_rows, 0, 1e10, 0);
-      columns.push_back(std::make_shared<ChunkedArray>(values));
-      fields.push_back(::arrow::field("timestamp", values->type()));
-    }
-    {
-      arrow::random::RandomArrayGenerator generator(1);
-      std::shared_ptr<Array> values = generator.Int32(num_rows, 0, 1e9, 0);
-      columns.push_back(std::make_shared<ChunkedArray>(values));
-      fields.push_back(::arrow::field("val", values->type()));
-    }
-
+    FieldVector fields{::arrow::field("timestamp", int64(), /*nullable=*/true,

Review comment:
       Nit: I think the explicit namespaces are not necessary here
   ```suggestion
       FieldVector fields{field("timestamp", int64(), /*nullable=*/true,
   ```

##########
File path: cpp/src/arrow/testing/random_test.cc
##########
@@ -0,0 +1,195 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <gtest/gtest.h>
+
+#include "arrow/array.h"
+#include "arrow/record_batch.h"
+#include "arrow/testing/gtest_util.h"
+#include "arrow/testing/random.h"
+#include "arrow/type.h"
+#include "arrow/util/key_value_metadata.h"
+
+namespace arrow {
+namespace random {
+
+class RandomArrayTest : public 
::testing::TestWithParam<std::shared_ptr<Field>> {
+ protected:
+  std::shared_ptr<Field> GetField() { return GetParam(); }
+};
+
+template <typename T>
+class RandomNumericArrayTest : public ::testing::Test {
+ protected:
+  std::shared_ptr<Field> GetField() { return field("field0", 
std::make_shared<T>()); }
+
+  std::shared_ptr<NumericArray<T>> Downcast(std::shared_ptr<Array> array) {
+    return internal::checked_pointer_cast<NumericArray<T>>(array);
+  }
+};
+
+TEST_P(RandomArrayTest, GenerateArray) {
+  auto field = GetField();
+  auto batch = Generate({field}, 128, 0xDEADBEEF);
+  AssertSchemaEqual(schema({field}), batch->schema());
+  auto array = batch->column(0);
+  ASSERT_EQ(128, array->length());
+  ASSERT_OK(array->ValidateFull());
+}
+
+TEST_P(RandomArrayTest, GenerateNonNullArray) {

Review comment:
       Nit: too similar to the next case's name
   ```suggestion
   TEST_P(RandomArrayTest, GenerateArrayWithZeroNullProbability) {
   ```

##########
File path: cpp/src/arrow/testing/random_test.cc
##########
@@ -0,0 +1,195 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <gtest/gtest.h>
+
+#include "arrow/array.h"
+#include "arrow/record_batch.h"
+#include "arrow/testing/gtest_util.h"
+#include "arrow/testing/random.h"
+#include "arrow/type.h"
+#include "arrow/util/key_value_metadata.h"
+
+namespace arrow {
+namespace random {
+
+class RandomArrayTest : public 
::testing::TestWithParam<std::shared_ptr<Field>> {
+ protected:
+  std::shared_ptr<Field> GetField() { return GetParam(); }
+};
+
+template <typename T>
+class RandomNumericArrayTest : public ::testing::Test {
+ protected:
+  std::shared_ptr<Field> GetField() { return field("field0", 
std::make_shared<T>()); }
+
+  std::shared_ptr<NumericArray<T>> Downcast(std::shared_ptr<Array> array) {
+    return internal::checked_pointer_cast<NumericArray<T>>(array);
+  }
+};
+
+TEST_P(RandomArrayTest, GenerateArray) {
+  auto field = GetField();
+  auto batch = Generate({field}, 128, 0xDEADBEEF);
+  AssertSchemaEqual(schema({field}), batch->schema());
+  auto array = batch->column(0);
+  ASSERT_EQ(128, array->length());
+  ASSERT_OK(array->ValidateFull());
+}
+
+TEST_P(RandomArrayTest, GenerateNonNullArray) {
+  auto field =
+      GetField()->WithMetadata(key_value_metadata({{"null_probability", 
"0.0"}}));
+  if (field->type()->id() == Type::type::NA) {
+    GTEST_SKIP() << "Cannot generate non-null null arrays";
+  }
+  auto batch = Generate({field}, 128, 0xDEADBEEF);
+  AssertSchemaEqual(schema({field}), batch->schema());
+  auto array = batch->column(0);
+  ASSERT_OK(array->ValidateFull());
+  ASSERT_EQ(0, array->null_count());
+}
+
+TEST_P(RandomArrayTest, GenerateNonNullableArray) {
+  auto field = GetField()->WithNullable(false);
+  if (field->type()->id() == Type::type::NA) {
+    GTEST_SKIP() << "Cannot generate non-null null arrays";
+  }
+  auto batch = Generate({field}, 128, 0xDEADBEEF);
+  AssertSchemaEqual(schema({field}), batch->schema());
+  auto array = batch->column(0);
+  ASSERT_OK(array->ValidateFull());
+  ASSERT_EQ(0, array->null_count());
+}
+
+struct FieldParamName {
+  template <class ParamType>
+  std::string operator()(const ::testing::TestParamInfo<ParamType>& info) 
const {
+    return std::to_string(info.index) + info.param->name();
+  }
+};
+
+auto values = ::testing::Values(
+    field("null", null()), field("bool", boolean()), field("uint8", uint8()),
+    field("int8", int8()), field("uint16", uint16()), field("int16", int16()),
+    field("uint32", uint32()), field("int32", int32()), field("uint64", 
uint64()),
+    field("int64", int64()), field("float16", float16()), field("float32", 
float32()),
+    field("float64", float64()), field("string", utf8()), field("binary", 
binary()),
+    field("fixed_size_binary", fixed_size_binary(8)),
+    field("decimal128", decimal128(8, 3)), field("decimal256", decimal256(16, 
4)),
+    field("date32", date32()), field("date64", date64()),
+    field("timestampns", timestamp(TimeUnit::NANO)),
+    field("timestamps", timestamp(TimeUnit::SECOND, "America/Phoenix")),
+    field("time32ms", time32(TimeUnit::MILLI)), field("time64ns", 
time64(TimeUnit::NANO)),
+    field("time32s", time32(TimeUnit::SECOND)),
+    field("time64us", time64(TimeUnit::MICRO)), field("month_interval", 
month_interval()),
+    field("daytime_interval", day_time_interval()), field("listint8", 
list(int8())),
+    field("listlistint8", list(list(int8()))),
+    field("listint8emptynulls", list(int8()), true,
+          key_value_metadata({{"force_empty_nulls", "true"}})),
+    field("listint81024values", list(int8()), true,
+          key_value_metadata({{"values", "1024"}})),
+    field("structints", struct_({
+                            field("int8", int8()),
+                            field("int16", int16()),
+                            field("int32", int32()),
+                        })),
+    field("structnested", struct_({
+                              field("string", utf8()),
+                              field("list", list(int64())),
+                              field("timestamp", timestamp(TimeUnit::MILLI)),
+                          })),
+    field("sparseunion", sparse_union({
+                             field("int8", int8()),
+                             field("int16", int16()),
+                             field("int32", int32()),
+                         })),
+    field("denseunion", dense_union({
+                            field("int8", int8()),
+                            field("int16", int16()),
+                            field("int32", int32()),
+                        })),
+    field("dictionary", dictionary(int8(), utf8())), field("map", map(int8(), 
utf8())),
+    field("fixedsizelist", fixed_size_list(int8(), 4)),
+    field("durationns", duration(TimeUnit::NANO)), field("largestring", 
large_utf8()),
+    field("largebinary", large_binary()),
+    field("largelistlistint8", large_list(list(int8()))));
+
+INSTANTIATE_TEST_SUITE_P(
+    TestRandomArrayGeneration, RandomArrayTest, values,
+    [](const ::testing::TestParamInfo<RandomArrayTest::ParamType>& info) {
+      return std::to_string(info.index) + info.param->name();
+    });
+
+using NumericTypes =
+    ::testing::Types<UInt8Type, Int8Type, UInt16Type, Int16Type, UInt32Type, 
Int32Type,
+                     HalfFloatType, FloatType, DoubleType>;
+TYPED_TEST_SUITE(RandomNumericArrayTest, NumericTypes);
+
+TYPED_TEST(RandomNumericArrayTest, GenerateMinMax) {
+  auto field = this->GetField()->WithMetadata(
+      key_value_metadata({{"min", "0"}, {"max", "127"}, {"nan_probability", 
"0.0"}}));
+  auto batch = Generate({field}, 128, 0xDEADBEEF);
+  AssertSchemaEqual(schema({field}), batch->schema());
+  auto array = this->Downcast(batch->column(0));
+  auto it = array->begin();
+  while (it != array->end()) {
+    if ((*it).has_value()) {
+      ASSERT_GE(**it, typename TypeParam::c_type(0));
+      ASSERT_LE(**it, typename TypeParam::c_type(127));
+    }
+    it++;
+  }

Review comment:
       You can use range-for and I believe `optional` supports direct comparison
   ```suggestion
     for (auto slot : *array) {
       if (!slot.has_value()) continue;
       ASSERT_GE(slot, 0);
       ASSERT_LE(slot, 127);
     }
   ```

##########
File path: cpp/src/arrow/testing/random.h
##########
@@ -358,6 +362,10 @@ class ARROW_TESTING_EXPORT RandomArrayGenerator {
   std::default_random_engine seed_rng_;
 };
 

Review comment:
       This function warrants a sprawling docstring.

##########
File path: cpp/src/arrow/testing/random.cc
##########
@@ -558,5 +584,248 @@ std::shared_ptr<Array> 
RandomArrayGenerator::ArrayOf(std::shared_ptr<DataType> t
   return RandomArrayGeneratorOfImpl{this, type, size, null_probability, 
nullptr}.Finish();
 }
 
+namespace {
+template <typename T>
+typename T::c_type GetMetadata(const KeyValueMetadata* metadata, const 
std::string& key,
+                               typename T::c_type default_value) {
+  if (!metadata) return default_value;
+  const auto index = metadata->FindKey(key);
+  if (index < 0) return default_value;
+  const auto& value = metadata->value(index);
+  typename T::c_type output{};
+  auto type = checked_pointer_cast<T>(TypeTraits<T>::type_singleton());
+  if (!internal::ParseValue(*type, value.data(), value.length(), &output)) {
+    ABORT_NOT_OK(Status::Invalid("Could not parse ", key, " = ", value));

Review comment:
       ```suggestion
     T output{};
     if (!internal::ParseValue<ArrowType>(value.data(), value.length(), 
&output)) {
       ABORT_NOT_OK(Status::Invalid("Could not parse ", key, " = ", value));
   ```

##########
File path: cpp/src/arrow/testing/random_test.cc
##########
@@ -0,0 +1,195 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <gtest/gtest.h>
+
+#include "arrow/array.h"
+#include "arrow/record_batch.h"
+#include "arrow/testing/gtest_util.h"
+#include "arrow/testing/random.h"
+#include "arrow/type.h"
+#include "arrow/util/key_value_metadata.h"
+
+namespace arrow {
+namespace random {
+
+class RandomArrayTest : public 
::testing::TestWithParam<std::shared_ptr<Field>> {
+ protected:
+  std::shared_ptr<Field> GetField() { return GetParam(); }
+};
+
+template <typename T>
+class RandomNumericArrayTest : public ::testing::Test {
+ protected:
+  std::shared_ptr<Field> GetField() { return field("field0", 
std::make_shared<T>()); }
+
+  std::shared_ptr<NumericArray<T>> Downcast(std::shared_ptr<Array> array) {
+    return internal::checked_pointer_cast<NumericArray<T>>(array);
+  }
+};
+
+TEST_P(RandomArrayTest, GenerateArray) {
+  auto field = GetField();
+  auto batch = Generate({field}, 128, 0xDEADBEEF);
+  AssertSchemaEqual(schema({field}), batch->schema());
+  auto array = batch->column(0);
+  ASSERT_EQ(128, array->length());
+  ASSERT_OK(array->ValidateFull());
+}
+
+TEST_P(RandomArrayTest, GenerateNonNullArray) {
+  auto field =
+      GetField()->WithMetadata(key_value_metadata({{"null_probability", 
"0.0"}}));
+  if (field->type()->id() == Type::type::NA) {
+    GTEST_SKIP() << "Cannot generate non-null null arrays";
+  }
+  auto batch = Generate({field}, 128, 0xDEADBEEF);
+  AssertSchemaEqual(schema({field}), batch->schema());
+  auto array = batch->column(0);
+  ASSERT_OK(array->ValidateFull());
+  ASSERT_EQ(0, array->null_count());
+}
+
+TEST_P(RandomArrayTest, GenerateNonNullableArray) {
+  auto field = GetField()->WithNullable(false);
+  if (field->type()->id() == Type::type::NA) {
+    GTEST_SKIP() << "Cannot generate non-null null arrays";
+  }
+  auto batch = Generate({field}, 128, 0xDEADBEEF);
+  AssertSchemaEqual(schema({field}), batch->schema());
+  auto array = batch->column(0);
+  ASSERT_OK(array->ValidateFull());
+  ASSERT_EQ(0, array->null_count());
+}
+
+struct FieldParamName {
+  template <class ParamType>
+  std::string operator()(const ::testing::TestParamInfo<ParamType>& info) 
const {
+    return std::to_string(info.index) + info.param->name();
+  }
+};
+
+auto values = ::testing::Values(
+    field("null", null()), field("bool", boolean()), field("uint8", uint8()),
+    field("int8", int8()), field("uint16", uint16()), field("int16", int16()),
+    field("uint32", uint32()), field("int32", int32()), field("uint64", 
uint64()),
+    field("int64", int64()), field("float16", float16()), field("float32", 
float32()),
+    field("float64", float64()), field("string", utf8()), field("binary", 
binary()),
+    field("fixed_size_binary", fixed_size_binary(8)),
+    field("decimal128", decimal128(8, 3)), field("decimal256", decimal256(16, 
4)),
+    field("date32", date32()), field("date64", date64()),
+    field("timestampns", timestamp(TimeUnit::NANO)),
+    field("timestamps", timestamp(TimeUnit::SECOND, "America/Phoenix")),
+    field("time32ms", time32(TimeUnit::MILLI)), field("time64ns", 
time64(TimeUnit::NANO)),
+    field("time32s", time32(TimeUnit::SECOND)),
+    field("time64us", time64(TimeUnit::MICRO)), field("month_interval", 
month_interval()),
+    field("daytime_interval", day_time_interval()), field("listint8", 
list(int8())),
+    field("listlistint8", list(list(int8()))),
+    field("listint8emptynulls", list(int8()), true,
+          key_value_metadata({{"force_empty_nulls", "true"}})),
+    field("listint81024values", list(int8()), true,
+          key_value_metadata({{"values", "1024"}})),
+    field("structints", struct_({
+                            field("int8", int8()),
+                            field("int16", int16()),
+                            field("int32", int32()),
+                        })),
+    field("structnested", struct_({
+                              field("string", utf8()),
+                              field("list", list(int64())),
+                              field("timestamp", timestamp(TimeUnit::MILLI)),
+                          })),
+    field("sparseunion", sparse_union({
+                             field("int8", int8()),
+                             field("int16", int16()),
+                             field("int32", int32()),
+                         })),
+    field("denseunion", dense_union({
+                            field("int8", int8()),
+                            field("int16", int16()),
+                            field("int32", int32()),
+                        })),
+    field("dictionary", dictionary(int8(), utf8())), field("map", map(int8(), 
utf8())),
+    field("fixedsizelist", fixed_size_list(int8(), 4)),
+    field("durationns", duration(TimeUnit::NANO)), field("largestring", 
large_utf8()),
+    field("largebinary", large_binary()),
+    field("largelistlistint8", large_list(list(int8()))));
+
+INSTANTIATE_TEST_SUITE_P(
+    TestRandomArrayGeneration, RandomArrayTest, values,
+    [](const ::testing::TestParamInfo<RandomArrayTest::ParamType>& info) {
+      return std::to_string(info.index) + info.param->name();
+    });
+
+using NumericTypes =
+    ::testing::Types<UInt8Type, Int8Type, UInt16Type, Int16Type, UInt32Type, 
Int32Type,
+                     HalfFloatType, FloatType, DoubleType>;
+TYPED_TEST_SUITE(RandomNumericArrayTest, NumericTypes);
+
+TYPED_TEST(RandomNumericArrayTest, GenerateMinMax) {
+  auto field = this->GetField()->WithMetadata(
+      key_value_metadata({{"min", "0"}, {"max", "127"}, {"nan_probability", 
"0.0"}}));
+  auto batch = Generate({field}, 128, 0xDEADBEEF);
+  AssertSchemaEqual(schema({field}), batch->schema());
+  auto array = this->Downcast(batch->column(0));
+  auto it = array->begin();
+  while (it != array->end()) {
+    if ((*it).has_value()) {
+      ASSERT_GE(**it, typename TypeParam::c_type(0));
+      ASSERT_LE(**it, typename TypeParam::c_type(127));
+    }
+    it++;
+  }
+}
+
+TEST(TypeSpecificTests, FloatNan) {
+  auto field = arrow::field("float32", float32())
+                   ->WithMetadata(key_value_metadata({{"nan_probability", 
"1.0"}}));
+  auto batch = Generate({field}, 128, 0xDEADBEEF);
+  AssertSchemaEqual(schema({field}), batch->schema());
+  auto array = 
internal::checked_pointer_cast<NumericArray<FloatType>>(batch->column(0));
+  auto it = array->begin();
+  while (it != array->end()) {
+    if ((*it).has_value()) {
+      ASSERT_TRUE(std::isnan(**it));
+    }
+    it++;
+  }
+}
+
+TEST(TypeSpecificTests, RepeatedStrings) {
+  auto field =
+      arrow::field("string", 
utf8())->WithMetadata(key_value_metadata({{"unique", "1"}}));
+  auto batch = Generate({field}, 128, 0xDEADBEEF);
+  AssertSchemaEqual(schema({field}), batch->schema());
+  auto array = internal::checked_pointer_cast<StringArray>(batch->column(0));
+  auto it = array->begin();
+  util::optional<util::string_view> singular_value;
+  while (it != array->end()) {
+    if ((*it).has_value()) {
+      if (!singular_value.has_value()) {
+        singular_value = *it;
+      } else {
+        ASSERT_EQ(*singular_value, **it);
+      }
+    }
+    it++;
+  }
+}
+

Review comment:
       There are numerous untested options, please ensure each recognized 
metadata key is hit at least once

##########
File path: cpp/src/arrow/testing/random_test.cc
##########
@@ -0,0 +1,195 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <gtest/gtest.h>
+
+#include "arrow/array.h"
+#include "arrow/record_batch.h"
+#include "arrow/testing/gtest_util.h"
+#include "arrow/testing/random.h"
+#include "arrow/type.h"
+#include "arrow/util/key_value_metadata.h"
+
+namespace arrow {
+namespace random {
+
+class RandomArrayTest : public 
::testing::TestWithParam<std::shared_ptr<Field>> {
+ protected:
+  std::shared_ptr<Field> GetField() { return GetParam(); }
+};
+
+template <typename T>
+class RandomNumericArrayTest : public ::testing::Test {
+ protected:
+  std::shared_ptr<Field> GetField() { return field("field0", 
std::make_shared<T>()); }
+
+  std::shared_ptr<NumericArray<T>> Downcast(std::shared_ptr<Array> array) {
+    return internal::checked_pointer_cast<NumericArray<T>>(array);
+  }
+};
+
+TEST_P(RandomArrayTest, GenerateArray) {
+  auto field = GetField();
+  auto batch = Generate({field}, 128, 0xDEADBEEF);
+  AssertSchemaEqual(schema({field}), batch->schema());
+  auto array = batch->column(0);
+  ASSERT_EQ(128, array->length());
+  ASSERT_OK(array->ValidateFull());
+}
+
+TEST_P(RandomArrayTest, GenerateNonNullArray) {
+  auto field =
+      GetField()->WithMetadata(key_value_metadata({{"null_probability", 
"0.0"}}));
+  if (field->type()->id() == Type::type::NA) {
+    GTEST_SKIP() << "Cannot generate non-null null arrays";
+  }
+  auto batch = Generate({field}, 128, 0xDEADBEEF);
+  AssertSchemaEqual(schema({field}), batch->schema());
+  auto array = batch->column(0);
+  ASSERT_OK(array->ValidateFull());
+  ASSERT_EQ(0, array->null_count());
+}
+
+TEST_P(RandomArrayTest, GenerateNonNullableArray) {
+  auto field = GetField()->WithNullable(false);
+  if (field->type()->id() == Type::type::NA) {
+    GTEST_SKIP() << "Cannot generate non-null null arrays";
+  }
+  auto batch = Generate({field}, 128, 0xDEADBEEF);
+  AssertSchemaEqual(schema({field}), batch->schema());
+  auto array = batch->column(0);
+  ASSERT_OK(array->ValidateFull());
+  ASSERT_EQ(0, array->null_count());
+}
+
+struct FieldParamName {
+  template <class ParamType>
+  std::string operator()(const ::testing::TestParamInfo<ParamType>& info) 
const {
+    return std::to_string(info.index) + info.param->name();
+  }
+};

Review comment:
       Looks like you replaced this with a lambda?
   ```suggestion
   ```




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


Reply via email to