benibus commented on code in PR #14355: URL: https://github.com/apache/arrow/pull/14355#discussion_r1046023452
########## cpp/src/arrow/json/reader.cc: ########## @@ -42,132 +42,443 @@ 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 ChunkedBlock { + std::shared_ptr<Buffer> partial; + std::shared_ptr<Buffer> completion; + std::shared_ptr<Buffer> whole; + int64_t index = -1; +}; + +struct DecodedBlock { + std::shared_ptr<RecordBatch> record_batch; + int64_t num_bytes = 0; +}; + +} // namespace +} // namespace json + +template <> +struct IterationTraits<json::ChunkedBlock> { + static json::ChunkedBlock End() { return json::ChunkedBlock{}; } + static bool IsEnd(const json::ChunkedBlock& val) { return val.index < 0; } +}; + +template <> +struct IterationTraits<json::DecodedBlock> { + 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<Schema> 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<Schema> 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<Schema> explicit_schema) { + auto unexpected_field_behavior = parse_options_.unexpected_field_behavior; + if (unexpected_field_behavior == UnexpectedFieldBehavior::InferType) { + unexpected_field_behavior = UnexpectedFieldBehavior::Error; + } + 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<DataType>& conversion_type() const { + return conversion_type_; + } + + private: + ParseOptions parse_options_; + std::shared_ptr<DataType> conversion_type_; + const PromotionGraph* promotion_graph_; + MemoryPool* pool_; +}; + +Result<std::shared_ptr<Array>> ParseBlock(const ChunkedBlock& block, + const ParseOptions& parse_options, + MemoryPool* pool, int64_t* out_size = nullptr) { + std::unique_ptr<BlockParser> 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<Buffer> 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<Array> parsed; + RETURN_NOT_OK(parser->Finish(&parsed)); + + if (out_size) *out_size = size; + + return parsed; +} + +class ChunkingTransformer { + public: + explicit ChunkingTransformer(std::unique_ptr<Chunker> chunker) + : chunker_(std::move(chunker)) {} + + template <typename... Args> + static Transformer<std::shared_ptr<Buffer>, ChunkedBlock> Make(Args&&... args) { + return [self = std::make_shared<ChunkingTransformer>(std::forward<Args>(args)...)]( + std::shared_ptr<Buffer> buffer) { return (*self)(std::move(buffer)); }; + } + + private: + Result<TransformFlow<ChunkedBlock>> operator()(std::shared_ptr<Buffer> next_buffer) { + if (!buffer_) { + if (ARROW_PREDICT_TRUE(!next_buffer)) { + DCHECK_EQ(partial_, nullptr) << "Logic error: non-null partial with null buffer"; + return TransformFinish(); + } + partial_ = std::make_shared<Buffer>(""); + buffer_ = std::move(next_buffer); + return TransformSkip(); + } + DCHECK_NE(partial_, nullptr); + + std::shared_ptr<Buffer> 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<Buffer> 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(ChunkedBlock{std::exchange(partial_, next_partial), + std::move(completion), std::move(whole), + index_++}); + } + + std::unique_ptr<Chunker> chunker_; + std::shared_ptr<Buffer> partial_; + std::shared_ptr<Buffer> buffer_; + int64_t index_ = 0; +}; + +template <typename... Args> +Iterator<ChunkedBlock> MakeChunkingIterator(Iterator<std::shared_ptr<Buffer>> source, + Args&&... args) { + return MakeTransformedIterator(std::move(source), + ChunkingTransformer::Make(std::forward<Args>(args)...)); +} + +// NOTE: Not reentrant. Incoming buffers are processed sequentially and the transformer's +// internal state gets updated on each call. +template <typename... Args> +AsyncGenerator<ChunkedBlock> MakeChunkingGenerator( + AsyncGenerator<std::shared_ptr<Buffer>> source, Args&&... args) { + return MakeTransformedGenerator(std::move(source), + ChunkingTransformer::Make(std::forward<Args>(args)...)); +} class TableReaderImpl : public TableReader, public std::enable_shared_from_this<TableReaderImpl> { public: TableReaderImpl(MemoryPool* pool, const ReadOptions& read_options, const ParseOptions& parse_options, std::shared_ptr<TaskGroup> task_group) - : pool_(pool), + : decode_context_(parse_options, pool), read_options_(read_options), - parse_options_(parse_options), - chunker_(MakeChunker(parse_options_)), task_group_(std::move(task_group)) {} Status Init(std::shared_ptr<io::InputStream> 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<std::shared_ptr<Table>> Read() override { - RETURN_NOT_OK(MakeBuilder()); - - ARROW_ASSIGN_OR_RAISE(auto block, block_iterator_.Next()); - if (block == nullptr) { + auto block_it = MakeChunkingIterator(std::move(buffer_iterator_), + MakeChunker(decode_context_.parse_options())); + + 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"); } - auto self = shared_from_this(); - auto empty = std::make_shared<Buffer>(""); + std::shared_ptr<ChunkedArray> array; + RETURN_NOT_OK(builder_->Finish(&array)); + return Table::FromChunkedStructArray(array); + } - int64_t block_index = 0; - std::shared_ptr<Buffer> partial = empty; + private: + Status MakeBuilder() { + return MakeChunkedArrayBuilder(task_group_, decode_context_.pool(), + decode_context_.promotion_graph(), + decode_context_.conversion_type(), &builder_); + } - while (block != nullptr) { - std::shared_ptr<Buffer> next_block, whole, completion, next_partial; + 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(); + } - ARROW_ASSIGN_OR_RAISE(next_block, block_iterator_.Next()); + DecodeContext decode_context_; + ReadOptions read_options_; + std::shared_ptr<TaskGroup> task_group_; + Iterator<std::shared_ptr<Buffer>> buffer_iterator_; + std::shared_ptr<ChunkedArrayBuilder> builder_; +}; - 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<Buffer> starts_with_whole; - // Get completion of partial from previous block. - RETURN_NOT_OK(chunker_->ProcessWithPartial(partial, block, &completion, - &starts_with_whole)); +// 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: + explicit DecodingOperator(std::shared_ptr<const DecodeContext> context) + : context_(std::move(context)) {} - // Get all whole objects entirely inside the current buffer - RETURN_NOT_OK(chunker_->Process(starts_with_whole, &whole, &next_partial)); - } + Result<DecodedBlock> operator()(const ChunkedBlock& block) const { + int64_t num_bytes; + ARROW_ASSIGN_OR_RAISE(auto unconverted, ParseBlock(block, context_->parse_options(), + context_->pool(), &num_bytes)); - // Launch parse task - task_group_->Append([self, partial, completion, whole, block_index] { - return self->ParseAndInsert(partial, completion, whole, block_index); - }); - block_index++; + std::shared_ptr<ChunkedArrayBuilder> builder; + RETURN_NOT_OK(MakeChunkedArrayBuilder(TaskGroup::MakeSerial(), context_->pool(), + context_->promotion_graph(), + context_->conversion_type(), &builder)); + builder->Insert(0, field("", unconverted->type()), unconverted); - partial = next_partial; - block = next_block; - } + std::shared_ptr<ChunkedArray> chunked; + RETURN_NOT_OK(builder->Finish(&chunked)); + ARROW_ASSIGN_OR_RAISE(auto batch, RecordBatch::FromStructArray(chunked->chunk(0))); - std::shared_ptr<ChunkedArray> array; - RETURN_NOT_OK(builder_->Finish(&array)); - return Table::FromChunkedStructArray(array); + return DecodedBlock{std::move(batch), num_bytes}; } private: - Status MakeBuilder() { - auto type = parse_options_.explicit_schema - ? struct_(parse_options_.explicit_schema->fields()) - : struct_({}); + std::shared_ptr<const DecodeContext> context_; +}; - auto promotion_graph = - parse_options_.unexpected_field_behavior == UnexpectedFieldBehavior::InferType - ? GetPromotionGraph() - : nullptr; +// 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<DecodedBlock> MakeDecodingGenerator( + Iterator<ChunkedBlock> source, + std::function<Result<DecodedBlock>(const ChunkedBlock&)> decoder) { + struct State { + Iterator<ChunkedBlock> source; + std::function<Result<DecodedBlock>(const ChunkedBlock&)> decoder; + } state{std::move(source), std::move(decoder)}; + return [state = std::make_shared<State>(std::move(state))] { + auto maybe_block = state->source.Next(); + if (!maybe_block.ok()) { + return Future<DecodedBlock>::MakeFinished(maybe_block.status()); + } + const auto& block = maybe_block.ValueUnsafe(); + if (IsIterationEnd(block)) { + return ToFuture(IterationEnd<DecodedBlock>()); + } + return ToFuture(state->decoder(block)); + }; +} + +class StreamingReaderImpl : public StreamingReader { + public: + StreamingReaderImpl(DecodedBlock first_block, AsyncGenerator<DecodedBlock> source, + const std::shared_ptr<DecodeContext>& context, int max_readahead) + : first_block_(std::move(first_block)), + schema_(first_block_->record_batch->schema()), + bytes_processed_(std::make_shared<std::atomic<int64_t>>(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; + }); + } - return MakeChunkedArrayBuilder(task_group_, pool_, promotion_graph, type, &builder_); - } - - Status ParseAndInsert(const std::shared_ptr<Buffer>& partial, - const std::shared_ptr<Buffer>& completion, - const std::shared_ptr<Buffer>& whole, int64_t block_index) { - std::unique_ptr<BlockParser> 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<Buffer> straddling; - if (partial->size() == 0) { - straddling = completion; - } else if (completion->size() == 0) { - straddling = partial; - } else { - ARROW_ASSIGN_OR_RAISE(straddling, - ConcatenateBuffers({partial, completion}, pool_)); + static Future<std::shared_ptr<StreamingReaderImpl>> MakeAsync( + std::shared_ptr<DecodeContext> context, std::shared_ptr<io::InputStream> 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())); + + AsyncGenerator<DecodedBlock> decoding_gen; + int max_readahead = 0; + if (read_options.use_threads) { + // Prepare a source generator capable of async-reentrancy and parallel execution + if (!cpu_executor) { + cpu_executor = GetCpuThreadPool(); } - RETURN_NOT_OK(parser->Parse(straddling)); + 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())); + + // 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 Review Comment: Yeah, I'll add something. On an unrelated note, I'm pretty sure this is partially relevant to https://github.com/apache/arrow/issues/14792. The documentation for `MappingGenerator` heavily undersells its usefulness IMO. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: github-unsubscr...@arrow.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org