benibus commented on code in PR #14355:
URL: https://github.com/apache/arrow/pull/14355#discussion_r1043547530


##########
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:
   It just means that a separate CPU thread won't be spawned to do the decoding 
- but due to the way `MappingGenerator` works, the decoding can still run in 
parallel. Each callback from its source starts the next pull before running the 
map function.



-- 
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

Reply via email to