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


##########
cpp/src/arrow/json/reader.cc:
##########
@@ -42,132 +42,476 @@ 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;
+// 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<DecodedBlock> MakeDecodingGenerator(
+    AsyncGenerator<ChunkedBlock> source,
+    std::function<Result<DecodedBlock>(const ChunkedBlock&)> decoder,
+    Executor* executor) {
+  struct State {
+    AsyncGenerator<ChunkedBlock> source;
+    std::function<Result<DecodedBlock>(const ChunkedBlock&)> decoder;
+    Executor* executor;
+  } state{std::move(source), std::move(decoder), executor};
+
+  return [state = std::make_shared<State>(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;

Review Comment:
   @pitrou After splitting the async-reentrant tests into a separate fixture, 
the stress tests now get run first. As a result, I'm frequently seeing `DCHECK` 
output after all the tests have passed - `Invalid: operation forbidden during 
or after shutdown` at  
[future.cc:107](https://github.com/benibus/arrow/blob/25a57e170c5966ddb84d6a4d7de5304c0893cd0a/cpp/src/arrow/util/future.cc#L107).
 From what I can tell, this is from the threaded tests that exit early with 
readahead futures still on the global thread pool.
   
   The easiest way to prevent this is to change this line to 
`ShouldSchedule::Never` (the default) - or just use `MakeMappedGenerator`, i 
guess - since the source futures have already been transferred to the cpu 
executor by then. Async-reentrancy would still work, although parallelism would 
be less aggressive.
   
   Anyways, I can push the changes - but I'm not sure if you had any deeper 
insights here. 
   
   



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