diff --git a/cpp/src/arrow/flight/integration_tests/test_integration_client.cc b/cpp/src/arrow/flight/integration_tests/test_integration_client.cc index 9c5c985c06efd..db72c9f4b0613 100644 --- a/cpp/src/arrow/flight/integration_tests/test_integration_client.cc +++ b/cpp/src/arrow/flight/integration_tests/test_integration_client.cc @@ -56,11 +56,10 @@ namespace integration_tests { /// \brief Helper to read all batches from a JsonReader Status ReadBatches(std::unique_ptr& reader, std::vector>* chunks) { - std::shared_ptr chunk; for (int i = 0; i < reader->num_record_batches(); i++) { - RETURN_NOT_OK(reader->ReadRecordBatch(i, &chunk)); + ARROW_ASSIGN_OR_RAISE(auto chunk, reader->ReadRecordBatch(i)); RETURN_NOT_OK(chunk->ValidateFull()); - chunks->push_back(chunk); + chunks->push_back(std::move(chunk)); } return Status::OK(); } @@ -150,11 +149,10 @@ class IntegrationTestScenario : public Scenario { FlightDescriptor descr{FlightDescriptor::PATH, "", {FLAGS_path}}; // 1. Put the data to the server. - std::unique_ptr reader; std::cout << "Opening JSON file '" << FLAGS_path << "'" << std::endl; auto in_file = *io::ReadableFile::Open(FLAGS_path); - ABORT_NOT_OK( - testing::IntegrationJsonReader::Open(default_memory_pool(), in_file, &reader)); + ARROW_ASSIGN_OR_RAISE(auto reader, testing::IntegrationJsonReader::Open( + default_memory_pool(), in_file)); std::shared_ptr original_schema = reader->schema(); std::vector> original_data; diff --git a/cpp/src/arrow/ipc/type_fwd.h b/cpp/src/arrow/ipc/type_fwd.h index 3493c4f1409a9..b0d3afa922f78 100644 --- a/cpp/src/arrow/ipc/type_fwd.h +++ b/cpp/src/arrow/ipc/type_fwd.h @@ -56,6 +56,9 @@ class RecordBatchStreamReader; class RecordBatchFileReader; class RecordBatchWriter; +class DictionaryFieldMapper; +class DictionaryMemo; + namespace feather { class Reader; diff --git a/cpp/src/arrow/testing/json_integration.cc b/cpp/src/arrow/testing/json_integration.cc index a6bed8ca195a1..e87566502b1dc 100644 --- a/cpp/src/arrow/testing/json_integration.cc +++ b/cpp/src/arrow/testing/json_integration.cc @@ -37,8 +37,6 @@ #include #include -using std::size_t; - namespace arrow { using ipc::DictionaryFieldMapper; @@ -82,12 +80,11 @@ class IntegrationJsonWriter::Impl { return Status::OK(); } - Status Finish(std::string* result) { + Result Finish() { writer_->EndArray(); // Record batches writer_->EndObject(); - *result = string_buffer_.GetString(); - return Status::OK(); + return string_buffer_.GetString(); } Status WriteRecordBatch(const RecordBatch& batch) { @@ -115,15 +112,14 @@ IntegrationJsonWriter::IntegrationJsonWriter(const std::shared_ptr& sche IntegrationJsonWriter::~IntegrationJsonWriter() {} -Status IntegrationJsonWriter::Open(const std::shared_ptr& schema, - std::unique_ptr* writer) { - *writer = std::unique_ptr(new IntegrationJsonWriter(schema)); - return (*writer)->impl_->Start(); +Result> IntegrationJsonWriter::Open( + const std::shared_ptr& schema) { + auto writer = std::unique_ptr(new IntegrationJsonWriter(schema)); + RETURN_NOT_OK(writer->impl_->Start()); + return writer; } -Status IntegrationJsonWriter::Finish(std::string* result) { - return impl_->Finish(result); -} +Result IntegrationJsonWriter::Finish() { return impl_->Finish(); } Status IntegrationJsonWriter::WriteRecordBatch(const RecordBatch& batch) { return impl_->WriteRecordBatch(batch); @@ -144,7 +140,7 @@ class IntegrationJsonReader::Impl { return Status::IOError("JSON parsing failed"); } - RETURN_NOT_OK(json::ReadSchema(doc_, pool_, &dictionary_memo_, &schema_)); + ARROW_ASSIGN_OR_RAISE(schema_, json::ReadSchema(doc_, pool_, &dictionary_memo_)); auto it = std::as_const(doc_).FindMember("batches"); RETURN_NOT_ARRAY("batches", it, doc_); @@ -153,13 +149,13 @@ class IntegrationJsonReader::Impl { return Status::OK(); } - Status ReadRecordBatch(int i, std::shared_ptr* batch) { + Result> ReadRecordBatch(int i) { DCHECK_GE(i, 0) << "i out of bounds"; DCHECK_LT(i, static_cast(record_batches_->GetArray().Size())) << "i out of bounds"; return json::ReadRecordBatch(record_batches_->GetArray()[i], schema_, - &dictionary_memo_, pool_, batch); + &dictionary_memo_, pool_); } std::shared_ptr schema() const { return schema_; } @@ -179,29 +175,30 @@ class IntegrationJsonReader::Impl { }; IntegrationJsonReader::IntegrationJsonReader(MemoryPool* pool, - const std::shared_ptr& data) { - impl_.reset(new Impl(pool, data)); + std::shared_ptr data) { + impl_.reset(new Impl(pool, std::move(data))); } IntegrationJsonReader::~IntegrationJsonReader() {} -Status IntegrationJsonReader::Open(const std::shared_ptr& data, - std::unique_ptr* reader) { - return Open(default_memory_pool(), data, reader); +Result> IntegrationJsonReader::Open( + MemoryPool* pool, std::shared_ptr data) { + auto reader = std::unique_ptr( + new IntegrationJsonReader(pool, std::move(data))); + RETURN_NOT_OK(reader->impl_->ParseAndReadSchema()); + return reader; } -Status IntegrationJsonReader::Open(MemoryPool* pool, const std::shared_ptr& data, - std::unique_ptr* reader) { - *reader = std::unique_ptr(new IntegrationJsonReader(pool, data)); - return (*reader)->impl_->ParseAndReadSchema(); +Result> IntegrationJsonReader::Open( + std::shared_ptr data) { + return Open(default_memory_pool(), std::move(data)); } -Status IntegrationJsonReader::Open(MemoryPool* pool, - const std::shared_ptr& in_file, - std::unique_ptr* reader) { +Result> IntegrationJsonReader::Open( + MemoryPool* pool, const std::shared_ptr& in_file) { ARROW_ASSIGN_OR_RAISE(int64_t file_size, in_file->GetSize()); ARROW_ASSIGN_OR_RAISE(auto json_buffer, in_file->Read(file_size)); - return Open(pool, json_buffer, reader); + return Open(pool, std::move(json_buffer)); } std::shared_ptr IntegrationJsonReader::schema() const { return impl_->schema(); } @@ -210,9 +207,8 @@ int IntegrationJsonReader::num_record_batches() const { return impl_->num_record_batches(); } -Status IntegrationJsonReader::ReadRecordBatch(int i, - std::shared_ptr* batch) const { - return impl_->ReadRecordBatch(i, batch); +Result> IntegrationJsonReader::ReadRecordBatch(int i) const { + return impl_->ReadRecordBatch(i); } } // namespace testing diff --git a/cpp/src/arrow/testing/json_integration.h b/cpp/src/arrow/testing/json_integration.h index 3486bb5d958be..c2d0c104a6077 100644 --- a/cpp/src/arrow/testing/json_integration.h +++ b/cpp/src/arrow/testing/json_integration.h @@ -22,21 +22,12 @@ #include #include -#include "arrow/status.h" +#include "arrow/io/type_fwd.h" +#include "arrow/result.h" #include "arrow/testing/visibility.h" +#include "arrow/type_fwd.h" -namespace arrow { - -class Buffer; -class MemoryPool; -class RecordBatch; -class Schema; - -namespace io { -class ReadableFile; -} // namespace io - -namespace testing { +namespace arrow::testing { /// \class IntegrationJsonWriter /// \brief Write the JSON representation of an Arrow record batch file or stream @@ -49,19 +40,17 @@ class ARROW_TESTING_EXPORT IntegrationJsonWriter { /// \brief Create a new JSON writer that writes to memory /// /// \param[in] schema the schema of record batches - /// \param[out] out the returned writer object - /// \return Status - static Status Open(const std::shared_ptr& schema, - std::unique_ptr* out); + /// \return the creater writer object + static Result> Open( + const std::shared_ptr& schema); /// \brief Append a record batch Status WriteRecordBatch(const RecordBatch& batch); /// \brief Finish the JSON payload and return as a std::string /// - /// \param[out] result the JSON as as a std::string - /// \return Status - Status Finish(std::string* result); + /// \return the JSON payload as a string + Result Finish(); private: explicit IntegrationJsonWriter(const std::shared_ptr& schema); @@ -83,27 +72,24 @@ class ARROW_TESTING_EXPORT IntegrationJsonReader { /// /// \param[in] pool a MemoryPool to use for buffer allocations /// \param[in] data a Buffer containing the JSON data - /// \param[out] reader the returned reader object - /// \return Status - static Status Open(MemoryPool* pool, const std::shared_ptr& data, - std::unique_ptr* reader); + /// \return the created JSON reader + static Result> Open( + MemoryPool* pool, std::shared_ptr data); /// \brief Create a new JSON reader that uses the default memory pool /// /// \param[in] data a Buffer containing the JSON data - /// \param[out] reader the returned reader object - /// \return Status - static Status Open(const std::shared_ptr& data, - std::unique_ptr* reader); + /// \return the created JSON reader + static Result> Open( + std::shared_ptr data); /// \brief Create a new JSON reader from a file /// /// \param[in] pool a MemoryPool to use for buffer allocations /// \param[in] in_file a ReadableFile containing JSON data - /// \param[out] reader the returned reader object - /// \return Status - static Status Open(MemoryPool* pool, const std::shared_ptr& in_file, - std::unique_ptr* reader); + /// \return the created JSON reader + static Result> Open( + MemoryPool* pool, const std::shared_ptr& in_file); /// \brief Return the schema read from the JSON std::shared_ptr schema() const; @@ -114,16 +100,15 @@ class ARROW_TESTING_EXPORT IntegrationJsonReader { /// \brief Read a particular record batch from the file /// /// \param[in] i the record batch index, does not boundscheck - /// \param[out] batch the read record batch - Status ReadRecordBatch(int i, std::shared_ptr* batch) const; + /// \return the record batch read + Result> ReadRecordBatch(int i) const; private: - IntegrationJsonReader(MemoryPool* pool, const std::shared_ptr& data); + IntegrationJsonReader(MemoryPool* pool, std::shared_ptr data); // Hide RapidJSON details from public API class Impl; std::unique_ptr impl_; }; -} // namespace testing -} // namespace arrow +} // namespace arrow::testing diff --git a/cpp/src/arrow/testing/json_integration_test.cc b/cpp/src/arrow/testing/json_integration_test.cc index 7f1c706684d42..e950d36ce00d7 100644 --- a/cpp/src/arrow/testing/json_integration_test.cc +++ b/cpp/src/arrow/testing/json_integration_test.cc @@ -86,8 +86,7 @@ static Status ConvertJsonToArrow(const std::string& json_path, ARROW_ASSIGN_OR_RAISE(int64_t file_size, in_file->GetSize()); ARROW_ASSIGN_OR_RAISE(auto json_buffer, in_file->Read(file_size)); - std::unique_ptr reader; - RETURN_NOT_OK(IntegrationJsonReader::Open(json_buffer, &reader)); + ARROW_ASSIGN_OR_RAISE(auto reader, IntegrationJsonReader::Open(json_buffer)); if (FLAGS_verbose) { std::cout << "Found schema:\n" @@ -97,8 +96,7 @@ static Status ConvertJsonToArrow(const std::string& json_path, ARROW_ASSIGN_OR_RAISE(auto writer, ipc::MakeFileWriter(out_file, reader->schema(), IpcWriteOptions::Defaults())); for (int i = 0; i < reader->num_record_batches(); ++i) { - std::shared_ptr batch; - RETURN_NOT_OK(reader->ReadRecordBatch(i, &batch)); + ARROW_ASSIGN_OR_RAISE(auto batch, reader->ReadRecordBatch(i)); RETURN_NOT_OK(writer->WriteRecordBatch(*batch)); } return writer->Close(); @@ -117,17 +115,15 @@ static Status ConvertArrowToJson(const std::string& arrow_path, std::cout << "Found schema:\n" << reader->schema()->ToString() << std::endl; } - std::unique_ptr writer; - RETURN_NOT_OK(IntegrationJsonWriter::Open(reader->schema(), &writer)); + ARROW_ASSIGN_OR_RAISE(auto writer, IntegrationJsonWriter::Open(reader->schema())); for (int i = 0; i < reader->num_record_batches(); ++i) { ARROW_ASSIGN_OR_RAISE(std::shared_ptr batch, reader->ReadRecordBatch(i)); RETURN_NOT_OK(writer->WriteRecordBatch(*batch)); } - std::string result; - RETURN_NOT_OK(writer->Finish(&result)); - return out_file->Write(result.c_str(), static_cast(result.size())); + ARROW_ASSIGN_OR_RAISE(auto json_data, writer->Finish()); + return out_file->Write(std::string_view(json_data)); } // Validate the batch, accounting for the -validate_decimals , -validate_date64, and @@ -162,8 +158,7 @@ static Status ValidateArrowVsJson(const std::string& arrow_path, ARROW_ASSIGN_OR_RAISE(int64_t file_size, json_file->GetSize()); ARROW_ASSIGN_OR_RAISE(auto json_buffer, json_file->Read(file_size)); - std::unique_ptr json_reader; - RETURN_NOT_OK(IntegrationJsonReader::Open(json_buffer, &json_reader)); + ARROW_ASSIGN_OR_RAISE(auto json_reader, IntegrationJsonReader::Open(json_buffer)); // Construct Arrow reader ARROW_ASSIGN_OR_RAISE(auto arrow_file, io::ReadableFile::Open(arrow_path)); @@ -198,7 +193,7 @@ static Status ValidateArrowVsJson(const std::string& arrow_path, std::shared_ptr arrow_batch; std::shared_ptr json_batch; for (int i = 0; i < json_nbatches; ++i) { - RETURN_NOT_OK(json_reader->ReadRecordBatch(i, &json_batch)); + ARROW_ASSIGN_OR_RAISE(json_batch, json_reader->ReadRecordBatch(i)); ARROW_ASSIGN_OR_RAISE(arrow_batch, arrow_reader->ReadRecordBatch(i)); Status valid_st = ValidateFull(*json_batch); if (!valid_st.ok()) { @@ -729,14 +724,14 @@ static const char* json_example6 = R"example( } )example"; -void TestSchemaRoundTrip(const Schema& schema) { +void TestSchemaRoundTrip(const std::shared_ptr& schema) { rj::StringBuffer sb; rj::Writer writer(sb); - DictionaryFieldMapper mapper(schema); + DictionaryFieldMapper mapper(*schema); writer.StartObject(); - ASSERT_OK(json::WriteSchema(schema, mapper, &writer)); + ASSERT_OK(json::WriteSchema(*schema, mapper, &writer)); writer.EndObject(); std::string json_schema = sb.GetString(); @@ -747,15 +742,9 @@ void TestSchemaRoundTrip(const Schema& schema) { d.Parse(json_schema.data(), json_schema.size()); DictionaryMemo in_memo; - std::shared_ptr out; - const auto status = json::ReadSchema(d, default_memory_pool(), &in_memo, &out); - if (!status.ok()) { - FAIL() << "Unable to read JSON schema: " << json_schema << "\nStatus: " << status; - } - - if (!schema.Equals(*out)) { - FAIL() << "In schema: " << schema.ToString() << "\nOut schema: " << out->ToString(); - } + ASSERT_OK_AND_ASSIGN(auto result_schema, + json::ReadSchema(d, default_memory_pool(), &in_memo)); + AssertSchemaEqual(schema, result_schema, /*check_metadata=*/true); } void TestArrayRoundTrip(const Array& array) { @@ -772,17 +761,17 @@ void TestArrayRoundTrip(const Array& array) { // Pass explicit size to avoid ASAN issues with // SIMD loads in RapidJson. d.Parse(array_as_json.data(), array_as_json.size()); - if (d.HasParseError()) { FAIL() << "JSON parsing failed"; } - std::shared_ptr out; - ASSERT_OK(json::ReadArray(default_memory_pool(), d, ::arrow::field(name, array.type()), - &out)); + ASSERT_OK_AND_ASSIGN( + auto result_array, + json::ReadArray(default_memory_pool(), d, ::arrow::field(name, array.type()))); + ASSERT_OK(result_array->ValidateFull()); // std::cout << array_as_json << std::endl; - CompareArraysDetailed(0, *out, array); + CompareArraysDetailed(0, *result_array, array); } template @@ -823,6 +812,7 @@ TEST(TestJsonSchemaWriter, FlatTypes) { field("f11", binary()), field("f12", list(int32())), field("f13", struct_({field("s1", int32()), field("s2", utf8())})), + field("f14", date32()), field("f15", date64()), field("f16", timestamp(TimeUnit::NANO)), field("f17", time64(TimeUnit::MICRO)), @@ -836,7 +826,7 @@ TEST(TestJsonSchemaWriter, FlatTypes) { field("f23", run_end_encoded(int64(), utf8())), }; - Schema schema(fields); + auto schema = ::arrow::schema(fields); TestSchemaRoundTrip(schema); } @@ -982,8 +972,7 @@ TEST(TestJsonFileReadWrite, BasicRoundTrip) { ::arrow::schema({field("f1", v1_type), field("f2", v2_type), field("f3", v3_type), field("f4", v4_type), field("f5", v5_type)}); - std::unique_ptr writer; - ASSERT_OK(IntegrationJsonWriter::Open(schema, &writer)); + ASSERT_OK_AND_ASSIGN(auto writer, IntegrationJsonWriter::Open(schema)); const int nbatches = 3; std::vector> batches; @@ -997,21 +986,17 @@ TEST(TestJsonFileReadWrite, BasicRoundTrip) { ASSERT_OK(writer->WriteRecordBatch(*batch)); } - std::string result; - ASSERT_OK(writer->Finish(&result)); + ASSERT_OK_AND_ASSIGN(auto json_data, writer->Finish()); - std::unique_ptr reader; - - auto buffer = std::make_shared(result); - - ASSERT_OK(IntegrationJsonReader::Open(buffer, &reader)); - ASSERT_TRUE(reader->schema()->Equals(*schema)); + ASSERT_OK_AND_ASSIGN(auto reader, + IntegrationJsonReader::Open(std::make_shared(json_data))); + AssertSchemaEqual(reader->schema(), schema, /*check_metadata=*/true); ASSERT_EQ(nbatches, reader->num_record_batches()); for (int i = 0; i < nbatches; ++i) { - std::shared_ptr batch; - ASSERT_OK(reader->ReadRecordBatch(i, &batch)); + ASSERT_OK_AND_ASSIGN(auto batch, reader->ReadRecordBatch(i)); + ASSERT_OK(batch->ValidateFull()); ASSERT_BATCHES_EQUAL(*batch, *batches[i]); } } @@ -1020,13 +1005,13 @@ static void ReadOneBatchJson(const char* json, const Schema& expected_schema, std::shared_ptr* out) { auto buffer = Buffer::Wrap(json, strlen(json)); - std::unique_ptr reader; - ASSERT_OK(IntegrationJsonReader::Open(buffer, &reader)); + ASSERT_OK_AND_ASSIGN(auto reader, IntegrationJsonReader::Open(buffer)); AssertSchemaEqual(*reader->schema(), expected_schema, /*check_metadata=*/true); ASSERT_EQ(1, reader->num_record_batches()); - ASSERT_OK(reader->ReadRecordBatch(0, out)); + ASSERT_OK_AND_ASSIGN(*out, reader->ReadRecordBatch(0)); + ASSERT_OK((*out)->ValidateFull()); } TEST(TestJsonFileReadWrite, JsonExample1) { @@ -1047,11 +1032,10 @@ TEST(TestJsonFileReadWrite, JsonExample2) { auto uuid_type = uuid(); auto buffer = Buffer::Wrap(json_example2, strlen(json_example2)); - std::unique_ptr reader; { ExtensionTypeGuard ext_guard(uuid_type); - ASSERT_OK(IntegrationJsonReader::Open(buffer, &reader)); + ASSERT_OK_AND_ASSIGN(auto reader, IntegrationJsonReader::Open(buffer)); // The second field is an unregistered extension and will be read as // its underlying storage. Schema ex_schema({field("uuids", uuid_type), field("things", null())}); @@ -1059,8 +1043,8 @@ TEST(TestJsonFileReadWrite, JsonExample2) { AssertSchemaEqual(ex_schema, *reader->schema()); ASSERT_EQ(1, reader->num_record_batches()); - std::shared_ptr batch; - ASSERT_OK(reader->ReadRecordBatch(0, &batch)); + ASSERT_OK_AND_ASSIGN(auto batch, reader->ReadRecordBatch(0)); + ASSERT_OK(batch->ValidateFull()); auto storage_array = ArrayFromJSON(fixed_size_binary(16), R"(["0123456789abcdef", null])"); @@ -1070,7 +1054,7 @@ TEST(TestJsonFileReadWrite, JsonExample2) { } // Should fail now that the Uuid extension is unregistered - ASSERT_RAISES(KeyError, IntegrationJsonReader::Open(buffer, &reader)); + ASSERT_RAISES(KeyError, IntegrationJsonReader::Open(buffer)); } TEST(TestJsonFileReadWrite, JsonExample3) { @@ -1139,22 +1123,17 @@ class TestJsonRoundTrip : public ::testing::TestWithParam { void CheckRoundtrip(const RecordBatch& batch) { ExtensionTypeGuard guard({uuid(), dict_extension_type(), complex128()}); - TestSchemaRoundTrip(*batch.schema()); + TestSchemaRoundTrip(batch.schema()); - std::unique_ptr writer; - ASSERT_OK(IntegrationJsonWriter::Open(batch.schema(), &writer)); + ASSERT_OK_AND_ASSIGN(auto writer, IntegrationJsonWriter::Open(batch.schema())); ASSERT_OK(writer->WriteRecordBatch(batch)); - std::string result; - ASSERT_OK(writer->Finish(&result)); - - auto buffer = std::make_shared(result); - - std::unique_ptr reader; - ASSERT_OK(IntegrationJsonReader::Open(buffer, &reader)); + ASSERT_OK_AND_ASSIGN(auto json_data, writer->Finish()); - std::shared_ptr result_batch; - ASSERT_OK(reader->ReadRecordBatch(0, &result_batch)); + ASSERT_OK_AND_ASSIGN(auto reader, + IntegrationJsonReader::Open(std::make_shared(json_data))); + ASSERT_OK_AND_ASSIGN(auto result_batch, reader->ReadRecordBatch(0)); + ASSERT_OK(result_batch->ValidateFull()); // take care of float rounding error in the text representation ApproxCompareBatch(batch, *result_batch); diff --git a/cpp/src/arrow/testing/json_internal.cc b/cpp/src/arrow/testing/json_internal.cc index 81c4befbf2bdc..ead0625e7bf3d 100644 --- a/cpp/src/arrow/testing/json_internal.cc +++ b/cpp/src/arrow/testing/json_internal.cc @@ -363,8 +363,7 @@ class SchemaWriter { return Status::OK(); } - Status WriteChildren(const std::vector>& children, - FieldPosition field_pos) { + Status WriteChildren(const FieldVector& children, FieldPosition field_pos) { writer_->Key("children"); writer_->StartArray(); for (auto [i, field] : Zip(Enumerate, children)) { @@ -666,7 +665,7 @@ class ArrayWriter { // but that fails the Java parser (ARROW-11483). } - Status WriteChildren(const std::vector>& fields, + Status WriteChildren(const FieldVector& fields, const std::vector>& arrays) { // NOTE: the Java parser fails on an empty "children" member (ARROW-11483). if (fields.size() == 0) return Status::OK(); @@ -822,77 +821,64 @@ Result GetMemberTimeUnit(const RjObject& obj, const std::string& return GetUnitFromString(unit_str); } -Status GetInteger(const rj::Value::ConstObject& json_type, - std::shared_ptr* type) { +Result> GetInteger(const rj::Value::ConstObject& json_type) { ARROW_ASSIGN_OR_RAISE(const bool is_signed, GetMemberBool(json_type, "isSigned")); ARROW_ASSIGN_OR_RAISE(const int bit_width, GetMemberInt(json_type, "bitWidth")); switch (bit_width) { case 8: - *type = is_signed ? int8() : uint8(); - break; + return is_signed ? int8() : uint8(); case 16: - *type = is_signed ? int16() : uint16(); - break; + return is_signed ? int16() : uint16(); case 32: - *type = is_signed ? int32() : uint32(); - break; + return is_signed ? int32() : uint32(); case 64: - *type = is_signed ? int64() : uint64(); - break; - default: - return Status::Invalid("Invalid bit width: ", bit_width); + return is_signed ? int64() : uint64(); } - return Status::OK(); + return Status::Invalid("Invalid bit width: ", bit_width); } -Status GetFloatingPoint(const RjObject& json_type, std::shared_ptr* type) { +Result> GetFloatingPoint(const RjObject& json_type) { ARROW_ASSIGN_OR_RAISE(const auto precision, GetMemberString(json_type, "precision")); if (precision == "DOUBLE") { - *type = float64(); + return float64(); } else if (precision == "SINGLE") { - *type = float32(); + return float32(); } else if (precision == "HALF") { - *type = float16(); - } else { - return Status::Invalid("Invalid precision: ", precision); + return float16(); } - return Status::OK(); + return Status::Invalid("Invalid precision: ", precision); } -Status GetMap(const RjObject& json_type, - const std::vector>& children, - std::shared_ptr* type) { +Result> GetMap(const RjObject& json_type, + const FieldVector& children) { if (children.size() != 1) { return Status::Invalid("Map must have exactly one child"); } ARROW_ASSIGN_OR_RAISE(const bool keys_sorted, GetMemberBool(json_type, "keysSorted")); - return MapType::Make(children[0], keys_sorted).Value(type); + return MapType::Make(children[0], keys_sorted); } -Status GetFixedSizeBinary(const RjObject& json_type, std::shared_ptr* type) { +Result> GetFixedSizeBinary(const RjObject& json_type) { ARROW_ASSIGN_OR_RAISE(const int32_t byte_width, GetMemberInt(json_type, "byteWidth")); - *type = fixed_size_binary(byte_width); - return Status::OK(); + return fixed_size_binary(byte_width); } -Status GetFixedSizeList(const RjObject& json_type, - const std::vector>& children, - std::shared_ptr* type) { +Result> GetFixedSizeList(const RjObject& json_type, + const FieldVector& children) { if (children.size() != 1) { return Status::Invalid("FixedSizeList must have exactly one child"); } ARROW_ASSIGN_OR_RAISE(const int32_t list_size, GetMemberInt(json_type, "listSize")); - *type = fixed_size_list(children[0], list_size); - return Status::OK(); + return fixed_size_list(children[0], list_size); } -Status GetDecimal(const RjObject& json_type, std::shared_ptr* type) { +Result> GetDecimal(const RjObject& json_type) { ARROW_ASSIGN_OR_RAISE(const int32_t precision, GetMemberInt(json_type, "precision")); ARROW_ASSIGN_OR_RAISE(const int32_t scale, GetMemberInt(json_type, "scale")); @@ -903,92 +889,82 @@ Status GetDecimal(const RjObject& json_type, std::shared_ptr* type) { } if (bit_width == 128) { - *type = decimal128(precision, scale); + return decimal128(precision, scale); } else if (bit_width == 256) { - *type = decimal256(precision, scale); - } else { - return Status::Invalid("Only 128 bit and 256 Decimals are supported. Received", - bit_width); + return decimal256(precision, scale); } - return Status::OK(); + return Status::Invalid("Only 128 bit and 256 Decimals are supported. Received", + bit_width); } -Status GetDate(const RjObject& json_type, std::shared_ptr* type) { +Result> GetDate(const RjObject& json_type) { ARROW_ASSIGN_OR_RAISE(const auto unit_str, GetMemberString(json_type, "unit")); if (unit_str == "DAY") { - *type = date32(); + return date32(); } else if (unit_str == "MILLISECOND") { - *type = date64(); - } else { - return Status::Invalid("Invalid date unit: ", unit_str); + return date64(); } - return Status::OK(); + return Status::Invalid("Invalid date unit: ", unit_str); } -Status GetTime(const RjObject& json_type, std::shared_ptr* type) { +Result> GetTime(const RjObject& json_type) { ARROW_ASSIGN_OR_RAISE(const auto unit_str, GetMemberString(json_type, "unit")); ARROW_ASSIGN_OR_RAISE(const int bit_width, GetMemberInt(json_type, "bitWidth")); + std::shared_ptr type; + if (unit_str == "SECOND") { - *type = time32(TimeUnit::SECOND); + type = time32(TimeUnit::SECOND); } else if (unit_str == "MILLISECOND") { - *type = time32(TimeUnit::MILLI); + type = time32(TimeUnit::MILLI); } else if (unit_str == "MICROSECOND") { - *type = time64(TimeUnit::MICRO); + type = time64(TimeUnit::MICRO); } else if (unit_str == "NANOSECOND") { - *type = time64(TimeUnit::NANO); + type = time64(TimeUnit::NANO); } else { return Status::Invalid("Invalid time unit: ", unit_str); } - const auto& fw_type = checked_cast(**type); - + const auto& fw_type = checked_cast(*type); if (bit_width != fw_type.bit_width()) { return Status::Invalid("Indicated bit width does not match unit"); } - - return Status::OK(); + return type; } -Status GetDuration(const RjObject& json_type, std::shared_ptr* type) { +Result> GetDuration(const RjObject& json_type) { ARROW_ASSIGN_OR_RAISE(const TimeUnit::type unit, GetMemberTimeUnit(json_type, "unit")); - *type = duration(unit); - return Status::OK(); + return duration(unit); } -Status GetTimestamp(const RjObject& json_type, std::shared_ptr* type) { +Result> GetTimestamp(const RjObject& json_type) { ARROW_ASSIGN_OR_RAISE(const TimeUnit::type unit, GetMemberTimeUnit(json_type, "unit")); const auto& it_tz = json_type.FindMember("timezone"); if (it_tz == json_type.MemberEnd()) { - *type = timestamp(unit); + return timestamp(unit); } else { RETURN_NOT_STRING("timezone", it_tz, json_type); - *type = timestamp(unit, it_tz->value.GetString()); + return timestamp(unit, it_tz->value.GetString()); } - - return Status::OK(); } -Status GetInterval(const RjObject& json_type, std::shared_ptr* type) { +Result> GetInterval(const RjObject& json_type) { ARROW_ASSIGN_OR_RAISE(const auto unit_str, GetMemberString(json_type, "unit")); if (unit_str == kDayTime) { - *type = day_time_interval(); + return day_time_interval(); } else if (unit_str == kYearMonth) { - *type = month_interval(); + return month_interval(); } else if (unit_str == kMonthDayNano) { - *type = month_day_nano_interval(); - } else { - return Status::Invalid("Invalid interval unit: " + unit_str); + return month_day_nano_interval(); } - return Status::OK(); + return Status::Invalid("Invalid interval unit: " + unit_str); } -Status GetUnion(const RjObject& json_type, - const std::vector>& children, - std::shared_ptr* type) { +Result> GetUnion(const RjObject& json_type, + const FieldVector& children) { ARROW_ASSIGN_OR_RAISE(const auto mode_str, GetMemberString(json_type, "mode")); UnionMode::type mode; @@ -1012,17 +988,14 @@ Status GetUnion(const RjObject& json_type, } if (mode == UnionMode::SPARSE) { - *type = sparse_union(std::move(children), std::move(type_codes)); + return sparse_union(std::move(children), std::move(type_codes)); } else { - *type = dense_union(std::move(children), std::move(type_codes)); + return dense_union(std::move(children), std::move(type_codes)); } - - return Status::OK(); } -Status GetRunEndEncoded(const RjObject& json_type, - const std::vector>& children, - std::shared_ptr* type) { +Result> GetRunEndEncoded(const RjObject& json_type, + const FieldVector& children) { if (children.size() != 2) { return Status::Invalid("Run-end encoded array must have exactly 2 fields, but got ", children.size()); @@ -1046,82 +1019,79 @@ Status GetRunEndEncoded(const RjObject& json_type, if (children[0]->nullable()) { return Status::Invalid("Run ends array should not be nullable"); } - *type = run_end_encoded(children[0]->type(), children[1]->type()); - return Status::OK(); + return run_end_encoded(children[0]->type(), children[1]->type()); } -Status GetType(const RjObject& json_type, - const std::vector>& children, - std::shared_ptr* type) { +Result> GetType(const RjObject& json_type, + const FieldVector& children) { ARROW_ASSIGN_OR_RAISE(const auto type_name, GetMemberString(json_type, "name")); if (type_name == "int") { - return GetInteger(json_type, type); + return GetInteger(json_type); } else if (type_name == "floatingpoint") { - return GetFloatingPoint(json_type, type); + return GetFloatingPoint(json_type); } else if (type_name == "bool") { - *type = boolean(); + return boolean(); } else if (type_name == "utf8") { - *type = utf8(); + return utf8(); } else if (type_name == "binary") { - *type = binary(); + return binary(); } else if (type_name == "largeutf8") { - *type = large_utf8(); + return large_utf8(); } else if (type_name == "largebinary") { - *type = large_binary(); + return large_binary(); } else if (type_name == "fixedsizebinary") { - return GetFixedSizeBinary(json_type, type); + return GetFixedSizeBinary(json_type); } else if (type_name == "decimal") { - return GetDecimal(json_type, type); + return GetDecimal(json_type); } else if (type_name == "null") { - *type = null(); + return null(); } else if (type_name == "date") { - return GetDate(json_type, type); + return GetDate(json_type); } else if (type_name == "time") { - return GetTime(json_type, type); + return GetTime(json_type); } else if (type_name == "timestamp") { - return GetTimestamp(json_type, type); + return GetTimestamp(json_type); } else if (type_name == "interval") { - return GetInterval(json_type, type); + return GetInterval(json_type); } else if (type_name == kDuration) { - return GetDuration(json_type, type); + return GetDuration(json_type); } else if (type_name == "list") { if (children.size() != 1) { return Status::Invalid("List must have exactly one child"); } - *type = list(children[0]); + return list(children[0]); } else if (type_name == "largelist") { if (children.size() != 1) { return Status::Invalid("Large list must have exactly one child"); } - *type = large_list(children[0]); + return large_list(children[0]); } else if (type_name == "map") { - return GetMap(json_type, children, type); + return GetMap(json_type, children); } else if (type_name == "fixedsizelist") { - return GetFixedSizeList(json_type, children, type); + return GetFixedSizeList(json_type, children); } else if (type_name == "struct") { - *type = struct_(children); + return struct_(children); } else if (type_name == "union") { - return GetUnion(json_type, children, type); + return GetUnion(json_type, children); } else if (type_name == "runendencoded") { - return GetRunEndEncoded(json_type, children, type); - } else { - return Status::Invalid("Unrecognized type name: ", type_name); + return GetRunEndEncoded(json_type, children); } - return Status::OK(); + return Status::Invalid("Unrecognized type name: ", type_name); } -Status GetField(const rj::Value& obj, FieldPosition field_pos, - DictionaryMemo* dictionary_memo, std::shared_ptr* field); +Result> GetField(const rj::Value& obj, FieldPosition field_pos, + DictionaryMemo* dictionary_memo); -Status GetFieldsFromArray(const RjArray& json_fields, FieldPosition parent_pos, - DictionaryMemo* dictionary_memo, - std::vector>* fields) { - fields->resize(json_fields.Size()); - for (auto [json_field, field, i] : Zip(json_fields, *fields, Enumerate)) { - RETURN_NOT_OK(GetField(json_field, parent_pos.child(i), dictionary_memo, &field)); +Result GetFieldsFromArray(const RjArray& json_fields, + FieldPosition parent_pos, + DictionaryMemo* dictionary_memo) { + FieldVector fields(json_fields.Size()); + for (auto [json_field, field, i] : Zip(json_fields, fields, Enumerate)) { + ARROW_ASSIGN_OR_RAISE(field, + GetField(json_field, parent_pos.child(i), dictionary_memo)); } - return Status::OK(); + return fields; } Status ParseDictionary(const RjObject& obj, int64_t* id, bool* is_ordered, @@ -1135,16 +1105,16 @@ Status ParseDictionary(const RjObject& obj, int64_t* id, bool* is_ordered, if (type_name != "int") { return Status::Invalid("Dictionary indices can only be integers"); } - return GetInteger(json_index_type, index_type); + return GetInteger(json_index_type).Value(index_type); } template -Status GetKeyValueMetadata(const FieldOrStruct& field_or_struct, - std::shared_ptr* out) { - out->reset(new KeyValueMetadata); +Result> GetKeyValueMetadata( + const FieldOrStruct& field_or_struct) { + auto metadata = std::make_shared(); auto it = field_or_struct.FindMember("metadata"); if (it == field_or_struct.MemberEnd() || it->value.IsNull()) { - return Status::OK(); + return metadata; } if (!it->value.IsArray()) { return Status::Invalid("Metadata was not a JSON array"); @@ -1159,32 +1129,30 @@ Status GetKeyValueMetadata(const FieldOrStruct& field_or_struct, ARROW_ASSIGN_OR_RAISE(const auto key, GetMemberString(key_value_pair, "key")); ARROW_ASSIGN_OR_RAISE(const auto value, GetMemberString(key_value_pair, "value")); - (*out)->Append(std::move(key), std::move(value)); + metadata->Append(std::move(key), std::move(value)); } - return Status::OK(); + return metadata; } -Status GetField(const rj::Value& obj, FieldPosition field_pos, - DictionaryMemo* dictionary_memo, std::shared_ptr* field) { +Result> GetField(const rj::Value& obj, FieldPosition field_pos, + DictionaryMemo* dictionary_memo) { if (!obj.IsObject()) { return Status::Invalid("Field was not a JSON object"); } const auto& json_field = obj.GetObject(); - std::shared_ptr type; - ARROW_ASSIGN_OR_RAISE(const auto name, GetMemberString(json_field, "name")); ARROW_ASSIGN_OR_RAISE(const bool nullable, GetMemberBool(json_field, "nullable")); ARROW_ASSIGN_OR_RAISE(const auto json_type, GetMemberObject(json_field, "type")); ARROW_ASSIGN_OR_RAISE(const auto json_children, GetMemberArray(json_field, "children")); - std::vector> children; - RETURN_NOT_OK(GetFieldsFromArray(json_children, field_pos, dictionary_memo, &children)); - RETURN_NOT_OK(GetType(json_type, children, &type)); + ARROW_ASSIGN_OR_RAISE(FieldVector children, + GetFieldsFromArray(json_children, field_pos, dictionary_memo)); + ARROW_ASSIGN_OR_RAISE(std::shared_ptr type, GetType(json_type, children)); - std::shared_ptr metadata; - RETURN_NOT_OK(GetKeyValueMetadata(json_field, &metadata)); + ARROW_ASSIGN_OR_RAISE(std::shared_ptr metadata, + GetKeyValueMetadata(json_field)); // Is it a dictionary type? int64_t dictionary_id = -1; @@ -1226,13 +1194,13 @@ Status GetField(const rj::Value& obj, FieldPosition field_pos, } // Create field - *field = ::arrow::field(name, type, nullable, metadata); + auto field = ::arrow::field(name, type, nullable, metadata); if (dictionary_id != -1) { RETURN_NOT_OK(dictionary_memo->fields().AddField(dictionary_id, field_pos.path())); RETURN_NOT_OK(dictionary_memo->AddDictionaryType(dictionary_id, dict_value_type)); } - return Status::OK(); + return field; } template @@ -1753,38 +1721,34 @@ Status ReadDictionaries(const rj::Value& doc, MemoryPool* pool, } // namespace -Status ReadSchema(const rj::Value& json_schema, MemoryPool* pool, - DictionaryMemo* dictionary_memo, std::shared_ptr* schema) { +Result> ReadSchema(const rj::Value& json_schema, MemoryPool* pool, + DictionaryMemo* dictionary_memo) { DCHECK(json_schema.IsObject()); ARROW_ASSIGN_OR_RAISE(const auto obj_schema, GetMemberObject(json_schema.GetObject(), "schema")); ARROW_ASSIGN_OR_RAISE(const auto json_fields, GetMemberArray(obj_schema, "fields")); - std::shared_ptr metadata; - RETURN_NOT_OK(GetKeyValueMetadata(obj_schema, &metadata)); - - std::vector> fields; - RETURN_NOT_OK( - GetFieldsFromArray(json_fields, FieldPosition(), dictionary_memo, &fields)); - + ARROW_ASSIGN_OR_RAISE(auto metadata, GetKeyValueMetadata(obj_schema)); + ARROW_ASSIGN_OR_RAISE( + FieldVector fields, + GetFieldsFromArray(json_fields, FieldPosition(), dictionary_memo)); // Read the dictionaries (if any) and cache in the memo RETURN_NOT_OK(ReadDictionaries(json_schema, pool, dictionary_memo)); - *schema = ::arrow::schema(fields, metadata); + return ::arrow::schema(fields, metadata); return Status::OK(); } -Status ReadArray(MemoryPool* pool, const rj::Value& json_array, - const std::shared_ptr& field, std::shared_ptr* out) { +Result> ReadArray(MemoryPool* pool, const rj::Value& json_array, + const std::shared_ptr& field) { ARROW_ASSIGN_OR_RAISE(auto data, ReadArrayData(pool, json_array, field)); - *out = MakeArray(data); - return Status::OK(); + return MakeArray(data); } -Status ReadRecordBatch(const rj::Value& json_obj, const std::shared_ptr& schema, - DictionaryMemo* dictionary_memo, MemoryPool* pool, - std::shared_ptr* batch) { +Result> ReadRecordBatch( + const rj::Value& json_obj, const std::shared_ptr& schema, + DictionaryMemo* dictionary_memo, MemoryPool* pool) { DCHECK(json_obj.IsObject()); const auto& batch_obj = json_obj.GetObject(); @@ -1800,8 +1764,7 @@ Status ReadRecordBatch(const rj::Value& json_obj, const std::shared_ptr& RETURN_NOT_OK(ResolveDictionaries(columns, *dictionary_memo, pool)); - *batch = RecordBatch::Make(schema, num_rows, columns); - return Status::OK(); + return RecordBatch::Make(schema, num_rows, columns); } Status WriteSchema(const Schema& schema, const DictionaryFieldMapper& mapper, diff --git a/cpp/src/arrow/testing/json_internal.h b/cpp/src/arrow/testing/json_internal.h index 0870dd1e797d9..ca382782af1c1 100644 --- a/cpp/src/arrow/testing/json_internal.h +++ b/cpp/src/arrow/testing/json_internal.h @@ -30,9 +30,10 @@ #include // IWYU pragma: export #include // IWYU pragma: export -#include "arrow/status.h" // IWYU pragma: export +#include "arrow/ipc/type_fwd.h" +#include "arrow/result.h" #include "arrow/testing/visibility.h" -#include "arrow/type_fwd.h" // IWYU pragma: keep +#include "arrow/type_fwd.h" namespace rj = arrow::rapidjson; using RjWriter = rj::Writer; @@ -74,23 +75,7 @@ using RjObject = rj::Value::ConstObject; return Status::Invalid("field was not an object line ", __LINE__); \ } -namespace arrow { - -class Array; -class Field; -class MemoryPool; -class RecordBatch; -class Schema; - -namespace ipc { - -class DictionaryFieldMapper; -class DictionaryMemo; - -} // namespace ipc - -namespace testing { -namespace json { +namespace arrow::testing::json { /// \brief Append integration test Schema format to rapidjson writer ARROW_TESTING_EXPORT @@ -108,19 +93,17 @@ ARROW_TESTING_EXPORT Status WriteArray(const std::string& name, const Array& array, RjWriter* writer); ARROW_TESTING_EXPORT -Status ReadSchema(const rj::Value& json_obj, MemoryPool* pool, - ipc::DictionaryMemo* dictionary_memo, std::shared_ptr* schema); +Result> ReadSchema(const rj::Value& json_obj, MemoryPool* pool, + ipc::DictionaryMemo* dictionary_memo); ARROW_TESTING_EXPORT -Status ReadRecordBatch(const rj::Value& json_obj, const std::shared_ptr& schema, - ipc::DictionaryMemo* dict_memo, MemoryPool* pool, - std::shared_ptr* batch); +Result> ReadRecordBatch( + const rj::Value& json_obj, const std::shared_ptr& schema, + ipc::DictionaryMemo* dict_memo, MemoryPool* pool); // NOTE: Doesn't work with dictionary arrays, use ReadRecordBatch instead. ARROW_TESTING_EXPORT -Status ReadArray(MemoryPool* pool, const rj::Value& json_obj, - const std::shared_ptr& type, std::shared_ptr* array); +Result> ReadArray(MemoryPool* pool, const rj::Value& json_obj, + const std::shared_ptr& field); -} // namespace json -} // namespace testing -} // namespace arrow +} // namespace arrow::testing::json diff --git a/dev/archery/archery/cli.py b/dev/archery/archery/cli.py index 48022176a2d99..70f865cc2fa70 100644 --- a/dev/archery/archery/cli.py +++ b/dev/archery/archery/cli.py @@ -764,6 +764,8 @@ def integration(with_all=False, random_seed=12345, **args): enabled_languages += 1 if gen_path: + # XXX See GH-37575: this option is only used by the JS test suite + # and might not be useful anymore. os.makedirs(gen_path, exist_ok=True) write_js_test_json(gen_path) else: