From 8ecb653c33ef39c45c6e31e12b5e96e4002ae48d Mon Sep 17 00:00:00 2001 From: benibus Date: Sun, 9 Oct 2022 01:43:53 -0400 Subject: [PATCH 01/18] Implement StreamingReader for JSON --- cpp/src/arrow/json/reader.cc | 407 ++++++++++++++++++++++++------ cpp/src/arrow/json/reader.h | 50 +++- cpp/src/arrow/json/reader_test.cc | 188 ++++++++++++++ 3 files changed, 560 insertions(+), 85 deletions(-) diff --git a/cpp/src/arrow/json/reader.cc b/cpp/src/arrow/json/reader.cc index 85e527c8bda..a348ff5d513 100644 --- a/cpp/src/arrow/json/reader.cc +++ b/cpp/src/arrow/json/reader.cc @@ -42,10 +42,149 @@ namespace arrow { using std::string_view; using internal::checked_cast; +using internal::Executor; using internal::GetCpuThreadPool; using internal::TaskGroup; using internal::ThreadPool; +namespace json { +namespace { + +struct JSONBlock { + std::shared_ptr partial; + std::shared_ptr completion; + std::shared_ptr whole; + int64_t index = -1; +}; + +struct DecodedBlock { + std::shared_ptr record_batch; + int64_t num_bytes = 0; + int64_t index = -1; +}; + +Result> ToRecordBatch(const StructArray& converted) { + std::vector> columns; + columns.reserve(converted.num_fields()); + for (const auto& f : converted.fields()) columns.push_back(f); + return RecordBatch::Make(schema(converted.type()->fields()), converted.length(), + std::move(columns)); +} + +auto ToRecordBatch(const Array& converted) { + return ToRecordBatch(checked_cast(converted)); +} + +Result> ParseBlock(const JSONBlock& block, + const ParseOptions& parse_options, + MemoryPool* pool, int64_t* out_size = nullptr) { + std::unique_ptr parser; + RETURN_NOT_OK(BlockParser::Make(pool, parse_options, &parser)); + + int64_t size = block.partial->size() + block.completion->size() + block.whole->size(); + RETURN_NOT_OK(parser->ReserveScalarStorage(size)); + + if (block.partial->size() || block.completion->size()) { + std::shared_ptr straddling; + if (!block.completion->size()) { + straddling = block.partial; + } else if (!block.partial->size()) { + straddling = block.completion; + } else { + ARROW_ASSIGN_OR_RAISE(straddling, + ConcatenateBuffers({block.partial, block.completion}, pool)); + } + RETURN_NOT_OK(parser->Parse(straddling)); + } + if (block.whole->size()) { + RETURN_NOT_OK(parser->Parse(block.whole)); + } + + std::shared_ptr parsed; + RETURN_NOT_OK(parser->Finish(&parsed)); + + if (out_size) *out_size = size; + + return parsed; +} + +// Utility for incrementally generating chunked JSON blocks from source buffers +// +// Note: Retains state from prior calls +class BlockReader { + public: + BlockReader(std::unique_ptr chunker, std::shared_ptr first_buffer) + : chunker_(std::move(chunker)), + partial_(std::make_shared("")), + buffer_(std::move(first_buffer)) {} + + // Compose an iterator from a source iterator + template + static Iterator MakeIterator(Iterator> buf_it, + Args&&... args) { + auto reader = std::make_shared(std::forward(args)...); + Transformer, JSONBlock> transformer = + [reader](std::shared_ptr next_buffer) { + return (*reader)(std::move(next_buffer)); + }; + return MakeTransformedIterator(std::move(buf_it), transformer); + } + + // Compose a callable generator from a source generator + template + static AsyncGenerator MakeGenerator( + AsyncGenerator> buf_gen, Args&&... args) { + auto reader = std::make_shared(std::forward(args)...); + Transformer, JSONBlock> transformer = + [reader](std::shared_ptr next_buffer) { + return (*reader)(std::move(next_buffer)); + }; + return MakeTransformedGenerator(std::move(buf_gen), transformer); + } + + Result> operator()(std::shared_ptr next_buffer) { + if (!buffer_) return TransformFinish(); + + std::shared_ptr whole, completion, next_partial; + if (!next_buffer) { + // End of file reached => compute completion from penultimate block + RETURN_NOT_OK(chunker_->ProcessFinal(partial_, buffer_, &completion, &whole)); + } else { + std::shared_ptr starts_with_whole; + // Get completion of partial from previous block. + RETURN_NOT_OK(chunker_->ProcessWithPartial(partial_, buffer_, &completion, + &starts_with_whole)); + // Get all whole objects entirely inside the current buffer + RETURN_NOT_OK(chunker_->Process(starts_with_whole, &whole, &next_partial)); + } + + buffer_ = std::move(next_buffer); + return TransformYield(JSONBlock{std::exchange(partial_, next_partial), + std::move(completion), std::move(whole), index_++}); + } + + private: + std::unique_ptr chunker_; + std::shared_ptr partial_; + std::shared_ptr buffer_; + int64_t index_ = 0; +}; + +} // namespace +} // namespace json + +template <> +struct IterationTraits { + static json::JSONBlock End() { return json::JSONBlock{}; } + static bool IsEnd(const json::JSONBlock& val) { return val.index < 0; } +}; + +template <> +struct IterationTraits { + static json::DecodedBlock End() { return json::DecodedBlock{}; } + static bool IsEnd(const json::DecodedBlock& val) { return val.index < 0; } +}; + namespace json { class TableReaderImpl : public TableReader, @@ -57,56 +196,30 @@ class TableReaderImpl : public TableReader, : pool_(pool), read_options_(read_options), parse_options_(parse_options), - chunker_(MakeChunker(parse_options_)), task_group_(std::move(task_group)) {} Status Init(std::shared_ptr input) { ARROW_ASSIGN_OR_RAISE(auto it, io::MakeInputStreamIterator(input, read_options_.block_size)); return MakeReadaheadIterator(std::move(it), task_group_->parallelism()) - .Value(&block_iterator_); + .Value(&buffer_iterator_); } Result> Read() override { - RETURN_NOT_OK(MakeBuilder()); - - ARROW_ASSIGN_OR_RAISE(auto block, block_iterator_.Next()); - if (block == nullptr) { + ARROW_ASSIGN_OR_RAISE(auto buffer, buffer_iterator_.Next()); + if (buffer == nullptr) { return Status::Invalid("Empty JSON file"); } - auto self = shared_from_this(); - auto empty = std::make_shared(""); - - int64_t block_index = 0; - std::shared_ptr partial = empty; - - while (block != nullptr) { - std::shared_ptr next_block, whole, completion, next_partial; - - ARROW_ASSIGN_OR_RAISE(next_block, block_iterator_.Next()); - - if (next_block == nullptr) { - // End of file reached => compute completion from penultimate block - RETURN_NOT_OK(chunker_->ProcessFinal(partial, block, &completion, &whole)); - } else { - std::shared_ptr starts_with_whole; - // Get completion of partial from previous block. - RETURN_NOT_OK(chunker_->ProcessWithPartial(partial, block, &completion, - &starts_with_whole)); - - // Get all whole objects entirely inside the current buffer - RETURN_NOT_OK(chunker_->Process(starts_with_whole, &whole, &next_partial)); - } - - // Launch parse task - task_group_->Append([self, partial, completion, whole, block_index] { - return self->ParseAndInsert(partial, completion, whole, block_index); - }); - block_index++; + RETURN_NOT_OK(MakeBuilder()); - partial = next_partial; - block = next_block; + auto block_it = BlockReader::MakeIterator( + std::move(buffer_iterator_), MakeChunker(parse_options_), std::move(buffer)); + while (true) { + ARROW_ASSIGN_OR_RAISE(auto block, block_it.Next()); + if (IsIterationEnd(block)) break; + task_group_->Append( + [self = shared_from_this(), block] { return self->ParseAndInsert(block); }); } std::shared_ptr array; @@ -128,43 +241,17 @@ class TableReaderImpl : public TableReader, return MakeChunkedArrayBuilder(task_group_, pool_, promotion_graph, type, &builder_); } - Status ParseAndInsert(const std::shared_ptr& partial, - const std::shared_ptr& completion, - const std::shared_ptr& whole, int64_t block_index) { - std::unique_ptr parser; - RETURN_NOT_OK(BlockParser::Make(pool_, parse_options_, &parser)); - RETURN_NOT_OK(parser->ReserveScalarStorage(partial->size() + completion->size() + - whole->size())); - - if (partial->size() != 0 || completion->size() != 0) { - std::shared_ptr straddling; - if (partial->size() == 0) { - straddling = completion; - } else if (completion->size() == 0) { - straddling = partial; - } else { - ARROW_ASSIGN_OR_RAISE(straddling, - ConcatenateBuffers({partial, completion}, pool_)); - } - RETURN_NOT_OK(parser->Parse(straddling)); - } - - if (whole->size() != 0) { - RETURN_NOT_OK(parser->Parse(whole)); - } - - std::shared_ptr parsed; - RETURN_NOT_OK(parser->Finish(&parsed)); - builder_->Insert(block_index, field("", parsed->type()), parsed); + Status ParseAndInsert(const JSONBlock& block) { + ARROW_ASSIGN_OR_RAISE(auto parsed, ParseBlock(block, parse_options_, pool_)); + builder_->Insert(block.index, field("", parsed->type()), parsed); return Status::OK(); } MemoryPool* pool_; ReadOptions read_options_; ParseOptions parse_options_; - std::unique_ptr chunker_; std::shared_ptr task_group_; - Iterator> block_iterator_; + Iterator> buffer_iterator_; std::shared_ptr builder_; }; @@ -204,14 +291,188 @@ Result> ParseOne(ParseOptions options, builder->Insert(0, field("", type), parsed); std::shared_ptr converted_chunked; RETURN_NOT_OK(builder->Finish(&converted_chunked)); - const auto& converted = checked_cast(*converted_chunked->chunk(0)); - std::vector> columns(converted.num_fields()); - for (int i = 0; i < converted.num_fields(); ++i) { - columns[i] = converted.field(i); + return ToRecordBatch(*converted_chunked->chunk(0)); +} + +namespace { + +// Callable object for decoding a pre-chunked JSON block into a RecordBatch +class BlockDecoder { + public: + BlockDecoder(MemoryPool* pool, const ParseOptions& parse_options) + : pool_(pool), + parse_options_(parse_options), + conversion_type_(parse_options_.explicit_schema + ? struct_(parse_options_.explicit_schema->fields()) + : struct_({})), + promotion_graph_(parse_options_.unexpected_field_behavior == + UnexpectedFieldBehavior::InferType + ? GetPromotionGraph() + : nullptr) {} + + Result operator()(const JSONBlock& block) const { + int64_t num_bytes; + ARROW_ASSIGN_OR_RAISE(auto unconverted, + ParseBlock(block, parse_options_, pool_, &num_bytes)); + + std::shared_ptr builder; + RETURN_NOT_OK(MakeChunkedArrayBuilder(TaskGroup::MakeSerial(), pool_, + promotion_graph_, conversion_type_, &builder)); + builder->Insert(0, field("", unconverted->type()), unconverted); + + std::shared_ptr chunked; + RETURN_NOT_OK(builder->Finish(&chunked)); + ARROW_ASSIGN_OR_RAISE(auto batch, ToRecordBatch(*chunked->chunk(0))); + + return DecodedBlock{std::move(batch), num_bytes, block.index}; } - return RecordBatch::Make(schema(converted.type()->fields()), converted.length(), - std::move(columns)); + + private: + MemoryPool* pool_; + ParseOptions parse_options_; + std::shared_ptr conversion_type_; + const PromotionGraph* promotion_graph_; +}; + +} // namespace + +class StreamingReaderImpl : public StreamingReader, + public std::enable_shared_from_this { + public: + StreamingReaderImpl(io::IOContext io_context, Executor* executor, + const ReadOptions& read_options, const ParseOptions& parse_options) + : io_context_(std::move(io_context)), + executor_(executor), + read_options_(read_options), + parse_options_(parse_options), + bytes_processed_(std::make_shared>(0)) {} + + Future<> Init(std::shared_ptr input) { + ARROW_ASSIGN_OR_RAISE(auto it, + io::MakeInputStreamIterator(input, read_options_.block_size)); + ARROW_ASSIGN_OR_RAISE(auto bg_it, + MakeBackgroundGenerator(std::move(it), io_context_.executor())); + auto buf_gen = MakeTransferredGenerator(bg_it, executor_); + // We pre-fetch the first buffer during instantiation to resolve the schema and ensure + // the stream isn't empty + return buf_gen().Then( + [self = shared_from_this(), buf_gen](const std::shared_ptr& buffer) { + return self->InitFromFirstBuffer(buffer, buf_gen); + }); + } + + std::shared_ptr schema() const override { + return parse_options_.explicit_schema; + } + + Status ReadNext(std::shared_ptr* out) override { + auto future = ReadNextAsync(); + auto result = future.result(); + return std::move(result).Value(out); + } + + Future> ReadNextAsync() override { + return record_batch_gen_(); + } + + int64_t bytes_read() const override { return bytes_processed_->load(); } + + private: + Future<> InitFromFirstBuffer(const std::shared_ptr& buffer, + AsyncGenerator> buf_gen) { + if (!buffer) return Status::Invalid("Empty JSON stream"); + + // Generator for pre-chunked JSON data + auto block_gen = BlockReader::MakeGenerator(std::move(buf_gen), + MakeChunker(parse_options_), buffer); + // Decoder for the first block using the initial parse options + auto decoder = BlockDecoder(io_context_.pool(), parse_options_); + + return block_gen().Then( + [self = shared_from_this(), block_gen, decoder](JSONBlock block) -> Future<> { + // Skip any initial empty record batches so we can try to get a useful schema + int64_t skipped_bytes = 0; + ARROW_ASSIGN_OR_RAISE(auto decoded, decoder(block)); + while (!IsIterationEnd(decoded) && !decoded.record_batch->num_rows()) { + skipped_bytes = decoded.num_bytes; + auto fut = block_gen(); + ARROW_ASSIGN_OR_RAISE(block, fut.result()); + if (IsIterationEnd(block)) { + decoded = IterationEnd(); + } else { + ARROW_ASSIGN_OR_RAISE(decoded, decoder(block)); + decoded.num_bytes += skipped_bytes; + } + } + return self->InitFromFirstDecoded(decoded, block_gen); + }); + } + + Future<> InitFromFirstDecoded(const DecodedBlock& decoded, + AsyncGenerator block_gen) { + // End of stream and no non-empty batches were yielded, so just return empty ones + if (IsIterationEnd(decoded)) { + record_batch_gen_ = MakeEmptyGenerator>(); + parse_options_.explicit_schema = nullptr; + return Status::OK(); + } + + // Use the schema from the first batch as the basis for all future reads. If type + // inference wasn't requested then this should be the same as the provided + // explicit_schema. Otherwise, ignore unexpected fields for future batches to ensure + // their schemas are consistent + parse_options_.explicit_schema = decoded.record_batch->schema(); + if (parse_options_.unexpected_field_behavior == UnexpectedFieldBehavior::InferType) { + parse_options_.unexpected_field_behavior = UnexpectedFieldBehavior::Ignore; + } + + // The final decoder, which uses the resolved parse options for type deduction + auto decoded_gen = MakeMappedGenerator( + std::move(block_gen), BlockDecoder(io_context_.pool(), parse_options_)); + if (read_options_.use_threads) { + decoded_gen = + MakeReadaheadGenerator(std::move(decoded_gen), executor_->GetCapacity()); + } + // Return the batch we just read on first invocation + decoded_gen = MakeGeneratorStartsWith({decoded}, std::move(decoded_gen)); + + // Compose the final generator + record_batch_gen_ = MakeMappedGenerator( + std::move(decoded_gen), + [bytes_processed = bytes_processed_](const DecodedBlock& decoded) { + bytes_processed->fetch_add(decoded.num_bytes); + return decoded.record_batch; + }); + record_batch_gen_ = + MakeCancellable(std::move(record_batch_gen_), io_context_.stop_token()); + + return Status::OK(); + } + + io::IOContext io_context_; + Executor* executor_; + ReadOptions read_options_; + ParseOptions parse_options_; + AsyncGenerator> record_batch_gen_; + std::shared_ptr> bytes_processed_; +}; + +Future> StreamingReader::MakeAsync( + io::IOContext io_context, std::shared_ptr input, Executor* executor, + const ReadOptions& read_options, const ParseOptions& parse_options) { + auto reader = std::make_shared(io_context, executor, read_options, + parse_options); + return reader->Init(input).Then( + [reader] { return std::static_pointer_cast(reader); }); +} + +Result> StreamingReader::Make( + io::IOContext io_context, std::shared_ptr input, + const ReadOptions& read_options, const ParseOptions& parse_options) { + auto future = StreamingReader::MakeAsync(io_context, input, GetCpuThreadPool(), + read_options, parse_options); + return future.result(); } } // namespace json diff --git a/cpp/src/arrow/json/reader.h b/cpp/src/arrow/json/reader.h index 3374931a043..251793bec83 100644 --- a/cpp/src/arrow/json/reader.h +++ b/cpp/src/arrow/json/reader.h @@ -19,25 +19,15 @@ #include +#include "arrow/io/interfaces.h" #include "arrow/json/options.h" +#include "arrow/record_batch.h" #include "arrow/result.h" #include "arrow/status.h" #include "arrow/util/macros.h" #include "arrow/util/visibility.h" namespace arrow { - -class Buffer; -class MemoryPool; -class Table; -class RecordBatch; -class Array; -class DataType; - -namespace io { -class InputStream; -} // namespace io - namespace json { /// A class that reads an entire JSON file into a Arrow Table @@ -60,5 +50,41 @@ class ARROW_EXPORT TableReader { ARROW_EXPORT Result> ParseOne(ParseOptions options, std::shared_ptr json); +/// \brief A class that reads a JSON file incrementally +/// +/// JSON data is read from a stream in fixed-size blocks (configurable with +/// `ReadOptions::block_size`). Each block is converted to a `RecordBatch`. Yielded +/// batches have a consistent schema but may differ in row count. +/// +/// The supplied `ParseOptions` are used to determine a schema on the first non-empty +/// block. Afterwards, the schema is frozen and unexpected fields will be ignored on +/// subsequent reads (unless `UnexpectedFieldBehavior::Error` was specified). +class ARROW_EXPORT StreamingReader : public RecordBatchReader { + public: + virtual ~StreamingReader() = default; + + virtual Future> ReadNextAsync() = 0; + + /// \brief Return the number of bytes which have been read and processed + /// + /// The returned number includes JSON bytes which the StreamingReader has finished + /// processing, but not bytes for which some processing (e.g. JSON parsing or conversion + /// to Arrow layout) is still ongoing. + [[nodiscard]] virtual int64_t bytes_read() const = 0; + + /// Create a StreamingReader instance + /// + /// This involves some I/O as the first batch must be loaded during the creation process + /// so it is returned as a future + static Future> MakeAsync( + io::IOContext io_context, std::shared_ptr input, + ::arrow::internal::Executor* cpu_executor, const ReadOptions&, const ParseOptions&); + + /// Create a StreamingReader instance + static Result> Make( + io::IOContext io_context, std::shared_ptr input, + const ReadOptions&, const ParseOptions&); +}; + } // namespace json } // namespace arrow diff --git a/cpp/src/arrow/json/reader_test.cc b/cpp/src/arrow/json/reader_test.cc index 83f5956a64c..035114e3402 100644 --- a/cpp/src/arrow/json/reader_test.cc +++ b/cpp/src/arrow/json/reader_test.cc @@ -320,5 +320,193 @@ TEST(ReaderTest, FailOnInvalidEOF) { } } +class StreamingReaderTest : public ::testing::TestWithParam { + public: + ParseOptions parse_options_ = ParseOptions::Defaults(); + ReadOptions read_options_ = DefaultReadOptions(); + io::IOContext io_context_ = io::default_io_context(); + std::shared_ptr input_; + std::shared_ptr reader_; + + private: + [[nodiscard]] ReadOptions DefaultReadOptions() const { + auto read_options = ReadOptions::Defaults(); + read_options.use_threads = GetParam(); + return read_options; + } +}; + +INSTANTIATE_TEST_SUITE_P(StreamingReaderTest, StreamingReaderTest, + ::testing::Values(false, true)); + +TEST_P(StreamingReaderTest, FailOnEmptyInput) { + ASSERT_OK(MakeStream("", &input_)); + ASSERT_RAISES( + Invalid, StreamingReader::Make(io_context_, input_, read_options_, parse_options_)); +} + +TEST_P(StreamingReaderTest, FailOnParseError) { + std::string json = R"( +{"n": 10000} +{"n": "foo"})"; + + read_options_.block_size = 16; + ASSERT_OK(MakeStream(json, &input_)); + ASSERT_OK_AND_ASSIGN( + reader_, StreamingReader::Make(io_context_, input_, read_options_, parse_options_)); + std::shared_ptr batch; + ASSERT_OK(reader_->ReadNext(&batch)); + ASSERT_EQ(14, reader_->bytes_read()); + ASSERT_RAISES(Invalid, reader_->ReadNext(&batch)); +} + +TEST_P(StreamingReaderTest, IgnoreLeadingEmptyBlocks) { + std::string json; + json.insert(json.end(), 32, '\n'); + json += R"({"b": true, "s": "foo"})"; + auto json_len = static_cast(json.length()); + + parse_options_.explicit_schema = schema({field("b", boolean()), field("s", utf8())}); + read_options_.block_size = 24; + ASSERT_OK(MakeStream(json, &input_)); + ASSERT_OK_AND_ASSIGN( + reader_, StreamingReader::Make(io_context_, input_, read_options_, parse_options_)); + + auto expected_schema = parse_options_.explicit_schema; + auto expected_batch = + RecordBatchFromJSON(expected_schema, R"([{"b": true, "s": "foo"}])"); + std::shared_ptr actual_batch; + + ASSERT_EQ(*reader_->schema(), *expected_schema); + ASSERT_OK(reader_->ReadNext(&actual_batch)); + ASSERT_TRUE(actual_batch); + ASSERT_EQ(json_len, reader_->bytes_read()); + ASSERT_BATCHES_EQUAL(*actual_batch, *expected_batch); + + ASSERT_OK(reader_->ReadNext(&actual_batch)); + ASSERT_FALSE(actual_batch); +} + +TEST_P(StreamingReaderTest, ExplicitSchema) { + std::string json = R"({"s": "foo", "t": "2022-01-01", "b": true})"; + auto json_len = static_cast(json.length()); + + parse_options_.explicit_schema = schema({field("s", utf8()), field("t", utf8())}); + parse_options_.unexpected_field_behavior = UnexpectedFieldBehavior::Ignore; + + ASSERT_OK(MakeStream(json, &input_)); + ASSERT_OK_AND_ASSIGN( + reader_, StreamingReader::Make(io_context_, input_, read_options_, parse_options_)); + + auto expected_schema = parse_options_.explicit_schema; + auto expected_batch = + RecordBatchFromJSON(expected_schema, R"([{"s": "foo", "t": "2022-01-01"}])"); + std::shared_ptr actual_batch; + + ASSERT_EQ(*reader_->schema(), *expected_schema); + ASSERT_OK(reader_->ReadNext(&actual_batch)); + ASSERT_TRUE(actual_batch); + ASSERT_EQ(json_len, reader_->bytes_read()); + ASSERT_BATCHES_EQUAL(*actual_batch, *expected_batch); + + parse_options_.unexpected_field_behavior = UnexpectedFieldBehavior::Error; + ASSERT_OK(MakeStream(json, &input_)); + ASSERT_RAISES( + Invalid, StreamingReader::Make(io_context_, input_, read_options_, parse_options_)); +} + +TEST_P(StreamingReaderTest, InferredSchema) { + std::string json = R"( +{"a": 0, "b": "foo" } +{"a": 1, "c": true } +{"a": 2, "d": "2022-01-01"} +)"; + + std::shared_ptr expected_schema; + std::shared_ptr expected_batch; + std::shared_ptr actual_batch; + + FieldVector fields = {field("a", int64()), field("b", utf8())}; + + parse_options_.unexpected_field_behavior = UnexpectedFieldBehavior::InferType; + + // Schema derived from the first line + read_options_.block_size = 32; + ASSERT_OK(MakeStream(json, &input_)); + ASSERT_OK_AND_ASSIGN( + reader_, StreamingReader::Make(io_context_, input_, read_options_, parse_options_)); + + expected_schema = schema(fields); + ASSERT_EQ(*reader_->schema(), *expected_schema); + + expected_batch = RecordBatchFromJSON(expected_schema, R"([{"a": 0, "b": "foo"}])"); + ASSERT_OK(reader_->ReadNext(&actual_batch)); + ASSERT_TRUE(actual_batch); + ASSERT_EQ(29, reader_->bytes_read()); + ASSERT_BATCHES_EQUAL(*actual_batch, *expected_batch); + + expected_batch = RecordBatchFromJSON(expected_schema, R"([{"a": 1, "b": null}])"); + ASSERT_OK(reader_->ReadNext(&actual_batch)); + ASSERT_TRUE(actual_batch); + ASSERT_EQ(57, reader_->bytes_read()); + ASSERT_BATCHES_EQUAL(*actual_batch, *expected_batch); + + expected_batch = RecordBatchFromJSON(expected_schema, R"([{"a": 2, "b": null}])"); + ASSERT_OK(reader_->ReadNext(&actual_batch)); + ASSERT_TRUE(actual_batch); + ASSERT_EQ(85, reader_->bytes_read()); + ASSERT_BATCHES_EQUAL(*actual_batch, *expected_batch); + + // Schema derived from the first 2 lines + read_options_.block_size = 64; + ASSERT_OK(MakeStream(json, &input_)); + ASSERT_OK_AND_ASSIGN( + reader_, StreamingReader::Make(io_context_, input_, read_options_, parse_options_)); + + fields.push_back(field("c", boolean())); + expected_schema = schema(fields); + ASSERT_EQ(*reader_->schema(), *expected_schema); + + expected_batch = RecordBatchFromJSON(expected_schema, R"([ + {"a": 0, "b": "foo", "c": null}, + {"a": 1, "b": null, "c": true} + ])"); + ASSERT_OK(reader_->ReadNext(&actual_batch)); + ASSERT_TRUE(actual_batch); + ASSERT_EQ(57, reader_->bytes_read()); + ASSERT_BATCHES_EQUAL(*actual_batch, *expected_batch); + + expected_batch = RecordBatchFromJSON(expected_schema, R"([ + {"a": 2, "b": null, "c": null} + ])"); + ASSERT_OK(reader_->ReadNext(&actual_batch)); + ASSERT_TRUE(actual_batch); + ASSERT_EQ(85, reader_->bytes_read()); + ASSERT_BATCHES_EQUAL(*actual_batch, *expected_batch); + + // Schema derived from all 3 lines + read_options_.block_size = 96; + ASSERT_OK(MakeStream(json, &input_)); + ASSERT_OK_AND_ASSIGN( + reader_, StreamingReader::Make(io_context_, input_, read_options_, parse_options_)); + + fields.push_back(field("d", timestamp(TimeUnit::SECOND))); + expected_schema = schema(fields); + ASSERT_EQ(*reader_->schema(), *expected_schema); + + expected_batch = RecordBatchFromJSON(expected_schema, R"([ + {"a": 0, "b": "foo", "c": null, "d": null}, + {"a": 1, "b": null, "c": true, "d": null}, + {"a": 2, "b": null, "c": null, "d": "2022-01-01"} + ])"); + ASSERT_OK(reader_->ReadNext(&actual_batch)); + ASSERT_TRUE(actual_batch); + ASSERT_EQ(85, reader_->bytes_read()); + ASSERT_BATCHES_EQUAL(*actual_batch, *expected_batch); + + ASSERT_OK(reader_->ReadNext(&actual_batch)); + ASSERT_FALSE(actual_batch); +} + } // namespace json } // namespace arrow From ef307a142ef6328a16d6609a9c5258c49008f2c4 Mon Sep 17 00:00:00 2001 From: benibus Date: Mon, 14 Nov 2022 11:29:53 -0500 Subject: [PATCH 02/18] Add async-reentrancy, parallelism, and tests --- cpp/src/arrow/json/reader.cc | 567 +++++++++++++++++------------- cpp/src/arrow/json/reader.h | 18 +- cpp/src/arrow/json/reader_test.cc | 567 ++++++++++++++++++++++++------ 3 files changed, 786 insertions(+), 366 deletions(-) diff --git a/cpp/src/arrow/json/reader.cc b/cpp/src/arrow/json/reader.cc index a348ff5d513..17d9f570e89 100644 --- a/cpp/src/arrow/json/reader.cc +++ b/cpp/src/arrow/json/reader.cc @@ -50,7 +50,7 @@ using internal::ThreadPool; namespace json { namespace { -struct JSONBlock { +struct ChunkedBlock { std::shared_ptr partial; std::shared_ptr completion; std::shared_ptr whole; @@ -60,7 +60,81 @@ struct JSONBlock { struct DecodedBlock { std::shared_ptr record_batch; int64_t num_bytes = 0; - int64_t index = -1; +}; + +} // namespace +} // namespace json + +template <> +struct IterationTraits { + static json::ChunkedBlock End() { return json::ChunkedBlock{}; } + static bool IsEnd(const json::ChunkedBlock& val) { return val.index < 0; } +}; + +template <> +struct IterationTraits { + static json::DecodedBlock End() { return json::DecodedBlock{}; } + static bool IsEnd(const json::DecodedBlock& val) { return !val.record_batch; } +}; + +namespace json { +namespace { + +// Holds related parameters for parsing and type conversion +class DecodeContext { + public: + explicit DecodeContext(MemoryPool* pool) + : DecodeContext(ParseOptions::Defaults(), pool) {} + explicit DecodeContext(ParseOptions options = ParseOptions::Defaults(), + MemoryPool* pool = default_memory_pool()) + : pool_(pool) { + SetParseOptions(std::move(options)); + } + + void SetParseOptions(ParseOptions options) { + parse_options_ = std::move(options); + if (parse_options_.explicit_schema) { + conversion_type_ = struct_(parse_options_.explicit_schema->fields()); + } else { + parse_options_.unexpected_field_behavior = UnexpectedFieldBehavior::InferType; + conversion_type_ = struct_({}); + } + promotion_graph_ = + parse_options_.unexpected_field_behavior == UnexpectedFieldBehavior::InferType + ? GetPromotionGraph() + : nullptr; + } + + void SetSchema(std::shared_ptr explicit_schema, + UnexpectedFieldBehavior unexpected_field_behavior) { + parse_options_.explicit_schema = std::move(explicit_schema); + parse_options_.unexpected_field_behavior = unexpected_field_behavior; + SetParseOptions(std::move(parse_options_)); + } + void SetSchema(std::shared_ptr explicit_schema) { + SetSchema(std::move(explicit_schema), parse_options_.unexpected_field_behavior); + } + // Set the schema but ensure unexpected fields won't be accepted + void SetStrictSchema(std::shared_ptr explicit_schema) { + auto unexpected_field_behavior = parse_options_.unexpected_field_behavior; + if (unexpected_field_behavior == UnexpectedFieldBehavior::InferType) { + unexpected_field_behavior = UnexpectedFieldBehavior::Ignore; + } + SetSchema(std::move(explicit_schema), unexpected_field_behavior); + } + + [[nodiscard]] MemoryPool* pool() const { return pool_; } + [[nodiscard]] const ParseOptions& parse_options() const { return parse_options_; } + [[nodiscard]] const PromotionGraph* promotion_graph() const { return promotion_graph_; } + [[nodiscard]] const std::shared_ptr& conversion_type() const { + return conversion_type_; + } + + private: + ParseOptions parse_options_; + std::shared_ptr conversion_type_; + const PromotionGraph* promotion_graph_; + MemoryPool* pool_; }; Result> ToRecordBatch(const StructArray& converted) { @@ -75,7 +149,7 @@ auto ToRecordBatch(const Array& converted) { return ToRecordBatch(checked_cast(converted)); } -Result> ParseBlock(const JSONBlock& block, +Result> ParseBlock(const ChunkedBlock& block, const ParseOptions& parse_options, MemoryPool* pool, int64_t* out_size = nullptr) { std::unique_ptr parser; @@ -108,42 +182,29 @@ Result> ParseBlock(const JSONBlock& block, return parsed; } -// Utility for incrementally generating chunked JSON blocks from source buffers -// -// Note: Retains state from prior calls -class BlockReader { +class ChunkingTransformer { public: - BlockReader(std::unique_ptr chunker, std::shared_ptr first_buffer) - : chunker_(std::move(chunker)), - partial_(std::make_shared("")), - buffer_(std::move(first_buffer)) {} - - // Compose an iterator from a source iterator - template - static Iterator MakeIterator(Iterator> buf_it, - Args&&... args) { - auto reader = std::make_shared(std::forward(args)...); - Transformer, JSONBlock> transformer = - [reader](std::shared_ptr next_buffer) { - return (*reader)(std::move(next_buffer)); - }; - return MakeTransformedIterator(std::move(buf_it), transformer); - } + explicit ChunkingTransformer(std::unique_ptr chunker) + : chunker_(std::move(chunker)) {} - // Compose a callable generator from a source generator template - static AsyncGenerator MakeGenerator( - AsyncGenerator> buf_gen, Args&&... args) { - auto reader = std::make_shared(std::forward(args)...); - Transformer, JSONBlock> transformer = - [reader](std::shared_ptr next_buffer) { - return (*reader)(std::move(next_buffer)); - }; - return MakeTransformedGenerator(std::move(buf_gen), transformer); + static Transformer, ChunkedBlock> Make(Args&&... args) { + return [self = std::make_shared(std::forward(args)...)]( + std::shared_ptr buffer) { return (*self)(std::move(buffer)); }; } - Result> operator()(std::shared_ptr next_buffer) { - if (!buffer_) return TransformFinish(); + private: + Result> operator()(std::shared_ptr next_buffer) { + if (!buffer_) { + if (ARROW_PREDICT_TRUE(!next_buffer)) { + partial_ = nullptr; + return TransformFinish(); + } + partial_ = std::make_shared(""); + buffer_ = std::move(next_buffer); + return TransformSkip(); + } + DCHECK_NE(partial_, nullptr); std::shared_ptr whole, completion, next_partial; if (!next_buffer) { @@ -159,33 +220,30 @@ class BlockReader { } buffer_ = std::move(next_buffer); - return TransformYield(JSONBlock{std::exchange(partial_, next_partial), - std::move(completion), std::move(whole), index_++}); + return TransformYield(ChunkedBlock{std::exchange(partial_, next_partial), + std::move(completion), std::move(whole), + index_++}); } - private: std::unique_ptr chunker_; std::shared_ptr partial_; std::shared_ptr buffer_; int64_t index_ = 0; }; -} // namespace -} // namespace json - -template <> -struct IterationTraits { - static json::JSONBlock End() { return json::JSONBlock{}; } - static bool IsEnd(const json::JSONBlock& val) { return val.index < 0; } -}; - -template <> -struct IterationTraits { - static json::DecodedBlock End() { return json::DecodedBlock{}; } - static bool IsEnd(const json::DecodedBlock& val) { return val.index < 0; } -}; +template +Iterator MakeChunkingIterator(Iterator> source, + Args&&... args) { + return MakeTransformedIterator(std::move(source), + ChunkingTransformer::Make(std::forward(args)...)); +} -namespace json { +template +AsyncGenerator MakeChunkingGenerator( + AsyncGenerator> source, Args&&... args) { + return MakeTransformedGenerator(std::move(source), + ChunkingTransformer::Make(std::forward(args)...)); +} class TableReaderImpl : public TableReader, public std::enable_shared_from_this { @@ -193,9 +251,8 @@ class TableReaderImpl : public TableReader, TableReaderImpl(MemoryPool* pool, const ReadOptions& read_options, const ParseOptions& parse_options, std::shared_ptr task_group) - : pool_(pool), + : decode_context_(parse_options, pool), read_options_(read_options), - parse_options_(parse_options), task_group_(std::move(task_group)) {} Status Init(std::shared_ptr input) { @@ -206,21 +263,23 @@ class TableReaderImpl : public TableReader, } Result> Read() override { - ARROW_ASSIGN_OR_RAISE(auto buffer, buffer_iterator_.Next()); - if (buffer == nullptr) { - return Status::Invalid("Empty JSON file"); - } - - RETURN_NOT_OK(MakeBuilder()); + auto block_it = MakeChunkingIterator(std::move(buffer_iterator_), + MakeChunker(decode_context_.parse_options())); - auto block_it = BlockReader::MakeIterator( - std::move(buffer_iterator_), MakeChunker(parse_options_), std::move(buffer)); + bool did_read = false; while (true) { ARROW_ASSIGN_OR_RAISE(auto block, block_it.Next()); if (IsIterationEnd(block)) break; + if (!did_read) { + did_read = true; + RETURN_NOT_OK(MakeBuilder()); + } task_group_->Append( [self = shared_from_this(), block] { return self->ParseAndInsert(block); }); } + if (!did_read) { + return Status::Invalid("Empty JSON file"); + } std::shared_ptr array; RETURN_NOT_OK(builder_->Finish(&array)); @@ -229,251 +288,263 @@ class TableReaderImpl : public TableReader, private: Status MakeBuilder() { - auto type = parse_options_.explicit_schema - ? struct_(parse_options_.explicit_schema->fields()) - : struct_({}); - - auto promotion_graph = - parse_options_.unexpected_field_behavior == UnexpectedFieldBehavior::InferType - ? GetPromotionGraph() - : nullptr; - - return MakeChunkedArrayBuilder(task_group_, pool_, promotion_graph, type, &builder_); + return MakeChunkedArrayBuilder(task_group_, decode_context_.pool(), + decode_context_.promotion_graph(), + decode_context_.conversion_type(), &builder_); } - Status ParseAndInsert(const JSONBlock& block) { - ARROW_ASSIGN_OR_RAISE(auto parsed, ParseBlock(block, parse_options_, pool_)); + Status ParseAndInsert(const ChunkedBlock& block) { + ARROW_ASSIGN_OR_RAISE(auto parsed, ParseBlock(block, decode_context_.parse_options(), + decode_context_.pool())); builder_->Insert(block.index, field("", parsed->type()), parsed); return Status::OK(); } - MemoryPool* pool_; + DecodeContext decode_context_; ReadOptions read_options_; - ParseOptions parse_options_; std::shared_ptr task_group_; Iterator> buffer_iterator_; std::shared_ptr builder_; }; -Result> TableReader::Make( - MemoryPool* pool, std::shared_ptr input, - const ReadOptions& read_options, const ParseOptions& parse_options) { - std::shared_ptr ptr; - if (read_options.use_threads) { - ptr = std::make_shared(pool, read_options, parse_options, - TaskGroup::MakeThreaded(GetCpuThreadPool())); - } else { - ptr = std::make_shared(pool, read_options, parse_options, - TaskGroup::MakeSerial()); - } - RETURN_NOT_OK(ptr->Init(input)); - return ptr; -} - -Result> ParseOne(ParseOptions options, - std::shared_ptr json) { - std::unique_ptr parser; - RETURN_NOT_OK(BlockParser::Make(options, &parser)); - RETURN_NOT_OK(parser->Parse(json)); - std::shared_ptr parsed; - RETURN_NOT_OK(parser->Finish(&parsed)); - - auto type = - options.explicit_schema ? struct_(options.explicit_schema->fields()) : struct_({}); - auto promotion_graph = - options.unexpected_field_behavior == UnexpectedFieldBehavior::InferType - ? GetPromotionGraph() - : nullptr; - std::shared_ptr builder; - RETURN_NOT_OK(MakeChunkedArrayBuilder(TaskGroup::MakeSerial(), default_memory_pool(), - promotion_graph, type, &builder)); - - builder->Insert(0, field("", type), parsed); - std::shared_ptr converted_chunked; - RETURN_NOT_OK(builder->Finish(&converted_chunked)); - - return ToRecordBatch(*converted_chunked->chunk(0)); -} - -namespace { - -// Callable object for decoding a pre-chunked JSON block into a RecordBatch -class BlockDecoder { +// Callable object for parsing/converting individual JSON blocks. The class itself can be +// called concurrently but reads from the `DecodeContext` aren't synchronized +class DecodingOperator { public: - BlockDecoder(MemoryPool* pool, const ParseOptions& parse_options) - : pool_(pool), - parse_options_(parse_options), - conversion_type_(parse_options_.explicit_schema - ? struct_(parse_options_.explicit_schema->fields()) - : struct_({})), - promotion_graph_(parse_options_.unexpected_field_behavior == - UnexpectedFieldBehavior::InferType - ? GetPromotionGraph() - : nullptr) {} - - Result operator()(const JSONBlock& block) const { + explicit DecodingOperator(std::shared_ptr context) + : context_(std::move(context)) {} + + Result operator()(const ChunkedBlock& block) const { int64_t num_bytes; - ARROW_ASSIGN_OR_RAISE(auto unconverted, - ParseBlock(block, parse_options_, pool_, &num_bytes)); + ARROW_ASSIGN_OR_RAISE(auto unconverted, ParseBlock(block, context_->parse_options(), + context_->pool(), &num_bytes)); std::shared_ptr builder; - RETURN_NOT_OK(MakeChunkedArrayBuilder(TaskGroup::MakeSerial(), pool_, - promotion_graph_, conversion_type_, &builder)); + RETURN_NOT_OK(MakeChunkedArrayBuilder(TaskGroup::MakeSerial(), context_->pool(), + context_->promotion_graph(), + context_->conversion_type(), &builder)); builder->Insert(0, field("", unconverted->type()), unconverted); std::shared_ptr chunked; RETURN_NOT_OK(builder->Finish(&chunked)); ARROW_ASSIGN_OR_RAISE(auto batch, ToRecordBatch(*chunked->chunk(0))); - return DecodedBlock{std::move(batch), num_bytes, block.index}; + return DecodedBlock{std::move(batch), num_bytes}; } private: - MemoryPool* pool_; - ParseOptions parse_options_; - std::shared_ptr conversion_type_; - const PromotionGraph* promotion_graph_; + std::shared_ptr context_; }; -} // namespace +// TODO(benibus): Replace with `MakeApplyGenerator` from +// github.com/apache/arrow/pull/14269 if/when it gets merged +// +// Reads from the source and spawns fan-out decoding tasks on the given executor +AsyncGenerator MakeDecodingGenerator( + AsyncGenerator source, + std::function(const ChunkedBlock&)> decoder, + Executor* executor) { + struct State { + AsyncGenerator source; + std::function(const ChunkedBlock&)> decoder; + Executor* executor; + } state{std::move(source), std::move(decoder), executor}; + + return [state = std::make_shared(std::move(state))] { + auto options = CallbackOptions::Defaults(); + options.executor = state->executor; + options.should_schedule = ShouldSchedule::Always; + + return state->source().Then( + [state](const ChunkedBlock& block) -> Result { + if (IsIterationEnd(block)) { + return IterationEnd(); + } else { + return state->decoder(block); + } + }, + {}, options); + }; +} -class StreamingReaderImpl : public StreamingReader, - public std::enable_shared_from_this { +class StreamingReaderImpl : public StreamingReader { public: - StreamingReaderImpl(io::IOContext io_context, Executor* executor, - const ReadOptions& read_options, const ParseOptions& parse_options) - : io_context_(std::move(io_context)), - executor_(executor), - read_options_(read_options), - parse_options_(parse_options), - bytes_processed_(std::make_shared>(0)) {} - - Future<> Init(std::shared_ptr input) { - ARROW_ASSIGN_OR_RAISE(auto it, - io::MakeInputStreamIterator(input, read_options_.block_size)); - ARROW_ASSIGN_OR_RAISE(auto bg_it, - MakeBackgroundGenerator(std::move(it), io_context_.executor())); - auto buf_gen = MakeTransferredGenerator(bg_it, executor_); - // We pre-fetch the first buffer during instantiation to resolve the schema and ensure - // the stream isn't empty - return buf_gen().Then( - [self = shared_from_this(), buf_gen](const std::shared_ptr& buffer) { - return self->InitFromFirstBuffer(buffer, buf_gen); + StreamingReaderImpl(DecodedBlock first_block, AsyncGenerator source, + const std::shared_ptr& context, int max_readahead) + : first_block_(std::move(first_block)), + schema_(first_block_->record_batch->schema()), + bytes_processed_(std::make_shared>(0)) { + // Set the final schema for future invocations of the source generator + context->SetStrictSchema(schema_); + if (max_readahead > 0) { + source = MakeReadaheadGenerator(std::move(source), max_readahead); + } + generator_ = MakeMappedGenerator( + std::move(source), [counter = bytes_processed_](const DecodedBlock& out) { + counter->fetch_add(out.num_bytes); + return out.record_batch; }); } - std::shared_ptr schema() const override { - return parse_options_.explicit_schema; + static Future> MakeAsync( + AsyncGenerator chunking_gen, std::shared_ptr context, + Executor* cpu_executor, bool use_threads) { + auto source = MakeDecodingGenerator(std::move(chunking_gen), + DecodingOperator(context), cpu_executor); + const int max_readahead = use_threads ? cpu_executor->GetCapacity() : 0; + return FirstBlock(source).Then([source = std::move(source), + context = std::move(context), + max_readahead](const DecodedBlock& block) { + return std::make_shared(block, std::move(source), context, + max_readahead); + }); } + [[nodiscard]] std::shared_ptr schema() const override { return schema_; } + Status ReadNext(std::shared_ptr* out) override { - auto future = ReadNextAsync(); - auto result = future.result(); + auto result = ReadNextAsync().result(); return std::move(result).Value(out); } Future> ReadNextAsync() override { - return record_batch_gen_(); + // On the first call, return the batch we used for initialization + if (ARROW_PREDICT_FALSE(first_block_)) { + bytes_processed_->fetch_add(first_block_->num_bytes); + auto batch = std::exchange(first_block_, std::nullopt)->record_batch; + return ToFuture(std::move(batch)); + } + return generator_(); } - int64_t bytes_read() const override { return bytes_processed_->load(); } + [[nodiscard]] int64_t bytes_read() const override { return bytes_processed_->load(); } private: - Future<> InitFromFirstBuffer(const std::shared_ptr& buffer, - AsyncGenerator> buf_gen) { - if (!buffer) return Status::Invalid("Empty JSON stream"); - - // Generator for pre-chunked JSON data - auto block_gen = BlockReader::MakeGenerator(std::move(buf_gen), - MakeChunker(parse_options_), buffer); - // Decoder for the first block using the initial parse options - auto decoder = BlockDecoder(io_context_.pool(), parse_options_); - - return block_gen().Then( - [self = shared_from_this(), block_gen, decoder](JSONBlock block) -> Future<> { - // Skip any initial empty record batches so we can try to get a useful schema - int64_t skipped_bytes = 0; - ARROW_ASSIGN_OR_RAISE(auto decoded, decoder(block)); - while (!IsIterationEnd(decoded) && !decoded.record_batch->num_rows()) { - skipped_bytes = decoded.num_bytes; - auto fut = block_gen(); - ARROW_ASSIGN_OR_RAISE(block, fut.result()); + static Future FirstBlock(AsyncGenerator gen) { + // Read from the stream until we get a non-empty record batch that we can use to + // declare the schema. Along the way, accumulate the bytes read so they can be + // recorded on the first `ReadNextAsync` + auto out = std::make_shared(); + DCHECK_EQ(out->num_bytes, 0); + auto loop_body = [gen = std::move(gen), + out = std::move(out)]() -> Future> { + return gen().Then( + [out](const DecodedBlock& block) -> Result> { if (IsIterationEnd(block)) { - decoded = IterationEnd(); - } else { - ARROW_ASSIGN_OR_RAISE(decoded, decoder(block)); - decoded.num_bytes += skipped_bytes; + return Status::Invalid("Empty JSON stream"); } - } - return self->InitFromFirstDecoded(decoded, block_gen); - }); + out->num_bytes += block.num_bytes; + if (block.record_batch->num_rows() == 0) { + return Continue(); + } + out->record_batch = block.record_batch; + return Break(*out); + }); + }; + return Loop(std::move(loop_body)); } - Future<> InitFromFirstDecoded(const DecodedBlock& decoded, - AsyncGenerator block_gen) { - // End of stream and no non-empty batches were yielded, so just return empty ones - if (IsIterationEnd(decoded)) { - record_batch_gen_ = MakeEmptyGenerator>(); - parse_options_.explicit_schema = nullptr; - return Status::OK(); - } + std::optional first_block_; + std::shared_ptr schema_; + std::shared_ptr> bytes_processed_; + AsyncGenerator> generator_; +}; - // Use the schema from the first batch as the basis for all future reads. If type - // inference wasn't requested then this should be the same as the provided - // explicit_schema. Otherwise, ignore unexpected fields for future batches to ensure - // their schemas are consistent - parse_options_.explicit_schema = decoded.record_batch->schema(); - if (parse_options_.unexpected_field_behavior == UnexpectedFieldBehavior::InferType) { - parse_options_.unexpected_field_behavior = UnexpectedFieldBehavior::Ignore; - } +template +Result> MakeReentrantGenerator(AsyncGenerator source) { + struct State { + AsyncGenerator source; + std::shared_ptr thread_pool; + } state{std::move(source), nullptr}; + ARROW_ASSIGN_OR_RAISE(state.thread_pool, ThreadPool::Make(1)); + + return [state = std::make_shared(std::move(state))]() -> Future { + auto maybe_future = + state->thread_pool->Submit([state] { return state->source().result(); }); + return DeferNotOk(std::move(maybe_future)); + }; +} - // The final decoder, which uses the resolved parse options for type deduction - auto decoded_gen = MakeMappedGenerator( - std::move(block_gen), BlockDecoder(io_context_.pool(), parse_options_)); - if (read_options_.use_threads) { - decoded_gen = - MakeReadaheadGenerator(std::move(decoded_gen), executor_->GetCapacity()); - } - // Return the batch we just read on first invocation - decoded_gen = MakeGeneratorStartsWith({decoded}, std::move(decoded_gen)); - - // Compose the final generator - record_batch_gen_ = MakeMappedGenerator( - std::move(decoded_gen), - [bytes_processed = bytes_processed_](const DecodedBlock& decoded) { - bytes_processed->fetch_add(decoded.num_bytes); - return decoded.record_batch; - }); - record_batch_gen_ = - MakeCancellable(std::move(record_batch_gen_), io_context_.stop_token()); +// Compose an async-reentrant `ChunkedBlock` generator using a sequentially-accessed +// `InputStream` +Result> MakeChunkingGenerator( + std::shared_ptr stream, int32_t block_size, + std::unique_ptr chunker, Executor* io_executor, Executor* cpu_executor) { + ARROW_ASSIGN_OR_RAISE(auto source_it, + io::MakeInputStreamIterator(std::move(stream), block_size)); + ARROW_ASSIGN_OR_RAISE(auto source_gen, + MakeBackgroundGenerator(std::move(source_it), io_executor)); + source_gen = MakeTransferredGenerator(std::move(source_gen), cpu_executor); + + auto gen = MakeChunkingGenerator(std::move(source_gen), std::move(chunker)); + ARROW_ASSIGN_OR_RAISE(gen, MakeReentrantGenerator(std::move(gen))); + return gen; +} - return Status::OK(); - } +} // namespace - io::IOContext io_context_; - Executor* executor_; - ReadOptions read_options_; - ParseOptions parse_options_; - AsyncGenerator> record_batch_gen_; - std::shared_ptr> bytes_processed_; -}; +Result> TableReader::Make( + MemoryPool* pool, std::shared_ptr input, + const ReadOptions& read_options, const ParseOptions& parse_options) { + std::shared_ptr ptr; + if (read_options.use_threads) { + ptr = std::make_shared(pool, read_options, parse_options, + TaskGroup::MakeThreaded(GetCpuThreadPool())); + } else { + ptr = std::make_shared(pool, read_options, parse_options, + TaskGroup::MakeSerial()); + } + RETURN_NOT_OK(ptr->Init(input)); + return ptr; +} Future> StreamingReader::MakeAsync( - io::IOContext io_context, std::shared_ptr input, Executor* executor, - const ReadOptions& read_options, const ParseOptions& parse_options) { - auto reader = std::make_shared(io_context, executor, read_options, - parse_options); - return reader->Init(input).Then( - [reader] { return std::static_pointer_cast(reader); }); + std::shared_ptr stream, io::IOContext io_context, + Executor* cpu_executor, const ReadOptions& read_options, + const ParseOptions& parse_options) { + ARROW_ASSIGN_OR_RAISE(auto chunking_gen, + MakeChunkingGenerator(std::move(stream), read_options.block_size, + MakeChunker(parse_options), + io_context.executor(), cpu_executor)); + auto decode_context = std::make_shared(parse_options, io_context.pool()); + auto future = + StreamingReaderImpl::MakeAsync(std::move(chunking_gen), std::move(decode_context), + cpu_executor, read_options.use_threads); + return future.Then([](const std::shared_ptr& reader) { + return std::static_pointer_cast(reader); + }); } Result> StreamingReader::Make( - io::IOContext io_context, std::shared_ptr input, - const ReadOptions& read_options, const ParseOptions& parse_options) { - auto future = StreamingReader::MakeAsync(io_context, input, GetCpuThreadPool(), + std::shared_ptr stream, io::IOContext io_context, + Executor* cpu_executor, const ReadOptions& read_options, + const ParseOptions& parse_options) { + auto future = StreamingReader::MakeAsync(std::move(stream), io_context, cpu_executor, read_options, parse_options); return future.result(); } +Result> ParseOne(ParseOptions options, + std::shared_ptr json) { + DecodeContext context(std::move(options)); + + std::unique_ptr parser; + RETURN_NOT_OK(BlockParser::Make(context.parse_options(), &parser)); + RETURN_NOT_OK(parser->Parse(json)); + std::shared_ptr parsed; + RETURN_NOT_OK(parser->Finish(&parsed)); + + std::shared_ptr builder; + RETURN_NOT_OK(MakeChunkedArrayBuilder(TaskGroup::MakeSerial(), context.pool(), + context.promotion_graph(), + context.conversion_type(), &builder)); + + builder->Insert(0, field("", context.conversion_type()), parsed); + std::shared_ptr converted_chunked; + RETURN_NOT_OK(builder->Finish(&converted_chunked)); + + return ToRecordBatch(*converted_chunked->chunk(0)); +} + } // namespace json } // namespace arrow diff --git a/cpp/src/arrow/json/reader.h b/cpp/src/arrow/json/reader.h index 251793bec83..22257ddf91b 100644 --- a/cpp/src/arrow/json/reader.h +++ b/cpp/src/arrow/json/reader.h @@ -59,10 +59,18 @@ ARROW_EXPORT Result> ParseOne(ParseOptions options, /// The supplied `ParseOptions` are used to determine a schema on the first non-empty /// block. Afterwards, the schema is frozen and unexpected fields will be ignored on /// subsequent reads (unless `UnexpectedFieldBehavior::Error` was specified). +/// +/// For each block, the reader will launch its subsequent parsing/decoding task on the +/// given `cpu_executor` - potentially in parallel. If `ReadOptions::use_threads` is +/// specified, readahead will be applied to these tasks in accordance with the executor's +/// capacity. class ARROW_EXPORT StreamingReader : public RecordBatchReader { public: virtual ~StreamingReader() = default; + /// \brief Read the next `RecordBatch` asynchronously + /// + /// This function is async-reentrant (but not synchronously reentrant) virtual Future> ReadNextAsync() = 0; /// \brief Return the number of bytes which have been read and processed @@ -72,18 +80,18 @@ class ARROW_EXPORT StreamingReader : public RecordBatchReader { /// to Arrow layout) is still ongoing. [[nodiscard]] virtual int64_t bytes_read() const = 0; - /// Create a StreamingReader instance + /// \brief Create a `StreamingReader` instance asynchronously /// /// This involves some I/O as the first batch must be loaded during the creation process /// so it is returned as a future static Future> MakeAsync( - io::IOContext io_context, std::shared_ptr input, + std::shared_ptr stream, io::IOContext io_context, ::arrow::internal::Executor* cpu_executor, const ReadOptions&, const ParseOptions&); - /// Create a StreamingReader instance + /// \brief Create a `StreamingReader` instance static Result> Make( - io::IOContext io_context, std::shared_ptr input, - const ReadOptions&, const ParseOptions&); + std::shared_ptr stream, io::IOContext io_context, + ::arrow::internal::Executor* cpu_executor, const ReadOptions&, const ParseOptions&); }; } // namespace json diff --git a/cpp/src/arrow/json/reader_test.cc b/cpp/src/arrow/json/reader_test.cc index 035114e3402..731ecd7e272 100644 --- a/cpp/src/arrow/json/reader_test.cc +++ b/cpp/src/arrow/json/reader_test.cc @@ -22,12 +22,16 @@ #include #include "arrow/io/interfaces.h" +#include "arrow/io/slow.h" #include "arrow/json/options.h" #include "arrow/json/reader.h" #include "arrow/json/test_common.h" #include "arrow/table.h" +#include "arrow/testing/async_test_util.h" +#include "arrow/testing/future_util.h" #include "arrow/testing/gtest_util.h" #include "arrow/type_fwd.h" +#include "arrow/util/vector.h" namespace arrow { namespace json { @@ -321,191 +325,528 @@ TEST(ReaderTest, FailOnInvalidEOF) { } class StreamingReaderTest : public ::testing::TestWithParam { - public: - ParseOptions parse_options_ = ParseOptions::Defaults(); - ReadOptions read_options_ = DefaultReadOptions(); - io::IOContext io_context_ = io::default_io_context(); - std::shared_ptr input_; - std::shared_ptr reader_; + protected: + void SetUp() override { read_options_.use_threads = GetParam(); } + + static std::shared_ptr MakeTestStream(const std::string& str) { + auto buffer = std::make_shared(str); + return std::make_shared(std::move(buffer)); + } + // Stream with simulated latency + static std::shared_ptr MakeTestStream(const std::string& str, + double latency) { + return std::make_shared(MakeTestStream(str), latency); + } - private: - [[nodiscard]] ReadOptions DefaultReadOptions() const { - auto read_options = ReadOptions::Defaults(); - read_options.use_threads = GetParam(); - return read_options; + Result> MakeReader( + std::shared_ptr stream) { + return StreamingReader::Make(std::move(stream), io_context_, executor_, read_options_, + parse_options_); + } + template + Result> MakeReader(Args&&... args) { + return MakeReader(MakeTestStream(std::forward(args)...)); + } + + AsyncGenerator> MakeGenerator( + std::shared_ptr reader) { + return [reader = std::move(reader)] { return reader->ReadNextAsync(); }; + } + template + Result>> MakeGenerator(Args&&... args) { + ARROW_ASSIGN_OR_RAISE(auto reader, MakeReader(std::forward(args)...)); + return MakeGenerator(std::move(reader)); + } + + static void AssertReadNext(const std::shared_ptr& reader, + std::shared_ptr* out) { + ASSERT_OK(reader->ReadNext(out)); + ASSERT_FALSE(IsIterationEnd(*out)); + } + static void AssertReadEnd(const std::shared_ptr& reader) { + std::shared_ptr out; + ASSERT_OK(reader->ReadNext(&out)); + ASSERT_TRUE(IsIterationEnd(out)); } + + struct TestCase { + std::string json; + int json_size; + int block_size; + int num_rows; + int num_batches; + std::shared_ptr schema; + RecordBatchVector batches; + std::shared_ptr table; + }; + + // Creates a test case from valid JSON objects with a human-readable index field and a + // struct field of random data. `block_size_multiplier` is applied to the largest + // generated row length to determine the target block_size. i.e - higher multiplier + // means fewer batches + static TestCase GenerateTestCase(int num_rows, double block_size_multiplier = 3.0) { + FieldVector data_fields = {field("s", utf8()), field("f", float64()), + field("b", boolean())}; + FieldVector fields = {field("i", int64()), field("d", struct_({data_fields}))}; + TestCase out; + out.schema = schema(fields); + out.num_rows = num_rows; + + constexpr int kSeed = 0x432432; + std::default_random_engine engine(kSeed); + std::vector rows(num_rows); + size_t max_row_size = 1; + + auto options = GenerateOptions::Defaults(); + options.null_probability = 0; + for (int i = 0; i < num_rows; ++i) { + StringBuffer string_buffer; + Writer writer(string_buffer); + ABORT_NOT_OK(Generate(data_fields, engine, &writer, options)); + std::string json = string_buffer.GetString(); + rows[i] = Join({"{\"i\":", std::to_string(i), ",\"d\":", json, "}\n"}); + max_row_size = std::max(max_row_size, rows[i].size()); + } + + auto block_size = static_cast(max_row_size * block_size_multiplier); + // Deduce the expected record batches from the target block size. + std::vector batch_rows; + size_t pos = 0; + for (const auto& row : rows) { + pos += row.size(); + if (pos > block_size) { + out.batches.push_back( + RecordBatchFromJSON(out.schema, Join({"[", Join(batch_rows, ","), "]"}))); + batch_rows.clear(); + pos -= block_size; + } + batch_rows.push_back(row); + out.json += row; + } + if (!batch_rows.empty()) { + out.batches.push_back( + RecordBatchFromJSON(out.schema, Join({"[", Join(batch_rows, ","), "]"}))); + } + + out.json_size = static_cast(out.json.size()); + out.block_size = static_cast(block_size); + out.num_batches = static_cast(out.batches.size()); + out.table = *Table::FromRecordBatches(out.batches); + + return out; + } + + static std::string Join(const std::vector& strings, + const std::string& delim = "", bool trailing_delim = false) { + std::string out; + for (size_t i = 0; i < strings.size();) { + out += strings[i++]; + if (i != strings.size() || trailing_delim) { + out += delim; + } + } + return out; + } + + internal::Executor* executor_ = internal::GetCpuThreadPool(); + ParseOptions parse_options_ = ParseOptions::Defaults(); + ReadOptions read_options_ = ReadOptions::Defaults(); + io::IOContext io_context_ = io::default_io_context(); }; INSTANTIATE_TEST_SUITE_P(StreamingReaderTest, StreamingReaderTest, ::testing::Values(false, true)); -TEST_P(StreamingReaderTest, FailOnEmptyInput) { - ASSERT_OK(MakeStream("", &input_)); - ASSERT_RAISES( - Invalid, StreamingReader::Make(io_context_, input_, read_options_, parse_options_)); +TEST_P(StreamingReaderTest, ErrorOnEmptyStream) { + ASSERT_RAISES(Invalid, MakeReader("")); + std::string data(100, '\n'); + for (auto block_size : {25, 49, 50, 100, 200}) { + read_options_.block_size = block_size; + ASSERT_RAISES(Invalid, MakeReader(data)); + } } -TEST_P(StreamingReaderTest, FailOnParseError) { - std::string json = R"( -{"n": 10000} -{"n": "foo"})"; +TEST_P(StreamingReaderTest, PropagateChunkingErrors) { + constexpr double kIoLatency = 1e-3; + + auto test_schema = schema({field("i", int64())}); + auto bad_first_chunk = Join( + { + R"({"i": 0 })", + R"({"i": 1})", + }, + "\n"); + auto bad_middle_chunk = Join( + { + R"({"i": 0})", + R"({"i": 1})", + R"({"i": 2})", + }, + "\n"); + + read_options_.block_size = 10; + ASSERT_RAISES(Invalid, MakeReader(bad_first_chunk)); + + ASSERT_OK_AND_ASSIGN(auto reader, MakeReader(bad_middle_chunk, kIoLatency)); + + std::shared_ptr batch; + AssertReadNext(reader, &batch); + EXPECT_EQ(reader->bytes_read(), 9); + ASSERT_BATCHES_EQUAL(*RecordBatchFromJSON(test_schema, "[{\"i\":0}]"), *batch); + + ASSERT_RAISES(Invalid, reader->ReadNext(&batch)); + EXPECT_EQ(reader->bytes_read(), 9); + AssertReadEnd(reader); + AssertReadEnd(reader); + EXPECT_EQ(reader->bytes_read(), 9); +} + +TEST_P(StreamingReaderTest, PropagateParsingErrors) { + auto test_schema = schema({field("n", int64())}); + auto bad_first_block = Join( + { + R"({"n": })", + R"({"n": 10000})", + }, + "\n"); + auto bad_first_block_after_empty = Join( + { + R"( )", + R"({"n": })", + R"({"n": 10000})", + }, + "\n"); + auto bad_middle_block = Join( + { + R"({"n": 10000})", + R"({"n": 200 0})", + R"({"n": 30000})", + }, + "\n"); read_options_.block_size = 16; - ASSERT_OK(MakeStream(json, &input_)); - ASSERT_OK_AND_ASSIGN( - reader_, StreamingReader::Make(io_context_, input_, read_options_, parse_options_)); + ASSERT_RAISES(Invalid, MakeReader(bad_first_block)); + ASSERT_RAISES(Invalid, MakeReader(bad_first_block_after_empty)); + std::shared_ptr batch; - ASSERT_OK(reader_->ReadNext(&batch)); - ASSERT_EQ(14, reader_->bytes_read()); - ASSERT_RAISES(Invalid, reader_->ReadNext(&batch)); + ASSERT_OK_AND_ASSIGN(auto reader, MakeReader(bad_middle_block)); + EXPECT_EQ(reader->bytes_read(), 0); + ASSERT_NE(reader->schema(), nullptr); + EXPECT_EQ(*reader->schema(), *test_schema); + + AssertReadNext(reader, &batch); + EXPECT_EQ(reader->bytes_read(), 13); + ASSERT_BATCHES_EQUAL(*RecordBatchFromJSON(test_schema, R"([{"n":10000}])"), *batch); + + ASSERT_RAISES(Invalid, reader->ReadNext(&batch)); + EXPECT_EQ(reader->bytes_read(), 13); + AssertReadEnd(reader); + EXPECT_EQ(reader->bytes_read(), 13); } TEST_P(StreamingReaderTest, IgnoreLeadingEmptyBlocks) { - std::string json; - json.insert(json.end(), 32, '\n'); - json += R"({"b": true, "s": "foo"})"; - auto json_len = static_cast(json.length()); + std::string test_json(32, '\n'); + test_json += R"({"b": true, "s": "foo"})"; + ASSERT_EQ(test_json.length(), 55); parse_options_.explicit_schema = schema({field("b", boolean()), field("s", utf8())}); read_options_.block_size = 24; - ASSERT_OK(MakeStream(json, &input_)); - ASSERT_OK_AND_ASSIGN( - reader_, StreamingReader::Make(io_context_, input_, read_options_, parse_options_)); + ASSERT_OK_AND_ASSIGN(auto reader, MakeReader(test_json)); + EXPECT_EQ(reader->bytes_read(), 0); auto expected_schema = parse_options_.explicit_schema; - auto expected_batch = - RecordBatchFromJSON(expected_schema, R"([{"b": true, "s": "foo"}])"); - std::shared_ptr actual_batch; + auto expected_batch = RecordBatchFromJSON(expected_schema, R"([{"b":true,"s":"foo"}])"); + + ASSERT_NE(reader->schema(), nullptr); + ASSERT_EQ(*reader->schema(), *expected_schema); - ASSERT_EQ(*reader_->schema(), *expected_schema); - ASSERT_OK(reader_->ReadNext(&actual_batch)); - ASSERT_TRUE(actual_batch); - ASSERT_EQ(json_len, reader_->bytes_read()); - ASSERT_BATCHES_EQUAL(*actual_batch, *expected_batch); + std::shared_ptr actual_batch; + AssertReadNext(reader, &actual_batch); + EXPECT_EQ(reader->bytes_read(), 55); + ASSERT_BATCHES_EQUAL(*expected_batch, *actual_batch); - ASSERT_OK(reader_->ReadNext(&actual_batch)); - ASSERT_FALSE(actual_batch); + AssertReadEnd(reader); } -TEST_P(StreamingReaderTest, ExplicitSchema) { - std::string json = R"({"s": "foo", "t": "2022-01-01", "b": true})"; - auto json_len = static_cast(json.length()); +TEST_P(StreamingReaderTest, ExplicitSchemaErrorOnUnexpectedFields) { + std::string test_json = + Join({R"({"s": "foo", "t": "2022-01-01"})", R"({"s": "foo", "t": "2022-01-01"})", + R"({"s": "foo", "t": "2022-01-01", "b": true})"}, + "\n"); - parse_options_.explicit_schema = schema({field("s", utf8()), field("t", utf8())}); - parse_options_.unexpected_field_behavior = UnexpectedFieldBehavior::Ignore; + FieldVector expected_fields = {field("s", utf8())}; + std::shared_ptr expected_schema = schema(expected_fields); + std::shared_ptr expected_batch; + std::shared_ptr actual_batch; + std::shared_ptr reader; - ASSERT_OK(MakeStream(json, &input_)); - ASSERT_OK_AND_ASSIGN( - reader_, StreamingReader::Make(io_context_, input_, read_options_, parse_options_)); + parse_options_.explicit_schema = expected_schema; + parse_options_.unexpected_field_behavior = UnexpectedFieldBehavior::Error; + read_options_.block_size = 48; + ASSERT_RAISES(Invalid, MakeReader(test_json)); + + expected_fields.push_back(field("t", utf8())); + expected_schema = schema(expected_fields); + expected_batch = + RecordBatchFromJSON(expected_schema, R"([{"s":"foo","t":"2022-01-01"}])"); + + parse_options_.explicit_schema = expected_schema; + ASSERT_OK_AND_ASSIGN(reader, MakeReader(test_json)); + ASSERT_NE(reader->schema(), nullptr); + ASSERT_EQ(*reader->schema(), *expected_schema); + + AssertReadNext(reader, &actual_batch); + ASSERT_BATCHES_EQUAL(*expected_batch, *actual_batch); + EXPECT_EQ(reader->bytes_read(), 32); + + AssertReadNext(reader, &actual_batch); + ASSERT_BATCHES_EQUAL(*expected_batch, *actual_batch); + EXPECT_EQ(reader->bytes_read(), 64); + + ASSERT_RAISES(Invalid, reader->ReadNext(&actual_batch)); + EXPECT_EQ(reader->bytes_read(), 64); + AssertReadEnd(reader); +} - auto expected_schema = parse_options_.explicit_schema; - auto expected_batch = - RecordBatchFromJSON(expected_schema, R"([{"s": "foo", "t": "2022-01-01"}])"); - std::shared_ptr actual_batch; +TEST_P(StreamingReaderTest, ExplicitSchemaIgnoreUnexpectedFields) { + std::string test_json = + Join({R"({"s": "foo", "u": "2022-01-01"})", R"({"s": "foo", "t": "2022-01-01"})", + R"({"s": "foo", "t": "2022-01-01", "b": true})"}, + "\n"); - ASSERT_EQ(*reader_->schema(), *expected_schema); - ASSERT_OK(reader_->ReadNext(&actual_batch)); - ASSERT_TRUE(actual_batch); - ASSERT_EQ(json_len, reader_->bytes_read()); - ASSERT_BATCHES_EQUAL(*actual_batch, *expected_batch); + FieldVector expected_fields = {field("s", utf8()), field("t", utf8())}; + std::shared_ptr expected_schema = schema(expected_fields); + std::shared_ptr expected_batch; + std::shared_ptr actual_batch; + std::shared_ptr reader; - parse_options_.unexpected_field_behavior = UnexpectedFieldBehavior::Error; - ASSERT_OK(MakeStream(json, &input_)); - ASSERT_RAISES( - Invalid, StreamingReader::Make(io_context_, input_, read_options_, parse_options_)); + parse_options_.explicit_schema = expected_schema; + parse_options_.unexpected_field_behavior = UnexpectedFieldBehavior::Ignore; + read_options_.block_size = 48; + + ASSERT_OK_AND_ASSIGN(reader, MakeReader(test_json)); + ASSERT_NE(reader->schema(), nullptr); + ASSERT_EQ(*reader->schema(), *expected_schema); + + expected_batch = RecordBatchFromJSON(expected_schema, R"([{"s":"foo","t":null}])"); + AssertReadNext(reader, &actual_batch); + ASSERT_BATCHES_EQUAL(*expected_batch, *actual_batch); + EXPECT_EQ(reader->bytes_read(), 32); + + expected_batch = + RecordBatchFromJSON(expected_schema, R"([{"s":"foo","t":"2022-01-01"}])"); + AssertReadNext(reader, &actual_batch); + ASSERT_BATCHES_EQUAL(*expected_batch, *actual_batch); + EXPECT_EQ(reader->bytes_read(), 64); + + AssertReadNext(reader, &actual_batch); + ASSERT_BATCHES_EQUAL(*expected_batch, *actual_batch); + EXPECT_EQ(reader->bytes_read(), 106); + AssertReadEnd(reader); } TEST_P(StreamingReaderTest, InferredSchema) { - std::string json = R"( -{"a": 0, "b": "foo" } -{"a": 1, "c": true } -{"a": 2, "d": "2022-01-01"} -)"; + auto test_json = Join( + { + R"({"a": 0, "b": "foo" })", + R"({"a": 1, "c": true })", + R"({"a": 2, "d": "2022-01-01"})", + }, + "\n", true); + std::shared_ptr reader; std::shared_ptr expected_schema; std::shared_ptr expected_batch; std::shared_ptr actual_batch; FieldVector fields = {field("a", int64()), field("b", utf8())}; - parse_options_.unexpected_field_behavior = UnexpectedFieldBehavior::InferType; + parse_options_.explicit_schema = nullptr; // Schema derived from the first line - read_options_.block_size = 32; - ASSERT_OK(MakeStream(json, &input_)); - ASSERT_OK_AND_ASSIGN( - reader_, StreamingReader::Make(io_context_, input_, read_options_, parse_options_)); - expected_schema = schema(fields); - ASSERT_EQ(*reader_->schema(), *expected_schema); + + read_options_.block_size = 32; + ASSERT_OK_AND_ASSIGN(reader, MakeReader(test_json)); + ASSERT_NE(reader->schema(), nullptr); + ASSERT_EQ(*reader->schema(), *expected_schema); expected_batch = RecordBatchFromJSON(expected_schema, R"([{"a": 0, "b": "foo"}])"); - ASSERT_OK(reader_->ReadNext(&actual_batch)); - ASSERT_TRUE(actual_batch); - ASSERT_EQ(29, reader_->bytes_read()); - ASSERT_BATCHES_EQUAL(*actual_batch, *expected_batch); + AssertReadNext(reader, &actual_batch); + EXPECT_EQ(reader->bytes_read(), 28); + ASSERT_BATCHES_EQUAL(*expected_batch, *actual_batch); expected_batch = RecordBatchFromJSON(expected_schema, R"([{"a": 1, "b": null}])"); - ASSERT_OK(reader_->ReadNext(&actual_batch)); - ASSERT_TRUE(actual_batch); - ASSERT_EQ(57, reader_->bytes_read()); - ASSERT_BATCHES_EQUAL(*actual_batch, *expected_batch); + AssertReadNext(reader, &actual_batch); + EXPECT_EQ(reader->bytes_read(), 56); + ASSERT_BATCHES_EQUAL(*expected_batch, *actual_batch); expected_batch = RecordBatchFromJSON(expected_schema, R"([{"a": 2, "b": null}])"); - ASSERT_OK(reader_->ReadNext(&actual_batch)); - ASSERT_TRUE(actual_batch); - ASSERT_EQ(85, reader_->bytes_read()); - ASSERT_BATCHES_EQUAL(*actual_batch, *expected_batch); + AssertReadNext(reader, &actual_batch); + EXPECT_EQ(reader->bytes_read(), 84); + ASSERT_BATCHES_EQUAL(*expected_batch, *actual_batch); // Schema derived from the first 2 lines - read_options_.block_size = 64; - ASSERT_OK(MakeStream(json, &input_)); - ASSERT_OK_AND_ASSIGN( - reader_, StreamingReader::Make(io_context_, input_, read_options_, parse_options_)); - fields.push_back(field("c", boolean())); expected_schema = schema(fields); - ASSERT_EQ(*reader_->schema(), *expected_schema); + + read_options_.block_size = 64; + ASSERT_OK_AND_ASSIGN(reader, MakeReader(test_json)); + ASSERT_NE(reader->schema(), nullptr); + ASSERT_EQ(*reader->schema(), *expected_schema); expected_batch = RecordBatchFromJSON(expected_schema, R"([ {"a": 0, "b": "foo", "c": null}, {"a": 1, "b": null, "c": true} ])"); - ASSERT_OK(reader_->ReadNext(&actual_batch)); - ASSERT_TRUE(actual_batch); - ASSERT_EQ(57, reader_->bytes_read()); - ASSERT_BATCHES_EQUAL(*actual_batch, *expected_batch); + AssertReadNext(reader, &actual_batch); + EXPECT_EQ(reader->bytes_read(), 56); + ASSERT_BATCHES_EQUAL(*expected_batch, *actual_batch); expected_batch = RecordBatchFromJSON(expected_schema, R"([ {"a": 2, "b": null, "c": null} ])"); - ASSERT_OK(reader_->ReadNext(&actual_batch)); - ASSERT_TRUE(actual_batch); - ASSERT_EQ(85, reader_->bytes_read()); - ASSERT_BATCHES_EQUAL(*actual_batch, *expected_batch); + AssertReadNext(reader, &actual_batch); + EXPECT_EQ(reader->bytes_read(), 84); + ASSERT_BATCHES_EQUAL(*expected_batch, *actual_batch); // Schema derived from all 3 lines - read_options_.block_size = 96; - ASSERT_OK(MakeStream(json, &input_)); - ASSERT_OK_AND_ASSIGN( - reader_, StreamingReader::Make(io_context_, input_, read_options_, parse_options_)); - fields.push_back(field("d", timestamp(TimeUnit::SECOND))); expected_schema = schema(fields); - ASSERT_EQ(*reader_->schema(), *expected_schema); + + read_options_.block_size = 96; + ASSERT_OK_AND_ASSIGN(reader, MakeReader(test_json)); + ASSERT_NE(reader->schema(), nullptr); + ASSERT_EQ(*reader->schema(), *expected_schema); expected_batch = RecordBatchFromJSON(expected_schema, R"([ {"a": 0, "b": "foo", "c": null, "d": null}, {"a": 1, "b": null, "c": true, "d": null}, {"a": 2, "b": null, "c": null, "d": "2022-01-01"} ])"); - ASSERT_OK(reader_->ReadNext(&actual_batch)); - ASSERT_TRUE(actual_batch); - ASSERT_EQ(85, reader_->bytes_read()); - ASSERT_BATCHES_EQUAL(*actual_batch, *expected_batch); + AssertReadNext(reader, &actual_batch); + EXPECT_EQ(reader->bytes_read(), 84); + ASSERT_BATCHES_EQUAL(*expected_batch, *actual_batch); + + AssertReadEnd(reader); +} + +TEST_P(StreamingReaderTest, AsyncReentrancy) { + constexpr int kNumRows = 16; + constexpr double kIoLatency = 1e-2; + + auto expected = GenerateTestCase(kNumRows); + parse_options_.explicit_schema = expected.schema; + parse_options_.unexpected_field_behavior = UnexpectedFieldBehavior::Error; + read_options_.block_size = expected.block_size; + + std::vector>> futures(expected.num_batches + 1); + ASSERT_OK_AND_ASSIGN(auto reader, MakeReader(expected.json, kIoLatency)); + EXPECT_EQ(reader->bytes_read(), 0); + for (auto& future : futures) { + future = reader->ReadNextAsync(); + } + + ASSERT_FINISHES_OK_AND_ASSIGN(auto results, All(std::move(futures))); + EXPECT_EQ(reader->bytes_read(), expected.json_size); + ASSERT_OK_AND_ASSIGN(auto batches, internal::UnwrapOrRaise(std::move(results))); + EXPECT_EQ(batches.back(), nullptr); + batches.pop_back(); + ASSERT_OK_AND_ASSIGN(auto table, Table::FromRecordBatches(batches)); + ASSERT_TABLES_EQUAL(*expected.table, *table); +} + +TEST_P(StreamingReaderTest, FuturesOutliveReader) { + constexpr int kNumRows = 16; + constexpr double kIoLatency = 1e-2; + + auto expected = GenerateTestCase(kNumRows); + parse_options_.explicit_schema = expected.schema; + parse_options_.unexpected_field_behavior = UnexpectedFieldBehavior::Error; + read_options_.block_size = expected.block_size; + + auto stream = MakeTestStream(expected.json, kIoLatency); + std::vector>> futures(expected.num_batches); + std::weak_ptr weak_reader; + { + ASSERT_OK_AND_ASSIGN(auto reader, MakeReader(stream)); + weak_reader = reader; + EXPECT_EQ(reader->bytes_read(), 0); + for (auto& future : futures) { + future = reader->ReadNextAsync(); + } + } + + auto all_future = All(std::move(futures)); + AssertNotFinished(all_future); + EXPECT_EQ(weak_reader.use_count(), 0); + ASSERT_FINISHES_OK_AND_ASSIGN(auto results, all_future); + ASSERT_OK_AND_ASSIGN(auto batches, internal::UnwrapOrRaise(std::move(results))); + ASSERT_OK_AND_ASSIGN(auto table, Table::FromRecordBatches(batches)); + ASSERT_TABLES_EQUAL(*expected.table, *table); +} + +TEST_P(StreamingReaderTest, NestedParallelism) { + constexpr int kNumRows = 16; + + auto expected = GenerateTestCase(kNumRows); + parse_options_.explicit_schema = expected.schema; + parse_options_.unexpected_field_behavior = UnexpectedFieldBehavior::Error; + read_options_.block_size = expected.block_size; - ASSERT_OK(reader_->ReadNext(&actual_batch)); - ASSERT_FALSE(actual_batch); + AsyncGenerator> generator; + auto task = [&generator] { return CollectAsyncGenerator(generator).result(); }; + + ASSERT_OK_AND_ASSIGN(auto thread_pool, internal::ThreadPool::Make(1)); + ASSERT_OK_AND_ASSIGN(generator, MakeGenerator(expected.json)); + ASSERT_OK_AND_ASSIGN(auto batches_future, thread_pool->Submit(task)); + + ASSERT_FINISHES_OK_AND_ASSIGN(auto batches, batches_future); + ASSERT_EQ(batches.size(), expected.batches.size()); + ASSERT_OK_AND_ASSIGN(auto table, Table::FromRecordBatches(batches)); + ASSERT_TABLES_EQUAL(*expected.table, *table); +} + +TEST_P(StreamingReaderTest, StressBufferedReads) { + constexpr int kNumRows = 500; + + auto expected = GenerateTestCase(kNumRows); + parse_options_.explicit_schema = expected.schema; + parse_options_.unexpected_field_behavior = UnexpectedFieldBehavior::Error; + read_options_.block_size = expected.block_size; + + std::vector>> futures(expected.num_batches); + ASSERT_OK_AND_ASSIGN(auto reader, MakeReader(expected.json)); + EXPECT_EQ(reader->bytes_read(), 0); + for (auto& future : futures) { + future = reader->ReadNextAsync(); + } + + ASSERT_FINISHES_OK_AND_ASSIGN(auto results, All(std::move(futures))); + ASSERT_OK_AND_ASSIGN(auto batches, internal::UnwrapOrRaise(results)); + ASSERT_OK_AND_ASSIGN(auto table, Table::FromRecordBatches(batches)); + ASSERT_TABLES_EQUAL(*expected.table, *table); +} + +TEST_P(StreamingReaderTest, StressSharedIoAndCpuExecutor) { + constexpr int kNumRows = 500; + constexpr double kIoLatency = 1e-4; + + auto expected = GenerateTestCase(kNumRows); + parse_options_.explicit_schema = expected.schema; + parse_options_.unexpected_field_behavior = UnexpectedFieldBehavior::Error; + read_options_.block_size = expected.block_size; + + // Force the serial -> parallel pipeline to contend for a single thread + ASSERT_OK_AND_ASSIGN(auto thread_pool, internal::ThreadPool::Make(1)); + io_context_ = io::IOContext(thread_pool.get()); + executor_ = thread_pool.get(); + + ASSERT_OK_AND_ASSIGN(auto generator, MakeGenerator(expected.json, kIoLatency)); + ASSERT_FINISHES_OK_AND_ASSIGN(auto batches, CollectAsyncGenerator(generator)); + ASSERT_EQ(batches.size(), expected.batches.size()); + ASSERT_OK_AND_ASSIGN(auto table, Table::FromRecordBatches(batches)); + ASSERT_TABLES_EQUAL(*expected.table, *table); } } // namespace json From 65997862c2c2b74e5ff8d4367fabb7fc4911d1eb Mon Sep 17 00:00:00 2001 From: benibus Date: Mon, 21 Nov 2022 10:34:48 -0500 Subject: [PATCH 03/18] Use RecordBatch::FromStructArray --- cpp/src/arrow/json/reader.cc | 16 ++-------------- 1 file changed, 2 insertions(+), 14 deletions(-) diff --git a/cpp/src/arrow/json/reader.cc b/cpp/src/arrow/json/reader.cc index 17d9f570e89..925ca932433 100644 --- a/cpp/src/arrow/json/reader.cc +++ b/cpp/src/arrow/json/reader.cc @@ -137,18 +137,6 @@ class DecodeContext { MemoryPool* pool_; }; -Result> ToRecordBatch(const StructArray& converted) { - std::vector> columns; - columns.reserve(converted.num_fields()); - for (const auto& f : converted.fields()) columns.push_back(f); - return RecordBatch::Make(schema(converted.type()->fields()), converted.length(), - std::move(columns)); -} - -auto ToRecordBatch(const Array& converted) { - return ToRecordBatch(checked_cast(converted)); -} - Result> ParseBlock(const ChunkedBlock& block, const ParseOptions& parse_options, MemoryPool* pool, int64_t* out_size = nullptr) { @@ -327,7 +315,7 @@ class DecodingOperator { std::shared_ptr chunked; RETURN_NOT_OK(builder->Finish(&chunked)); - ARROW_ASSIGN_OR_RAISE(auto batch, ToRecordBatch(*chunked->chunk(0))); + ARROW_ASSIGN_OR_RAISE(auto batch, RecordBatch::FromStructArray(chunked->chunk(0))); return DecodedBlock{std::move(batch), num_bytes}; } @@ -543,7 +531,7 @@ Result> ParseOne(ParseOptions options, std::shared_ptr converted_chunked; RETURN_NOT_OK(builder->Finish(&converted_chunked)); - return ToRecordBatch(*converted_chunked->chunk(0)); + return RecordBatch::FromStructArray(converted_chunked->chunk(0)); } } // namespace json From 2084b6c04485a76d9d7f4973b495879e086c2bb8 Mon Sep 17 00:00:00 2001 From: Ben Harkins <60872452+benibus@users.noreply.github.com> Date: Mon, 21 Nov 2022 11:43:21 -0500 Subject: [PATCH 04/18] Apply docs suggestion Co-authored-by: Will Jones --- cpp/src/arrow/json/reader.h | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/cpp/src/arrow/json/reader.h b/cpp/src/arrow/json/reader.h index 22257ddf91b..8ac950c1b36 100644 --- a/cpp/src/arrow/json/reader.h +++ b/cpp/src/arrow/json/reader.h @@ -56,8 +56,9 @@ ARROW_EXPORT Result> ParseOne(ParseOptions options, /// `ReadOptions::block_size`). Each block is converted to a `RecordBatch`. Yielded /// batches have a consistent schema but may differ in row count. /// -/// The supplied `ParseOptions` are used to determine a schema on the first non-empty -/// block. Afterwards, the schema is frozen and unexpected fields will be ignored on +/// The supplied `ParseOptions` are used to determine a schema, based either on a +/// provided explicit schema or inferred from the first non-empty block. +/// Afterwards, the schema is frozen and unexpected fields will be ignored on /// subsequent reads (unless `UnexpectedFieldBehavior::Error` was specified). /// /// For each block, the reader will launch its subsequent parsing/decoding task on the From 431fe62586dbc36f0b98ebf5b019fe99857bde64 Mon Sep 17 00:00:00 2001 From: Ben Harkins <60872452+benibus@users.noreply.github.com> Date: Wed, 30 Nov 2022 15:57:14 -0500 Subject: [PATCH 05/18] Apply some suggestions Co-authored-by: Antoine Pitrou --- cpp/src/arrow/json/reader.cc | 4 +++- cpp/src/arrow/json/reader.h | 3 ++- 2 files changed, 5 insertions(+), 2 deletions(-) diff --git a/cpp/src/arrow/json/reader.cc b/cpp/src/arrow/json/reader.cc index 925ca932433..8946334c5e0 100644 --- a/cpp/src/arrow/json/reader.cc +++ b/cpp/src/arrow/json/reader.cc @@ -185,7 +185,7 @@ class ChunkingTransformer { Result> operator()(std::shared_ptr next_buffer) { if (!buffer_) { if (ARROW_PREDICT_TRUE(!next_buffer)) { - partial_ = nullptr; + DCHECK_EQ(partial_, nullptr) << "Logic error: non-null partial with null buffer"; return TransformFinish(); } partial_ = std::make_shared(""); @@ -341,6 +341,8 @@ AsyncGenerator MakeDecodingGenerator( return [state = std::make_shared(std::move(state))] { auto options = CallbackOptions::Defaults(); options.executor = state->executor; + // Since the decode step is heavy we want to schedule it as + // a separate task so as to maximize task distribution accross CPU cores options.should_schedule = ShouldSchedule::Always; return state->source().Then( diff --git a/cpp/src/arrow/json/reader.h b/cpp/src/arrow/json/reader.h index 8ac950c1b36..c8362610f64 100644 --- a/cpp/src/arrow/json/reader.h +++ b/cpp/src/arrow/json/reader.h @@ -19,12 +19,13 @@ #include -#include "arrow/io/interfaces.h" +#include "arrow/io/type_fwd.h" #include "arrow/json/options.h" #include "arrow/record_batch.h" #include "arrow/result.h" #include "arrow/status.h" #include "arrow/util/macros.h" +#include "arrow/util/type_fwd.h" #include "arrow/util/visibility.h" namespace arrow { From a5f729299b159df96839493abb93bec2728083a6 Mon Sep 17 00:00:00 2001 From: benibus Date: Thu, 1 Dec 2022 12:11:50 -0500 Subject: [PATCH 06/18] Address most points from code review --- cpp/src/arrow/json/reader.cc | 104 ++++++++++++++---------------- cpp/src/arrow/json/reader_test.cc | 81 ++++++----------------- 2 files changed, 70 insertions(+), 115 deletions(-) diff --git a/cpp/src/arrow/json/reader.cc b/cpp/src/arrow/json/reader.cc index 8946334c5e0..c43f6be5813 100644 --- a/cpp/src/arrow/json/reader.cc +++ b/cpp/src/arrow/json/reader.cc @@ -118,7 +118,7 @@ class DecodeContext { void SetStrictSchema(std::shared_ptr explicit_schema) { auto unexpected_field_behavior = parse_options_.unexpected_field_behavior; if (unexpected_field_behavior == UnexpectedFieldBehavior::InferType) { - unexpected_field_behavior = UnexpectedFieldBehavior::Ignore; + unexpected_field_behavior = UnexpectedFieldBehavior::Error; } SetSchema(std::move(explicit_schema), unexpected_field_behavior); } @@ -226,6 +226,8 @@ Iterator MakeChunkingIterator(Iterator> so ChunkingTransformer::Make(std::forward(args)...)); } +// NOTE: Not reentrant. Incoming buffers are processed sequentially and the transformer's +// internal state gets updated on each call. template AsyncGenerator MakeChunkingGenerator( AsyncGenerator> source, Args&&... args) { @@ -357,6 +359,23 @@ AsyncGenerator MakeDecodingGenerator( }; } +// Adds async-reentrancy to `source` by submitting tasks to a single-threaded executor +// (FIFO order) - ensuring, at most, one future is pending at a time +template +Result> MakeReentrantGenerator(AsyncGenerator source) { + struct State { + AsyncGenerator source; + std::shared_ptr thread_pool; + } state{std::move(source), nullptr}; + ARROW_ASSIGN_OR_RAISE(state.thread_pool, ThreadPool::Make(1)); + + return [state = std::make_shared(std::move(state))]() -> Future { + auto maybe_future = + state->thread_pool->Submit([state] { return state->source().result(); }); + return DeferNotOk(std::move(maybe_future)); + }; +} + class StreamingReaderImpl : public StreamingReader { public: StreamingReaderImpl(DecodedBlock first_block, AsyncGenerator source, @@ -377,17 +396,29 @@ class StreamingReaderImpl : public StreamingReader { } static Future> MakeAsync( - AsyncGenerator chunking_gen, std::shared_ptr context, - Executor* cpu_executor, bool use_threads) { - auto source = MakeDecodingGenerator(std::move(chunking_gen), - DecodingOperator(context), cpu_executor); - const int max_readahead = use_threads ? cpu_executor->GetCapacity() : 0; - return FirstBlock(source).Then([source = std::move(source), - context = std::move(context), - max_readahead](const DecodedBlock& block) { - return std::make_shared(block, std::move(source), context, - max_readahead); - }); + std::shared_ptr context, std::shared_ptr stream, + io::IOContext io_context, Executor* cpu_executor, const ReadOptions& read_options) { + ARROW_ASSIGN_OR_RAISE( + auto buffer_it, + io::MakeInputStreamIterator(std::move(stream), read_options.block_size)); + ARROW_ASSIGN_OR_RAISE( + auto buffer_gen, + MakeBackgroundGenerator(std::move(buffer_it), io_context.executor())); + buffer_gen = MakeTransferredGenerator(std::move(buffer_gen), cpu_executor); + + auto chunking_gen = MakeChunkingGenerator(std::move(buffer_gen), + MakeChunker(context->parse_options())); + ARROW_ASSIGN_OR_RAISE(chunking_gen, MakeReentrantGenerator(std::move(chunking_gen))); + + auto decoding_gen = MakeDecodingGenerator(std::move(chunking_gen), + DecodingOperator(context), cpu_executor); + const int max_readahead = read_options.use_threads ? cpu_executor->GetCapacity() : 0; + return FirstBlock(decoding_gen) + .Then([source = std::move(decoding_gen), context = std::move(context), + max_readahead](const DecodedBlock& block) { + return std::make_shared(block, std::move(source), context, + max_readahead); + }); } [[nodiscard]] std::shared_ptr schema() const override { return schema_; } @@ -414,10 +445,9 @@ class StreamingReaderImpl : public StreamingReader { // Read from the stream until we get a non-empty record batch that we can use to // declare the schema. Along the way, accumulate the bytes read so they can be // recorded on the first `ReadNextAsync` - auto out = std::make_shared(); - DCHECK_EQ(out->num_bytes, 0); - auto loop_body = [gen = std::move(gen), - out = std::move(out)]() -> Future> { + auto loop_body = + [gen = std::move(gen), + out = std::make_shared()]() -> Future> { return gen().Then( [out](const DecodedBlock& block) -> Result> { if (IsIterationEnd(block)) { @@ -440,37 +470,6 @@ class StreamingReaderImpl : public StreamingReader { AsyncGenerator> generator_; }; -template -Result> MakeReentrantGenerator(AsyncGenerator source) { - struct State { - AsyncGenerator source; - std::shared_ptr thread_pool; - } state{std::move(source), nullptr}; - ARROW_ASSIGN_OR_RAISE(state.thread_pool, ThreadPool::Make(1)); - - return [state = std::make_shared(std::move(state))]() -> Future { - auto maybe_future = - state->thread_pool->Submit([state] { return state->source().result(); }); - return DeferNotOk(std::move(maybe_future)); - }; -} - -// Compose an async-reentrant `ChunkedBlock` generator using a sequentially-accessed -// `InputStream` -Result> MakeChunkingGenerator( - std::shared_ptr stream, int32_t block_size, - std::unique_ptr chunker, Executor* io_executor, Executor* cpu_executor) { - ARROW_ASSIGN_OR_RAISE(auto source_it, - io::MakeInputStreamIterator(std::move(stream), block_size)); - ARROW_ASSIGN_OR_RAISE(auto source_gen, - MakeBackgroundGenerator(std::move(source_it), io_executor)); - source_gen = MakeTransferredGenerator(std::move(source_gen), cpu_executor); - - auto gen = MakeChunkingGenerator(std::move(source_gen), std::move(chunker)); - ARROW_ASSIGN_OR_RAISE(gen, MakeReentrantGenerator(std::move(gen))); - return gen; -} - } // namespace Result> TableReader::Make( @@ -492,14 +491,9 @@ Future> StreamingReader::MakeAsync( std::shared_ptr stream, io::IOContext io_context, Executor* cpu_executor, const ReadOptions& read_options, const ParseOptions& parse_options) { - ARROW_ASSIGN_OR_RAISE(auto chunking_gen, - MakeChunkingGenerator(std::move(stream), read_options.block_size, - MakeChunker(parse_options), - io_context.executor(), cpu_executor)); - auto decode_context = std::make_shared(parse_options, io_context.pool()); - auto future = - StreamingReaderImpl::MakeAsync(std::move(chunking_gen), std::move(decode_context), - cpu_executor, read_options.use_threads); + auto future = StreamingReaderImpl::MakeAsync( + std::make_shared(parse_options, io_context.pool()), + std::move(stream), io_context, cpu_executor, read_options); return future.Then([](const std::shared_ptr& reader) { return std::static_pointer_cast(reader); }); diff --git a/cpp/src/arrow/json/reader_test.cc b/cpp/src/arrow/json/reader_test.cc index 731ecd7e272..967145710cb 100644 --- a/cpp/src/arrow/json/reader_test.cc +++ b/cpp/src/arrow/json/reader_test.cc @@ -531,8 +531,7 @@ TEST_P(StreamingReaderTest, PropagateParsingErrors) { std::shared_ptr batch; ASSERT_OK_AND_ASSIGN(auto reader, MakeReader(bad_middle_block)); EXPECT_EQ(reader->bytes_read(), 0); - ASSERT_NE(reader->schema(), nullptr); - EXPECT_EQ(*reader->schema(), *test_schema); + AssertSchemaEqual(reader->schema(), test_schema); AssertReadNext(reader, &batch); EXPECT_EQ(reader->bytes_read(), 13); @@ -557,8 +556,7 @@ TEST_P(StreamingReaderTest, IgnoreLeadingEmptyBlocks) { auto expected_schema = parse_options_.explicit_schema; auto expected_batch = RecordBatchFromJSON(expected_schema, R"([{"b":true,"s":"foo"}])"); - ASSERT_NE(reader->schema(), nullptr); - ASSERT_EQ(*reader->schema(), *expected_schema); + AssertSchemaEqual(reader->schema(), expected_schema); std::shared_ptr actual_batch; AssertReadNext(reader, &actual_batch); @@ -592,8 +590,7 @@ TEST_P(StreamingReaderTest, ExplicitSchemaErrorOnUnexpectedFields) { parse_options_.explicit_schema = expected_schema; ASSERT_OK_AND_ASSIGN(reader, MakeReader(test_json)); - ASSERT_NE(reader->schema(), nullptr); - ASSERT_EQ(*reader->schema(), *expected_schema); + AssertSchemaEqual(reader->schema(), expected_schema); AssertReadNext(reader, &actual_batch); ASSERT_BATCHES_EQUAL(*expected_batch, *actual_batch); @@ -625,8 +622,7 @@ TEST_P(StreamingReaderTest, ExplicitSchemaIgnoreUnexpectedFields) { read_options_.block_size = 48; ASSERT_OK_AND_ASSIGN(reader, MakeReader(test_json)); - ASSERT_NE(reader->schema(), nullptr); - ASSERT_EQ(*reader->schema(), *expected_schema); + AssertSchemaEqual(reader->schema(), expected_schema); expected_batch = RecordBatchFromJSON(expected_schema, R"([{"s":"foo","t":null}])"); AssertReadNext(reader, &actual_batch); @@ -668,23 +664,14 @@ TEST_P(StreamingReaderTest, InferredSchema) { read_options_.block_size = 32; ASSERT_OK_AND_ASSIGN(reader, MakeReader(test_json)); - ASSERT_NE(reader->schema(), nullptr); - ASSERT_EQ(*reader->schema(), *expected_schema); + AssertSchemaEqual(reader->schema(), expected_schema); expected_batch = RecordBatchFromJSON(expected_schema, R"([{"a": 0, "b": "foo"}])"); AssertReadNext(reader, &actual_batch); EXPECT_EQ(reader->bytes_read(), 28); ASSERT_BATCHES_EQUAL(*expected_batch, *actual_batch); - expected_batch = RecordBatchFromJSON(expected_schema, R"([{"a": 1, "b": null}])"); - AssertReadNext(reader, &actual_batch); - EXPECT_EQ(reader->bytes_read(), 56); - ASSERT_BATCHES_EQUAL(*expected_batch, *actual_batch); - - expected_batch = RecordBatchFromJSON(expected_schema, R"([{"a": 2, "b": null}])"); - AssertReadNext(reader, &actual_batch); - EXPECT_EQ(reader->bytes_read(), 84); - ASSERT_BATCHES_EQUAL(*expected_batch, *actual_batch); + ASSERT_RAISES(Invalid, reader->ReadNext(&actual_batch)); // Schema derived from the first 2 lines fields.push_back(field("c", boolean())); @@ -692,8 +679,7 @@ TEST_P(StreamingReaderTest, InferredSchema) { read_options_.block_size = 64; ASSERT_OK_AND_ASSIGN(reader, MakeReader(test_json)); - ASSERT_NE(reader->schema(), nullptr); - ASSERT_EQ(*reader->schema(), *expected_schema); + AssertSchemaEqual(reader->schema(), expected_schema); expected_batch = RecordBatchFromJSON(expected_schema, R"([ {"a": 0, "b": "foo", "c": null}, @@ -703,12 +689,7 @@ TEST_P(StreamingReaderTest, InferredSchema) { EXPECT_EQ(reader->bytes_read(), 56); ASSERT_BATCHES_EQUAL(*expected_batch, *actual_batch); - expected_batch = RecordBatchFromJSON(expected_schema, R"([ - {"a": 2, "b": null, "c": null} - ])"); - AssertReadNext(reader, &actual_batch); - EXPECT_EQ(reader->bytes_read(), 84); - ASSERT_BATCHES_EQUAL(*expected_batch, *actual_batch); + ASSERT_RAISES(Invalid, reader->ReadNext(&actual_batch)); // Schema derived from all 3 lines fields.push_back(field("d", timestamp(TimeUnit::SECOND))); @@ -716,8 +697,7 @@ TEST_P(StreamingReaderTest, InferredSchema) { read_options_.block_size = 96; ASSERT_OK_AND_ASSIGN(reader, MakeReader(test_json)); - ASSERT_NE(reader->schema(), nullptr); - ASSERT_EQ(*reader->schema(), *expected_schema); + AssertSchemaEqual(reader->schema(), expected_schema); expected_batch = RecordBatchFromJSON(expected_schema, R"([ {"a": 0, "b": "foo", "c": null, "d": null}, @@ -740,7 +720,7 @@ TEST_P(StreamingReaderTest, AsyncReentrancy) { parse_options_.unexpected_field_behavior = UnexpectedFieldBehavior::Error; read_options_.block_size = expected.block_size; - std::vector>> futures(expected.num_batches + 1); + std::vector>> futures(expected.num_batches + 2); ASSERT_OK_AND_ASSIGN(auto reader, MakeReader(expected.json, kIoLatency)); EXPECT_EQ(reader->bytes_read(), 0); for (auto& future : futures) { @@ -750,8 +730,9 @@ TEST_P(StreamingReaderTest, AsyncReentrancy) { ASSERT_FINISHES_OK_AND_ASSIGN(auto results, All(std::move(futures))); EXPECT_EQ(reader->bytes_read(), expected.json_size); ASSERT_OK_AND_ASSIGN(auto batches, internal::UnwrapOrRaise(std::move(results))); - EXPECT_EQ(batches.back(), nullptr); - batches.pop_back(); + batches.erase(std::remove(batches.begin(), batches.end(), nullptr), batches.end()); + EXPECT_EQ(batches.size(), static_cast(expected.num_batches)); + ASSERT_OK_AND_ASSIGN(auto table, Table::FromRecordBatches(batches)); ASSERT_TABLES_EQUAL(*expected.table, *table); } @@ -766,43 +747,20 @@ TEST_P(StreamingReaderTest, FuturesOutliveReader) { read_options_.block_size = expected.block_size; auto stream = MakeTestStream(expected.json, kIoLatency); - std::vector>> futures(expected.num_batches); - std::weak_ptr weak_reader; + std::vector>> futures(expected.num_batches + 2); { ASSERT_OK_AND_ASSIGN(auto reader, MakeReader(stream)); - weak_reader = reader; EXPECT_EQ(reader->bytes_read(), 0); for (auto& future : futures) { future = reader->ReadNextAsync(); } } - auto all_future = All(std::move(futures)); - AssertNotFinished(all_future); - EXPECT_EQ(weak_reader.use_count(), 0); - ASSERT_FINISHES_OK_AND_ASSIGN(auto results, all_future); + ASSERT_FINISHES_OK_AND_ASSIGN(auto results, All(std::move(futures))); ASSERT_OK_AND_ASSIGN(auto batches, internal::UnwrapOrRaise(std::move(results))); - ASSERT_OK_AND_ASSIGN(auto table, Table::FromRecordBatches(batches)); - ASSERT_TABLES_EQUAL(*expected.table, *table); -} - -TEST_P(StreamingReaderTest, NestedParallelism) { - constexpr int kNumRows = 16; + batches.erase(std::remove(batches.begin(), batches.end(), nullptr), batches.end()); + EXPECT_EQ(batches.size(), static_cast(expected.num_batches)); - auto expected = GenerateTestCase(kNumRows); - parse_options_.explicit_schema = expected.schema; - parse_options_.unexpected_field_behavior = UnexpectedFieldBehavior::Error; - read_options_.block_size = expected.block_size; - - AsyncGenerator> generator; - auto task = [&generator] { return CollectAsyncGenerator(generator).result(); }; - - ASSERT_OK_AND_ASSIGN(auto thread_pool, internal::ThreadPool::Make(1)); - ASSERT_OK_AND_ASSIGN(generator, MakeGenerator(expected.json)); - ASSERT_OK_AND_ASSIGN(auto batches_future, thread_pool->Submit(task)); - - ASSERT_FINISHES_OK_AND_ASSIGN(auto batches, batches_future); - ASSERT_EQ(batches.size(), expected.batches.size()); ASSERT_OK_AND_ASSIGN(auto table, Table::FromRecordBatches(batches)); ASSERT_TABLES_EQUAL(*expected.table, *table); } @@ -815,7 +773,7 @@ TEST_P(StreamingReaderTest, StressBufferedReads) { parse_options_.unexpected_field_behavior = UnexpectedFieldBehavior::Error; read_options_.block_size = expected.block_size; - std::vector>> futures(expected.num_batches); + std::vector>> futures(expected.num_batches + 2); ASSERT_OK_AND_ASSIGN(auto reader, MakeReader(expected.json)); EXPECT_EQ(reader->bytes_read(), 0); for (auto& future : futures) { @@ -824,6 +782,9 @@ TEST_P(StreamingReaderTest, StressBufferedReads) { ASSERT_FINISHES_OK_AND_ASSIGN(auto results, All(std::move(futures))); ASSERT_OK_AND_ASSIGN(auto batches, internal::UnwrapOrRaise(results)); + batches.erase(std::remove(batches.begin(), batches.end(), nullptr), batches.end()); + EXPECT_EQ(batches.size(), static_cast(expected.num_batches)); + ASSERT_OK_AND_ASSIGN(auto table, Table::FromRecordBatches(batches)); ASSERT_TABLES_EQUAL(*expected.table, *table); } From 3e826f25b45bc39a19030b3dc768460122eb6a09 Mon Sep 17 00:00:00 2001 From: benibus Date: Fri, 2 Dec 2022 00:56:04 -0500 Subject: [PATCH 07/18] Tweak public API/comments --- cpp/src/arrow/json/reader.cc | 24 ++++++++------ cpp/src/arrow/json/reader.h | 52 ++++++++++++++++++++----------- cpp/src/arrow/json/reader_test.cc | 50 ++++++++++++++--------------- 3 files changed, 73 insertions(+), 53 deletions(-) diff --git a/cpp/src/arrow/json/reader.cc b/cpp/src/arrow/json/reader.cc index c43f6be5813..3399a505f68 100644 --- a/cpp/src/arrow/json/reader.cc +++ b/cpp/src/arrow/json/reader.cc @@ -398,6 +398,10 @@ class StreamingReaderImpl : public StreamingReader { static Future> MakeAsync( std::shared_ptr context, std::shared_ptr stream, io::IOContext io_context, Executor* cpu_executor, const ReadOptions& read_options) { + if (!cpu_executor) { + cpu_executor = GetCpuThreadPool(); + } + ARROW_ASSIGN_OR_RAISE( auto buffer_it, io::MakeInputStreamIterator(std::move(stream), read_options.block_size)); @@ -438,7 +442,9 @@ class StreamingReaderImpl : public StreamingReader { return generator_(); } - [[nodiscard]] int64_t bytes_read() const override { return bytes_processed_->load(); } + [[nodiscard]] int64_t bytes_processed() const override { + return bytes_processed_->load(); + } private: static Future FirstBlock(AsyncGenerator gen) { @@ -488,9 +494,9 @@ Result> TableReader::Make( } Future> StreamingReader::MakeAsync( - std::shared_ptr stream, io::IOContext io_context, - Executor* cpu_executor, const ReadOptions& read_options, - const ParseOptions& parse_options) { + std::shared_ptr stream, const ReadOptions& read_options, + const ParseOptions& parse_options, const io::IOContext& io_context, + Executor* cpu_executor) { auto future = StreamingReaderImpl::MakeAsync( std::make_shared(parse_options, io_context.pool()), std::move(stream), io_context, cpu_executor, read_options); @@ -500,11 +506,11 @@ Future> StreamingReader::MakeAsync( } Result> StreamingReader::Make( - std::shared_ptr stream, io::IOContext io_context, - Executor* cpu_executor, const ReadOptions& read_options, - const ParseOptions& parse_options) { - auto future = StreamingReader::MakeAsync(std::move(stream), io_context, cpu_executor, - read_options, parse_options); + std::shared_ptr stream, const ReadOptions& read_options, + const ParseOptions& parse_options, const io::IOContext& io_context, + Executor* cpu_executor) { + auto future = + MakeAsync(std::move(stream), read_options, parse_options, io_context, cpu_executor); return future.result(); } diff --git a/cpp/src/arrow/json/reader.h b/cpp/src/arrow/json/reader.h index c8362610f64..240d30da267 100644 --- a/cpp/src/arrow/json/reader.h +++ b/cpp/src/arrow/json/reader.h @@ -59,41 +59,55 @@ ARROW_EXPORT Result> ParseOne(ParseOptions options, /// /// The supplied `ParseOptions` are used to determine a schema, based either on a /// provided explicit schema or inferred from the first non-empty block. -/// Afterwards, the schema is frozen and unexpected fields will be ignored on -/// subsequent reads (unless `UnexpectedFieldBehavior::Error` was specified). +/// Afterwards, the target schema is frozen. If `UnexpectedFieldBehavior::InferType` is +/// specified, unexpected fields will only be inferred for the first block. Afterwards +/// they'll be treated as errors. /// /// For each block, the reader will launch its subsequent parsing/decoding task on the /// given `cpu_executor` - potentially in parallel. If `ReadOptions::use_threads` is /// specified, readahead will be applied to these tasks in accordance with the executor's -/// capacity. +/// capacity. If an executor is not provided, the global thread pool will be used. class ARROW_EXPORT StreamingReader : public RecordBatchReader { public: virtual ~StreamingReader() = default; /// \brief Read the next `RecordBatch` asynchronously - /// /// This function is async-reentrant (but not synchronously reentrant) virtual Future> ReadNextAsync() = 0; - /// \brief Return the number of bytes which have been read and processed + /// Get the number of bytes which have been succesfully converted to record batches + /// and consumed + [[nodiscard]] virtual int64_t bytes_processed() const = 0; + + /// \brief Create a `StreamingReader` from an `InputStream` + /// Blocks until the initial batch is loaded /// - /// The returned number includes JSON bytes which the StreamingReader has finished - /// processing, but not bytes for which some processing (e.g. JSON parsing or conversion - /// to Arrow layout) is still ongoing. - [[nodiscard]] virtual int64_t bytes_read() const = 0; + /// \param[in] stream JSON source stream + /// \param[in] read_options Options for reading + /// \param[in] parse_options Options for chunking, parsing, and conversion + /// \param[in] io_context Context for IO operations (optional) + /// \param[in] cpu_executor Executor for computation tasks (optional) + /// \return The initialized reader + static Result> Make( + std::shared_ptr stream, const ReadOptions& read_options, + const ParseOptions& parse_options, + const io::IOContext& io_context = io::default_io_context(), + ::arrow::internal::Executor* cpu_executor = NULLPTR); - /// \brief Create a `StreamingReader` instance asynchronously + /// \brief Create a `StreamingReader` from an `InputStream` asynchronously + /// Returned future completes after loading the first batch /// - /// This involves some I/O as the first batch must be loaded during the creation process - /// so it is returned as a future + /// \param[in] stream JSON source stream + /// \param[in] read_options Options for reading + /// \param[in] parse_options Options for chunking, parsing, and conversion + /// \param[in] io_context Context for IO operations (optional) + /// \param[in] cpu_executor Executor for computation tasks (optional) + /// \return Future for the initialized reader static Future> MakeAsync( - std::shared_ptr stream, io::IOContext io_context, - ::arrow::internal::Executor* cpu_executor, const ReadOptions&, const ParseOptions&); - - /// \brief Create a `StreamingReader` instance - static Result> Make( - std::shared_ptr stream, io::IOContext io_context, - ::arrow::internal::Executor* cpu_executor, const ReadOptions&, const ParseOptions&); + std::shared_ptr stream, const ReadOptions& read_options, + const ParseOptions& parse_options, + const io::IOContext& io_context = io::default_io_context(), + ::arrow::internal::Executor* cpu_executor = NULLPTR); }; } // namespace json diff --git a/cpp/src/arrow/json/reader_test.cc b/cpp/src/arrow/json/reader_test.cc index 967145710cb..46ba8bb90d7 100644 --- a/cpp/src/arrow/json/reader_test.cc +++ b/cpp/src/arrow/json/reader_test.cc @@ -340,8 +340,8 @@ class StreamingReaderTest : public ::testing::TestWithParam { Result> MakeReader( std::shared_ptr stream) { - return StreamingReader::Make(std::move(stream), io_context_, executor_, read_options_, - parse_options_); + return StreamingReader::Make(std::move(stream), read_options_, parse_options_, + io_context_, executor_); } template Result> MakeReader(Args&&... args) { @@ -448,7 +448,7 @@ class StreamingReaderTest : public ::testing::TestWithParam { return out; } - internal::Executor* executor_ = internal::GetCpuThreadPool(); + internal::Executor* executor_ = nullptr; ParseOptions parse_options_ = ParseOptions::Defaults(); ReadOptions read_options_ = ReadOptions::Defaults(); io::IOContext io_context_ = io::default_io_context(); @@ -491,14 +491,14 @@ TEST_P(StreamingReaderTest, PropagateChunkingErrors) { std::shared_ptr batch; AssertReadNext(reader, &batch); - EXPECT_EQ(reader->bytes_read(), 9); + EXPECT_EQ(reader->bytes_processed(), 9); ASSERT_BATCHES_EQUAL(*RecordBatchFromJSON(test_schema, "[{\"i\":0}]"), *batch); ASSERT_RAISES(Invalid, reader->ReadNext(&batch)); - EXPECT_EQ(reader->bytes_read(), 9); + EXPECT_EQ(reader->bytes_processed(), 9); AssertReadEnd(reader); AssertReadEnd(reader); - EXPECT_EQ(reader->bytes_read(), 9); + EXPECT_EQ(reader->bytes_processed(), 9); } TEST_P(StreamingReaderTest, PropagateParsingErrors) { @@ -530,17 +530,17 @@ TEST_P(StreamingReaderTest, PropagateParsingErrors) { std::shared_ptr batch; ASSERT_OK_AND_ASSIGN(auto reader, MakeReader(bad_middle_block)); - EXPECT_EQ(reader->bytes_read(), 0); + EXPECT_EQ(reader->bytes_processed(), 0); AssertSchemaEqual(reader->schema(), test_schema); AssertReadNext(reader, &batch); - EXPECT_EQ(reader->bytes_read(), 13); + EXPECT_EQ(reader->bytes_processed(), 13); ASSERT_BATCHES_EQUAL(*RecordBatchFromJSON(test_schema, R"([{"n":10000}])"), *batch); ASSERT_RAISES(Invalid, reader->ReadNext(&batch)); - EXPECT_EQ(reader->bytes_read(), 13); + EXPECT_EQ(reader->bytes_processed(), 13); AssertReadEnd(reader); - EXPECT_EQ(reader->bytes_read(), 13); + EXPECT_EQ(reader->bytes_processed(), 13); } TEST_P(StreamingReaderTest, IgnoreLeadingEmptyBlocks) { @@ -551,7 +551,7 @@ TEST_P(StreamingReaderTest, IgnoreLeadingEmptyBlocks) { parse_options_.explicit_schema = schema({field("b", boolean()), field("s", utf8())}); read_options_.block_size = 24; ASSERT_OK_AND_ASSIGN(auto reader, MakeReader(test_json)); - EXPECT_EQ(reader->bytes_read(), 0); + EXPECT_EQ(reader->bytes_processed(), 0); auto expected_schema = parse_options_.explicit_schema; auto expected_batch = RecordBatchFromJSON(expected_schema, R"([{"b":true,"s":"foo"}])"); @@ -560,7 +560,7 @@ TEST_P(StreamingReaderTest, IgnoreLeadingEmptyBlocks) { std::shared_ptr actual_batch; AssertReadNext(reader, &actual_batch); - EXPECT_EQ(reader->bytes_read(), 55); + EXPECT_EQ(reader->bytes_processed(), 55); ASSERT_BATCHES_EQUAL(*expected_batch, *actual_batch); AssertReadEnd(reader); @@ -594,14 +594,14 @@ TEST_P(StreamingReaderTest, ExplicitSchemaErrorOnUnexpectedFields) { AssertReadNext(reader, &actual_batch); ASSERT_BATCHES_EQUAL(*expected_batch, *actual_batch); - EXPECT_EQ(reader->bytes_read(), 32); + EXPECT_EQ(reader->bytes_processed(), 32); AssertReadNext(reader, &actual_batch); ASSERT_BATCHES_EQUAL(*expected_batch, *actual_batch); - EXPECT_EQ(reader->bytes_read(), 64); + EXPECT_EQ(reader->bytes_processed(), 64); ASSERT_RAISES(Invalid, reader->ReadNext(&actual_batch)); - EXPECT_EQ(reader->bytes_read(), 64); + EXPECT_EQ(reader->bytes_processed(), 64); AssertReadEnd(reader); } @@ -627,17 +627,17 @@ TEST_P(StreamingReaderTest, ExplicitSchemaIgnoreUnexpectedFields) { expected_batch = RecordBatchFromJSON(expected_schema, R"([{"s":"foo","t":null}])"); AssertReadNext(reader, &actual_batch); ASSERT_BATCHES_EQUAL(*expected_batch, *actual_batch); - EXPECT_EQ(reader->bytes_read(), 32); + EXPECT_EQ(reader->bytes_processed(), 32); expected_batch = RecordBatchFromJSON(expected_schema, R"([{"s":"foo","t":"2022-01-01"}])"); AssertReadNext(reader, &actual_batch); ASSERT_BATCHES_EQUAL(*expected_batch, *actual_batch); - EXPECT_EQ(reader->bytes_read(), 64); + EXPECT_EQ(reader->bytes_processed(), 64); AssertReadNext(reader, &actual_batch); ASSERT_BATCHES_EQUAL(*expected_batch, *actual_batch); - EXPECT_EQ(reader->bytes_read(), 106); + EXPECT_EQ(reader->bytes_processed(), 106); AssertReadEnd(reader); } @@ -668,7 +668,7 @@ TEST_P(StreamingReaderTest, InferredSchema) { expected_batch = RecordBatchFromJSON(expected_schema, R"([{"a": 0, "b": "foo"}])"); AssertReadNext(reader, &actual_batch); - EXPECT_EQ(reader->bytes_read(), 28); + EXPECT_EQ(reader->bytes_processed(), 28); ASSERT_BATCHES_EQUAL(*expected_batch, *actual_batch); ASSERT_RAISES(Invalid, reader->ReadNext(&actual_batch)); @@ -686,7 +686,7 @@ TEST_P(StreamingReaderTest, InferredSchema) { {"a": 1, "b": null, "c": true} ])"); AssertReadNext(reader, &actual_batch); - EXPECT_EQ(reader->bytes_read(), 56); + EXPECT_EQ(reader->bytes_processed(), 56); ASSERT_BATCHES_EQUAL(*expected_batch, *actual_batch); ASSERT_RAISES(Invalid, reader->ReadNext(&actual_batch)); @@ -705,7 +705,7 @@ TEST_P(StreamingReaderTest, InferredSchema) { {"a": 2, "b": null, "c": null, "d": "2022-01-01"} ])"); AssertReadNext(reader, &actual_batch); - EXPECT_EQ(reader->bytes_read(), 84); + EXPECT_EQ(reader->bytes_processed(), 84); ASSERT_BATCHES_EQUAL(*expected_batch, *actual_batch); AssertReadEnd(reader); @@ -722,13 +722,13 @@ TEST_P(StreamingReaderTest, AsyncReentrancy) { std::vector>> futures(expected.num_batches + 2); ASSERT_OK_AND_ASSIGN(auto reader, MakeReader(expected.json, kIoLatency)); - EXPECT_EQ(reader->bytes_read(), 0); + EXPECT_EQ(reader->bytes_processed(), 0); for (auto& future : futures) { future = reader->ReadNextAsync(); } ASSERT_FINISHES_OK_AND_ASSIGN(auto results, All(std::move(futures))); - EXPECT_EQ(reader->bytes_read(), expected.json_size); + EXPECT_EQ(reader->bytes_processed(), expected.json_size); ASSERT_OK_AND_ASSIGN(auto batches, internal::UnwrapOrRaise(std::move(results))); batches.erase(std::remove(batches.begin(), batches.end(), nullptr), batches.end()); EXPECT_EQ(batches.size(), static_cast(expected.num_batches)); @@ -750,7 +750,7 @@ TEST_P(StreamingReaderTest, FuturesOutliveReader) { std::vector>> futures(expected.num_batches + 2); { ASSERT_OK_AND_ASSIGN(auto reader, MakeReader(stream)); - EXPECT_EQ(reader->bytes_read(), 0); + EXPECT_EQ(reader->bytes_processed(), 0); for (auto& future : futures) { future = reader->ReadNextAsync(); } @@ -775,7 +775,7 @@ TEST_P(StreamingReaderTest, StressBufferedReads) { std::vector>> futures(expected.num_batches + 2); ASSERT_OK_AND_ASSIGN(auto reader, MakeReader(expected.json)); - EXPECT_EQ(reader->bytes_read(), 0); + EXPECT_EQ(reader->bytes_processed(), 0); for (auto& future : futures) { future = reader->ReadNextAsync(); } From 697c94f6ded2c363b57297df9d1bc32778746578 Mon Sep 17 00:00:00 2001 From: benibus Date: Fri, 2 Dec 2022 14:39:05 -0500 Subject: [PATCH 08/18] Update class docs --- docs/source/cpp/api/formats.rst | 3 ++ docs/source/cpp/json.rst | 62 +++++++++++++++++++++++++++++---- 2 files changed, 59 insertions(+), 6 deletions(-) diff --git a/docs/source/cpp/api/formats.rst b/docs/source/cpp/api/formats.rst index 49fa5645f8b..264b9e4e7c6 100644 --- a/docs/source/cpp/api/formats.rst +++ b/docs/source/cpp/api/formats.rst @@ -67,6 +67,9 @@ Line-separated JSON .. doxygenclass:: arrow::json::TableReader :members: +.. doxygenclass:: arrow::json::StreamingReader + :members: + .. _cpp-api-parquet: Parquet reader diff --git a/docs/source/cpp/json.rst b/docs/source/cpp/json.rst index cdb742e6ce1..4ede5a5da8d 100644 --- a/docs/source/cpp/json.rst +++ b/docs/source/cpp/json.rst @@ -24,17 +24,24 @@ Reading JSON files ================== -Arrow allows reading line-separated JSON files as Arrow tables. Each -independent JSON object in the input file is converted to a row in -the target Arrow table. +Line-separated JSON files can either be read as a single Arrow Table +with a :class:`~TableReader` or streamed as RecordBatches with a +:class:`~StreamingReader`. + +Both of these readers require an :class:`arrow::io::InputStream` instance +representing the input file. Their behavior can be customized using a +combination of :class:`~ReadOptions`, :class:`~ParseOptions`, and +other parameters. .. seealso:: :ref:`JSON reader API reference `. -Basic usage +TableReader =========== -A JSON file is read from a :class:`~arrow::io::InputStream`. +Reads an entire file in one shot as a :class:`~arrow::Table`. Each +independent JSON object in the input file is converted to a row in +the output table. .. code-block:: cpp @@ -66,6 +73,49 @@ A JSON file is read from a :class:`~arrow::io::InputStream`. } } +StreamingReader +=============== + +Reads a file incrementally in fixed-size blocks, each yielding a +:class:`~arrow::RecordBatch`. Each independent JSON object in a block +is converted to a row in the output batch. + +All batches adhere to a consistent :class:`~arrow:Schema`, which is +derived from the first loaded batch. + +.. code-block:: cpp + + #include "arrow/json/api.h" + + { + // ... + auto read_options = arrow::json::ReadOptions::Defaults(); + auto parse_options = arrow::json::ParseOptions::Defaults(); + + std::shared_ptr stream; + auto result = arrow::json::StreamingReader::Make(stream, + read_options, + parse_options); + if (!result.ok()) { + // Handle instantiation error + } + std::shared_ptr reader = *result; + + std::shared_ptr batch; + while (true) { + arrow::Status status = reader->ReadNext(&batch); + + if (!status.ok()) { + // Handle read/parse error + } + + if (batch == nullptr) { + // Handle end of file + break; + } + } + } + Data types ========== @@ -75,7 +125,7 @@ objects. The fields of top-level objects are taken to represent columns in the Arrow data. For each name/value pair in a JSON object, there are two possible modes of deciding the output data type: -* if the name is in :class:`ConvertOptions::explicit_schema`, +* if the name is in :member:`ParseOptions::explicit_schema`, conversion of the JSON value to the corresponding Arrow data type is attempted; From e4ac337153758acef4db3045f2ad7e4852590d26 Mon Sep 17 00:00:00 2001 From: benibus Date: Mon, 5 Dec 2022 15:19:40 -0500 Subject: [PATCH 09/18] Add tests with JSON chunker --- cpp/src/arrow/json/reader_test.cc | 56 +++++++++++++++++++++++++++++++ 1 file changed, 56 insertions(+) diff --git a/cpp/src/arrow/json/reader_test.cc b/cpp/src/arrow/json/reader_test.cc index 46ba8bb90d7..4f9371b15f6 100644 --- a/cpp/src/arrow/json/reader_test.cc +++ b/cpp/src/arrow/json/reader_test.cc @@ -19,6 +19,7 @@ #include #include +#include #include #include "arrow/io/interfaces.h" @@ -470,6 +471,7 @@ TEST_P(StreamingReaderTest, PropagateChunkingErrors) { constexpr double kIoLatency = 1e-3; auto test_schema = schema({field("i", int64())}); + // Object straddles multiple blocks auto bad_first_chunk = Join( { R"({"i": 0 })", @@ -543,6 +545,60 @@ TEST_P(StreamingReaderTest, PropagateParsingErrors) { EXPECT_EQ(reader->bytes_processed(), 13); } +TEST_P(StreamingReaderTest, PropagateErrorsNonLinewiseChunker) { + auto test_schema = schema({field("i", int64())}); + auto bad_first_block = Join( + { + R"({"i":0}{1})", + R"({"i":2})", + }, + "\n"); + auto bad_middle_blocks = Join( + { + R"({"i": 0})", + R"({"i": 1})", + R"({}"i":2})", + R"({"i": 3})", + }, + "\n"); + + std::shared_ptr batch; + std::shared_ptr reader; + Status status; + read_options_.block_size = 10; + parse_options_.newlines_in_values = true; + + ASSERT_OK_AND_ASSIGN(reader, MakeReader(bad_first_block)); + AssertReadNext(reader, &batch); + EXPECT_EQ(reader->bytes_processed(), 7); + ASSERT_BATCHES_EQUAL(*RecordBatchFromJSON(test_schema, "[{\"i\":0}]"), *batch); + status = reader->ReadNext(&batch); + EXPECT_EQ(reader->bytes_processed(), 7); + ASSERT_RAISES(Invalid, status); + EXPECT_THAT(status.message(), ::testing::StartsWith("JSON parse error")); + AssertReadEnd(reader); + + ASSERT_OK_AND_ASSIGN(reader, MakeReader(bad_middle_blocks)); + AssertReadNext(reader, &batch); + EXPECT_EQ(reader->bytes_processed(), 9); + ASSERT_BATCHES_EQUAL(*RecordBatchFromJSON(test_schema, "[{\"i\":0}]"), *batch); + // Chunker doesn't require newline delimiters, so this should be valid + AssertReadNext(reader, &batch); + EXPECT_EQ(reader->bytes_processed(), 20); + ASSERT_BATCHES_EQUAL(*RecordBatchFromJSON(test_schema, "[{\"i\":1}]"), *batch); + + status = reader->ReadNext(&batch); + EXPECT_EQ(reader->bytes_processed(), 20); + // Should fail to parse "{}\"i\"" + ASSERT_RAISES(Invalid, status); + EXPECT_THAT(status.message(), ::testing::StartsWith("JSON parse error")); + // Incoming chunker error from ":2}" shouldn't leak through after the first failure, + // which is a possibility if async tasks are still outstanding due to readahead. + AssertReadEnd(reader); + AssertReadEnd(reader); + EXPECT_EQ(reader->bytes_processed(), 20); +} + TEST_P(StreamingReaderTest, IgnoreLeadingEmptyBlocks) { std::string test_json(32, '\n'); test_json += R"({"b": true, "s": "foo"})"; From 588773f9c3a2a8ae933915197ec3acbd49d913f1 Mon Sep 17 00:00:00 2001 From: Ben Harkins <60872452+benibus@users.noreply.github.com> Date: Mon, 5 Dec 2022 15:27:09 -0500 Subject: [PATCH 10/18] Apply docs suggestions Co-authored-by: Will Jones --- docs/source/cpp/json.rst | 16 ++++++---------- 1 file changed, 6 insertions(+), 10 deletions(-) diff --git a/docs/source/cpp/json.rst b/docs/source/cpp/json.rst index 4ede5a5da8d..a35a4f6a0bc 100644 --- a/docs/source/cpp/json.rst +++ b/docs/source/cpp/json.rst @@ -81,7 +81,8 @@ Reads a file incrementally in fixed-size blocks, each yielding a is converted to a row in the output batch. All batches adhere to a consistent :class:`~arrow:Schema`, which is -derived from the first loaded batch. +derived from the first loaded batch. Alternatively, an explicit schema +may be passed via :class:`~ParseOptions`. .. code-block:: cpp @@ -102,17 +103,12 @@ derived from the first loaded batch. std::shared_ptr reader = *result; std::shared_ptr batch; - while (true) { - arrow::Status status = reader->ReadNext(&batch); - - if (!status.ok()) { + for (arrow::Result> maybe_batch : *reader) { + if (!result.ok()) { // Handle read/parse error } - - if (batch == nullptr) { - // Handle end of file - break; - } + batch = *maybe_batch; + // Operate on each batch... } } From 85dd02bbe01081f75bfd872a81be8dfa6d358f07 Mon Sep 17 00:00:00 2001 From: Ben Harkins <60872452+benibus@users.noreply.github.com> Date: Tue, 6 Dec 2022 11:14:40 -0500 Subject: [PATCH 11/18] Apply suggestions from code review Co-authored-by: Antoine Pitrou --- cpp/src/arrow/json/reader_test.cc | 2 ++ docs/source/cpp/json.rst | 6 +++--- 2 files changed, 5 insertions(+), 3 deletions(-) diff --git a/cpp/src/arrow/json/reader_test.cc b/cpp/src/arrow/json/reader_test.cc index 4f9371b15f6..6f7933bb4f5 100644 --- a/cpp/src/arrow/json/reader_test.cc +++ b/cpp/src/arrow/json/reader_test.cc @@ -363,6 +363,7 @@ class StreamingReaderTest : public ::testing::TestWithParam { std::shared_ptr* out) { ASSERT_OK(reader->ReadNext(out)); ASSERT_FALSE(IsIterationEnd(*out)); + ASSERT_OK((**out).ValidateFull()); } static void AssertReadEnd(const std::shared_ptr& reader) { std::shared_ptr out; @@ -790,6 +791,7 @@ TEST_P(StreamingReaderTest, AsyncReentrancy) { EXPECT_EQ(batches.size(), static_cast(expected.num_batches)); ASSERT_OK_AND_ASSIGN(auto table, Table::FromRecordBatches(batches)); + ASSERT_OK(table->ValidateFull()); ASSERT_TABLES_EQUAL(*expected.table, *table); } diff --git a/docs/source/cpp/json.rst b/docs/source/cpp/json.rst index a35a4f6a0bc..6ea5d09f803 100644 --- a/docs/source/cpp/json.rst +++ b/docs/source/cpp/json.rst @@ -39,7 +39,7 @@ other parameters. TableReader =========== -Reads an entire file in one shot as a :class:`~arrow::Table`. Each +:class:`~TableReader` reads an entire file in one shot as a :class:`~arrow::Table`. Each independent JSON object in the input file is converted to a row in the output table. @@ -76,7 +76,7 @@ the output table. StreamingReader =============== -Reads a file incrementally in fixed-size blocks, each yielding a +:class:`~StreamingReader` reads a file incrementally from blocks of a roughly equal byte size, each yielding a :class:`~arrow::RecordBatch`. Each independent JSON object in a block is converted to a row in the output batch. @@ -104,7 +104,7 @@ may be passed via :class:`~ParseOptions`. std::shared_ptr batch; for (arrow::Result> maybe_batch : *reader) { - if (!result.ok()) { + if (!maybe_batch.ok()) { // Handle read/parse error } batch = *maybe_batch; From 2187242dfc2c8ec14799ce69266d8770ef1e2931 Mon Sep 17 00:00:00 2001 From: benibus Date: Tue, 6 Dec 2022 13:19:40 -0500 Subject: [PATCH 12/18] Only use cpu executor if threading is desired --- cpp/src/arrow/json/reader.cc | 56 +++++++++++++++++++++++++------ cpp/src/arrow/json/reader.h | 14 +++++--- cpp/src/arrow/json/reader_test.cc | 26 ++++++++++---- 3 files changed, 73 insertions(+), 23 deletions(-) diff --git a/cpp/src/arrow/json/reader.cc b/cpp/src/arrow/json/reader.cc index 3399a505f68..b3fc5e59aef 100644 --- a/cpp/src/arrow/json/reader.cc +++ b/cpp/src/arrow/json/reader.cc @@ -359,6 +359,28 @@ AsyncGenerator MakeDecodingGenerator( }; } +// Reads from a source iterator serially, completes subsequent decode tasks on the calling +// thread. +AsyncGenerator MakeDecodingGenerator( + Iterator source, + std::function(const ChunkedBlock&)> decoder) { + struct State { + Iterator source; + std::function(const ChunkedBlock&)> decoder; + } state{std::move(source), std::move(decoder)}; + return [state = std::make_shared(std::move(state))] { + auto maybe_block = state->source.Next(); + if (!maybe_block.ok()) { + return Future::MakeFinished(maybe_block.status()); + } + auto block = maybe_block.MoveValueUnsafe(); + if (IsIterationEnd(block)) { + return ToFuture(IterationEnd()); + } + return ToFuture(state->decoder(block)); + }; +} + // Adds async-reentrancy to `source` by submitting tasks to a single-threaded executor // (FIFO order) - ensuring, at most, one future is pending at a time template @@ -398,25 +420,37 @@ class StreamingReaderImpl : public StreamingReader { static Future> MakeAsync( std::shared_ptr context, std::shared_ptr stream, io::IOContext io_context, Executor* cpu_executor, const ReadOptions& read_options) { - if (!cpu_executor) { - cpu_executor = GetCpuThreadPool(); - } - ARROW_ASSIGN_OR_RAISE( auto buffer_it, io::MakeInputStreamIterator(std::move(stream), read_options.block_size)); ARROW_ASSIGN_OR_RAISE( auto buffer_gen, MakeBackgroundGenerator(std::move(buffer_it), io_context.executor())); - buffer_gen = MakeTransferredGenerator(std::move(buffer_gen), cpu_executor); - auto chunking_gen = MakeChunkingGenerator(std::move(buffer_gen), - MakeChunker(context->parse_options())); - ARROW_ASSIGN_OR_RAISE(chunking_gen, MakeReentrantGenerator(std::move(chunking_gen))); + AsyncGenerator decoding_gen; + int max_readahead = 0; + if (read_options.use_threads) { + // Prepare a source generator capable of async-reentrancy and parallel exececution + if (!cpu_executor) { + cpu_executor = GetCpuThreadPool(); + max_readahead = cpu_executor->GetCapacity(); + } + buffer_gen = MakeTransferredGenerator(std::move(buffer_gen), cpu_executor); + auto chunking_gen = MakeChunkingGenerator(std::move(buffer_gen), + MakeChunker(context->parse_options())); + ARROW_ASSIGN_OR_RAISE(chunking_gen, + MakeReentrantGenerator(std::move(chunking_gen))); + decoding_gen = MakeDecodingGenerator(std::move(chunking_gen), + DecodingOperator(context), cpu_executor); + } else { + // Prepare a source generator without a separate cpu executor + auto chunking_it = + MakeChunkingIterator(MakeGeneratorIterator(std::move(buffer_gen)), + MakeChunker(context->parse_options())); + decoding_gen = + MakeDecodingGenerator(std::move(chunking_it), DecodingOperator(context)); + } - auto decoding_gen = MakeDecodingGenerator(std::move(chunking_gen), - DecodingOperator(context), cpu_executor); - const int max_readahead = read_options.use_threads ? cpu_executor->GetCapacity() : 0; return FirstBlock(decoding_gen) .Then([source = std::move(decoding_gen), context = std::move(context), max_readahead](const DecodedBlock& block) { diff --git a/cpp/src/arrow/json/reader.h b/cpp/src/arrow/json/reader.h index 240d30da267..8a83f9fce1f 100644 --- a/cpp/src/arrow/json/reader.h +++ b/cpp/src/arrow/json/reader.h @@ -63,16 +63,20 @@ ARROW_EXPORT Result> ParseOne(ParseOptions options, /// specified, unexpected fields will only be inferred for the first block. Afterwards /// they'll be treated as errors. /// -/// For each block, the reader will launch its subsequent parsing/decoding task on the -/// given `cpu_executor` - potentially in parallel. If `ReadOptions::use_threads` is -/// specified, readahead will be applied to these tasks in accordance with the executor's -/// capacity. If an executor is not provided, the global thread pool will be used. +/// If `ReadOptions::use_threads` is `true`, each block's parsing/decoding task will be +/// parallelized on the given `cpu_executor` (with readahead corresponding to the +/// executor's capacity). If an executor isn't provided, the global thread pool will be +/// used. +/// +/// If `ReadOptions::use_threads` is `false`, computations will be run on the calling +/// thread and `cpu_executor` will be ignored. class ARROW_EXPORT StreamingReader : public RecordBatchReader { public: virtual ~StreamingReader() = default; /// \brief Read the next `RecordBatch` asynchronously - /// This function is async-reentrant (but not synchronously reentrant) + /// If threading is enabled, this function is async-reentrant (but not synchronously + /// reentrant). virtual Future> ReadNextAsync() = 0; /// Get the number of bytes which have been succesfully converted to record batches diff --git a/cpp/src/arrow/json/reader_test.cc b/cpp/src/arrow/json/reader_test.cc index 6f7933bb4f5..b7b29afb4ec 100644 --- a/cpp/src/arrow/json/reader_test.cc +++ b/cpp/src/arrow/json/reader_test.cc @@ -325,10 +325,11 @@ TEST(ReaderTest, FailOnInvalidEOF) { } } -class StreamingReaderTest : public ::testing::TestWithParam { - protected: - void SetUp() override { read_options_.use_threads = GetParam(); } +class StreamingReaderTestBase { + public: + virtual ~StreamingReaderTestBase() = default; + protected: static std::shared_ptr MakeTestStream(const std::string& str) { auto buffer = std::make_shared(str); return std::make_shared(std::move(buffer)); @@ -456,6 +457,17 @@ class StreamingReaderTest : public ::testing::TestWithParam { io::IOContext io_context_ = io::default_io_context(); }; +class AsyncStreamingReaderTest : public StreamingReaderTestBase, public ::testing::Test { + protected: + void SetUp() override { read_options_.use_threads = true; } +}; + +class StreamingReaderTest : public StreamingReaderTestBase, + public ::testing::TestWithParam { + protected: + void SetUp() override { read_options_.use_threads = GetParam(); } +}; + INSTANTIATE_TEST_SUITE_P(StreamingReaderTest, StreamingReaderTest, ::testing::Values(false, true)); @@ -768,7 +780,7 @@ TEST_P(StreamingReaderTest, InferredSchema) { AssertReadEnd(reader); } -TEST_P(StreamingReaderTest, AsyncReentrancy) { +TEST_F(AsyncStreamingReaderTest, AsyncReentrancy) { constexpr int kNumRows = 16; constexpr double kIoLatency = 1e-2; @@ -795,7 +807,7 @@ TEST_P(StreamingReaderTest, AsyncReentrancy) { ASSERT_TABLES_EQUAL(*expected.table, *table); } -TEST_P(StreamingReaderTest, FuturesOutliveReader) { +TEST_F(AsyncStreamingReaderTest, FuturesOutliveReader) { constexpr int kNumRows = 16; constexpr double kIoLatency = 1e-2; @@ -823,7 +835,7 @@ TEST_P(StreamingReaderTest, FuturesOutliveReader) { ASSERT_TABLES_EQUAL(*expected.table, *table); } -TEST_P(StreamingReaderTest, StressBufferedReads) { +TEST_F(AsyncStreamingReaderTest, StressBufferedReads) { constexpr int kNumRows = 500; auto expected = GenerateTestCase(kNumRows); @@ -847,7 +859,7 @@ TEST_P(StreamingReaderTest, StressBufferedReads) { ASSERT_TABLES_EQUAL(*expected.table, *table); } -TEST_P(StreamingReaderTest, StressSharedIoAndCpuExecutor) { +TEST_F(AsyncStreamingReaderTest, StressSharedIoAndCpuExecutor) { constexpr int kNumRows = 500; constexpr double kIoLatency = 1e-4; From 29a61162cc4ca668fa9bd9adf5f2a398d6c834b4 Mon Sep 17 00:00:00 2001 From: benibus Date: Tue, 6 Dec 2022 17:02:13 -0500 Subject: [PATCH 13/18] Address requested test/docs changes --- cpp/src/arrow/json/reader_test.cc | 64 +++++++++++++++++-------------- docs/source/cpp/json.rst | 3 +- 2 files changed, 36 insertions(+), 31 deletions(-) diff --git a/cpp/src/arrow/json/reader_test.cc b/cpp/src/arrow/json/reader_test.cc index b7b29afb4ec..cd59cc98820 100644 --- a/cpp/src/arrow/json/reader_test.cc +++ b/cpp/src/arrow/json/reader_test.cc @@ -637,75 +637,78 @@ TEST_P(StreamingReaderTest, IgnoreLeadingEmptyBlocks) { TEST_P(StreamingReaderTest, ExplicitSchemaErrorOnUnexpectedFields) { std::string test_json = - Join({R"({"s": "foo", "t": "2022-01-01"})", R"({"s": "foo", "t": "2022-01-01"})", - R"({"s": "foo", "t": "2022-01-01", "b": true})"}, + Join({R"({"s": "foo", "t": "2022-01-01"})", R"({"s": "bar", "t": "2022-01-02"})", + R"({"s": "baz", "t": "2022-01-03", "b": true})"}, "\n"); FieldVector expected_fields = {field("s", utf8())}; std::shared_ptr expected_schema = schema(expected_fields); - std::shared_ptr expected_batch; - std::shared_ptr actual_batch; - std::shared_ptr reader; parse_options_.explicit_schema = expected_schema; parse_options_.unexpected_field_behavior = UnexpectedFieldBehavior::Error; read_options_.block_size = 48; - ASSERT_RAISES(Invalid, MakeReader(test_json)); + + auto result = MakeReader(test_json); + ASSERT_RAISES(Invalid, result); + EXPECT_THAT(result.status().message(), + ::testing::StartsWith("JSON parse error: unexpected field")); expected_fields.push_back(field("t", utf8())); expected_schema = schema(expected_fields); - expected_batch = - RecordBatchFromJSON(expected_schema, R"([{"s":"foo","t":"2022-01-01"}])"); parse_options_.explicit_schema = expected_schema; - ASSERT_OK_AND_ASSIGN(reader, MakeReader(test_json)); + ASSERT_OK_AND_ASSIGN(auto reader, MakeReader(test_json)); AssertSchemaEqual(reader->schema(), expected_schema); - AssertReadNext(reader, &actual_batch); - ASSERT_BATCHES_EQUAL(*expected_batch, *actual_batch); + std::shared_ptr batch; + AssertReadNext(reader, &batch); + ASSERT_BATCHES_EQUAL( + *RecordBatchFromJSON(expected_schema, R"([{"s":"foo","t":"2022-01-01"}])"), *batch); EXPECT_EQ(reader->bytes_processed(), 32); - AssertReadNext(reader, &actual_batch); - ASSERT_BATCHES_EQUAL(*expected_batch, *actual_batch); + AssertReadNext(reader, &batch); + ASSERT_BATCHES_EQUAL( + *RecordBatchFromJSON(expected_schema, R"([{"s":"bar","t":"2022-01-02"}])"), *batch); EXPECT_EQ(reader->bytes_processed(), 64); - ASSERT_RAISES(Invalid, reader->ReadNext(&actual_batch)); + auto status = reader->ReadNext(&batch); EXPECT_EQ(reader->bytes_processed(), 64); + ASSERT_RAISES(Invalid, status); + EXPECT_THAT(status.message(), + ::testing::StartsWith("JSON parse error: unexpected field")); AssertReadEnd(reader); } TEST_P(StreamingReaderTest, ExplicitSchemaIgnoreUnexpectedFields) { std::string test_json = - Join({R"({"s": "foo", "u": "2022-01-01"})", R"({"s": "foo", "t": "2022-01-01"})", - R"({"s": "foo", "t": "2022-01-01", "b": true})"}, + Join({R"({"s": "foo", "u": "2022-01-01"})", R"({"s": "bar", "t": "2022-01-02"})", + R"({"s": "baz", "t": "2022-01-03", "b": true})"}, "\n"); FieldVector expected_fields = {field("s", utf8()), field("t", utf8())}; std::shared_ptr expected_schema = schema(expected_fields); - std::shared_ptr expected_batch; - std::shared_ptr actual_batch; - std::shared_ptr reader; parse_options_.explicit_schema = expected_schema; parse_options_.unexpected_field_behavior = UnexpectedFieldBehavior::Ignore; read_options_.block_size = 48; - ASSERT_OK_AND_ASSIGN(reader, MakeReader(test_json)); + ASSERT_OK_AND_ASSIGN(auto reader, MakeReader(test_json)); AssertSchemaEqual(reader->schema(), expected_schema); - expected_batch = RecordBatchFromJSON(expected_schema, R"([{"s":"foo","t":null}])"); - AssertReadNext(reader, &actual_batch); - ASSERT_BATCHES_EQUAL(*expected_batch, *actual_batch); + std::shared_ptr batch; + AssertReadNext(reader, &batch); + ASSERT_BATCHES_EQUAL(*RecordBatchFromJSON(expected_schema, R"([{"s":"foo","t":null}])"), + *batch); EXPECT_EQ(reader->bytes_processed(), 32); - expected_batch = - RecordBatchFromJSON(expected_schema, R"([{"s":"foo","t":"2022-01-01"}])"); - AssertReadNext(reader, &actual_batch); - ASSERT_BATCHES_EQUAL(*expected_batch, *actual_batch); + AssertReadNext(reader, &batch); + ASSERT_BATCHES_EQUAL( + *RecordBatchFromJSON(expected_schema, R"([{"s":"bar","t":"2022-01-02"}])"), *batch); EXPECT_EQ(reader->bytes_processed(), 64); - AssertReadNext(reader, &actual_batch); - ASSERT_BATCHES_EQUAL(*expected_batch, *actual_batch); + AssertReadNext(reader, &batch); + ASSERT_BATCHES_EQUAL( + *RecordBatchFromJSON(expected_schema, R"([{"s":"baz","t":"2022-01-03"}])"), *batch); EXPECT_EQ(reader->bytes_processed(), 106); AssertReadEnd(reader); } @@ -832,6 +835,7 @@ TEST_F(AsyncStreamingReaderTest, FuturesOutliveReader) { EXPECT_EQ(batches.size(), static_cast(expected.num_batches)); ASSERT_OK_AND_ASSIGN(auto table, Table::FromRecordBatches(batches)); + ASSERT_OK(table->ValidateFull()); ASSERT_TABLES_EQUAL(*expected.table, *table); } @@ -856,6 +860,7 @@ TEST_F(AsyncStreamingReaderTest, StressBufferedReads) { EXPECT_EQ(batches.size(), static_cast(expected.num_batches)); ASSERT_OK_AND_ASSIGN(auto table, Table::FromRecordBatches(batches)); + ASSERT_OK(table->ValidateFull()); ASSERT_TABLES_EQUAL(*expected.table, *table); } @@ -877,6 +882,7 @@ TEST_F(AsyncStreamingReaderTest, StressSharedIoAndCpuExecutor) { ASSERT_FINISHES_OK_AND_ASSIGN(auto batches, CollectAsyncGenerator(generator)); ASSERT_EQ(batches.size(), expected.batches.size()); ASSERT_OK_AND_ASSIGN(auto table, Table::FromRecordBatches(batches)); + ASSERT_OK(table->ValidateFull()); ASSERT_TABLES_EQUAL(*expected.table, *table); } diff --git a/docs/source/cpp/json.rst b/docs/source/cpp/json.rst index 6ea5d09f803..e2c55d00d10 100644 --- a/docs/source/cpp/json.rst +++ b/docs/source/cpp/json.rst @@ -102,12 +102,11 @@ may be passed via :class:`~ParseOptions`. } std::shared_ptr reader = *result; - std::shared_ptr batch; for (arrow::Result> maybe_batch : *reader) { if (!maybe_batch.ok()) { // Handle read/parse error } - batch = *maybe_batch; + std::shared_ptr batch = *maybe_batch; // Operate on each batch... } } From c940acf9908562fb246efefd9dae17ecc72c093b Mon Sep 17 00:00:00 2001 From: benibus Date: Wed, 7 Dec 2022 11:07:37 -0500 Subject: [PATCH 14/18] Minor docs correction --- cpp/src/arrow/json/reader.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/cpp/src/arrow/json/reader.h b/cpp/src/arrow/json/reader.h index 8a83f9fce1f..7776cb0b7d8 100644 --- a/cpp/src/arrow/json/reader.h +++ b/cpp/src/arrow/json/reader.h @@ -75,8 +75,8 @@ class ARROW_EXPORT StreamingReader : public RecordBatchReader { virtual ~StreamingReader() = default; /// \brief Read the next `RecordBatch` asynchronously - /// If threading is enabled, this function is async-reentrant (but not synchronously - /// reentrant). + /// This function is async-reentrant (but not synchronously reentrant). However, if + /// threading is disabled, this will block until completion. virtual Future> ReadNextAsync() = 0; /// Get the number of bytes which have been succesfully converted to record batches From 09454a5b96372edf6983ac8af442cc39a76c9481 Mon Sep 17 00:00:00 2001 From: benibus Date: Wed, 7 Dec 2022 11:10:05 -0500 Subject: [PATCH 15/18] More robust cpu executor transfer --- cpp/src/arrow/json/reader.cc | 30 +++++------------------------- 1 file changed, 5 insertions(+), 25 deletions(-) diff --git a/cpp/src/arrow/json/reader.cc b/cpp/src/arrow/json/reader.cc index b3fc5e59aef..99d275f9c4e 100644 --- a/cpp/src/arrow/json/reader.cc +++ b/cpp/src/arrow/json/reader.cc @@ -326,37 +326,17 @@ class DecodingOperator { std::shared_ptr context_; }; -// TODO(benibus): Replace with `MakeApplyGenerator` from -// github.com/apache/arrow/pull/14269 if/when it gets merged -// // Reads from the source and spawns fan-out decoding tasks on the given executor AsyncGenerator MakeDecodingGenerator( AsyncGenerator source, std::function(const ChunkedBlock&)> decoder, Executor* executor) { - struct State { - AsyncGenerator source; - std::function(const ChunkedBlock&)> decoder; - Executor* executor; - } state{std::move(source), std::move(decoder), executor}; - - return [state = std::make_shared(std::move(state))] { - auto options = CallbackOptions::Defaults(); - options.executor = state->executor; - // Since the decode step is heavy we want to schedule it as - // a separate task so as to maximize task distribution accross CPU cores - options.should_schedule = ShouldSchedule::Always; - - return state->source().Then( - [state](const ChunkedBlock& block) -> Result { - if (IsIterationEnd(block)) { - return IterationEnd(); - } else { - return state->decoder(block); - } - }, - {}, options); + AsyncGenerator gen = [source = std::move(source), executor] { + // Since the decode step is heavy we want to schedule it as a separate task so as to + // maximize task distribution accross CPU cores + return executor->TransferAlways(source()); }; + return MakeMappedGenerator(std::move(gen), std::move(decoder)); } // Reads from a source iterator serially, completes subsequent decode tasks on the calling From 8720e26cc6de37cb00e92082ac4e2dc9d697e59a Mon Sep 17 00:00:00 2001 From: benibus Date: Wed, 7 Dec 2022 22:29:36 -0500 Subject: [PATCH 16/18] Refactor, remove cruft --- cpp/src/arrow/json/reader.cc | 69 +++++++++++++++--------------------- 1 file changed, 28 insertions(+), 41 deletions(-) diff --git a/cpp/src/arrow/json/reader.cc b/cpp/src/arrow/json/reader.cc index 99d275f9c4e..f8cd82c2bbe 100644 --- a/cpp/src/arrow/json/reader.cc +++ b/cpp/src/arrow/json/reader.cc @@ -326,21 +326,9 @@ class DecodingOperator { std::shared_ptr context_; }; -// Reads from the source and spawns fan-out decoding tasks on the given executor -AsyncGenerator MakeDecodingGenerator( - AsyncGenerator source, - std::function(const ChunkedBlock&)> decoder, - Executor* executor) { - AsyncGenerator gen = [source = std::move(source), executor] { - // Since the decode step is heavy we want to schedule it as a separate task so as to - // maximize task distribution accross CPU cores - return executor->TransferAlways(source()); - }; - return MakeMappedGenerator(std::move(gen), std::move(decoder)); -} - -// Reads from a source iterator serially, completes subsequent decode tasks on the calling -// thread. +// Reads from a source iterator, completes the subsequent decode task on the calling +// thread. This is only really used for compatibility with the async pipeline when CPU +// threading is disabled AsyncGenerator MakeDecodingGenerator( Iterator source, std::function(const ChunkedBlock&)> decoder) { @@ -353,7 +341,7 @@ AsyncGenerator MakeDecodingGenerator( if (!maybe_block.ok()) { return Future::MakeFinished(maybe_block.status()); } - auto block = maybe_block.MoveValueUnsafe(); + const auto& block = maybe_block.ValueUnsafe(); if (IsIterationEnd(block)) { return ToFuture(IterationEnd()); } @@ -361,23 +349,6 @@ AsyncGenerator MakeDecodingGenerator( }; } -// Adds async-reentrancy to `source` by submitting tasks to a single-threaded executor -// (FIFO order) - ensuring, at most, one future is pending at a time -template -Result> MakeReentrantGenerator(AsyncGenerator source) { - struct State { - AsyncGenerator source; - std::shared_ptr thread_pool; - } state{std::move(source), nullptr}; - ARROW_ASSIGN_OR_RAISE(state.thread_pool, ThreadPool::Make(1)); - - return [state = std::make_shared(std::move(state))]() -> Future { - auto maybe_future = - state->thread_pool->Submit([state] { return state->source().result(); }); - return DeferNotOk(std::move(maybe_future)); - }; -} - class StreamingReaderImpl : public StreamingReader { public: StreamingReaderImpl(DecodedBlock first_block, AsyncGenerator source, @@ -410,20 +381,36 @@ class StreamingReaderImpl : public StreamingReader { AsyncGenerator decoding_gen; int max_readahead = 0; if (read_options.use_threads) { - // Prepare a source generator capable of async-reentrancy and parallel exececution + // Prepare a source generator capable of async-reentrancy and parallel execution if (!cpu_executor) { cpu_executor = GetCpuThreadPool(); - max_readahead = cpu_executor->GetCapacity(); } - buffer_gen = MakeTransferredGenerator(std::move(buffer_gen), cpu_executor); + max_readahead = cpu_executor->GetCapacity(); + + // Since the chunking/decoding steps are heavy we want to schedule them as a + // separate task so as to maximize task distribution across CPU cores + // + // TODO: Add an `always_transfer` parameter to `MakeTransferredGenerator`? + buffer_gen = [source = std::move(buffer_gen), cpu_executor] { + return cpu_executor->TransferAlways(source()); + }; auto chunking_gen = MakeChunkingGenerator(std::move(buffer_gen), MakeChunker(context->parse_options())); - ARROW_ASSIGN_OR_RAISE(chunking_gen, - MakeReentrantGenerator(std::move(chunking_gen))); - decoding_gen = MakeDecodingGenerator(std::move(chunking_gen), - DecodingOperator(context), cpu_executor); + + // Despite having already transferred to the CPU executor, we don't bother + // synchronizing access to the chunking generator because `MappingGenerator` queues + // jobs and keeps only one future from its source active at a time. This is also + // why we can apply readahead later despite the generator we're providing not being + // async-reentrant + // + // The subsequent decoding task should run on the same CPU thread as the chunking + // continuation. However, the next read can be initialized before then + decoding_gen = + MakeMappedGenerator(std::move(chunking_gen), DecodingOperator(context)); } else { - // Prepare a source generator without a separate cpu executor + buffer_gen = MakeTransferredGenerator(std::move(buffer_gen), io_context.executor()); + // We convert the background generator back to an iterator so its work can remain on + // the IO pool while we process its buffers on the calling thread auto chunking_it = MakeChunkingIterator(MakeGeneratorIterator(std::move(buffer_gen)), MakeChunker(context->parse_options())); From 17fc378f6b1ebffec4b877895511bd100e9c70e8 Mon Sep 17 00:00:00 2001 From: benibus Date: Thu, 8 Dec 2022 15:01:10 -0500 Subject: [PATCH 17/18] Test improvements --- cpp/src/arrow/json/reader_test.cc | 111 +++++++++++++++++------------- 1 file changed, 63 insertions(+), 48 deletions(-) diff --git a/cpp/src/arrow/json/reader_test.cc b/cpp/src/arrow/json/reader_test.cc index cd59cc98820..7758da29a52 100644 --- a/cpp/src/arrow/json/reader_test.cc +++ b/cpp/src/arrow/json/reader_test.cc @@ -372,6 +372,23 @@ class StreamingReaderTestBase { ASSERT_TRUE(IsIterationEnd(out)); } + static void AssertBatchSequenceEquals(const RecordBatchVector& expected_batches, + const RecordBatchVector& sequence) { + ASSERT_OK_AND_ASSIGN(auto expected_table, Table::FromRecordBatches(expected_batches)); + ASSERT_OK(expected_table->ValidateFull()); + + auto first_null = std::find(sequence.cbegin(), sequence.cend(), nullptr); + for (auto it = first_null; it != sequence.cend(); ++it) { + ASSERT_EQ(*it, nullptr); + } + + RecordBatchVector batches(sequence.cbegin(), first_null); + EXPECT_EQ(batches.size(), expected_batches.size()); + ASSERT_OK_AND_ASSIGN(auto table, Table::FromRecordBatches(batches)); + ASSERT_OK(table->ValidateFull()); + ASSERT_TABLES_EQUAL(*expected_table, *table); + } + struct TestCase { std::string json; int json_size; @@ -380,7 +397,6 @@ class StreamingReaderTestBase { int num_batches; std::shared_ptr schema; RecordBatchVector batches; - std::shared_ptr
table; }; // Creates a test case from valid JSON objects with a human-readable index field and a @@ -434,7 +450,6 @@ class StreamingReaderTestBase { out.json_size = static_cast(out.json.size()); out.block_size = static_cast(block_size); out.num_batches = static_cast(out.batches.size()); - out.table = *Table::FromRecordBatches(out.batches); return out; } @@ -472,11 +487,13 @@ INSTANTIATE_TEST_SUITE_P(StreamingReaderTest, StreamingReaderTest, ::testing::Values(false, true)); TEST_P(StreamingReaderTest, ErrorOnEmptyStream) { - ASSERT_RAISES(Invalid, MakeReader("")); + EXPECT_RAISES_WITH_MESSAGE_THAT( + Invalid, ::testing::StartsWith("Invalid: Empty JSON stream"), MakeReader("")); std::string data(100, '\n'); for (auto block_size : {25, 49, 50, 100, 200}) { read_options_.block_size = block_size; - ASSERT_RAISES(Invalid, MakeReader(data)); + EXPECT_RAISES_WITH_MESSAGE_THAT( + Invalid, ::testing::StartsWith("Invalid: Empty JSON stream"), MakeReader(data)); } } @@ -500,7 +517,10 @@ TEST_P(StreamingReaderTest, PropagateChunkingErrors) { "\n"); read_options_.block_size = 10; - ASSERT_RAISES(Invalid, MakeReader(bad_first_chunk)); + EXPECT_RAISES_WITH_MESSAGE_THAT( + Invalid, + ::testing::StartsWith("Invalid: straddling object straddles two block boundaries"), + MakeReader(bad_first_chunk)); ASSERT_OK_AND_ASSIGN(auto reader, MakeReader(bad_middle_chunk, kIoLatency)); @@ -509,7 +529,10 @@ TEST_P(StreamingReaderTest, PropagateChunkingErrors) { EXPECT_EQ(reader->bytes_processed(), 9); ASSERT_BATCHES_EQUAL(*RecordBatchFromJSON(test_schema, "[{\"i\":0}]"), *batch); - ASSERT_RAISES(Invalid, reader->ReadNext(&batch)); + EXPECT_RAISES_WITH_MESSAGE_THAT( + Invalid, + ::testing::StartsWith("Invalid: straddling object straddles two block boundaries"), + reader->ReadNext(&batch)); EXPECT_EQ(reader->bytes_processed(), 9); AssertReadEnd(reader); AssertReadEnd(reader); @@ -540,8 +563,12 @@ TEST_P(StreamingReaderTest, PropagateParsingErrors) { "\n"); read_options_.block_size = 16; - ASSERT_RAISES(Invalid, MakeReader(bad_first_block)); - ASSERT_RAISES(Invalid, MakeReader(bad_first_block_after_empty)); + EXPECT_RAISES_WITH_MESSAGE_THAT( + Invalid, ::testing::StartsWith("Invalid: JSON parse error: Invalid value"), + MakeReader(bad_first_block)); + EXPECT_RAISES_WITH_MESSAGE_THAT( + Invalid, ::testing::StartsWith("Invalid: JSON parse error: Invalid value"), + MakeReader(bad_first_block_after_empty)); std::shared_ptr batch; ASSERT_OK_AND_ASSIGN(auto reader, MakeReader(bad_middle_block)); @@ -552,7 +579,11 @@ TEST_P(StreamingReaderTest, PropagateParsingErrors) { EXPECT_EQ(reader->bytes_processed(), 13); ASSERT_BATCHES_EQUAL(*RecordBatchFromJSON(test_schema, R"([{"n":10000}])"), *batch); - ASSERT_RAISES(Invalid, reader->ReadNext(&batch)); + EXPECT_RAISES_WITH_MESSAGE_THAT( + Invalid, + ::testing::StartsWith( + "Invalid: JSON parse error: Missing a comma or '}' after an object member"), + reader->ReadNext(&batch)); EXPECT_EQ(reader->bytes_processed(), 13); AssertReadEnd(reader); EXPECT_EQ(reader->bytes_processed(), 13); @@ -585,10 +616,11 @@ TEST_P(StreamingReaderTest, PropagateErrorsNonLinewiseChunker) { AssertReadNext(reader, &batch); EXPECT_EQ(reader->bytes_processed(), 7); ASSERT_BATCHES_EQUAL(*RecordBatchFromJSON(test_schema, "[{\"i\":0}]"), *batch); - status = reader->ReadNext(&batch); + + EXPECT_RAISES_WITH_MESSAGE_THAT(Invalid, + ::testing::StartsWith("Invalid: JSON parse error"), + reader->ReadNext(&batch)); EXPECT_EQ(reader->bytes_processed(), 7); - ASSERT_RAISES(Invalid, status); - EXPECT_THAT(status.message(), ::testing::StartsWith("JSON parse error")); AssertReadEnd(reader); ASSERT_OK_AND_ASSIGN(reader, MakeReader(bad_middle_blocks)); @@ -600,11 +632,10 @@ TEST_P(StreamingReaderTest, PropagateErrorsNonLinewiseChunker) { EXPECT_EQ(reader->bytes_processed(), 20); ASSERT_BATCHES_EQUAL(*RecordBatchFromJSON(test_schema, "[{\"i\":1}]"), *batch); - status = reader->ReadNext(&batch); + EXPECT_RAISES_WITH_MESSAGE_THAT(Invalid, + ::testing::StartsWith("Invalid: JSON parse error"), + reader->ReadNext(&batch)); EXPECT_EQ(reader->bytes_processed(), 20); - // Should fail to parse "{}\"i\"" - ASSERT_RAISES(Invalid, status); - EXPECT_THAT(status.message(), ::testing::StartsWith("JSON parse error")); // Incoming chunker error from ":2}" shouldn't leak through after the first failure, // which is a possibility if async tasks are still outstanding due to readahead. AssertReadEnd(reader); @@ -648,10 +679,9 @@ TEST_P(StreamingReaderTest, ExplicitSchemaErrorOnUnexpectedFields) { parse_options_.unexpected_field_behavior = UnexpectedFieldBehavior::Error; read_options_.block_size = 48; - auto result = MakeReader(test_json); - ASSERT_RAISES(Invalid, result); - EXPECT_THAT(result.status().message(), - ::testing::StartsWith("JSON parse error: unexpected field")); + EXPECT_RAISES_WITH_MESSAGE_THAT( + Invalid, ::testing::StartsWith("Invalid: JSON parse error: unexpected field"), + MakeReader(test_json)); expected_fields.push_back(field("t", utf8())); expected_schema = schema(expected_fields); @@ -671,11 +701,10 @@ TEST_P(StreamingReaderTest, ExplicitSchemaErrorOnUnexpectedFields) { *RecordBatchFromJSON(expected_schema, R"([{"s":"bar","t":"2022-01-02"}])"), *batch); EXPECT_EQ(reader->bytes_processed(), 64); - auto status = reader->ReadNext(&batch); + EXPECT_RAISES_WITH_MESSAGE_THAT( + Invalid, ::testing::StartsWith("Invalid: JSON parse error: unexpected field"), + reader->ReadNext(&batch)); EXPECT_EQ(reader->bytes_processed(), 64); - ASSERT_RAISES(Invalid, status); - EXPECT_THAT(status.message(), - ::testing::StartsWith("JSON parse error: unexpected field")); AssertReadEnd(reader); } @@ -743,7 +772,9 @@ TEST_P(StreamingReaderTest, InferredSchema) { EXPECT_EQ(reader->bytes_processed(), 28); ASSERT_BATCHES_EQUAL(*expected_batch, *actual_batch); - ASSERT_RAISES(Invalid, reader->ReadNext(&actual_batch)); + EXPECT_RAISES_WITH_MESSAGE_THAT( + Invalid, ::testing::StartsWith("Invalid: JSON parse error: unexpected field"), + reader->ReadNext(&actual_batch)); // Schema derived from the first 2 lines fields.push_back(field("c", boolean())); @@ -761,7 +792,9 @@ TEST_P(StreamingReaderTest, InferredSchema) { EXPECT_EQ(reader->bytes_processed(), 56); ASSERT_BATCHES_EQUAL(*expected_batch, *actual_batch); - ASSERT_RAISES(Invalid, reader->ReadNext(&actual_batch)); + EXPECT_RAISES_WITH_MESSAGE_THAT( + Invalid, ::testing::StartsWith("Invalid: JSON parse error: unexpected field"), + reader->ReadNext(&actual_batch)); // Schema derived from all 3 lines fields.push_back(field("d", timestamp(TimeUnit::SECOND))); @@ -802,12 +835,7 @@ TEST_F(AsyncStreamingReaderTest, AsyncReentrancy) { ASSERT_FINISHES_OK_AND_ASSIGN(auto results, All(std::move(futures))); EXPECT_EQ(reader->bytes_processed(), expected.json_size); ASSERT_OK_AND_ASSIGN(auto batches, internal::UnwrapOrRaise(std::move(results))); - batches.erase(std::remove(batches.begin(), batches.end(), nullptr), batches.end()); - EXPECT_EQ(batches.size(), static_cast(expected.num_batches)); - - ASSERT_OK_AND_ASSIGN(auto table, Table::FromRecordBatches(batches)); - ASSERT_OK(table->ValidateFull()); - ASSERT_TABLES_EQUAL(*expected.table, *table); + AssertBatchSequenceEquals(expected.batches, batches); } TEST_F(AsyncStreamingReaderTest, FuturesOutliveReader) { @@ -831,12 +859,7 @@ TEST_F(AsyncStreamingReaderTest, FuturesOutliveReader) { ASSERT_FINISHES_OK_AND_ASSIGN(auto results, All(std::move(futures))); ASSERT_OK_AND_ASSIGN(auto batches, internal::UnwrapOrRaise(std::move(results))); - batches.erase(std::remove(batches.begin(), batches.end(), nullptr), batches.end()); - EXPECT_EQ(batches.size(), static_cast(expected.num_batches)); - - ASSERT_OK_AND_ASSIGN(auto table, Table::FromRecordBatches(batches)); - ASSERT_OK(table->ValidateFull()); - ASSERT_TABLES_EQUAL(*expected.table, *table); + AssertBatchSequenceEquals(expected.batches, batches); } TEST_F(AsyncStreamingReaderTest, StressBufferedReads) { @@ -856,12 +879,7 @@ TEST_F(AsyncStreamingReaderTest, StressBufferedReads) { ASSERT_FINISHES_OK_AND_ASSIGN(auto results, All(std::move(futures))); ASSERT_OK_AND_ASSIGN(auto batches, internal::UnwrapOrRaise(results)); - batches.erase(std::remove(batches.begin(), batches.end(), nullptr), batches.end()); - EXPECT_EQ(batches.size(), static_cast(expected.num_batches)); - - ASSERT_OK_AND_ASSIGN(auto table, Table::FromRecordBatches(batches)); - ASSERT_OK(table->ValidateFull()); - ASSERT_TABLES_EQUAL(*expected.table, *table); + AssertBatchSequenceEquals(expected.batches, batches); } TEST_F(AsyncStreamingReaderTest, StressSharedIoAndCpuExecutor) { @@ -880,10 +898,7 @@ TEST_F(AsyncStreamingReaderTest, StressSharedIoAndCpuExecutor) { ASSERT_OK_AND_ASSIGN(auto generator, MakeGenerator(expected.json, kIoLatency)); ASSERT_FINISHES_OK_AND_ASSIGN(auto batches, CollectAsyncGenerator(generator)); - ASSERT_EQ(batches.size(), expected.batches.size()); - ASSERT_OK_AND_ASSIGN(auto table, Table::FromRecordBatches(batches)); - ASSERT_OK(table->ValidateFull()); - ASSERT_TABLES_EQUAL(*expected.table, *table); + AssertBatchSequenceEquals(expected.batches, batches); } } // namespace json From 71db48183dc380a52e4d98dd2555e2c6a3b4ee93 Mon Sep 17 00:00:00 2001 From: benibus Date: Mon, 12 Dec 2022 12:07:53 -0500 Subject: [PATCH 18/18] Add explanatory comment/diagram --- cpp/src/arrow/json/reader.cc | 32 +++++++++++++++++++++++++------- 1 file changed, 25 insertions(+), 7 deletions(-) diff --git a/cpp/src/arrow/json/reader.cc b/cpp/src/arrow/json/reader.cc index f8cd82c2bbe..dae06d5bf61 100644 --- a/cpp/src/arrow/json/reader.cc +++ b/cpp/src/arrow/json/reader.cc @@ -397,14 +397,32 @@ class StreamingReaderImpl : public StreamingReader { auto chunking_gen = MakeChunkingGenerator(std::move(buffer_gen), MakeChunker(context->parse_options())); - // Despite having already transferred to the CPU executor, we don't bother - // synchronizing access to the chunking generator because `MappingGenerator` queues - // jobs and keeps only one future from its source active at a time. This is also - // why we can apply readahead later despite the generator we're providing not being - // async-reentrant + // At this stage, we want to allow the decoding tasks for each chunked block to run + // in parallel on the CPU executor. However: + // - Chunking is inherently serial and not thread-safe + // - The chunking generator is not async-reentrant, won't play well with readahead // - // The subsequent decoding task should run on the same CPU thread as the chunking - // continuation. However, the next read can be initialized before then + // Fortunately, `MappingGenerator` queues pending jobs and keeps only one future + // from its source active at a time - which takes care of those concerns. In + // addition, it will start the next job within the continuation of the previous one, + // but before invoking its map function (in our case, `DecodingOperator`). This + // allows for decoding tasks to gradually saturate multiple CPU cores over multiple + // iterations. At a high level, this is how the full pipeline would operate in cases + // where decoding tasks are disproportionately expensive: + // + // -------------------------------------------------------------------------- + // Reading: IoThread(?) --> Chunking: CpuThread(0) ... Decoding: CpuThread(0) + // -------------------------------------------------------------------------- + // Decoding: CpuThread(0) + // Reading: IoThread(?) --> Chunking: CpuThread(1) ... Decoding: CpuThread(1) + // -------------------------------------------------------------------------- + // Decoding: CpuThread(0) + // Decoding: CpuThread(1) + // Reading: IoThread(?) --> Chunking: CpuThread(2) ... Decoding: CpuThread(2) + // -------------------------------------------------------------------------- + // + // Remember that we should already be on the CPU executor following chunking, so the + // decoding task simply continues to use that thread rather than spawning a new one. decoding_gen = MakeMappedGenerator(std::move(chunking_gen), DecodingOperator(context)); } else {