You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@arrow.apache.org by GitBox <gi...@apache.org> on 2021/05/25 21:06:03 UTC

[GitHub] [arrow] bkietz opened a new pull request #10397: ARROW-11930: [C++][Dataset][Compute] Use an ExecPlan for dataset scans

bkietz opened a new pull request #10397:
URL: https://github.com/apache/arrow/pull/10397


   So far this involved a lot of refactoring of Expressions to be compatible with ExecBatches. The next step is to add a ScanNode wrapping a ScannerBuilder


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] lidavidm commented on a change in pull request #10397: ARROW-11930: [C++][Dataset][Compute] Use an ExecPlan for dataset scans

Posted by GitBox <gi...@apache.org>.
lidavidm commented on a change in pull request #10397:
URL: https://github.com/apache/arrow/pull/10397#discussion_r640912315



##########
File path: cpp/src/arrow/compute/exec/exec_plan.cc
##########
@@ -170,48 +165,409 @@ Status ExecPlan::Validate() { return ToDerived(this)->Validate(); }
 
 Status ExecPlan::StartProducing() { return ToDerived(this)->StartProducing(); }
 
-ExecNode::ExecNode(ExecPlan* plan, std::string label,
-                   std::vector<BatchDescr> input_descrs,
+ExecNode::ExecNode(ExecPlan* plan, std::string label, NodeVector inputs,
                    std::vector<std::string> input_labels, BatchDescr output_descr,
                    int num_outputs)
     : plan_(plan),
       label_(std::move(label)),
-      input_descrs_(std::move(input_descrs)),
+      inputs_(std::move(inputs)),
       input_labels_(std::move(input_labels)),
       output_descr_(std::move(output_descr)),
-      num_outputs_(num_outputs) {}
+      num_outputs_(num_outputs) {
+  for (auto input : inputs_) {
+    input->outputs_.push_back(this);
+  }
+}
 
 Status ExecNode::Validate() const {
-  if (inputs_.size() != input_descrs_.size()) {
+  if (inputs_.size() != input_labels_.size()) {
     return Status::Invalid("Invalid number of inputs for '", label(), "' (expected ",
-                           num_inputs(), ", actual ", inputs_.size(), ")");
+                           num_inputs(), ", actual ", input_labels_.size(), ")");
   }
 
   if (static_cast<int>(outputs_.size()) != num_outputs_) {
     return Status::Invalid("Invalid number of outputs for '", label(), "' (expected ",
                            num_outputs(), ", actual ", outputs_.size(), ")");
   }
 
-  DCHECK_EQ(input_descrs_.size(), input_labels_.size());
-
   for (auto out : outputs_) {
     auto input_index = GetNodeIndex(out->inputs(), this);
     if (!input_index) {
       return Status::Invalid("Node '", label(), "' outputs to node '", out->label(),
                              "' but is not listed as an input.");
     }
+  }
 
-    const auto& in_descr = out->input_descrs_[*input_index];
-    if (in_descr != output_descr_) {
-      return Status::Invalid(
-          "Node '", label(), "' (bound to input ", input_labels_[*input_index],
-          ") produces batches with type '", ValueDescr::ToString(output_descr_),
-          "' inconsistent with consumer '", out->label(), "' which accepts '",
-          ValueDescr::ToString(in_descr), "'");
+  return Status::OK();
+}
+
+struct SourceNode : ExecNode {
+  SourceNode(ExecPlan* plan, std::string label, ExecNode::BatchDescr output_descr,
+             AsyncGenerator<util::optional<ExecBatch>> generator)
+      : ExecNode(plan, std::move(label), {}, {}, std::move(output_descr),
+                 /*num_outputs=*/1),
+        generator_(std::move(generator)) {}
+
+  const char* kind_name() override { return "SourceNode"; }
+
+  static void NoInputs() { DCHECK(false) << "no inputs; this should never be called"; }
+  void InputReceived(ExecNode*, int, ExecBatch) override { NoInputs(); }
+  void ErrorReceived(ExecNode*, Status) override { NoInputs(); }
+  void InputFinished(ExecNode*, int) override { NoInputs(); }
+
+  Status StartProducing() override {
+    if (finished_) {
+      return Status::Invalid("Restarted SourceNode '", label(), "'");
     }
+
+    auto gen = std::move(generator_);
+
+    /// XXX should we wait on this future anywhere? In StopProducing() maybe?
+    auto done_fut =
+        Loop([gen, this] {
+          std::unique_lock<std::mutex> lock(mutex_);
+          int seq = next_batch_index_++;
+          if (finished_) {
+            return Future<ControlFlow<int>>::MakeFinished(Break(seq));
+          }
+          lock.unlock();
+
+          return gen().Then(
+              [=](const util::optional<ExecBatch>& batch) -> ControlFlow<int> {
+                std::unique_lock<std::mutex> lock(mutex_);
+                if (!batch || finished_) {
+                  finished_ = true;
+                  return Break(seq);
+                }
+                lock.unlock();
+
+                outputs_[0]->InputReceived(this, seq, *batch);
+                return Continue();
+              },
+              [=](const Status& error) -> ControlFlow<int> {
+                std::unique_lock<std::mutex> lock(mutex_);
+                if (!finished_) {
+                  finished_ = true;
+                  lock.unlock();
+                  // unless we were already finished, push the error to our output
+                  // XXX is this correct? Is it reasonable for a consumer to ignore errors
+                  // from a finished producer?

Review comment:
       Ah, I see. I think I have the same inclination then; except for maybe a sink node that's already gotten all its results, in which case subsequent errors are probably irrelevant, propagating errors even when otherwise 'finished' makes sense.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] westonpace commented on a change in pull request #10397: ARROW-11930: [C++][Dataset][Compute] Use an ExecPlan for dataset scans

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #10397:
URL: https://github.com/apache/arrow/pull/10397#discussion_r643637975



##########
File path: cpp/src/arrow/compute/exec/expression_test.cc
##########
@@ -165,6 +165,56 @@ TEST(ExpressionUtils, StripOrderPreservingCasts) {
   Expect(cast(field_ref("i32"), uint64()), no_change);
 }
 
+TEST(ExpressionUtils, MakeExecBatch) {
+  auto Expect = [](std::shared_ptr<RecordBatch> partial_batch) {
+    SCOPED_TRACE(partial_batch->ToString());
+    ASSERT_OK_AND_ASSIGN(auto batch, MakeExecBatch(*kBoringSchema, partial_batch));
+
+    ASSERT_EQ(batch.num_values(), kBoringSchema->num_fields());
+    for (int i = 0; i < kBoringSchema->num_fields(); ++i) {
+      const auto& field = *kBoringSchema->field(i);
+
+      SCOPED_TRACE("Field#" + std::to_string(i) + " " + field.ToString());
+
+      EXPECT_TRUE(batch[i].type()->Equals(field.type()))
+          << "Incorrect type " << batch[i].type()->ToString();
+
+      ASSERT_OK_AND_ASSIGN(auto col, FieldRef(field.name()).GetOneOrNone(*partial_batch));

Review comment:
       Ah, I expected `GetColumnByName` to just return the first instance of the field (I thought duplicate field names were generally allowed outside of `compute`) but you are correct, it treats it the same as "not found".




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] bkietz commented on a change in pull request #10397: ARROW-11930: [C++][Dataset][Compute] Use an ExecPlan for dataset scans

Posted by GitBox <gi...@apache.org>.
bkietz commented on a change in pull request #10397:
URL: https://github.com/apache/arrow/pull/10397#discussion_r643447166



##########
File path: cpp/src/arrow/compute/exec/exec_plan.h
##########
@@ -225,22 +212,43 @@ class ARROW_EXPORT ExecNode {
   virtual void StopProducing() = 0;
 
  protected:
-  ExecNode(ExecPlan* plan, std::string label, std::vector<BatchDescr> input_descrs,
+  ExecNode(ExecPlan*, std::string label, NodeVector inputs,
            std::vector<std::string> input_labels, BatchDescr output_descr,
            int num_outputs);
 
   ExecPlan* plan_;
-
   std::string label_;
 
-  std::vector<BatchDescr> input_descrs_;
-  std::vector<std::string> input_labels_;
   NodeVector inputs_;
+  std::vector<std::string> input_labels_;
 
   BatchDescr output_descr_;
   int num_outputs_;
   NodeVector outputs_;
 };
 
+/// \brief Adapt an AsyncGenerator<ExecBatch> as a source node
+ARROW_EXPORT
+ExecNode* MakeSourceNode(ExecPlan*, std::string label, ExecNode::BatchDescr output_descr,
+                         AsyncGenerator<util::optional<ExecBatch>>);
+
+/// \brief Add a sink node which forwards to an AsyncGenerator<ExecBatch>
+ARROW_EXPORT
+AsyncGenerator<util::optional<ExecBatch>> MakeSinkNode(ExecNode* input,
+                                                       std::string label);
+
+/// \brief Make a node which excludes some rows from batches passed through it
+///
+/// filter Expression must be bound; no field references will be looked up by name
+ARROW_EXPORT
+ExecNode* MakeFilterNode(ExecNode* input, std::string label, Expression filter);
+
+/// \brief Make a node which executes expressions on input batches, producing new batches.
+///
+/// Expressions must be bound; no field references will be looked up by name
+ARROW_EXPORT
+ExecNode* MakeProjectNode(ExecNode* input, std::string label,

Review comment:
       I'll add clarification




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] pitrou commented on a change in pull request #10397: ARROW-11930: [C++][Dataset][Compute] Use an ExecPlan for dataset scans

Posted by GitBox <gi...@apache.org>.
pitrou commented on a change in pull request #10397:
URL: https://github.com/apache/arrow/pull/10397#discussion_r644308066



##########
File path: cpp/src/arrow/compute/exec/exec_plan.cc
##########
@@ -170,48 +165,409 @@ Status ExecPlan::Validate() { return ToDerived(this)->Validate(); }
 
 Status ExecPlan::StartProducing() { return ToDerived(this)->StartProducing(); }
 
-ExecNode::ExecNode(ExecPlan* plan, std::string label,
-                   std::vector<BatchDescr> input_descrs,
+ExecNode::ExecNode(ExecPlan* plan, std::string label, NodeVector inputs,
                    std::vector<std::string> input_labels, BatchDescr output_descr,
                    int num_outputs)
     : plan_(plan),
       label_(std::move(label)),
-      input_descrs_(std::move(input_descrs)),
+      inputs_(std::move(inputs)),
       input_labels_(std::move(input_labels)),
       output_descr_(std::move(output_descr)),
-      num_outputs_(num_outputs) {}
+      num_outputs_(num_outputs) {
+  for (auto input : inputs_) {
+    input->outputs_.push_back(this);
+  }
+}
 
 Status ExecNode::Validate() const {
-  if (inputs_.size() != input_descrs_.size()) {
+  if (inputs_.size() != input_labels_.size()) {
     return Status::Invalid("Invalid number of inputs for '", label(), "' (expected ",
-                           num_inputs(), ", actual ", inputs_.size(), ")");
+                           num_inputs(), ", actual ", input_labels_.size(), ")");
   }
 
   if (static_cast<int>(outputs_.size()) != num_outputs_) {
     return Status::Invalid("Invalid number of outputs for '", label(), "' (expected ",
                            num_outputs(), ", actual ", outputs_.size(), ")");
   }
 
-  DCHECK_EQ(input_descrs_.size(), input_labels_.size());
-
   for (auto out : outputs_) {
     auto input_index = GetNodeIndex(out->inputs(), this);
     if (!input_index) {
       return Status::Invalid("Node '", label(), "' outputs to node '", out->label(),
                              "' but is not listed as an input.");
     }
+  }
 
-    const auto& in_descr = out->input_descrs_[*input_index];
-    if (in_descr != output_descr_) {
-      return Status::Invalid(
-          "Node '", label(), "' (bound to input ", input_labels_[*input_index],
-          ") produces batches with type '", ValueDescr::ToString(output_descr_),
-          "' inconsistent with consumer '", out->label(), "' which accepts '",
-          ValueDescr::ToString(in_descr), "'");
+  return Status::OK();
+}
+
+struct SourceNode : ExecNode {
+  SourceNode(ExecPlan* plan, std::string label, ExecNode::BatchDescr output_descr,
+             AsyncGenerator<util::optional<ExecBatch>> generator)
+      : ExecNode(plan, std::move(label), {}, {}, std::move(output_descr),
+                 /*num_outputs=*/1),
+        generator_(std::move(generator)) {}
+
+  const char* kind_name() override { return "SourceNode"; }
+
+  static void NoInputs() { DCHECK(false) << "no inputs; this should never be called"; }
+  void InputReceived(ExecNode*, int, ExecBatch) override { NoInputs(); }
+  void ErrorReceived(ExecNode*, Status) override { NoInputs(); }
+  void InputFinished(ExecNode*, int) override { NoInputs(); }
+
+  Status StartProducing() override {
+    if (finished_) {
+      return Status::Invalid("Restarted SourceNode '", label(), "'");
     }
+
+    auto gen = std::move(generator_);
+
+    /// XXX should we wait on this future anywhere? In StopProducing() maybe?
+    auto done_fut =
+        Loop([gen, this] {
+          std::unique_lock<std::mutex> lock(mutex_);
+          int seq = next_batch_index_++;
+          if (finished_) {
+            return Future<ControlFlow<int>>::MakeFinished(Break(seq));
+          }
+          lock.unlock();
+
+          return gen().Then(
+              [=](const util::optional<ExecBatch>& batch) -> ControlFlow<int> {
+                std::unique_lock<std::mutex> lock(mutex_);
+                if (!batch || finished_) {
+                  finished_ = true;
+                  return Break(seq);
+                }
+                lock.unlock();
+
+                outputs_[0]->InputReceived(this, seq, *batch);
+                return Continue();
+              },
+              [=](const Status& error) -> ControlFlow<int> {
+                std::unique_lock<std::mutex> lock(mutex_);
+                if (!finished_) {
+                  finished_ = true;
+                  lock.unlock();
+                  // unless we were already finished, push the error to our output
+                  // XXX is this correct? Is it reasonable for a consumer to ignore errors
+                  // from a finished producer?
+                  outputs_[0]->ErrorReceived(this, error);
+                }
+                return Break(seq);
+              });
+        }).Then([&](int seq) {
+          /// XXX this is probably redundant: do we always call InputFinished after
+          /// ErrorReceived or will ErrorRecieved be sufficient?
+          outputs_[0]->InputFinished(this, seq);
+        });
+
+    return Status::OK();
   }
 
-  return Status::OK();
+  void PauseProducing(ExecNode* output) override {}
+
+  void ResumeProducing(ExecNode* output) override {}
+
+  void StopProducing(ExecNode* output) override {
+    DCHECK_EQ(output, outputs_[0]);
+    std::unique_lock<std::mutex> lock(mutex_);
+    finished_ = true;
+  }
+
+  void StopProducing() override { StopProducing(outputs_[0]); }
+
+ private:
+  std::mutex mutex_;
+  bool finished_{false};
+  int next_batch_index_{0};
+  AsyncGenerator<util::optional<ExecBatch>> generator_;
+};
+
+ExecNode* MakeSourceNode(ExecPlan* plan, std::string label,
+                         ExecNode::BatchDescr output_descr,
+                         AsyncGenerator<util::optional<ExecBatch>> generator) {
+  return plan->EmplaceNode<SourceNode>(plan, std::move(label), std::move(output_descr),
+                                       std::move(generator));
+}
+
+struct FilterNode : ExecNode {
+  FilterNode(ExecNode* input, std::string label, Expression filter)
+      : ExecNode(input->plan(), std::move(label), {input}, {"target"},
+                 /*output_descr=*/{input->output_descr()},
+                 /*num_outputs=*/1),
+        filter_(std::move(filter)) {}
+
+  const char* kind_name() override { return "FilterNode"; }
+
+  Result<ExecBatch> DoFilter(const ExecBatch& target) {
+    ARROW_ASSIGN_OR_RAISE(Expression simplified_filter,
+                          SimplifyWithGuarantee(filter_, target.guarantee));
+
+    // XXX get a non-default exec context
+    ARROW_ASSIGN_OR_RAISE(Datum mask, ExecuteScalarExpression(simplified_filter, target));
+
+    if (mask.is_scalar()) {
+      const auto& mask_scalar = mask.scalar_as<BooleanScalar>();
+      if (mask_scalar.is_valid && mask_scalar.value) {
+        return target;
+      }
+
+      return target.Slice(0, 0);
+    }
+
+    auto values = target.values;
+    for (auto& value : values) {
+      if (value.is_scalar()) continue;
+      ARROW_ASSIGN_OR_RAISE(value, Filter(value, mask, FilterOptions::Defaults()));
+    }
+    return ExecBatch::Make(std::move(values));
+  }
+
+  void InputReceived(ExecNode* input, int seq, ExecBatch batch) override {
+    DCHECK_EQ(input, inputs_[0]);
+
+    auto maybe_filtered = DoFilter(std::move(batch));
+    if (!maybe_filtered.ok()) {
+      outputs_[0]->ErrorReceived(this, maybe_filtered.status());
+      inputs_[0]->StopProducing(this);
+      return;
+    }
+
+    maybe_filtered->guarantee = batch.guarantee;
+    outputs_[0]->InputReceived(this, seq, maybe_filtered.MoveValueUnsafe());
+  }
+
+  void ErrorReceived(ExecNode* input, Status error) override {
+    DCHECK_EQ(input, inputs_[0]);
+    outputs_[0]->ErrorReceived(this, std::move(error));
+    inputs_[0]->StopProducing(this);
+  }
+
+  void InputFinished(ExecNode* input, int seq) override {
+    DCHECK_EQ(input, inputs_[0]);
+    outputs_[0]->InputFinished(this, seq);
+    inputs_[0]->StopProducing(this);
+  }
+
+  Status StartProducing() override {
+    // XXX validate inputs_[0]->output_descr() against filter_
+    return Status::OK();
+  }
+
+  void PauseProducing(ExecNode* output) override {}
+
+  void ResumeProducing(ExecNode* output) override {}
+
+  void StopProducing(ExecNode* output) override {
+    DCHECK_EQ(output, outputs_[0]);
+    inputs_[0]->StopProducing(this);
+  }
+
+  void StopProducing() override { StopProducing(outputs_[0]); }
+
+ private:
+  Expression filter_;
+};
+
+ExecNode* MakeFilterNode(ExecNode* input, std::string label, Expression filter) {
+  return input->plan()->EmplaceNode<FilterNode>(input, std::move(label),
+                                                std::move(filter));
+}
+
+struct ProjectNode : ExecNode {
+  ProjectNode(ExecNode* input, std::string label, std::vector<Expression> exprs)
+      : ExecNode(input->plan(), std::move(label), {input}, {"target"},
+                 /*output_descr=*/{input->output_descr()},
+                 /*num_outputs=*/1),
+        exprs_(std::move(exprs)) {}
+
+  const char* kind_name() override { return "ProjectNode"; }
+
+  Result<ExecBatch> DoProject(const ExecBatch& target) {
+    // XXX get a non-default exec context
+    std::vector<Datum> values{exprs_.size()};
+    for (size_t i = 0; i < exprs_.size(); ++i) {
+      ARROW_ASSIGN_OR_RAISE(Expression simplified_expr,
+                            SimplifyWithGuarantee(exprs_[i], target.guarantee));
+
+      ARROW_ASSIGN_OR_RAISE(values[i], ExecuteScalarExpression(simplified_expr, target));
+    }
+    return ExecBatch::Make(std::move(values));
+  }
+
+  void InputReceived(ExecNode* input, int seq, ExecBatch batch) override {
+    DCHECK_EQ(input, inputs_[0]);
+
+    auto maybe_projected = DoProject(std::move(batch));
+    if (!maybe_projected.ok()) {
+      outputs_[0]->ErrorReceived(this, maybe_projected.status());
+      inputs_[0]->StopProducing(this);
+      return;
+    }
+
+    maybe_projected->guarantee = batch.guarantee;
+    outputs_[0]->InputReceived(this, seq, maybe_projected.MoveValueUnsafe());
+  }
+
+  void ErrorReceived(ExecNode* input, Status error) override {
+    DCHECK_EQ(input, inputs_[0]);
+    outputs_[0]->ErrorReceived(this, std::move(error));
+    inputs_[0]->StopProducing(this);
+  }
+
+  void InputFinished(ExecNode* input, int seq) override {
+    DCHECK_EQ(input, inputs_[0]);
+    outputs_[0]->InputFinished(this, seq);
+    inputs_[0]->StopProducing(this);
+  }
+
+  Status StartProducing() override {
+    // XXX validate inputs_[0]->output_descr() against filter_
+    return Status::OK();
+  }
+
+  void PauseProducing(ExecNode* output) override {}
+
+  void ResumeProducing(ExecNode* output) override {}
+
+  void StopProducing(ExecNode* output) override {
+    DCHECK_EQ(output, outputs_[0]);
+    inputs_[0]->StopProducing(this);
+  }
+
+  void StopProducing() override { StopProducing(outputs_[0]); }
+
+ private:
+  std::vector<Expression> exprs_;
+};
+
+ExecNode* MakeProjectNode(ExecNode* input, std::string label,
+                          std::vector<Expression> exprs) {
+  return input->plan()->EmplaceNode<ProjectNode>(input, std::move(label),
+                                                 std::move(exprs));
+}
+
+struct SinkNode : ExecNode {
+  SinkNode(ExecNode* input, std::string label,
+           AsyncGenerator<util::optional<ExecBatch>>* generator)
+      : ExecNode(input->plan(), std::move(label), {input}, {"collected"}, {},
+                 /*num_outputs=*/0),
+        producer_(MakeProducer(generator)) {}
+
+  static PushGenerator<util::optional<ExecBatch>>::Producer MakeProducer(
+      AsyncGenerator<util::optional<ExecBatch>>* out_gen) {
+    PushGenerator<util::optional<ExecBatch>> gen;
+    auto out = gen.producer();
+    *out_gen = std::move(gen);
+    return out;
+  }
+
+  const char* kind_name() override { return "SinkNode"; }
+
+  Status StartProducing() override { return Status::OK(); }
+
+  // sink nodes have no outputs from which to feel backpressure
+  static void NoOutputs() { DCHECK(false) << "no outputs; this should never be called"; }
+  void ResumeProducing(ExecNode* output) override { NoOutputs(); }
+  void PauseProducing(ExecNode* output) override { NoOutputs(); }
+  void StopProducing(ExecNode* output) override { NoOutputs(); }
+
+  void StopProducing() override {
+    std::unique_lock<std::mutex> lock(mutex_);
+    StopProducingUnlocked();
+  }
+
+  void InputReceived(ExecNode* input, int seq_num, ExecBatch exec_batch) override {
+    std::unique_lock<std::mutex> lock(mutex_);
+    if (stopped_) return;
+
+    // TODO would be nice to factor this out in a ReorderQueue
+    if (seq_num <= static_cast<int>(received_batches_.size())) {
+      received_batches_.resize(seq_num + 1);
+      emitted_.resize(seq_num + 1, false);
+    }
+    received_batches_[seq_num] = std::move(exec_batch);
+    ++num_received_;
+
+    if (seq_num != num_emitted_) {
+      // Cannot emit yet as there is a hole at `num_emitted_`
+      DCHECK_GT(seq_num, num_emitted_);
+      return;
+    }
+
+    if (num_received_ == emit_stop_) {
+      StopProducingUnlocked();
+    }
+
+    // Emit batches in order as far as possible
+    // First collect these batches, then unlock before producing.
+    const auto seq_start = seq_num;
+    while (seq_num < static_cast<int>(emitted_.size()) && !emitted_[seq_num]) {
+      emitted_[seq_num] = true;
+      ++seq_num;
+    }
+    DCHECK_GT(seq_num, seq_start);
+    // By moving the values now, we make sure another thread won't emit the same values
+    // below
+    std::vector<ExecBatch> to_emit(
+        std::make_move_iterator(received_batches_.begin() + seq_start),
+        std::make_move_iterator(received_batches_.begin() + seq_num));
+
+    lock.unlock();
+    for (auto&& batch : to_emit) {
+      producer_.Push(std::move(batch));
+    }
+    lock.lock();
+
+    DCHECK_EQ(seq_start, num_emitted_);  // num_emitted_ wasn't bumped in the meantime
+    num_emitted_ = seq_num;

Review comment:
       You're definitely right that there should be some kind of test at the end to see if more batches need to be more emitted.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] lidavidm commented on a change in pull request #10397: ARROW-11930: [C++][Dataset][Compute] Use an ExecPlan for dataset scans

Posted by GitBox <gi...@apache.org>.
lidavidm commented on a change in pull request #10397:
URL: https://github.com/apache/arrow/pull/10397#discussion_r640905731



##########
File path: cpp/src/arrow/compute/exec/expression.cc
##########
@@ -492,9 +448,18 @@ Result<Expression> Expression::Bind(ValueDescr in,
   if (literal()) return *this;
 
   if (auto ref = field_ref()) {
-    ARROW_ASSIGN_OR_RAISE(auto field, ref->GetOneOrNone(*in.type));
-    auto descr = field ? ValueDescr{field->type(), in.shape} : ValueDescr::Scalar(null());
-    return Expression{Parameter{*ref, std::move(descr)}};
+    if (ref->IsNested()) {
+      return Status::NotImplemented("nested field references");
+    }
+
+    ARROW_ASSIGN_OR_RAISE(auto path, ref->FindOne(*in.type));

Review comment:
       Yeah, when I was digging into the wide-dataframe performance regression, it boiled almost entirely down to us doing a linear field lookup for every field, for every batch (ARROW-11469)




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] westonpace commented on a change in pull request #10397: ARROW-11930: [C++][Dataset][Compute] Use an ExecPlan for dataset scans

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #10397:
URL: https://github.com/apache/arrow/pull/10397#discussion_r644146368



##########
File path: cpp/src/arrow/compute/exec/exec_plan.cc
##########
@@ -170,48 +165,409 @@ Status ExecPlan::Validate() { return ToDerived(this)->Validate(); }
 
 Status ExecPlan::StartProducing() { return ToDerived(this)->StartProducing(); }
 
-ExecNode::ExecNode(ExecPlan* plan, std::string label,
-                   std::vector<BatchDescr> input_descrs,
+ExecNode::ExecNode(ExecPlan* plan, std::string label, NodeVector inputs,
                    std::vector<std::string> input_labels, BatchDescr output_descr,
                    int num_outputs)
     : plan_(plan),
       label_(std::move(label)),
-      input_descrs_(std::move(input_descrs)),
+      inputs_(std::move(inputs)),
       input_labels_(std::move(input_labels)),
       output_descr_(std::move(output_descr)),
-      num_outputs_(num_outputs) {}
+      num_outputs_(num_outputs) {
+  for (auto input : inputs_) {
+    input->outputs_.push_back(this);
+  }
+}
 
 Status ExecNode::Validate() const {
-  if (inputs_.size() != input_descrs_.size()) {
+  if (inputs_.size() != input_labels_.size()) {
     return Status::Invalid("Invalid number of inputs for '", label(), "' (expected ",
-                           num_inputs(), ", actual ", inputs_.size(), ")");
+                           num_inputs(), ", actual ", input_labels_.size(), ")");
   }
 
   if (static_cast<int>(outputs_.size()) != num_outputs_) {
     return Status::Invalid("Invalid number of outputs for '", label(), "' (expected ",
                            num_outputs(), ", actual ", outputs_.size(), ")");
   }
 
-  DCHECK_EQ(input_descrs_.size(), input_labels_.size());
-
   for (auto out : outputs_) {
     auto input_index = GetNodeIndex(out->inputs(), this);
     if (!input_index) {
       return Status::Invalid("Node '", label(), "' outputs to node '", out->label(),
                              "' but is not listed as an input.");
     }
+  }
 
-    const auto& in_descr = out->input_descrs_[*input_index];
-    if (in_descr != output_descr_) {
-      return Status::Invalid(
-          "Node '", label(), "' (bound to input ", input_labels_[*input_index],
-          ") produces batches with type '", ValueDescr::ToString(output_descr_),
-          "' inconsistent with consumer '", out->label(), "' which accepts '",
-          ValueDescr::ToString(in_descr), "'");
+  return Status::OK();
+}
+
+struct SourceNode : ExecNode {
+  SourceNode(ExecPlan* plan, std::string label, ExecNode::BatchDescr output_descr,
+             AsyncGenerator<util::optional<ExecBatch>> generator)
+      : ExecNode(plan, std::move(label), {}, {}, std::move(output_descr),
+                 /*num_outputs=*/1),
+        generator_(std::move(generator)) {}
+
+  const char* kind_name() override { return "SourceNode"; }
+
+  static void NoInputs() { DCHECK(false) << "no inputs; this should never be called"; }
+  void InputReceived(ExecNode*, int, ExecBatch) override { NoInputs(); }
+  void ErrorReceived(ExecNode*, Status) override { NoInputs(); }
+  void InputFinished(ExecNode*, int) override { NoInputs(); }
+
+  Status StartProducing() override {
+    if (finished_) {
+      return Status::Invalid("Restarted SourceNode '", label(), "'");
     }
+
+    auto gen = std::move(generator_);
+
+    /// XXX should we wait on this future anywhere? In StopProducing() maybe?
+    auto done_fut =
+        Loop([gen, this] {

Review comment:
       We have `VisitAsyncGenerator` which could probably be used here and seems more intuitive but it wouldn't apply back pressure.  We could enhance visit so that `VisitAsyncGenerator` takes a mandatory visitor function (like it does today) and an optional  `should_pause` or `is_ready` function (which returns an unfinished future if it is not ready).




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] bkietz commented on a change in pull request #10397: ARROW-11930: [C++][Dataset][Compute] Use an ExecPlan for dataset scans

Posted by GitBox <gi...@apache.org>.
bkietz commented on a change in pull request #10397:
URL: https://github.com/apache/arrow/pull/10397#discussion_r640897940



##########
File path: cpp/src/arrow/compute/exec/expression.h
##########
@@ -64,8 +64,11 @@ class ARROW_EXPORT Expression {
   /// Bind this expression to the given input type, looking up Kernels and field types.
   /// Some expression simplification may be performed and implicit casts will be inserted.
   /// Any state necessary for execution will be initialized and returned.
-  Result<Expression> Bind(ValueDescr in, compute::ExecContext* = NULLPTR) const;
-  Result<Expression> Bind(const Schema& in_schema, compute::ExecContext* = NULLPTR) const;
+  Result<Expression> Bind(ValueDescr in, ExecContext* = NULLPTR) const;
+  Result<Expression> Bind(const Schema& in_schema, ExecContext* = NULLPTR) const;
+
+  Result<Expression> BindFlattened(ValueDescr in, ExecContext* = NULLPTR) const;

Review comment:
       No, this should be removed.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] bkietz commented on pull request #10397: ARROW-11930: [C++][Dataset][Compute] Use an ExecPlan for dataset scans

Posted by GitBox <gi...@apache.org>.
bkietz commented on pull request #10397:
URL: https://github.com/apache/arrow/pull/10397#issuecomment-865269157


   @pitrou PTAL


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] westonpace commented on a change in pull request #10397: ARROW-11930: [C++][Dataset][Compute] Use an ExecPlan for dataset scans

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #10397:
URL: https://github.com/apache/arrow/pull/10397#discussion_r644148134



##########
File path: cpp/src/arrow/compute/exec/exec_plan.cc
##########
@@ -170,48 +165,409 @@ Status ExecPlan::Validate() { return ToDerived(this)->Validate(); }
 
 Status ExecPlan::StartProducing() { return ToDerived(this)->StartProducing(); }
 
-ExecNode::ExecNode(ExecPlan* plan, std::string label,
-                   std::vector<BatchDescr> input_descrs,
+ExecNode::ExecNode(ExecPlan* plan, std::string label, NodeVector inputs,
                    std::vector<std::string> input_labels, BatchDescr output_descr,
                    int num_outputs)
     : plan_(plan),
       label_(std::move(label)),
-      input_descrs_(std::move(input_descrs)),
+      inputs_(std::move(inputs)),
       input_labels_(std::move(input_labels)),
       output_descr_(std::move(output_descr)),
-      num_outputs_(num_outputs) {}
+      num_outputs_(num_outputs) {
+  for (auto input : inputs_) {
+    input->outputs_.push_back(this);
+  }
+}
 
 Status ExecNode::Validate() const {
-  if (inputs_.size() != input_descrs_.size()) {
+  if (inputs_.size() != input_labels_.size()) {
     return Status::Invalid("Invalid number of inputs for '", label(), "' (expected ",
-                           num_inputs(), ", actual ", inputs_.size(), ")");
+                           num_inputs(), ", actual ", input_labels_.size(), ")");
   }
 
   if (static_cast<int>(outputs_.size()) != num_outputs_) {
     return Status::Invalid("Invalid number of outputs for '", label(), "' (expected ",
                            num_outputs(), ", actual ", outputs_.size(), ")");
   }
 
-  DCHECK_EQ(input_descrs_.size(), input_labels_.size());
-
   for (auto out : outputs_) {
     auto input_index = GetNodeIndex(out->inputs(), this);
     if (!input_index) {
       return Status::Invalid("Node '", label(), "' outputs to node '", out->label(),
                              "' but is not listed as an input.");
     }
+  }
 
-    const auto& in_descr = out->input_descrs_[*input_index];
-    if (in_descr != output_descr_) {
-      return Status::Invalid(
-          "Node '", label(), "' (bound to input ", input_labels_[*input_index],
-          ") produces batches with type '", ValueDescr::ToString(output_descr_),
-          "' inconsistent with consumer '", out->label(), "' which accepts '",
-          ValueDescr::ToString(in_descr), "'");
+  return Status::OK();
+}
+
+struct SourceNode : ExecNode {
+  SourceNode(ExecPlan* plan, std::string label, ExecNode::BatchDescr output_descr,
+             AsyncGenerator<util::optional<ExecBatch>> generator)
+      : ExecNode(plan, std::move(label), {}, {}, std::move(output_descr),
+                 /*num_outputs=*/1),
+        generator_(std::move(generator)) {}
+
+  const char* kind_name() override { return "SourceNode"; }
+
+  static void NoInputs() { DCHECK(false) << "no inputs; this should never be called"; }
+  void InputReceived(ExecNode*, int, ExecBatch) override { NoInputs(); }
+  void ErrorReceived(ExecNode*, Status) override { NoInputs(); }
+  void InputFinished(ExecNode*, int) override { NoInputs(); }
+
+  Status StartProducing() override {
+    if (finished_) {
+      return Status::Invalid("Restarted SourceNode '", label(), "'");
     }
+
+    auto gen = std::move(generator_);
+
+    /// XXX should we wait on this future anywhere? In StopProducing() maybe?
+    auto done_fut =
+        Loop([gen, this] {
+          std::unique_lock<std::mutex> lock(mutex_);
+          int seq = next_batch_index_++;
+          if (finished_) {
+            return Future<ControlFlow<int>>::MakeFinished(Break(seq));
+          }
+          lock.unlock();
+
+          return gen().Then(
+              [=](const util::optional<ExecBatch>& batch) -> ControlFlow<int> {
+                std::unique_lock<std::mutex> lock(mutex_);
+                if (!batch || finished_) {
+                  finished_ = true;
+                  return Break(seq);
+                }
+                lock.unlock();
+
+                outputs_[0]->InputReceived(this, seq, *batch);
+                return Continue();
+              },
+              [=](const Status& error) -> ControlFlow<int> {
+                std::unique_lock<std::mutex> lock(mutex_);
+                if (!finished_) {
+                  finished_ = true;
+                  lock.unlock();
+                  // unless we were already finished, push the error to our output
+                  // XXX is this correct? Is it reasonable for a consumer to ignore errors
+                  // from a finished producer?
+                  outputs_[0]->ErrorReceived(this, error);
+                }
+                return Break(seq);
+              });
+        }).Then([&](int seq) {
+          /// XXX this is probably redundant: do we always call InputFinished after
+          /// ErrorReceived or will ErrorRecieved be sufficient?

Review comment:
       My vote is that `ErrorReceived` is sufficient.  I think a node could recover from a failure but, if it does so, it shouldn't call `ErrorReceived`.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] bkietz commented on a change in pull request #10397: ARROW-11930: [C++][Dataset][Compute] Use an ExecPlan for dataset scans

Posted by GitBox <gi...@apache.org>.
bkietz commented on a change in pull request #10397:
URL: https://github.com/apache/arrow/pull/10397#discussion_r660115795



##########
File path: cpp/src/arrow/compute/exec/expression.cc
##########
@@ -61,13 +61,22 @@ Expression call(std::string function, std::vector<Expression> arguments,
   call.function_name = std::move(function);
   call.arguments = std::move(arguments);
   call.options = std::move(options);
+
+  call.hash = std::hash<std::string>{}(call.function_name);
+  for (const auto& arg : call.arguments) {
+    call.hash ^= arg.hash();

Review comment:
       thanks, will replace




-- 
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-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] bkietz commented on a change in pull request #10397: ARROW-11930: [C++][Dataset][Compute] Use an ExecPlan for dataset scans

Posted by GitBox <gi...@apache.org>.
bkietz commented on a change in pull request #10397:
URL: https://github.com/apache/arrow/pull/10397#discussion_r644275371



##########
File path: cpp/src/arrow/compute/exec/test_util.cc
##########
@@ -124,277 +130,42 @@ struct DummyNode : ExecNode {
   bool started_ = false;
 };
 
-struct RecordBatchReaderNode : ExecNode {
-  RecordBatchReaderNode(ExecPlan* plan, std::string label,
-                        std::shared_ptr<RecordBatchReader> reader, Executor* io_executor)
-      : ExecNode(plan, std::move(label), {}, {},
-                 DescrFromSchemaColumns(*reader->schema()), /*num_outputs=*/1),
-        schema_(reader->schema()),
-        reader_(std::move(reader)),
-        io_executor_(io_executor) {}
-
-  RecordBatchReaderNode(ExecPlan* plan, std::string label, std::shared_ptr<Schema> schema,
-                        RecordBatchGenerator generator, Executor* io_executor)
-      : ExecNode(plan, std::move(label), {}, {}, DescrFromSchemaColumns(*schema),
-                 /*num_outputs=*/1),
-        schema_(std::move(schema)),
-        generator_(std::move(generator)),
-        io_executor_(io_executor) {}
-
-  const char* kind_name() override { return "RecordBatchReader"; }
-
-  void InputReceived(ExecNode* input, int seq_num, compute::ExecBatch batch) override {}
-
-  void ErrorReceived(ExecNode* input, Status error) override {}
-
-  void InputFinished(ExecNode* input, int seq_stop) override {}
-
-  Status StartProducing() override {
-    next_batch_index_ = 0;
-    if (!generator_) {
-      auto it = MakeIteratorFromReader(reader_);
-      ARROW_ASSIGN_OR_RAISE(generator_,
-                            MakeBackgroundGenerator(std::move(it), io_executor_));
-    }
-    GenerateOne(std::unique_lock<std::mutex>{mutex_});
-    return Status::OK();
-  }
-
-  void PauseProducing(ExecNode* output) override {}
-
-  void ResumeProducing(ExecNode* output) override {}
-
-  void StopProducing(ExecNode* output) override {
-    ASSERT_EQ(output, outputs_[0]);
-    std::unique_lock<std::mutex> lock(mutex_);
-    generator_ = nullptr;  // null function
-  }
-
-  void StopProducing() override { StopProducing(outputs_[0]); }
-
- private:
-  void GenerateOne(std::unique_lock<std::mutex>&& lock) {
-    if (!generator_) {
-      // Stopped
-      return;
-    }
-    auto plan = this->plan()->shared_from_this();
-    auto fut = generator_();
-    const auto batch_index = next_batch_index_++;
-
-    lock.unlock();
-    // TODO we want to transfer always here
-    io_executor_->Transfer(std::move(fut))
-        .AddCallback(
-            [plan, batch_index, this](const Result<std::shared_ptr<RecordBatch>>& res) {
-              std::unique_lock<std::mutex> lock(mutex_);
-              if (!res.ok()) {
-                for (auto out : outputs_) {
-                  out->ErrorReceived(this, res.status());
-                }
-                return;
-              }
-              const auto& batch = *res;
-              if (IsIterationEnd(batch)) {
-                lock.unlock();
-                for (auto out : outputs_) {
-                  out->InputFinished(this, batch_index);
-                }
-              } else {
-                lock.unlock();
-                for (auto out : outputs_) {
-                  out->InputReceived(this, batch_index, compute::ExecBatch(*batch));
-                }
-                lock.lock();
-                GenerateOne(std::move(lock));
-              }
-            });
-  }
-
-  std::mutex mutex_;
-  const std::shared_ptr<Schema> schema_;
-  const std::shared_ptr<RecordBatchReader> reader_;
-  RecordBatchGenerator generator_;
-  int next_batch_index_;
-
-  Executor* const io_executor_;
-};
-
-struct RecordBatchCollectNodeImpl : public RecordBatchCollectNode {
-  RecordBatchCollectNodeImpl(ExecPlan* plan, std::string label,
-                             std::shared_ptr<Schema> schema)
-      : RecordBatchCollectNode(plan, std::move(label), {DescrFromSchemaColumns(*schema)},
-                               {"batches_to_collect"}, {}, 0),
-        schema_(std::move(schema)) {}
-
-  RecordBatchGenerator generator() override { return generator_; }
-
-  const char* kind_name() override { return "RecordBatchReader"; }
-
-  Status StartProducing() override {
-    num_received_ = 0;
-    num_emitted_ = 0;
-    emit_stop_ = -1;
-    stopped_ = false;
-    producer_.emplace(generator_.producer());
-    return Status::OK();
-  }
-
-  // sink nodes have no outputs from which to feel backpressure
-  void ResumeProducing(ExecNode* output) override {
-    FAIL() << "no outputs; this should never be called";
-  }
-  void PauseProducing(ExecNode* output) override {
-    FAIL() << "no outputs; this should never be called";
-  }
-  void StopProducing(ExecNode* output) override {
-    FAIL() << "no outputs; this should never be called";
-  }
-
-  void StopProducing() override {
-    std::unique_lock<std::mutex> lock(mutex_);
-    StopProducingUnlocked();
-  }
-
-  void InputReceived(ExecNode* input, int seq_num,
-                     compute::ExecBatch exec_batch) override {
-    std::unique_lock<std::mutex> lock(mutex_);
-    if (stopped_) {
-      return;
-    }
-    auto maybe_batch = MakeBatch(std::move(exec_batch));
-    if (!maybe_batch.ok()) {
-      lock.unlock();
-      producer_->Push(std::move(maybe_batch));
-      return;
-    }
-
-    // TODO would be nice to factor this out in a ReorderQueue
-    auto batch = *std::move(maybe_batch);
-    if (seq_num <= static_cast<int>(received_batches_.size())) {
-      received_batches_.resize(seq_num + 1, nullptr);
-    }
-    DCHECK_EQ(received_batches_[seq_num], nullptr);
-    received_batches_[seq_num] = std::move(batch);
-    ++num_received_;
-
-    if (seq_num != num_emitted_) {
-      // Cannot emit yet as there is a hole at `num_emitted_`
-      DCHECK_GT(seq_num, num_emitted_);
-      DCHECK_EQ(received_batches_[num_emitted_], nullptr);
-      return;
-    }
-    if (num_received_ == emit_stop_) {
-      StopProducingUnlocked();
-    }
-
-    // Emit batches in order as far as possible
-    // First collect these batches, then unlock before producing.
-    const auto seq_start = seq_num;
-    while (seq_num < static_cast<int>(received_batches_.size()) &&
-           received_batches_[seq_num] != nullptr) {
-      ++seq_num;
-    }
-    DCHECK_GT(seq_num, seq_start);
-    // By moving the values now, we make sure another thread won't emit the same values
-    // below
-    RecordBatchVector to_emit(
-        std::make_move_iterator(received_batches_.begin() + seq_start),
-        std::make_move_iterator(received_batches_.begin() + seq_num));
-
-    lock.unlock();
-    for (auto&& batch : to_emit) {
-      producer_->Push(std::move(batch));
-    }
-    lock.lock();
-
-    DCHECK_EQ(seq_start, num_emitted_);  // num_emitted_ wasn't bumped in the meantime
-    num_emitted_ = seq_num;
-  }
-
-  void ErrorReceived(ExecNode* input, Status error) override {
-    // XXX do we care about properly sequencing the error?
-    producer_->Push(std::move(error));
-    std::unique_lock<std::mutex> lock(mutex_);
-    StopProducingUnlocked();
-  }
-
-  void InputFinished(ExecNode* input, int seq_stop) override {
-    std::unique_lock<std::mutex> lock(mutex_);
-    DCHECK_GE(seq_stop, static_cast<int>(received_batches_.size()));
-    received_batches_.reserve(seq_stop);
-    emit_stop_ = seq_stop;
-    if (emit_stop_ == num_received_) {
-      DCHECK_EQ(emit_stop_, num_emitted_);
-      StopProducingUnlocked();
-    }
-  }
-
- private:
-  void StopProducingUnlocked() {
-    if (!stopped_) {
-      stopped_ = true;
-      producer_->Close();
-      inputs_[0]->StopProducing(this);
-    }
-  }
-
-  // TODO factor this out as ExecBatch::ToRecordBatch()?
-  Result<std::shared_ptr<RecordBatch>> MakeBatch(compute::ExecBatch&& exec_batch) {
-    ArrayDataVector columns;
-    columns.reserve(exec_batch.values.size());
-    for (auto&& value : exec_batch.values) {
-      if (!value.is_array()) {
-        return Status::TypeError("Expected array input");
-      }
-      columns.push_back(std::move(value).array());
-    }
-    return RecordBatch::Make(schema_, exec_batch.length, std::move(columns));
-  }
-
-  const std::shared_ptr<Schema> schema_;
-
-  std::mutex mutex_;
-  RecordBatchVector received_batches_;
-  int num_received_;
-  int num_emitted_;
-  int emit_stop_;
-  bool stopped_;
-
-  PushGenerator<std::shared_ptr<RecordBatch>> generator_;
-  util::optional<PushGenerator<std::shared_ptr<RecordBatch>>::Producer> producer_;
-};
+AsyncGenerator<util::optional<ExecBatch>> Wrap(RecordBatchGenerator gen,
+                                               ::arrow::internal::Executor* io_executor) {
+  return MakeMappedGenerator(
+      MakeTransferredGenerator(std::move(gen), io_executor),

Review comment:
       The (half-baked) intent here is to emulate a reader which is loading batches from disk on an IO thread pool, which should then get passed off to an ExecPlan running in a CPU thread pool.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] bkietz commented on pull request #10397: ARROW-11930: [C++][Dataset][Compute] Use an ExecPlan for dataset scans

Posted by GitBox <gi...@apache.org>.
bkietz commented on pull request #10397:
URL: https://github.com/apache/arrow/pull/10397#issuecomment-871831272


   @pitrou I think I've addressed your comments. Could we merge this and address anything else in follow up?


-- 
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-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] bkietz commented on a change in pull request #10397: ARROW-11930: [C++][Dataset][Compute] Use an ExecPlan for dataset scans

Posted by GitBox <gi...@apache.org>.
bkietz commented on a change in pull request #10397:
URL: https://github.com/apache/arrow/pull/10397#discussion_r640899835



##########
File path: cpp/src/arrow/compute/exec/expression.cc
##########
@@ -492,9 +448,18 @@ Result<Expression> Expression::Bind(ValueDescr in,
   if (literal()) return *this;
 
   if (auto ref = field_ref()) {
-    ARROW_ASSIGN_OR_RAISE(auto field, ref->GetOneOrNone(*in.type));
-    auto descr = field ? ValueDescr{field->type(), in.shape} : ValueDescr::Scalar(null());
-    return Expression{Parameter{*ref, std::move(descr)}};
+    if (ref->IsNested()) {
+      return Status::NotImplemented("nested field references");
+    }
+
+    ARROW_ASSIGN_OR_RAISE(auto path, ref->FindOne(*in.type));

Review comment:
       Ah, I see: FindOne is currently linear in the number of fields. That could/should be fixed




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] bkietz commented on a change in pull request #10397: ARROW-11930: [C++][Dataset][Compute] Use an ExecPlan for dataset scans

Posted by GitBox <gi...@apache.org>.
bkietz commented on a change in pull request #10397:
URL: https://github.com/apache/arrow/pull/10397#discussion_r660110016



##########
File path: cpp/src/arrow/util/future.h
##########
@@ -976,4 +979,43 @@ Future<BreakValueType> Loop(Iterate iterate) {
   return break_fut;
 }
 
+template <typename T>
+struct EnsureFuture {
+  using type = Future<T>;
+};
+
+template <typename T>
+struct EnsureFuture<Result<T>> {
+  using type = Future<T>;
+};
+
+template <typename T>
+struct EnsureFuture<Future<T>> {
+  using type = Future<T>;
+};
+
+template <>
+struct EnsureFuture<Status> {
+  using type = Future<>;
+};

Review comment:
       Nice

##########
File path: cpp/src/arrow/util/thread_pool_test.cc
##########
@@ -395,6 +395,22 @@ TEST_F(TestThreadPool, StressSpawn) {
   SpawnAdds(pool.get(), 1000, task_add<int>);
 }
 
+TEST_F(TestThreadPool, OwnsCurrentThread) {
+  auto pool = this->MakeThreadPool(30);
+  std::atomic<bool> one_failed{false};
+
+  for (int i = 0; i < 1000; ++i) {
+    ASSERT_OK(pool->Spawn([&] {
+      if (pool->OwnsThisThread()) return;
+
+      one_failed = true;
+    }));
+  }
+
+  ASSERT_OK(pool->Shutdown());
+  ASSERT_FALSE(one_failed);

Review comment:
       will do




-- 
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-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] bkietz commented on a change in pull request #10397: ARROW-11930: [C++][Dataset][Compute] Use an ExecPlan for dataset scans

Posted by GitBox <gi...@apache.org>.
bkietz commented on a change in pull request #10397:
URL: https://github.com/apache/arrow/pull/10397#discussion_r644293348



##########
File path: cpp/src/arrow/compute/exec/exec_plan.h
##########
@@ -225,22 +212,43 @@ class ARROW_EXPORT ExecNode {
   virtual void StopProducing() = 0;
 
  protected:
-  ExecNode(ExecPlan* plan, std::string label, std::vector<BatchDescr> input_descrs,
+  ExecNode(ExecPlan*, std::string label, NodeVector inputs,
            std::vector<std::string> input_labels, BatchDescr output_descr,
            int num_outputs);
 
   ExecPlan* plan_;
-
   std::string label_;
 
-  std::vector<BatchDescr> input_descrs_;
-  std::vector<std::string> input_labels_;
   NodeVector inputs_;
+  std::vector<std::string> input_labels_;
 
   BatchDescr output_descr_;
   int num_outputs_;
   NodeVector outputs_;
 };
 
+/// \brief Adapt an AsyncGenerator<ExecBatch> as a source node
+ARROW_EXPORT
+ExecNode* MakeSourceNode(ExecPlan*, std::string label, ExecNode::BatchDescr output_descr,
+                         AsyncGenerator<util::optional<ExecBatch>>);

Review comment:
       That sounds necessary; eventually we'll have many concrete nodes and we might even need a registry and generic factory for them. However I think that separation can wait for a follow up




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] bkietz commented on a change in pull request #10397: ARROW-11930: [C++][Dataset][Compute] Use an ExecPlan for dataset scans

Posted by GitBox <gi...@apache.org>.
bkietz commented on a change in pull request #10397:
URL: https://github.com/apache/arrow/pull/10397#discussion_r644308307



##########
File path: cpp/src/arrow/compute/exec/exec_plan.cc
##########
@@ -170,48 +165,409 @@ Status ExecPlan::Validate() { return ToDerived(this)->Validate(); }
 
 Status ExecPlan::StartProducing() { return ToDerived(this)->StartProducing(); }
 
-ExecNode::ExecNode(ExecPlan* plan, std::string label,
-                   std::vector<BatchDescr> input_descrs,
+ExecNode::ExecNode(ExecPlan* plan, std::string label, NodeVector inputs,
                    std::vector<std::string> input_labels, BatchDescr output_descr,
                    int num_outputs)
     : plan_(plan),
       label_(std::move(label)),
-      input_descrs_(std::move(input_descrs)),
+      inputs_(std::move(inputs)),
       input_labels_(std::move(input_labels)),
       output_descr_(std::move(output_descr)),
-      num_outputs_(num_outputs) {}
+      num_outputs_(num_outputs) {
+  for (auto input : inputs_) {
+    input->outputs_.push_back(this);
+  }
+}
 
 Status ExecNode::Validate() const {
-  if (inputs_.size() != input_descrs_.size()) {
+  if (inputs_.size() != input_labels_.size()) {
     return Status::Invalid("Invalid number of inputs for '", label(), "' (expected ",
-                           num_inputs(), ", actual ", inputs_.size(), ")");
+                           num_inputs(), ", actual ", input_labels_.size(), ")");
   }
 
   if (static_cast<int>(outputs_.size()) != num_outputs_) {
     return Status::Invalid("Invalid number of outputs for '", label(), "' (expected ",
                            num_outputs(), ", actual ", outputs_.size(), ")");
   }
 
-  DCHECK_EQ(input_descrs_.size(), input_labels_.size());
-
   for (auto out : outputs_) {
     auto input_index = GetNodeIndex(out->inputs(), this);
     if (!input_index) {
       return Status::Invalid("Node '", label(), "' outputs to node '", out->label(),
                              "' but is not listed as an input.");
     }
+  }
 
-    const auto& in_descr = out->input_descrs_[*input_index];
-    if (in_descr != output_descr_) {
-      return Status::Invalid(
-          "Node '", label(), "' (bound to input ", input_labels_[*input_index],
-          ") produces batches with type '", ValueDescr::ToString(output_descr_),
-          "' inconsistent with consumer '", out->label(), "' which accepts '",
-          ValueDescr::ToString(in_descr), "'");
+  return Status::OK();
+}
+
+struct SourceNode : ExecNode {
+  SourceNode(ExecPlan* plan, std::string label, ExecNode::BatchDescr output_descr,
+             AsyncGenerator<util::optional<ExecBatch>> generator)
+      : ExecNode(plan, std::move(label), {}, {}, std::move(output_descr),
+                 /*num_outputs=*/1),
+        generator_(std::move(generator)) {}
+
+  const char* kind_name() override { return "SourceNode"; }
+
+  static void NoInputs() { DCHECK(false) << "no inputs; this should never be called"; }
+  void InputReceived(ExecNode*, int, ExecBatch) override { NoInputs(); }
+  void ErrorReceived(ExecNode*, Status) override { NoInputs(); }
+  void InputFinished(ExecNode*, int) override { NoInputs(); }
+
+  Status StartProducing() override {
+    if (finished_) {
+      return Status::Invalid("Restarted SourceNode '", label(), "'");
     }
+
+    auto gen = std::move(generator_);
+
+    /// XXX should we wait on this future anywhere? In StopProducing() maybe?
+    auto done_fut =
+        Loop([gen, this] {
+          std::unique_lock<std::mutex> lock(mutex_);
+          int seq = next_batch_index_++;
+          if (finished_) {
+            return Future<ControlFlow<int>>::MakeFinished(Break(seq));
+          }
+          lock.unlock();
+
+          return gen().Then(
+              [=](const util::optional<ExecBatch>& batch) -> ControlFlow<int> {
+                std::unique_lock<std::mutex> lock(mutex_);
+                if (!batch || finished_) {
+                  finished_ = true;
+                  return Break(seq);
+                }
+                lock.unlock();
+
+                outputs_[0]->InputReceived(this, seq, *batch);
+                return Continue();
+              },
+              [=](const Status& error) -> ControlFlow<int> {
+                std::unique_lock<std::mutex> lock(mutex_);
+                if (!finished_) {
+                  finished_ = true;
+                  lock.unlock();
+                  // unless we were already finished, push the error to our output
+                  // XXX is this correct? Is it reasonable for a consumer to ignore errors
+                  // from a finished producer?
+                  outputs_[0]->ErrorReceived(this, error);
+                }
+                return Break(seq);
+              });
+        }).Then([&](int seq) {
+          /// XXX this is probably redundant: do we always call InputFinished after
+          /// ErrorReceived or will ErrorRecieved be sufficient?
+          outputs_[0]->InputFinished(this, seq);
+        });
+
+    return Status::OK();
   }
 
-  return Status::OK();
+  void PauseProducing(ExecNode* output) override {}
+
+  void ResumeProducing(ExecNode* output) override {}
+
+  void StopProducing(ExecNode* output) override {
+    DCHECK_EQ(output, outputs_[0]);
+    std::unique_lock<std::mutex> lock(mutex_);
+    finished_ = true;
+  }
+
+  void StopProducing() override { StopProducing(outputs_[0]); }
+
+ private:
+  std::mutex mutex_;
+  bool finished_{false};
+  int next_batch_index_{0};
+  AsyncGenerator<util::optional<ExecBatch>> generator_;
+};
+
+ExecNode* MakeSourceNode(ExecPlan* plan, std::string label,
+                         ExecNode::BatchDescr output_descr,
+                         AsyncGenerator<util::optional<ExecBatch>> generator) {
+  return plan->EmplaceNode<SourceNode>(plan, std::move(label), std::move(output_descr),
+                                       std::move(generator));
+}
+
+struct FilterNode : ExecNode {
+  FilterNode(ExecNode* input, std::string label, Expression filter)
+      : ExecNode(input->plan(), std::move(label), {input}, {"target"},
+                 /*output_descr=*/{input->output_descr()},
+                 /*num_outputs=*/1),
+        filter_(std::move(filter)) {}
+
+  const char* kind_name() override { return "FilterNode"; }
+
+  Result<ExecBatch> DoFilter(const ExecBatch& target) {
+    ARROW_ASSIGN_OR_RAISE(Expression simplified_filter,
+                          SimplifyWithGuarantee(filter_, target.guarantee));
+
+    // XXX get a non-default exec context
+    ARROW_ASSIGN_OR_RAISE(Datum mask, ExecuteScalarExpression(simplified_filter, target));
+
+    if (mask.is_scalar()) {
+      const auto& mask_scalar = mask.scalar_as<BooleanScalar>();
+      if (mask_scalar.is_valid && mask_scalar.value) {
+        return target;
+      }
+
+      return target.Slice(0, 0);
+    }
+
+    auto values = target.values;
+    for (auto& value : values) {
+      if (value.is_scalar()) continue;
+      ARROW_ASSIGN_OR_RAISE(value, Filter(value, mask, FilterOptions::Defaults()));
+    }
+    return ExecBatch::Make(std::move(values));
+  }
+
+  void InputReceived(ExecNode* input, int seq, ExecBatch batch) override {
+    DCHECK_EQ(input, inputs_[0]);
+
+    auto maybe_filtered = DoFilter(std::move(batch));
+    if (!maybe_filtered.ok()) {
+      outputs_[0]->ErrorReceived(this, maybe_filtered.status());
+      inputs_[0]->StopProducing(this);
+      return;
+    }
+
+    maybe_filtered->guarantee = batch.guarantee;
+    outputs_[0]->InputReceived(this, seq, maybe_filtered.MoveValueUnsafe());
+  }
+
+  void ErrorReceived(ExecNode* input, Status error) override {
+    DCHECK_EQ(input, inputs_[0]);
+    outputs_[0]->ErrorReceived(this, std::move(error));
+    inputs_[0]->StopProducing(this);
+  }
+
+  void InputFinished(ExecNode* input, int seq) override {
+    DCHECK_EQ(input, inputs_[0]);
+    outputs_[0]->InputFinished(this, seq);
+    inputs_[0]->StopProducing(this);
+  }
+
+  Status StartProducing() override {
+    // XXX validate inputs_[0]->output_descr() against filter_
+    return Status::OK();
+  }
+
+  void PauseProducing(ExecNode* output) override {}
+
+  void ResumeProducing(ExecNode* output) override {}
+
+  void StopProducing(ExecNode* output) override {
+    DCHECK_EQ(output, outputs_[0]);
+    inputs_[0]->StopProducing(this);
+  }
+
+  void StopProducing() override { StopProducing(outputs_[0]); }
+
+ private:
+  Expression filter_;
+};
+
+ExecNode* MakeFilterNode(ExecNode* input, std::string label, Expression filter) {
+  return input->plan()->EmplaceNode<FilterNode>(input, std::move(label),
+                                                std::move(filter));
+}
+
+struct ProjectNode : ExecNode {
+  ProjectNode(ExecNode* input, std::string label, std::vector<Expression> exprs)
+      : ExecNode(input->plan(), std::move(label), {input}, {"target"},
+                 /*output_descr=*/{input->output_descr()},
+                 /*num_outputs=*/1),
+        exprs_(std::move(exprs)) {}
+
+  const char* kind_name() override { return "ProjectNode"; }
+
+  Result<ExecBatch> DoProject(const ExecBatch& target) {
+    // XXX get a non-default exec context
+    std::vector<Datum> values{exprs_.size()};
+    for (size_t i = 0; i < exprs_.size(); ++i) {
+      ARROW_ASSIGN_OR_RAISE(Expression simplified_expr,
+                            SimplifyWithGuarantee(exprs_[i], target.guarantee));
+
+      ARROW_ASSIGN_OR_RAISE(values[i], ExecuteScalarExpression(simplified_expr, target));
+    }
+    return ExecBatch::Make(std::move(values));
+  }
+
+  void InputReceived(ExecNode* input, int seq, ExecBatch batch) override {
+    DCHECK_EQ(input, inputs_[0]);
+
+    auto maybe_projected = DoProject(std::move(batch));
+    if (!maybe_projected.ok()) {
+      outputs_[0]->ErrorReceived(this, maybe_projected.status());
+      inputs_[0]->StopProducing(this);
+      return;
+    }
+
+    maybe_projected->guarantee = batch.guarantee;
+    outputs_[0]->InputReceived(this, seq, maybe_projected.MoveValueUnsafe());
+  }
+
+  void ErrorReceived(ExecNode* input, Status error) override {
+    DCHECK_EQ(input, inputs_[0]);
+    outputs_[0]->ErrorReceived(this, std::move(error));
+    inputs_[0]->StopProducing(this);
+  }
+
+  void InputFinished(ExecNode* input, int seq) override {
+    DCHECK_EQ(input, inputs_[0]);
+    outputs_[0]->InputFinished(this, seq);
+    inputs_[0]->StopProducing(this);
+  }
+
+  Status StartProducing() override {
+    // XXX validate inputs_[0]->output_descr() against filter_
+    return Status::OK();
+  }
+
+  void PauseProducing(ExecNode* output) override {}
+
+  void ResumeProducing(ExecNode* output) override {}
+
+  void StopProducing(ExecNode* output) override {
+    DCHECK_EQ(output, outputs_[0]);
+    inputs_[0]->StopProducing(this);
+  }
+
+  void StopProducing() override { StopProducing(outputs_[0]); }
+
+ private:
+  std::vector<Expression> exprs_;
+};
+
+ExecNode* MakeProjectNode(ExecNode* input, std::string label,
+                          std::vector<Expression> exprs) {
+  return input->plan()->EmplaceNode<ProjectNode>(input, std::move(label),
+                                                 std::move(exprs));
+}
+
+struct SinkNode : ExecNode {

Review comment:
       This sounds reasonable, but the prerequisite is to support pause/resume in a source node




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] bkietz commented on a change in pull request #10397: ARROW-11930: [C++][Dataset][Compute] Use an ExecPlan for dataset scans

Posted by GitBox <gi...@apache.org>.
bkietz commented on a change in pull request #10397:
URL: https://github.com/apache/arrow/pull/10397#discussion_r644304045



##########
File path: cpp/src/arrow/compute/exec/exec_plan.cc
##########
@@ -170,48 +165,409 @@ Status ExecPlan::Validate() { return ToDerived(this)->Validate(); }
 
 Status ExecPlan::StartProducing() { return ToDerived(this)->StartProducing(); }
 
-ExecNode::ExecNode(ExecPlan* plan, std::string label,
-                   std::vector<BatchDescr> input_descrs,
+ExecNode::ExecNode(ExecPlan* plan, std::string label, NodeVector inputs,
                    std::vector<std::string> input_labels, BatchDescr output_descr,
                    int num_outputs)
     : plan_(plan),
       label_(std::move(label)),
-      input_descrs_(std::move(input_descrs)),
+      inputs_(std::move(inputs)),
       input_labels_(std::move(input_labels)),
       output_descr_(std::move(output_descr)),
-      num_outputs_(num_outputs) {}
+      num_outputs_(num_outputs) {
+  for (auto input : inputs_) {
+    input->outputs_.push_back(this);
+  }
+}
 
 Status ExecNode::Validate() const {
-  if (inputs_.size() != input_descrs_.size()) {
+  if (inputs_.size() != input_labels_.size()) {
     return Status::Invalid("Invalid number of inputs for '", label(), "' (expected ",
-                           num_inputs(), ", actual ", inputs_.size(), ")");
+                           num_inputs(), ", actual ", input_labels_.size(), ")");
   }
 
   if (static_cast<int>(outputs_.size()) != num_outputs_) {
     return Status::Invalid("Invalid number of outputs for '", label(), "' (expected ",
                            num_outputs(), ", actual ", outputs_.size(), ")");
   }
 
-  DCHECK_EQ(input_descrs_.size(), input_labels_.size());
-
   for (auto out : outputs_) {
     auto input_index = GetNodeIndex(out->inputs(), this);
     if (!input_index) {
       return Status::Invalid("Node '", label(), "' outputs to node '", out->label(),
                              "' but is not listed as an input.");
     }
+  }
 
-    const auto& in_descr = out->input_descrs_[*input_index];
-    if (in_descr != output_descr_) {
-      return Status::Invalid(
-          "Node '", label(), "' (bound to input ", input_labels_[*input_index],
-          ") produces batches with type '", ValueDescr::ToString(output_descr_),
-          "' inconsistent with consumer '", out->label(), "' which accepts '",
-          ValueDescr::ToString(in_descr), "'");
+  return Status::OK();
+}
+
+struct SourceNode : ExecNode {
+  SourceNode(ExecPlan* plan, std::string label, ExecNode::BatchDescr output_descr,
+             AsyncGenerator<util::optional<ExecBatch>> generator)
+      : ExecNode(plan, std::move(label), {}, {}, std::move(output_descr),
+                 /*num_outputs=*/1),
+        generator_(std::move(generator)) {}
+
+  const char* kind_name() override { return "SourceNode"; }
+
+  static void NoInputs() { DCHECK(false) << "no inputs; this should never be called"; }
+  void InputReceived(ExecNode*, int, ExecBatch) override { NoInputs(); }
+  void ErrorReceived(ExecNode*, Status) override { NoInputs(); }
+  void InputFinished(ExecNode*, int) override { NoInputs(); }
+
+  Status StartProducing() override {
+    if (finished_) {
+      return Status::Invalid("Restarted SourceNode '", label(), "'");
     }
+
+    auto gen = std::move(generator_);
+
+    /// XXX should we wait on this future anywhere? In StopProducing() maybe?
+    auto done_fut =
+        Loop([gen, this] {

Review comment:
       The alternative is something like [GenerateOne](https://github.com/apache/arrow/pull/10397/commits/4bb7404b0101a3c4ebb6a929f28229da44b6bcc5#diff-f001186af47c4e8a2d4ca433e9eec579745bbd5e0bbb42fdafb970b625a7ed20L238), which is just as bad or worse 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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] westonpace commented on a change in pull request #10397: ARROW-11930: [C++][Dataset][Compute] Use an ExecPlan for dataset scans

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #10397:
URL: https://github.com/apache/arrow/pull/10397#discussion_r644136382



##########
File path: cpp/src/arrow/compute/exec/exec_plan.cc
##########
@@ -170,48 +165,409 @@ Status ExecPlan::Validate() { return ToDerived(this)->Validate(); }
 
 Status ExecPlan::StartProducing() { return ToDerived(this)->StartProducing(); }
 
-ExecNode::ExecNode(ExecPlan* plan, std::string label,
-                   std::vector<BatchDescr> input_descrs,
+ExecNode::ExecNode(ExecPlan* plan, std::string label, NodeVector inputs,
                    std::vector<std::string> input_labels, BatchDescr output_descr,
                    int num_outputs)
     : plan_(plan),
       label_(std::move(label)),
-      input_descrs_(std::move(input_descrs)),
+      inputs_(std::move(inputs)),
       input_labels_(std::move(input_labels)),
       output_descr_(std::move(output_descr)),
-      num_outputs_(num_outputs) {}
+      num_outputs_(num_outputs) {
+  for (auto input : inputs_) {
+    input->outputs_.push_back(this);
+  }
+}
 
 Status ExecNode::Validate() const {
-  if (inputs_.size() != input_descrs_.size()) {
+  if (inputs_.size() != input_labels_.size()) {
     return Status::Invalid("Invalid number of inputs for '", label(), "' (expected ",
-                           num_inputs(), ", actual ", inputs_.size(), ")");
+                           num_inputs(), ", actual ", input_labels_.size(), ")");
   }
 
   if (static_cast<int>(outputs_.size()) != num_outputs_) {
     return Status::Invalid("Invalid number of outputs for '", label(), "' (expected ",
                            num_outputs(), ", actual ", outputs_.size(), ")");
   }
 
-  DCHECK_EQ(input_descrs_.size(), input_labels_.size());
-
   for (auto out : outputs_) {
     auto input_index = GetNodeIndex(out->inputs(), this);
     if (!input_index) {
       return Status::Invalid("Node '", label(), "' outputs to node '", out->label(),
                              "' but is not listed as an input.");
     }
+  }
 
-    const auto& in_descr = out->input_descrs_[*input_index];
-    if (in_descr != output_descr_) {
-      return Status::Invalid(
-          "Node '", label(), "' (bound to input ", input_labels_[*input_index],
-          ") produces batches with type '", ValueDescr::ToString(output_descr_),
-          "' inconsistent with consumer '", out->label(), "' which accepts '",
-          ValueDescr::ToString(in_descr), "'");
+  return Status::OK();
+}
+
+struct SourceNode : ExecNode {
+  SourceNode(ExecPlan* plan, std::string label, ExecNode::BatchDescr output_descr,
+             AsyncGenerator<util::optional<ExecBatch>> generator)
+      : ExecNode(plan, std::move(label), {}, {}, std::move(output_descr),
+                 /*num_outputs=*/1),
+        generator_(std::move(generator)) {}
+
+  const char* kind_name() override { return "SourceNode"; }
+
+  static void NoInputs() { DCHECK(false) << "no inputs; this should never be called"; }
+  void InputReceived(ExecNode*, int, ExecBatch) override { NoInputs(); }
+  void ErrorReceived(ExecNode*, Status) override { NoInputs(); }
+  void InputFinished(ExecNode*, int) override { NoInputs(); }
+
+  Status StartProducing() override {
+    if (finished_) {
+      return Status::Invalid("Restarted SourceNode '", label(), "'");
     }
+
+    auto gen = std::move(generator_);
+
+    /// XXX should we wait on this future anywhere? In StopProducing() maybe?
+    auto done_fut =
+        Loop([gen, this] {
+          std::unique_lock<std::mutex> lock(mutex_);
+          int seq = next_batch_index_++;
+          if (finished_) {
+            return Future<ControlFlow<int>>::MakeFinished(Break(seq));
+          }
+          lock.unlock();
+
+          return gen().Then(
+              [=](const util::optional<ExecBatch>& batch) -> ControlFlow<int> {
+                std::unique_lock<std::mutex> lock(mutex_);
+                if (!batch || finished_) {
+                  finished_ = true;
+                  return Break(seq);
+                }
+                lock.unlock();
+
+                outputs_[0]->InputReceived(this, seq, *batch);
+                return Continue();
+              },
+              [=](const Status& error) -> ControlFlow<int> {
+                std::unique_lock<std::mutex> lock(mutex_);
+                if (!finished_) {
+                  finished_ = true;
+                  lock.unlock();
+                  // unless we were already finished, push the error to our output
+                  // XXX is this correct? Is it reasonable for a consumer to ignore errors
+                  // from a finished producer?
+                  outputs_[0]->ErrorReceived(this, error);
+                }
+                return Break(seq);
+              });
+        }).Then([&](int seq) {
+          /// XXX this is probably redundant: do we always call InputFinished after
+          /// ErrorReceived or will ErrorRecieved be sufficient?
+          outputs_[0]->InputFinished(this, seq);
+        });
+
+    return Status::OK();
   }
 
-  return Status::OK();
+  void PauseProducing(ExecNode* output) override {}
+
+  void ResumeProducing(ExecNode* output) override {}
+
+  void StopProducing(ExecNode* output) override {
+    DCHECK_EQ(output, outputs_[0]);
+    std::unique_lock<std::mutex> lock(mutex_);
+    finished_ = true;
+  }
+
+  void StopProducing() override { StopProducing(outputs_[0]); }
+
+ private:
+  std::mutex mutex_;
+  bool finished_{false};
+  int next_batch_index_{0};
+  AsyncGenerator<util::optional<ExecBatch>> generator_;
+};
+
+ExecNode* MakeSourceNode(ExecPlan* plan, std::string label,
+                         ExecNode::BatchDescr output_descr,
+                         AsyncGenerator<util::optional<ExecBatch>> generator) {
+  return plan->EmplaceNode<SourceNode>(plan, std::move(label), std::move(output_descr),
+                                       std::move(generator));
+}
+
+struct FilterNode : ExecNode {
+  FilterNode(ExecNode* input, std::string label, Expression filter)
+      : ExecNode(input->plan(), std::move(label), {input}, {"target"},
+                 /*output_descr=*/{input->output_descr()},
+                 /*num_outputs=*/1),
+        filter_(std::move(filter)) {}
+
+  const char* kind_name() override { return "FilterNode"; }
+
+  Result<ExecBatch> DoFilter(const ExecBatch& target) {
+    ARROW_ASSIGN_OR_RAISE(Expression simplified_filter,
+                          SimplifyWithGuarantee(filter_, target.guarantee));
+
+    // XXX get a non-default exec context
+    ARROW_ASSIGN_OR_RAISE(Datum mask, ExecuteScalarExpression(simplified_filter, target));
+
+    if (mask.is_scalar()) {
+      const auto& mask_scalar = mask.scalar_as<BooleanScalar>();
+      if (mask_scalar.is_valid && mask_scalar.value) {
+        return target;
+      }
+
+      return target.Slice(0, 0);
+    }
+
+    auto values = target.values;
+    for (auto& value : values) {
+      if (value.is_scalar()) continue;
+      ARROW_ASSIGN_OR_RAISE(value, Filter(value, mask, FilterOptions::Defaults()));
+    }
+    return ExecBatch::Make(std::move(values));
+  }
+
+  void InputReceived(ExecNode* input, int seq, ExecBatch batch) override {
+    DCHECK_EQ(input, inputs_[0]);
+
+    auto maybe_filtered = DoFilter(std::move(batch));
+    if (!maybe_filtered.ok()) {
+      outputs_[0]->ErrorReceived(this, maybe_filtered.status());
+      inputs_[0]->StopProducing(this);
+      return;
+    }
+
+    maybe_filtered->guarantee = batch.guarantee;
+    outputs_[0]->InputReceived(this, seq, maybe_filtered.MoveValueUnsafe());
+  }
+
+  void ErrorReceived(ExecNode* input, Status error) override {
+    DCHECK_EQ(input, inputs_[0]);
+    outputs_[0]->ErrorReceived(this, std::move(error));
+    inputs_[0]->StopProducing(this);
+  }
+
+  void InputFinished(ExecNode* input, int seq) override {
+    DCHECK_EQ(input, inputs_[0]);
+    outputs_[0]->InputFinished(this, seq);
+    inputs_[0]->StopProducing(this);
+  }
+
+  Status StartProducing() override {
+    // XXX validate inputs_[0]->output_descr() against filter_
+    return Status::OK();
+  }
+
+  void PauseProducing(ExecNode* output) override {}
+
+  void ResumeProducing(ExecNode* output) override {}
+
+  void StopProducing(ExecNode* output) override {
+    DCHECK_EQ(output, outputs_[0]);
+    inputs_[0]->StopProducing(this);
+  }
+
+  void StopProducing() override { StopProducing(outputs_[0]); }
+
+ private:
+  Expression filter_;
+};
+
+ExecNode* MakeFilterNode(ExecNode* input, std::string label, Expression filter) {
+  return input->plan()->EmplaceNode<FilterNode>(input, std::move(label),
+                                                std::move(filter));
+}
+
+struct ProjectNode : ExecNode {
+  ProjectNode(ExecNode* input, std::string label, std::vector<Expression> exprs)
+      : ExecNode(input->plan(), std::move(label), {input}, {"target"},
+                 /*output_descr=*/{input->output_descr()},
+                 /*num_outputs=*/1),
+        exprs_(std::move(exprs)) {}
+
+  const char* kind_name() override { return "ProjectNode"; }
+
+  Result<ExecBatch> DoProject(const ExecBatch& target) {
+    // XXX get a non-default exec context
+    std::vector<Datum> values{exprs_.size()};
+    for (size_t i = 0; i < exprs_.size(); ++i) {
+      ARROW_ASSIGN_OR_RAISE(Expression simplified_expr,
+                            SimplifyWithGuarantee(exprs_[i], target.guarantee));
+
+      ARROW_ASSIGN_OR_RAISE(values[i], ExecuteScalarExpression(simplified_expr, target));
+    }
+    return ExecBatch::Make(std::move(values));
+  }
+
+  void InputReceived(ExecNode* input, int seq, ExecBatch batch) override {
+    DCHECK_EQ(input, inputs_[0]);
+
+    auto maybe_projected = DoProject(std::move(batch));
+    if (!maybe_projected.ok()) {
+      outputs_[0]->ErrorReceived(this, maybe_projected.status());
+      inputs_[0]->StopProducing(this);
+      return;
+    }
+
+    maybe_projected->guarantee = batch.guarantee;
+    outputs_[0]->InputReceived(this, seq, maybe_projected.MoveValueUnsafe());
+  }
+
+  void ErrorReceived(ExecNode* input, Status error) override {
+    DCHECK_EQ(input, inputs_[0]);
+    outputs_[0]->ErrorReceived(this, std::move(error));
+    inputs_[0]->StopProducing(this);
+  }
+
+  void InputFinished(ExecNode* input, int seq) override {
+    DCHECK_EQ(input, inputs_[0]);
+    outputs_[0]->InputFinished(this, seq);
+    inputs_[0]->StopProducing(this);
+  }
+
+  Status StartProducing() override {
+    // XXX validate inputs_[0]->output_descr() against filter_
+    return Status::OK();
+  }
+
+  void PauseProducing(ExecNode* output) override {}

Review comment:
       For any kind of "map-like" node that has an input and an output a call to `PauseProducing` should always call `PauseProducing` on the input.  That's the only way to ensure that back pressure is properly channeled to the source (which can actually pause)




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] bkietz commented on a change in pull request #10397: ARROW-11930: [C++][Dataset][Compute] Use an ExecPlan for dataset scans

Posted by GitBox <gi...@apache.org>.
bkietz commented on a change in pull request #10397:
URL: https://github.com/apache/arrow/pull/10397#discussion_r660110016



##########
File path: cpp/src/arrow/util/future.h
##########
@@ -976,4 +979,43 @@ Future<BreakValueType> Loop(Iterate iterate) {
   return break_fut;
 }
 
+template <typename T>
+struct EnsureFuture {
+  using type = Future<T>;
+};
+
+template <typename T>
+struct EnsureFuture<Result<T>> {
+  using type = Future<T>;
+};
+
+template <typename T>
+struct EnsureFuture<Future<T>> {
+  using type = Future<T>;
+};
+
+template <>
+struct EnsureFuture<Status> {
+  using type = Future<>;
+};

Review comment:
       Nice

##########
File path: cpp/src/arrow/util/thread_pool_test.cc
##########
@@ -395,6 +395,22 @@ TEST_F(TestThreadPool, StressSpawn) {
   SpawnAdds(pool.get(), 1000, task_add<int>);
 }
 
+TEST_F(TestThreadPool, OwnsCurrentThread) {
+  auto pool = this->MakeThreadPool(30);
+  std::atomic<bool> one_failed{false};
+
+  for (int i = 0; i < 1000; ++i) {
+    ASSERT_OK(pool->Spawn([&] {
+      if (pool->OwnsThisThread()) return;
+
+      one_failed = true;
+    }));
+  }
+
+  ASSERT_OK(pool->Shutdown());
+  ASSERT_FALSE(one_failed);

Review comment:
       will do

##########
File path: cpp/src/arrow/testing/gtest_util.h
##########
@@ -28,6 +28,7 @@
 #include <utility>
 #include <vector>
 
+#include <gmock/gmock-matchers.h>

Review comment:
       I'll move these to `arrow/testing/matchers.h`, then

##########
File path: cpp/src/arrow/compute/exec/plan_test.cc
##########
@@ -20,27 +20,46 @@
 #include <functional>
 #include <memory>
 
+#include "arrow/compute/exec.h"
 #include "arrow/compute/exec/exec_plan.h"
+#include "arrow/compute/exec/expression.h"
 #include "arrow/compute/exec/test_util.h"
 #include "arrow/record_batch.h"
 #include "arrow/testing/future_util.h"
 #include "arrow/testing/gtest_util.h"
 #include "arrow/testing/random.h"
 #include "arrow/util/logging.h"
 #include "arrow/util/thread_pool.h"
+#include "arrow/util/vector.h"
 
-namespace arrow {
+using testing::ElementsAre;
+using testing::HasSubstr;
+using testing::UnorderedElementsAreArray;
 
-using internal::Executor;
+namespace arrow {
 
 namespace compute {
 
-void AssertBatchesEqual(const RecordBatchVector& expected,
-                        const RecordBatchVector& actual) {
-  ASSERT_EQ(expected.size(), actual.size());
-  for (size_t i = 0; i < expected.size(); ++i) {
-    AssertBatchesEqual(*expected[i], *actual[i]);
+ExecBatch ExecBatchFromJSON(const std::vector<ValueDescr>& descrs,

Review comment:
       I'll put it in arrow/compute/test_util.h

##########
File path: cpp/src/arrow/compute/exec/plan_test.cc
##########
@@ -20,27 +20,46 @@
 #include <functional>
 #include <memory>
 
+#include "arrow/compute/exec.h"
 #include "arrow/compute/exec/exec_plan.h"
+#include "arrow/compute/exec/expression.h"
 #include "arrow/compute/exec/test_util.h"
 #include "arrow/record_batch.h"
 #include "arrow/testing/future_util.h"
 #include "arrow/testing/gtest_util.h"
 #include "arrow/testing/random.h"
 #include "arrow/util/logging.h"
 #include "arrow/util/thread_pool.h"
+#include "arrow/util/vector.h"
 
-namespace arrow {
+using testing::ElementsAre;
+using testing::HasSubstr;
+using testing::UnorderedElementsAreArray;
 
-using internal::Executor;
+namespace arrow {
 
 namespace compute {
 
-void AssertBatchesEqual(const RecordBatchVector& expected,
-                        const RecordBatchVector& actual) {
-  ASSERT_EQ(expected.size(), actual.size());
-  for (size_t i = 0; i < expected.size(); ++i) {
-    AssertBatchesEqual(*expected[i], *actual[i]);
+ExecBatch ExecBatchFromJSON(const std::vector<ValueDescr>& descrs,

Review comment:
       I'll put it in arrow/compute/exec/test_util.h

##########
File path: cpp/src/arrow/compute/exec/expression.cc
##########
@@ -61,13 +61,22 @@ Expression call(std::string function, std::vector<Expression> arguments,
   call.function_name = std::move(function);
   call.arguments = std::move(arguments);
   call.options = std::move(options);
+
+  call.hash = std::hash<std::string>{}(call.function_name);
+  for (const auto& arg : call.arguments) {
+    call.hash ^= arg.hash();

Review comment:
       thanks, will replace

##########
File path: cpp/src/arrow/util/vector.h
##########
@@ -84,27 +84,49 @@ std::vector<T> FilterVector(std::vector<T> values, Predicate&& predicate) {
   return values;
 }
 
-/// \brief Like MapVector, but where the function can fail.
-template <typename Fn, typename From = internal::call_traits::argument_type<0, Fn>,
-          typename To = typename internal::call_traits::return_type<Fn>::ValueType>
-Result<std::vector<To>> MaybeMapVector(Fn&& map, const std::vector<From>& src) {
+template <typename Fn, typename From,
+          typename To = decltype(std::declval<Fn>()(std::declval<From>()))>
+std::vector<To> MapVector(Fn&& map, const std::vector<From>& source) {
   std::vector<To> out;
-  out.reserve(src.size());
-  ARROW_RETURN_NOT_OK(MaybeTransform(src.begin(), src.end(), std::back_inserter(out),
-                                     std::forward<Fn>(map)));
-  return std::move(out);
+  out.reserve(source.size());
+  std::transform(source.begin(), source.end(), std::back_inserter(out),
+                 std::forward<Fn>(map));
+  return out;
 }
 
 template <typename Fn, typename From,
           typename To = decltype(std::declval<Fn>()(std::declval<From>()))>
-std::vector<To> MapVector(Fn&& map, const std::vector<From>& source) {
+std::vector<To> MapVector(Fn&& map, std::vector<From>&& source) {
   std::vector<To> out;
   out.reserve(source.size());
-  std::transform(source.begin(), source.end(), std::back_inserter(out),
+  std::transform(std::make_move_iterator(source.begin()),
+                 std::make_move_iterator(source.end()), std::back_inserter(out),
                  std::forward<Fn>(map));
   return out;
 }
 
+/// \brief Like MapVector, but where the function can fail.
+template <typename Fn, typename From = internal::call_traits::argument_type<0, Fn>,
+          typename To = typename internal::call_traits::return_type<Fn>::ValueType>

Review comment:
       there's not a good reason; just uniformity with getting `From` from call_traits.

##########
File path: cpp/src/arrow/type.cc
##########
@@ -1195,6 +1195,10 @@ std::string FieldRef::ToString() const {
 }
 
 std::vector<FieldPath> FieldRef::FindAll(const Schema& schema) const {
+  if (auto name = this->name()) {
+    return internal::MapVector([](int i) { return FieldPath{i}; },
+                               schema.GetAllFieldIndices(*name));
+  }

Review comment:
       This is tested https://github.com/bkietz/arrow/blob/1b8bedcbf4b794858228c85644248300b64ce5a4/cpp/src/arrow/type_test.cc#L391-L392




-- 
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-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] pitrou commented on a change in pull request #10397: ARROW-11930: [C++][Dataset][Compute] Use an ExecPlan for dataset scans

Posted by GitBox <gi...@apache.org>.
pitrou commented on a change in pull request #10397:
URL: https://github.com/apache/arrow/pull/10397#discussion_r643987448



##########
File path: cpp/src/arrow/dataset/scanner.h
##########
@@ -25,6 +25,7 @@
 #include <utility>
 #include <vector>
 
+#include "arrow/compute/exec/exec_plan.h"

Review comment:
       Do we have a `type_fwd.h` for this instead? Only pointers are exposed in this header.

##########
File path: cpp/src/arrow/util/future.h
##########
@@ -937,4 +940,19 @@ Future<BreakValueType> Loop(Iterate iterate) {
   return break_fut;
 }
 
+template <typename T>
+struct EnsureFuture {
+  using type = Future<T>;
+};
+
+template <typename T>
+struct EnsureFuture<Result<T>> {

Review comment:
       Should there also be `EnsureFuture<Status>`?

##########
File path: cpp/src/arrow/compute/exec/exec_plan.h
##########
@@ -23,6 +23,7 @@
 
 #include "arrow/compute/type_fwd.h"
 #include "arrow/type_fwd.h"
+#include "arrow/util/async_generator.h"

Review comment:
       I would prefer if we didn't include this large header file here. We can use the `std::function<Future<T>()>` spelling directly instead.

##########
File path: cpp/src/arrow/compute/exec.h
##########
@@ -28,6 +28,7 @@
 #include <vector>
 
 #include "arrow/array/data.h"
+#include "arrow/compute/exec/expression.h"

Review comment:
       Usual concern: is there a way to avoid including too much (I see this pulls `unordered_map`, for example)?

##########
File path: cpp/src/arrow/util/iterator.h
##########
@@ -66,6 +66,12 @@ bool IsIterationEnd(const T& val) {
   return IterationTraits<T>::IsEnd(val);
 }
 
+template <typename T>
+bool IsIterationEnd(const Result<T>& maybe_val) {
+  if (!maybe_val.ok()) return true;

Review comment:
       Are we sure a non-ok status is an end of iteration? I don't think it's the case.

##########
File path: cpp/src/arrow/util/async_generator.h
##########
@@ -258,43 +258,27 @@ class MappingGenerator {
 /// Note: Errors returned from the `map` function will be propagated
 ///
 /// If the source generator is async-reentrant then this generator will be also
-template <typename T, typename V>
-AsyncGenerator<V> MakeMappedGenerator(AsyncGenerator<T> source_generator,
-                                      std::function<Result<V>(const T&)> map) {
-  std::function<Future<V>(const T&)> future_map = [map](const T& val) -> Future<V> {
-    return Future<V>::MakeFinished(map(val));
-  };
-  return MappingGenerator<T, V>(std::move(source_generator), std::move(future_map));
-}
-template <typename T, typename V>
-AsyncGenerator<V> MakeMappedGenerator(AsyncGenerator<T> source_generator,
-                                      std::function<V(const T&)> map) {
-  std::function<Future<V>(const T&)> maybe_future_map = [map](const T& val) -> Future<V> {
-    return Future<V>::MakeFinished(map(val));
-  };
-  return MappingGenerator<T, V>(std::move(source_generator), std::move(maybe_future_map));
-}
-template <typename T, typename V>
-AsyncGenerator<V> MakeMappedGenerator(AsyncGenerator<T> source_generator,
-                                      std::function<Future<V>(const T&)> map) {
-  return MappingGenerator<T, V>(std::move(source_generator), std::move(map));
-}
-
-template <typename V, typename T, typename MapFunc>
-AsyncGenerator<V> MakeMappedGenerator(AsyncGenerator<T> source_generator, MapFunc map) {
+template <typename T, typename MapFn,
+          typename Mapped = detail::result_of_t<MapFn(const T&)>,
+          typename V = typename EnsureFuture<Mapped>::type::ValueType>
+AsyncGenerator<V> MakeMappedGenerator(AsyncGenerator<T> source_generator, MapFn map) {
   struct MapCallback {
-    MapFunc map;
+    MapFn map_;
 
-    Future<V> operator()(const T& val) { return EnsureFuture(map(val)); }
+    Future<V> operator()(const T& val) { return EnsureFuture(map_(val)); }
+
+    Future<V> EnsureFuture(V mapped) {
+      return Future<V>::MakeFinished(std::move(mapped));
+    }
 
-    Future<V> EnsureFuture(Result<V> val) {
-      return Future<V>::MakeFinished(std::move(val));
+    Future<V> EnsureFuture(Result<V> mapped) {
+      return Future<V>::MakeFinished(std::move(mapped));
     }
-    Future<V> EnsureFuture(V val) { return Future<V>::MakeFinished(std::move(val)); }
-    Future<V> EnsureFuture(Future<V> val) { return val; }
+
+    Future<V> EnsureFuture(Future<V> mapped) { return mapped; }

Review comment:
       Is this related to the `EnsureFuture` struct in `future.h`? Should it be moved there?

##########
File path: cpp/src/arrow/compute/exec/exec_plan.h
##########
@@ -225,22 +212,43 @@ class ARROW_EXPORT ExecNode {
   virtual void StopProducing() = 0;
 
  protected:
-  ExecNode(ExecPlan* plan, std::string label, std::vector<BatchDescr> input_descrs,
+  ExecNode(ExecPlan*, std::string label, NodeVector inputs,
            std::vector<std::string> input_labels, BatchDescr output_descr,
            int num_outputs);
 
   ExecPlan* plan_;
-
   std::string label_;
 
-  std::vector<BatchDescr> input_descrs_;
-  std::vector<std::string> input_labels_;
   NodeVector inputs_;
+  std::vector<std::string> input_labels_;
 
   BatchDescr output_descr_;
   int num_outputs_;
   NodeVector outputs_;
 };
 
+/// \brief Adapt an AsyncGenerator<ExecBatch> as a source node
+ARROW_EXPORT
+ExecNode* MakeSourceNode(ExecPlan*, std::string label, ExecNode::BatchDescr output_descr,
+                         AsyncGenerator<util::optional<ExecBatch>>);

Review comment:
       I wonder if we should move the concrete ExecNode implementations to a separate file. `exec_plan.h` would only expose the base API. What do you think?

##########
File path: cpp/src/arrow/compute/exec/exec_plan.cc
##########
@@ -170,48 +165,409 @@ Status ExecPlan::Validate() { return ToDerived(this)->Validate(); }
 
 Status ExecPlan::StartProducing() { return ToDerived(this)->StartProducing(); }
 
-ExecNode::ExecNode(ExecPlan* plan, std::string label,
-                   std::vector<BatchDescr> input_descrs,
+ExecNode::ExecNode(ExecPlan* plan, std::string label, NodeVector inputs,
                    std::vector<std::string> input_labels, BatchDescr output_descr,
                    int num_outputs)
     : plan_(plan),
       label_(std::move(label)),
-      input_descrs_(std::move(input_descrs)),
+      inputs_(std::move(inputs)),
       input_labels_(std::move(input_labels)),
       output_descr_(std::move(output_descr)),
-      num_outputs_(num_outputs) {}
+      num_outputs_(num_outputs) {
+  for (auto input : inputs_) {
+    input->outputs_.push_back(this);
+  }
+}
 
 Status ExecNode::Validate() const {
-  if (inputs_.size() != input_descrs_.size()) {
+  if (inputs_.size() != input_labels_.size()) {
     return Status::Invalid("Invalid number of inputs for '", label(), "' (expected ",
-                           num_inputs(), ", actual ", inputs_.size(), ")");
+                           num_inputs(), ", actual ", input_labels_.size(), ")");
   }
 
   if (static_cast<int>(outputs_.size()) != num_outputs_) {
     return Status::Invalid("Invalid number of outputs for '", label(), "' (expected ",
                            num_outputs(), ", actual ", outputs_.size(), ")");
   }
 
-  DCHECK_EQ(input_descrs_.size(), input_labels_.size());
-
   for (auto out : outputs_) {
     auto input_index = GetNodeIndex(out->inputs(), this);
     if (!input_index) {
       return Status::Invalid("Node '", label(), "' outputs to node '", out->label(),
                              "' but is not listed as an input.");
     }
+  }
 
-    const auto& in_descr = out->input_descrs_[*input_index];
-    if (in_descr != output_descr_) {
-      return Status::Invalid(
-          "Node '", label(), "' (bound to input ", input_labels_[*input_index],
-          ") produces batches with type '", ValueDescr::ToString(output_descr_),
-          "' inconsistent with consumer '", out->label(), "' which accepts '",
-          ValueDescr::ToString(in_descr), "'");
+  return Status::OK();
+}
+
+struct SourceNode : ExecNode {
+  SourceNode(ExecPlan* plan, std::string label, ExecNode::BatchDescr output_descr,
+             AsyncGenerator<util::optional<ExecBatch>> generator)
+      : ExecNode(plan, std::move(label), {}, {}, std::move(output_descr),
+                 /*num_outputs=*/1),
+        generator_(std::move(generator)) {}
+
+  const char* kind_name() override { return "SourceNode"; }
+
+  static void NoInputs() { DCHECK(false) << "no inputs; this should never be called"; }
+  void InputReceived(ExecNode*, int, ExecBatch) override { NoInputs(); }
+  void ErrorReceived(ExecNode*, Status) override { NoInputs(); }
+  void InputFinished(ExecNode*, int) override { NoInputs(); }
+
+  Status StartProducing() override {
+    if (finished_) {
+      return Status::Invalid("Restarted SourceNode '", label(), "'");
     }
+
+    auto gen = std::move(generator_);
+
+    /// XXX should we wait on this future anywhere? In StopProducing() maybe?
+    auto done_fut =
+        Loop([gen, this] {

Review comment:
       Is the `Loop` construct necessary? I find that it produces unreadable code.

##########
File path: cpp/src/arrow/compute/exec/exec_plan.cc
##########
@@ -170,48 +165,409 @@ Status ExecPlan::Validate() { return ToDerived(this)->Validate(); }
 
 Status ExecPlan::StartProducing() { return ToDerived(this)->StartProducing(); }
 
-ExecNode::ExecNode(ExecPlan* plan, std::string label,
-                   std::vector<BatchDescr> input_descrs,
+ExecNode::ExecNode(ExecPlan* plan, std::string label, NodeVector inputs,
                    std::vector<std::string> input_labels, BatchDescr output_descr,
                    int num_outputs)
     : plan_(plan),
       label_(std::move(label)),
-      input_descrs_(std::move(input_descrs)),
+      inputs_(std::move(inputs)),
       input_labels_(std::move(input_labels)),
       output_descr_(std::move(output_descr)),
-      num_outputs_(num_outputs) {}
+      num_outputs_(num_outputs) {
+  for (auto input : inputs_) {
+    input->outputs_.push_back(this);
+  }
+}
 
 Status ExecNode::Validate() const {
-  if (inputs_.size() != input_descrs_.size()) {
+  if (inputs_.size() != input_labels_.size()) {
     return Status::Invalid("Invalid number of inputs for '", label(), "' (expected ",
-                           num_inputs(), ", actual ", inputs_.size(), ")");
+                           num_inputs(), ", actual ", input_labels_.size(), ")");
   }
 
   if (static_cast<int>(outputs_.size()) != num_outputs_) {
     return Status::Invalid("Invalid number of outputs for '", label(), "' (expected ",
                            num_outputs(), ", actual ", outputs_.size(), ")");
   }
 
-  DCHECK_EQ(input_descrs_.size(), input_labels_.size());
-
   for (auto out : outputs_) {
     auto input_index = GetNodeIndex(out->inputs(), this);
     if (!input_index) {
       return Status::Invalid("Node '", label(), "' outputs to node '", out->label(),
                              "' but is not listed as an input.");
     }
+  }
 
-    const auto& in_descr = out->input_descrs_[*input_index];
-    if (in_descr != output_descr_) {
-      return Status::Invalid(
-          "Node '", label(), "' (bound to input ", input_labels_[*input_index],
-          ") produces batches with type '", ValueDescr::ToString(output_descr_),
-          "' inconsistent with consumer '", out->label(), "' which accepts '",
-          ValueDescr::ToString(in_descr), "'");
+  return Status::OK();
+}
+
+struct SourceNode : ExecNode {
+  SourceNode(ExecPlan* plan, std::string label, ExecNode::BatchDescr output_descr,
+             AsyncGenerator<util::optional<ExecBatch>> generator)
+      : ExecNode(plan, std::move(label), {}, {}, std::move(output_descr),
+                 /*num_outputs=*/1),
+        generator_(std::move(generator)) {}
+
+  const char* kind_name() override { return "SourceNode"; }
+
+  static void NoInputs() { DCHECK(false) << "no inputs; this should never be called"; }
+  void InputReceived(ExecNode*, int, ExecBatch) override { NoInputs(); }
+  void ErrorReceived(ExecNode*, Status) override { NoInputs(); }
+  void InputFinished(ExecNode*, int) override { NoInputs(); }
+
+  Status StartProducing() override {
+    if (finished_) {
+      return Status::Invalid("Restarted SourceNode '", label(), "'");
     }
+
+    auto gen = std::move(generator_);
+
+    /// XXX should we wait on this future anywhere? In StopProducing() maybe?

Review comment:
       Perhaps in `~SourceNode()`... but destructors may not be called in deterministic order, so perhaps we need a `Close` method instead that would be called in order by `~ExecPlan()`.

##########
File path: cpp/src/arrow/compute/exec/exec_plan.cc
##########
@@ -105,18 +107,11 @@ struct ExecPlanImpl : public ExecPlan {
           return Status::OK();
         }
 
-        auto it_success = visiting.insert(node);
-        if (!it_success.second) {
-          // Insertion failed => node is already being visited
-          return Status::Invalid("Cycle detected in execution plan");
-        }

Review comment:
       Why did you remove this?

##########
File path: cpp/src/arrow/compute/exec/exec_plan.cc
##########
@@ -170,48 +165,409 @@ Status ExecPlan::Validate() { return ToDerived(this)->Validate(); }
 
 Status ExecPlan::StartProducing() { return ToDerived(this)->StartProducing(); }
 
-ExecNode::ExecNode(ExecPlan* plan, std::string label,
-                   std::vector<BatchDescr> input_descrs,
+ExecNode::ExecNode(ExecPlan* plan, std::string label, NodeVector inputs,
                    std::vector<std::string> input_labels, BatchDescr output_descr,
                    int num_outputs)
     : plan_(plan),
       label_(std::move(label)),
-      input_descrs_(std::move(input_descrs)),
+      inputs_(std::move(inputs)),
       input_labels_(std::move(input_labels)),
       output_descr_(std::move(output_descr)),
-      num_outputs_(num_outputs) {}
+      num_outputs_(num_outputs) {
+  for (auto input : inputs_) {
+    input->outputs_.push_back(this);
+  }
+}
 
 Status ExecNode::Validate() const {
-  if (inputs_.size() != input_descrs_.size()) {
+  if (inputs_.size() != input_labels_.size()) {
     return Status::Invalid("Invalid number of inputs for '", label(), "' (expected ",
-                           num_inputs(), ", actual ", inputs_.size(), ")");
+                           num_inputs(), ", actual ", input_labels_.size(), ")");
   }
 
   if (static_cast<int>(outputs_.size()) != num_outputs_) {
     return Status::Invalid("Invalid number of outputs for '", label(), "' (expected ",
                            num_outputs(), ", actual ", outputs_.size(), ")");
   }
 
-  DCHECK_EQ(input_descrs_.size(), input_labels_.size());
-
   for (auto out : outputs_) {
     auto input_index = GetNodeIndex(out->inputs(), this);
     if (!input_index) {
       return Status::Invalid("Node '", label(), "' outputs to node '", out->label(),
                              "' but is not listed as an input.");
     }
+  }
 
-    const auto& in_descr = out->input_descrs_[*input_index];
-    if (in_descr != output_descr_) {
-      return Status::Invalid(
-          "Node '", label(), "' (bound to input ", input_labels_[*input_index],
-          ") produces batches with type '", ValueDescr::ToString(output_descr_),
-          "' inconsistent with consumer '", out->label(), "' which accepts '",
-          ValueDescr::ToString(in_descr), "'");
+  return Status::OK();
+}
+
+struct SourceNode : ExecNode {
+  SourceNode(ExecPlan* plan, std::string label, ExecNode::BatchDescr output_descr,
+             AsyncGenerator<util::optional<ExecBatch>> generator)
+      : ExecNode(plan, std::move(label), {}, {}, std::move(output_descr),
+                 /*num_outputs=*/1),
+        generator_(std::move(generator)) {}
+
+  const char* kind_name() override { return "SourceNode"; }
+
+  static void NoInputs() { DCHECK(false) << "no inputs; this should never be called"; }
+  void InputReceived(ExecNode*, int, ExecBatch) override { NoInputs(); }
+  void ErrorReceived(ExecNode*, Status) override { NoInputs(); }
+  void InputFinished(ExecNode*, int) override { NoInputs(); }
+
+  Status StartProducing() override {
+    if (finished_) {
+      return Status::Invalid("Restarted SourceNode '", label(), "'");
     }
+
+    auto gen = std::move(generator_);
+
+    /// XXX should we wait on this future anywhere? In StopProducing() maybe?
+    auto done_fut =
+        Loop([gen, this] {
+          std::unique_lock<std::mutex> lock(mutex_);
+          int seq = next_batch_index_++;
+          if (finished_) {
+            return Future<ControlFlow<int>>::MakeFinished(Break(seq));
+          }
+          lock.unlock();
+
+          return gen().Then(
+              [=](const util::optional<ExecBatch>& batch) -> ControlFlow<int> {
+                std::unique_lock<std::mutex> lock(mutex_);
+                if (!batch || finished_) {
+                  finished_ = true;
+                  return Break(seq);
+                }
+                lock.unlock();
+
+                outputs_[0]->InputReceived(this, seq, *batch);
+                return Continue();
+              },
+              [=](const Status& error) -> ControlFlow<int> {
+                std::unique_lock<std::mutex> lock(mutex_);
+                if (!finished_) {
+                  finished_ = true;
+                  lock.unlock();
+                  // unless we were already finished, push the error to our output
+                  // XXX is this correct? Is it reasonable for a consumer to ignore errors
+                  // from a finished producer?
+                  outputs_[0]->ErrorReceived(this, error);
+                }
+                return Break(seq);
+              });
+        }).Then([&](int seq) {
+          /// XXX this is probably redundant: do we always call InputFinished after
+          /// ErrorReceived or will ErrorRecieved be sufficient?
+          outputs_[0]->InputFinished(this, seq);
+        });
+
+    return Status::OK();
   }
 
-  return Status::OK();
+  void PauseProducing(ExecNode* output) override {}
+
+  void ResumeProducing(ExecNode* output) override {}

Review comment:
       The problem is that we'll want to be able to apply backpressure at some point. But a generator doesn't allow for that. So it seems that, instead of wrapping a generator, you should really have a `ExecNode` that wraps a dataset scanner directly.

##########
File path: cpp/src/arrow/compute/exec/exec_plan.cc
##########
@@ -170,48 +165,409 @@ Status ExecPlan::Validate() { return ToDerived(this)->Validate(); }
 
 Status ExecPlan::StartProducing() { return ToDerived(this)->StartProducing(); }
 
-ExecNode::ExecNode(ExecPlan* plan, std::string label,
-                   std::vector<BatchDescr> input_descrs,
+ExecNode::ExecNode(ExecPlan* plan, std::string label, NodeVector inputs,
                    std::vector<std::string> input_labels, BatchDescr output_descr,
                    int num_outputs)
     : plan_(plan),
       label_(std::move(label)),
-      input_descrs_(std::move(input_descrs)),
+      inputs_(std::move(inputs)),
       input_labels_(std::move(input_labels)),
       output_descr_(std::move(output_descr)),
-      num_outputs_(num_outputs) {}
+      num_outputs_(num_outputs) {
+  for (auto input : inputs_) {
+    input->outputs_.push_back(this);
+  }
+}
 
 Status ExecNode::Validate() const {
-  if (inputs_.size() != input_descrs_.size()) {
+  if (inputs_.size() != input_labels_.size()) {
     return Status::Invalid("Invalid number of inputs for '", label(), "' (expected ",
-                           num_inputs(), ", actual ", inputs_.size(), ")");
+                           num_inputs(), ", actual ", input_labels_.size(), ")");
   }
 
   if (static_cast<int>(outputs_.size()) != num_outputs_) {
     return Status::Invalid("Invalid number of outputs for '", label(), "' (expected ",
                            num_outputs(), ", actual ", outputs_.size(), ")");
   }
 
-  DCHECK_EQ(input_descrs_.size(), input_labels_.size());
-
   for (auto out : outputs_) {
     auto input_index = GetNodeIndex(out->inputs(), this);
     if (!input_index) {
       return Status::Invalid("Node '", label(), "' outputs to node '", out->label(),
                              "' but is not listed as an input.");
     }
+  }
 
-    const auto& in_descr = out->input_descrs_[*input_index];
-    if (in_descr != output_descr_) {
-      return Status::Invalid(
-          "Node '", label(), "' (bound to input ", input_labels_[*input_index],
-          ") produces batches with type '", ValueDescr::ToString(output_descr_),
-          "' inconsistent with consumer '", out->label(), "' which accepts '",
-          ValueDescr::ToString(in_descr), "'");
+  return Status::OK();
+}
+
+struct SourceNode : ExecNode {
+  SourceNode(ExecPlan* plan, std::string label, ExecNode::BatchDescr output_descr,
+             AsyncGenerator<util::optional<ExecBatch>> generator)
+      : ExecNode(plan, std::move(label), {}, {}, std::move(output_descr),
+                 /*num_outputs=*/1),
+        generator_(std::move(generator)) {}
+
+  const char* kind_name() override { return "SourceNode"; }
+
+  static void NoInputs() { DCHECK(false) << "no inputs; this should never be called"; }
+  void InputReceived(ExecNode*, int, ExecBatch) override { NoInputs(); }
+  void ErrorReceived(ExecNode*, Status) override { NoInputs(); }
+  void InputFinished(ExecNode*, int) override { NoInputs(); }
+
+  Status StartProducing() override {
+    if (finished_) {
+      return Status::Invalid("Restarted SourceNode '", label(), "'");
     }
+
+    auto gen = std::move(generator_);
+
+    /// XXX should we wait on this future anywhere? In StopProducing() maybe?
+    auto done_fut =
+        Loop([gen, this] {
+          std::unique_lock<std::mutex> lock(mutex_);
+          int seq = next_batch_index_++;
+          if (finished_) {
+            return Future<ControlFlow<int>>::MakeFinished(Break(seq));
+          }
+          lock.unlock();
+
+          return gen().Then(
+              [=](const util::optional<ExecBatch>& batch) -> ControlFlow<int> {
+                std::unique_lock<std::mutex> lock(mutex_);
+                if (!batch || finished_) {
+                  finished_ = true;
+                  return Break(seq);
+                }
+                lock.unlock();
+
+                outputs_[0]->InputReceived(this, seq, *batch);
+                return Continue();
+              },
+              [=](const Status& error) -> ControlFlow<int> {
+                std::unique_lock<std::mutex> lock(mutex_);
+                if (!finished_) {
+                  finished_ = true;
+                  lock.unlock();
+                  // unless we were already finished, push the error to our output
+                  // XXX is this correct? Is it reasonable for a consumer to ignore errors
+                  // from a finished producer?
+                  outputs_[0]->ErrorReceived(this, error);
+                }
+                return Break(seq);
+              });
+        }).Then([&](int seq) {
+          /// XXX this is probably redundant: do we always call InputFinished after
+          /// ErrorReceived or will ErrorRecieved be sufficient?
+          outputs_[0]->InputFinished(this, seq);
+        });
+
+    return Status::OK();
   }
 
-  return Status::OK();
+  void PauseProducing(ExecNode* output) override {}
+
+  void ResumeProducing(ExecNode* output) override {}
+
+  void StopProducing(ExecNode* output) override {
+    DCHECK_EQ(output, outputs_[0]);
+    std::unique_lock<std::mutex> lock(mutex_);
+    finished_ = true;
+  }
+
+  void StopProducing() override { StopProducing(outputs_[0]); }
+
+ private:
+  std::mutex mutex_;
+  bool finished_{false};
+  int next_batch_index_{0};
+  AsyncGenerator<util::optional<ExecBatch>> generator_;
+};
+
+ExecNode* MakeSourceNode(ExecPlan* plan, std::string label,
+                         ExecNode::BatchDescr output_descr,
+                         AsyncGenerator<util::optional<ExecBatch>> generator) {
+  return plan->EmplaceNode<SourceNode>(plan, std::move(label), std::move(output_descr),
+                                       std::move(generator));
+}
+
+struct FilterNode : ExecNode {
+  FilterNode(ExecNode* input, std::string label, Expression filter)
+      : ExecNode(input->plan(), std::move(label), {input}, {"target"},
+                 /*output_descr=*/{input->output_descr()},
+                 /*num_outputs=*/1),
+        filter_(std::move(filter)) {}
+
+  const char* kind_name() override { return "FilterNode"; }
+
+  Result<ExecBatch> DoFilter(const ExecBatch& target) {
+    ARROW_ASSIGN_OR_RAISE(Expression simplified_filter,
+                          SimplifyWithGuarantee(filter_, target.guarantee));
+
+    // XXX get a non-default exec context
+    ARROW_ASSIGN_OR_RAISE(Datum mask, ExecuteScalarExpression(simplified_filter, target));
+
+    if (mask.is_scalar()) {
+      const auto& mask_scalar = mask.scalar_as<BooleanScalar>();
+      if (mask_scalar.is_valid && mask_scalar.value) {
+        return target;
+      }
+
+      return target.Slice(0, 0);
+    }
+
+    auto values = target.values;
+    for (auto& value : values) {
+      if (value.is_scalar()) continue;
+      ARROW_ASSIGN_OR_RAISE(value, Filter(value, mask, FilterOptions::Defaults()));
+    }
+    return ExecBatch::Make(std::move(values));
+  }
+
+  void InputReceived(ExecNode* input, int seq, ExecBatch batch) override {
+    DCHECK_EQ(input, inputs_[0]);
+
+    auto maybe_filtered = DoFilter(std::move(batch));
+    if (!maybe_filtered.ok()) {
+      outputs_[0]->ErrorReceived(this, maybe_filtered.status());
+      inputs_[0]->StopProducing(this);
+      return;
+    }
+
+    maybe_filtered->guarantee = batch.guarantee;
+    outputs_[0]->InputReceived(this, seq, maybe_filtered.MoveValueUnsafe());
+  }
+
+  void ErrorReceived(ExecNode* input, Status error) override {
+    DCHECK_EQ(input, inputs_[0]);
+    outputs_[0]->ErrorReceived(this, std::move(error));
+    inputs_[0]->StopProducing(this);
+  }
+
+  void InputFinished(ExecNode* input, int seq) override {
+    DCHECK_EQ(input, inputs_[0]);
+    outputs_[0]->InputFinished(this, seq);
+    inputs_[0]->StopProducing(this);
+  }
+
+  Status StartProducing() override {
+    // XXX validate inputs_[0]->output_descr() against filter_
+    return Status::OK();
+  }
+
+  void PauseProducing(ExecNode* output) override {}
+
+  void ResumeProducing(ExecNode* output) override {}
+
+  void StopProducing(ExecNode* output) override {
+    DCHECK_EQ(output, outputs_[0]);
+    inputs_[0]->StopProducing(this);
+  }
+
+  void StopProducing() override { StopProducing(outputs_[0]); }
+
+ private:
+  Expression filter_;
+};
+
+ExecNode* MakeFilterNode(ExecNode* input, std::string label, Expression filter) {
+  return input->plan()->EmplaceNode<FilterNode>(input, std::move(label),
+                                                std::move(filter));
+}
+
+struct ProjectNode : ExecNode {
+  ProjectNode(ExecNode* input, std::string label, std::vector<Expression> exprs)
+      : ExecNode(input->plan(), std::move(label), {input}, {"target"},
+                 /*output_descr=*/{input->output_descr()},
+                 /*num_outputs=*/1),
+        exprs_(std::move(exprs)) {}
+
+  const char* kind_name() override { return "ProjectNode"; }
+
+  Result<ExecBatch> DoProject(const ExecBatch& target) {
+    // XXX get a non-default exec context
+    std::vector<Datum> values{exprs_.size()};
+    for (size_t i = 0; i < exprs_.size(); ++i) {
+      ARROW_ASSIGN_OR_RAISE(Expression simplified_expr,
+                            SimplifyWithGuarantee(exprs_[i], target.guarantee));
+
+      ARROW_ASSIGN_OR_RAISE(values[i], ExecuteScalarExpression(simplified_expr, target));
+    }
+    return ExecBatch::Make(std::move(values));
+  }
+
+  void InputReceived(ExecNode* input, int seq, ExecBatch batch) override {
+    DCHECK_EQ(input, inputs_[0]);
+
+    auto maybe_projected = DoProject(std::move(batch));
+    if (!maybe_projected.ok()) {
+      outputs_[0]->ErrorReceived(this, maybe_projected.status());
+      inputs_[0]->StopProducing(this);
+      return;
+    }
+
+    maybe_projected->guarantee = batch.guarantee;
+    outputs_[0]->InputReceived(this, seq, maybe_projected.MoveValueUnsafe());
+  }
+
+  void ErrorReceived(ExecNode* input, Status error) override {
+    DCHECK_EQ(input, inputs_[0]);
+    outputs_[0]->ErrorReceived(this, std::move(error));
+    inputs_[0]->StopProducing(this);
+  }
+
+  void InputFinished(ExecNode* input, int seq) override {
+    DCHECK_EQ(input, inputs_[0]);
+    outputs_[0]->InputFinished(this, seq);
+    inputs_[0]->StopProducing(this);
+  }
+
+  Status StartProducing() override {
+    // XXX validate inputs_[0]->output_descr() against filter_
+    return Status::OK();
+  }
+
+  void PauseProducing(ExecNode* output) override {}
+
+  void ResumeProducing(ExecNode* output) override {}
+
+  void StopProducing(ExecNode* output) override {
+    DCHECK_EQ(output, outputs_[0]);
+    inputs_[0]->StopProducing(this);
+  }
+
+  void StopProducing() override { StopProducing(outputs_[0]); }
+
+ private:
+  std::vector<Expression> exprs_;
+};
+
+ExecNode* MakeProjectNode(ExecNode* input, std::string label,
+                          std::vector<Expression> exprs) {
+  return input->plan()->EmplaceNode<ProjectNode>(input, std::move(label),
+                                                 std::move(exprs));
+}
+
+struct SinkNode : ExecNode {
+  SinkNode(ExecNode* input, std::string label,
+           AsyncGenerator<util::optional<ExecBatch>>* generator)
+      : ExecNode(input->plan(), std::move(label), {input}, {"collected"}, {},
+                 /*num_outputs=*/0),
+        producer_(MakeProducer(generator)) {}
+
+  static PushGenerator<util::optional<ExecBatch>>::Producer MakeProducer(
+      AsyncGenerator<util::optional<ExecBatch>>* out_gen) {
+    PushGenerator<util::optional<ExecBatch>> gen;
+    auto out = gen.producer();
+    *out_gen = std::move(gen);
+    return out;
+  }
+
+  const char* kind_name() override { return "SinkNode"; }
+
+  Status StartProducing() override { return Status::OK(); }
+
+  // sink nodes have no outputs from which to feel backpressure
+  static void NoOutputs() { DCHECK(false) << "no outputs; this should never be called"; }
+  void ResumeProducing(ExecNode* output) override { NoOutputs(); }
+  void PauseProducing(ExecNode* output) override { NoOutputs(); }
+  void StopProducing(ExecNode* output) override { NoOutputs(); }
+
+  void StopProducing() override {
+    std::unique_lock<std::mutex> lock(mutex_);
+    StopProducingUnlocked();
+  }
+
+  void InputReceived(ExecNode* input, int seq_num, ExecBatch exec_batch) override {
+    std::unique_lock<std::mutex> lock(mutex_);
+    if (stopped_) return;
+
+    // TODO would be nice to factor this out in a ReorderQueue
+    if (seq_num <= static_cast<int>(received_batches_.size())) {
+      received_batches_.resize(seq_num + 1);
+      emitted_.resize(seq_num + 1, false);
+    }
+    received_batches_[seq_num] = std::move(exec_batch);
+    ++num_received_;
+
+    if (seq_num != num_emitted_) {
+      // Cannot emit yet as there is a hole at `num_emitted_`
+      DCHECK_GT(seq_num, num_emitted_);
+      return;
+    }
+
+    if (num_received_ == emit_stop_) {
+      StopProducingUnlocked();
+    }
+
+    // Emit batches in order as far as possible
+    // First collect these batches, then unlock before producing.
+    const auto seq_start = seq_num;
+    while (seq_num < static_cast<int>(emitted_.size()) && !emitted_[seq_num]) {
+      emitted_[seq_num] = true;
+      ++seq_num;
+    }
+    DCHECK_GT(seq_num, seq_start);
+    // By moving the values now, we make sure another thread won't emit the same values
+    // below
+    std::vector<ExecBatch> to_emit(
+        std::make_move_iterator(received_batches_.begin() + seq_start),
+        std::make_move_iterator(received_batches_.begin() + seq_num));
+
+    lock.unlock();
+    for (auto&& batch : to_emit) {
+      producer_.Push(std::move(batch));
+    }
+    lock.lock();
+
+    DCHECK_EQ(seq_start, num_emitted_);  // num_emitted_ wasn't bumped in the meantime
+    num_emitted_ = seq_num;

Review comment:
       @westonpace If another thread comes here, it will have a different `seq_num` and will hit the `seq_num != num_emitted_` condition above.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] bkietz commented on a change in pull request #10397: ARROW-11930: [C++][Dataset][Compute] Use an ExecPlan for dataset scans

Posted by GitBox <gi...@apache.org>.
bkietz commented on a change in pull request #10397:
URL: https://github.com/apache/arrow/pull/10397#discussion_r640907832



##########
File path: cpp/src/arrow/compute/exec/exec_plan.cc
##########
@@ -170,48 +165,409 @@ Status ExecPlan::Validate() { return ToDerived(this)->Validate(); }
 
 Status ExecPlan::StartProducing() { return ToDerived(this)->StartProducing(); }
 
-ExecNode::ExecNode(ExecPlan* plan, std::string label,
-                   std::vector<BatchDescr> input_descrs,
+ExecNode::ExecNode(ExecPlan* plan, std::string label, NodeVector inputs,
                    std::vector<std::string> input_labels, BatchDescr output_descr,
                    int num_outputs)
     : plan_(plan),
       label_(std::move(label)),
-      input_descrs_(std::move(input_descrs)),
+      inputs_(std::move(inputs)),
       input_labels_(std::move(input_labels)),
       output_descr_(std::move(output_descr)),
-      num_outputs_(num_outputs) {}
+      num_outputs_(num_outputs) {
+  for (auto input : inputs_) {
+    input->outputs_.push_back(this);
+  }
+}
 
 Status ExecNode::Validate() const {
-  if (inputs_.size() != input_descrs_.size()) {
+  if (inputs_.size() != input_labels_.size()) {
     return Status::Invalid("Invalid number of inputs for '", label(), "' (expected ",
-                           num_inputs(), ", actual ", inputs_.size(), ")");
+                           num_inputs(), ", actual ", input_labels_.size(), ")");
   }
 
   if (static_cast<int>(outputs_.size()) != num_outputs_) {
     return Status::Invalid("Invalid number of outputs for '", label(), "' (expected ",
                            num_outputs(), ", actual ", outputs_.size(), ")");
   }
 
-  DCHECK_EQ(input_descrs_.size(), input_labels_.size());
-
   for (auto out : outputs_) {
     auto input_index = GetNodeIndex(out->inputs(), this);
     if (!input_index) {
       return Status::Invalid("Node '", label(), "' outputs to node '", out->label(),
                              "' but is not listed as an input.");
     }
+  }
 
-    const auto& in_descr = out->input_descrs_[*input_index];
-    if (in_descr != output_descr_) {
-      return Status::Invalid(
-          "Node '", label(), "' (bound to input ", input_labels_[*input_index],
-          ") produces batches with type '", ValueDescr::ToString(output_descr_),
-          "' inconsistent with consumer '", out->label(), "' which accepts '",
-          ValueDescr::ToString(in_descr), "'");
+  return Status::OK();
+}
+
+struct SourceNode : ExecNode {
+  SourceNode(ExecPlan* plan, std::string label, ExecNode::BatchDescr output_descr,
+             AsyncGenerator<util::optional<ExecBatch>> generator)
+      : ExecNode(plan, std::move(label), {}, {}, std::move(output_descr),
+                 /*num_outputs=*/1),
+        generator_(std::move(generator)) {}
+
+  const char* kind_name() override { return "SourceNode"; }
+
+  static void NoInputs() { DCHECK(false) << "no inputs; this should never be called"; }
+  void InputReceived(ExecNode*, int, ExecBatch) override { NoInputs(); }
+  void ErrorReceived(ExecNode*, Status) override { NoInputs(); }
+  void InputFinished(ExecNode*, int) override { NoInputs(); }
+
+  Status StartProducing() override {
+    if (finished_) {
+      return Status::Invalid("Restarted SourceNode '", label(), "'");
     }
+
+    auto gen = std::move(generator_);
+
+    /// XXX should we wait on this future anywhere? In StopProducing() maybe?
+    auto done_fut =
+        Loop([gen, this] {
+          std::unique_lock<std::mutex> lock(mutex_);
+          int seq = next_batch_index_++;
+          if (finished_) {
+            return Future<ControlFlow<int>>::MakeFinished(Break(seq));
+          }
+          lock.unlock();
+
+          return gen().Then(
+              [=](const util::optional<ExecBatch>& batch) -> ControlFlow<int> {
+                std::unique_lock<std::mutex> lock(mutex_);
+                if (!batch || finished_) {
+                  finished_ = true;
+                  return Break(seq);
+                }
+                lock.unlock();
+
+                outputs_[0]->InputReceived(this, seq, *batch);
+                return Continue();
+              },
+              [=](const Status& error) -> ControlFlow<int> {
+                std::unique_lock<std::mutex> lock(mutex_);
+                if (!finished_) {
+                  finished_ = true;
+                  lock.unlock();
+                  // unless we were already finished, push the error to our output
+                  // XXX is this correct? Is it reasonable for a consumer to ignore errors
+                  // from a finished producer?

Review comment:
       I'd agree that handling/ignoring trailing batches is necessary; the producer may take a while to stop. However I wonder if it's reasonable to do the same for trailing errors. For example: let's say we have a plan where a LimitNode is taking the first 99 rows from EmitsErrorAfterHundredthRowNode. There's a race condition here (also depends on chunking) because the LimitNode will sometimes receive the trailing error before it can stop the producer and sometimes  will succeed in stopping its producer before it gets around to raising an error. I'm not sure what the correct answer here is, but I lean toward: if any node emits any error, that always puts all subsequent nodes into an error state too (unless explicitly intercepted). The above example seems like a problem we need to fix in EmitsErrorAfterHundredthRowNode rather than requiring all consumers to ignore post-stop errors




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] westonpace commented on a change in pull request #10397: ARROW-11930: [C++][Dataset][Compute] Use an ExecPlan for dataset scans

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #10397:
URL: https://github.com/apache/arrow/pull/10397#discussion_r644158198



##########
File path: cpp/src/arrow/compute/exec/exec_plan.cc
##########
@@ -170,48 +165,409 @@ Status ExecPlan::Validate() { return ToDerived(this)->Validate(); }
 
 Status ExecPlan::StartProducing() { return ToDerived(this)->StartProducing(); }
 
-ExecNode::ExecNode(ExecPlan* plan, std::string label,
-                   std::vector<BatchDescr> input_descrs,
+ExecNode::ExecNode(ExecPlan* plan, std::string label, NodeVector inputs,
                    std::vector<std::string> input_labels, BatchDescr output_descr,
                    int num_outputs)
     : plan_(plan),
       label_(std::move(label)),
-      input_descrs_(std::move(input_descrs)),
+      inputs_(std::move(inputs)),
       input_labels_(std::move(input_labels)),
       output_descr_(std::move(output_descr)),
-      num_outputs_(num_outputs) {}
+      num_outputs_(num_outputs) {
+  for (auto input : inputs_) {
+    input->outputs_.push_back(this);
+  }
+}
 
 Status ExecNode::Validate() const {
-  if (inputs_.size() != input_descrs_.size()) {
+  if (inputs_.size() != input_labels_.size()) {
     return Status::Invalid("Invalid number of inputs for '", label(), "' (expected ",
-                           num_inputs(), ", actual ", inputs_.size(), ")");
+                           num_inputs(), ", actual ", input_labels_.size(), ")");
   }
 
   if (static_cast<int>(outputs_.size()) != num_outputs_) {
     return Status::Invalid("Invalid number of outputs for '", label(), "' (expected ",
                            num_outputs(), ", actual ", outputs_.size(), ")");
   }
 
-  DCHECK_EQ(input_descrs_.size(), input_labels_.size());
-
   for (auto out : outputs_) {
     auto input_index = GetNodeIndex(out->inputs(), this);
     if (!input_index) {
       return Status::Invalid("Node '", label(), "' outputs to node '", out->label(),
                              "' but is not listed as an input.");
     }
+  }
 
-    const auto& in_descr = out->input_descrs_[*input_index];
-    if (in_descr != output_descr_) {
-      return Status::Invalid(
-          "Node '", label(), "' (bound to input ", input_labels_[*input_index],
-          ") produces batches with type '", ValueDescr::ToString(output_descr_),
-          "' inconsistent with consumer '", out->label(), "' which accepts '",
-          ValueDescr::ToString(in_descr), "'");
+  return Status::OK();
+}
+
+struct SourceNode : ExecNode {
+  SourceNode(ExecPlan* plan, std::string label, ExecNode::BatchDescr output_descr,
+             AsyncGenerator<util::optional<ExecBatch>> generator)
+      : ExecNode(plan, std::move(label), {}, {}, std::move(output_descr),
+                 /*num_outputs=*/1),
+        generator_(std::move(generator)) {}
+
+  const char* kind_name() override { return "SourceNode"; }
+
+  static void NoInputs() { DCHECK(false) << "no inputs; this should never be called"; }
+  void InputReceived(ExecNode*, int, ExecBatch) override { NoInputs(); }
+  void ErrorReceived(ExecNode*, Status) override { NoInputs(); }
+  void InputFinished(ExecNode*, int) override { NoInputs(); }
+
+  Status StartProducing() override {
+    if (finished_) {
+      return Status::Invalid("Restarted SourceNode '", label(), "'");
     }
+
+    auto gen = std::move(generator_);
+
+    /// XXX should we wait on this future anywhere? In StopProducing() maybe?
+    auto done_fut =
+        Loop([gen, this] {
+          std::unique_lock<std::mutex> lock(mutex_);
+          int seq = next_batch_index_++;
+          if (finished_) {
+            return Future<ControlFlow<int>>::MakeFinished(Break(seq));
+          }
+          lock.unlock();
+
+          return gen().Then(
+              [=](const util::optional<ExecBatch>& batch) -> ControlFlow<int> {
+                std::unique_lock<std::mutex> lock(mutex_);
+                if (!batch || finished_) {
+                  finished_ = true;
+                  return Break(seq);
+                }
+                lock.unlock();
+
+                outputs_[0]->InputReceived(this, seq, *batch);
+                return Continue();
+              },
+              [=](const Status& error) -> ControlFlow<int> {
+                std::unique_lock<std::mutex> lock(mutex_);
+                if (!finished_) {
+                  finished_ = true;
+                  lock.unlock();
+                  // unless we were already finished, push the error to our output
+                  // XXX is this correct? Is it reasonable for a consumer to ignore errors
+                  // from a finished producer?
+                  outputs_[0]->ErrorReceived(this, error);
+                }
+                return Break(seq);
+              });
+        }).Then([&](int seq) {
+          /// XXX this is probably redundant: do we always call InputFinished after
+          /// ErrorReceived or will ErrorRecieved be sufficient?
+          outputs_[0]->InputFinished(this, seq);
+        });
+
+    return Status::OK();
   }
 
-  return Status::OK();
+  void PauseProducing(ExecNode* output) override {}
+
+  void ResumeProducing(ExecNode* output) override {}
+
+  void StopProducing(ExecNode* output) override {
+    DCHECK_EQ(output, outputs_[0]);
+    std::unique_lock<std::mutex> lock(mutex_);
+    finished_ = true;
+  }
+
+  void StopProducing() override { StopProducing(outputs_[0]); }
+
+ private:
+  std::mutex mutex_;
+  bool finished_{false};
+  int next_batch_index_{0};
+  AsyncGenerator<util::optional<ExecBatch>> generator_;
+};
+
+ExecNode* MakeSourceNode(ExecPlan* plan, std::string label,
+                         ExecNode::BatchDescr output_descr,
+                         AsyncGenerator<util::optional<ExecBatch>> generator) {
+  return plan->EmplaceNode<SourceNode>(plan, std::move(label), std::move(output_descr),
+                                       std::move(generator));
+}
+
+struct FilterNode : ExecNode {
+  FilterNode(ExecNode* input, std::string label, Expression filter)
+      : ExecNode(input->plan(), std::move(label), {input}, {"target"},
+                 /*output_descr=*/{input->output_descr()},
+                 /*num_outputs=*/1),
+        filter_(std::move(filter)) {}
+
+  const char* kind_name() override { return "FilterNode"; }
+
+  Result<ExecBatch> DoFilter(const ExecBatch& target) {
+    ARROW_ASSIGN_OR_RAISE(Expression simplified_filter,
+                          SimplifyWithGuarantee(filter_, target.guarantee));
+
+    // XXX get a non-default exec context
+    ARROW_ASSIGN_OR_RAISE(Datum mask, ExecuteScalarExpression(simplified_filter, target));
+
+    if (mask.is_scalar()) {
+      const auto& mask_scalar = mask.scalar_as<BooleanScalar>();
+      if (mask_scalar.is_valid && mask_scalar.value) {
+        return target;
+      }
+
+      return target.Slice(0, 0);
+    }
+
+    auto values = target.values;
+    for (auto& value : values) {
+      if (value.is_scalar()) continue;
+      ARROW_ASSIGN_OR_RAISE(value, Filter(value, mask, FilterOptions::Defaults()));
+    }
+    return ExecBatch::Make(std::move(values));
+  }
+
+  void InputReceived(ExecNode* input, int seq, ExecBatch batch) override {
+    DCHECK_EQ(input, inputs_[0]);
+
+    auto maybe_filtered = DoFilter(std::move(batch));
+    if (!maybe_filtered.ok()) {
+      outputs_[0]->ErrorReceived(this, maybe_filtered.status());
+      inputs_[0]->StopProducing(this);
+      return;
+    }
+
+    maybe_filtered->guarantee = batch.guarantee;
+    outputs_[0]->InputReceived(this, seq, maybe_filtered.MoveValueUnsafe());
+  }
+
+  void ErrorReceived(ExecNode* input, Status error) override {
+    DCHECK_EQ(input, inputs_[0]);
+    outputs_[0]->ErrorReceived(this, std::move(error));
+    inputs_[0]->StopProducing(this);
+  }
+
+  void InputFinished(ExecNode* input, int seq) override {
+    DCHECK_EQ(input, inputs_[0]);
+    outputs_[0]->InputFinished(this, seq);
+    inputs_[0]->StopProducing(this);
+  }
+
+  Status StartProducing() override {
+    // XXX validate inputs_[0]->output_descr() against filter_
+    return Status::OK();
+  }
+
+  void PauseProducing(ExecNode* output) override {}
+
+  void ResumeProducing(ExecNode* output) override {}
+
+  void StopProducing(ExecNode* output) override {
+    DCHECK_EQ(output, outputs_[0]);
+    inputs_[0]->StopProducing(this);
+  }
+
+  void StopProducing() override { StopProducing(outputs_[0]); }
+
+ private:
+  Expression filter_;
+};
+
+ExecNode* MakeFilterNode(ExecNode* input, std::string label, Expression filter) {
+  return input->plan()->EmplaceNode<FilterNode>(input, std::move(label),
+                                                std::move(filter));
+}
+
+struct ProjectNode : ExecNode {
+  ProjectNode(ExecNode* input, std::string label, std::vector<Expression> exprs)
+      : ExecNode(input->plan(), std::move(label), {input}, {"target"},
+                 /*output_descr=*/{input->output_descr()},
+                 /*num_outputs=*/1),
+        exprs_(std::move(exprs)) {}
+
+  const char* kind_name() override { return "ProjectNode"; }
+
+  Result<ExecBatch> DoProject(const ExecBatch& target) {
+    // XXX get a non-default exec context
+    std::vector<Datum> values{exprs_.size()};
+    for (size_t i = 0; i < exprs_.size(); ++i) {
+      ARROW_ASSIGN_OR_RAISE(Expression simplified_expr,
+                            SimplifyWithGuarantee(exprs_[i], target.guarantee));
+
+      ARROW_ASSIGN_OR_RAISE(values[i], ExecuteScalarExpression(simplified_expr, target));
+    }
+    return ExecBatch::Make(std::move(values));
+  }
+
+  void InputReceived(ExecNode* input, int seq, ExecBatch batch) override {
+    DCHECK_EQ(input, inputs_[0]);
+
+    auto maybe_projected = DoProject(std::move(batch));
+    if (!maybe_projected.ok()) {
+      outputs_[0]->ErrorReceived(this, maybe_projected.status());
+      inputs_[0]->StopProducing(this);
+      return;
+    }
+
+    maybe_projected->guarantee = batch.guarantee;
+    outputs_[0]->InputReceived(this, seq, maybe_projected.MoveValueUnsafe());
+  }
+
+  void ErrorReceived(ExecNode* input, Status error) override {
+    DCHECK_EQ(input, inputs_[0]);
+    outputs_[0]->ErrorReceived(this, std::move(error));
+    inputs_[0]->StopProducing(this);
+  }
+
+  void InputFinished(ExecNode* input, int seq) override {
+    DCHECK_EQ(input, inputs_[0]);
+    outputs_[0]->InputFinished(this, seq);
+    inputs_[0]->StopProducing(this);
+  }
+
+  Status StartProducing() override {
+    // XXX validate inputs_[0]->output_descr() against filter_
+    return Status::OK();
+  }
+
+  void PauseProducing(ExecNode* output) override {}
+
+  void ResumeProducing(ExecNode* output) override {}
+
+  void StopProducing(ExecNode* output) override {
+    DCHECK_EQ(output, outputs_[0]);
+    inputs_[0]->StopProducing(this);
+  }
+
+  void StopProducing() override { StopProducing(outputs_[0]); }
+
+ private:
+  std::vector<Expression> exprs_;
+};
+
+ExecNode* MakeProjectNode(ExecNode* input, std::string label,
+                          std::vector<Expression> exprs) {
+  return input->plan()->EmplaceNode<ProjectNode>(input, std::move(label),
+                                                 std::move(exprs));
+}
+
+struct SinkNode : ExecNode {
+  SinkNode(ExecNode* input, std::string label,
+           AsyncGenerator<util::optional<ExecBatch>>* generator)
+      : ExecNode(input->plan(), std::move(label), {input}, {"collected"}, {},
+                 /*num_outputs=*/0),
+        producer_(MakeProducer(generator)) {}
+
+  static PushGenerator<util::optional<ExecBatch>>::Producer MakeProducer(
+      AsyncGenerator<util::optional<ExecBatch>>* out_gen) {
+    PushGenerator<util::optional<ExecBatch>> gen;
+    auto out = gen.producer();
+    *out_gen = std::move(gen);
+    return out;
+  }
+
+  const char* kind_name() override { return "SinkNode"; }
+
+  Status StartProducing() override { return Status::OK(); }
+
+  // sink nodes have no outputs from which to feel backpressure
+  static void NoOutputs() { DCHECK(false) << "no outputs; this should never be called"; }
+  void ResumeProducing(ExecNode* output) override { NoOutputs(); }
+  void PauseProducing(ExecNode* output) override { NoOutputs(); }
+  void StopProducing(ExecNode* output) override { NoOutputs(); }
+
+  void StopProducing() override {
+    std::unique_lock<std::mutex> lock(mutex_);
+    StopProducingUnlocked();
+  }
+
+  void InputReceived(ExecNode* input, int seq_num, ExecBatch exec_batch) override {
+    std::unique_lock<std::mutex> lock(mutex_);
+    if (stopped_) return;
+
+    // TODO would be nice to factor this out in a ReorderQueue
+    if (seq_num <= static_cast<int>(received_batches_.size())) {
+      received_batches_.resize(seq_num + 1);
+      emitted_.resize(seq_num + 1, false);
+    }
+    received_batches_[seq_num] = std::move(exec_batch);
+    ++num_received_;
+
+    if (seq_num != num_emitted_) {
+      // Cannot emit yet as there is a hole at `num_emitted_`
+      DCHECK_GT(seq_num, num_emitted_);
+      return;
+    }
+
+    if (num_received_ == emit_stop_) {
+      StopProducingUnlocked();
+    }
+
+    // Emit batches in order as far as possible
+    // First collect these batches, then unlock before producing.
+    const auto seq_start = seq_num;
+    while (seq_num < static_cast<int>(emitted_.size()) && !emitted_[seq_num]) {
+      emitted_[seq_num] = true;
+      ++seq_num;
+    }
+    DCHECK_GT(seq_num, seq_start);
+    // By moving the values now, we make sure another thread won't emit the same values
+    // below
+    std::vector<ExecBatch> to_emit(
+        std::make_move_iterator(received_batches_.begin() + seq_start),
+        std::make_move_iterator(received_batches_.begin() + seq_num));
+
+    lock.unlock();
+    for (auto&& batch : to_emit) {
+      producer_.Push(std::move(batch));
+    }
+    lock.lock();
+
+    DCHECK_EQ(seq_start, num_emitted_);  // num_emitted_ wasn't bumped in the meantime
+    num_emitted_ = seq_num;

Review comment:
       Ah, I see, you are saying that @lidavidm 's original concern is invalid.  I misread your comment, my mistake.  I agree that it wouldn't be an error as David described.  However, I still think there could be a problem at the end with the current code.  A thread could arrive to deliver the final item (let's say seq_num=100), see that `seq_num != num_emitted_` and return early.  Then the first thread finishes delivering the item with seq_num=99 and sets `num_emitted_` to 100.  At this point ~~`StopProducing`~~/`InputFinished` will have already been called, no one will call `InputReceived` anymore, and the last item will be lost.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] westonpace commented on a change in pull request #10397: ARROW-11930: [C++][Dataset][Compute] Use an ExecPlan for dataset scans

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #10397:
URL: https://github.com/apache/arrow/pull/10397#discussion_r644158198



##########
File path: cpp/src/arrow/compute/exec/exec_plan.cc
##########
@@ -170,48 +165,409 @@ Status ExecPlan::Validate() { return ToDerived(this)->Validate(); }
 
 Status ExecPlan::StartProducing() { return ToDerived(this)->StartProducing(); }
 
-ExecNode::ExecNode(ExecPlan* plan, std::string label,
-                   std::vector<BatchDescr> input_descrs,
+ExecNode::ExecNode(ExecPlan* plan, std::string label, NodeVector inputs,
                    std::vector<std::string> input_labels, BatchDescr output_descr,
                    int num_outputs)
     : plan_(plan),
       label_(std::move(label)),
-      input_descrs_(std::move(input_descrs)),
+      inputs_(std::move(inputs)),
       input_labels_(std::move(input_labels)),
       output_descr_(std::move(output_descr)),
-      num_outputs_(num_outputs) {}
+      num_outputs_(num_outputs) {
+  for (auto input : inputs_) {
+    input->outputs_.push_back(this);
+  }
+}
 
 Status ExecNode::Validate() const {
-  if (inputs_.size() != input_descrs_.size()) {
+  if (inputs_.size() != input_labels_.size()) {
     return Status::Invalid("Invalid number of inputs for '", label(), "' (expected ",
-                           num_inputs(), ", actual ", inputs_.size(), ")");
+                           num_inputs(), ", actual ", input_labels_.size(), ")");
   }
 
   if (static_cast<int>(outputs_.size()) != num_outputs_) {
     return Status::Invalid("Invalid number of outputs for '", label(), "' (expected ",
                            num_outputs(), ", actual ", outputs_.size(), ")");
   }
 
-  DCHECK_EQ(input_descrs_.size(), input_labels_.size());
-
   for (auto out : outputs_) {
     auto input_index = GetNodeIndex(out->inputs(), this);
     if (!input_index) {
       return Status::Invalid("Node '", label(), "' outputs to node '", out->label(),
                              "' but is not listed as an input.");
     }
+  }
 
-    const auto& in_descr = out->input_descrs_[*input_index];
-    if (in_descr != output_descr_) {
-      return Status::Invalid(
-          "Node '", label(), "' (bound to input ", input_labels_[*input_index],
-          ") produces batches with type '", ValueDescr::ToString(output_descr_),
-          "' inconsistent with consumer '", out->label(), "' which accepts '",
-          ValueDescr::ToString(in_descr), "'");
+  return Status::OK();
+}
+
+struct SourceNode : ExecNode {
+  SourceNode(ExecPlan* plan, std::string label, ExecNode::BatchDescr output_descr,
+             AsyncGenerator<util::optional<ExecBatch>> generator)
+      : ExecNode(plan, std::move(label), {}, {}, std::move(output_descr),
+                 /*num_outputs=*/1),
+        generator_(std::move(generator)) {}
+
+  const char* kind_name() override { return "SourceNode"; }
+
+  static void NoInputs() { DCHECK(false) << "no inputs; this should never be called"; }
+  void InputReceived(ExecNode*, int, ExecBatch) override { NoInputs(); }
+  void ErrorReceived(ExecNode*, Status) override { NoInputs(); }
+  void InputFinished(ExecNode*, int) override { NoInputs(); }
+
+  Status StartProducing() override {
+    if (finished_) {
+      return Status::Invalid("Restarted SourceNode '", label(), "'");
     }
+
+    auto gen = std::move(generator_);
+
+    /// XXX should we wait on this future anywhere? In StopProducing() maybe?
+    auto done_fut =
+        Loop([gen, this] {
+          std::unique_lock<std::mutex> lock(mutex_);
+          int seq = next_batch_index_++;
+          if (finished_) {
+            return Future<ControlFlow<int>>::MakeFinished(Break(seq));
+          }
+          lock.unlock();
+
+          return gen().Then(
+              [=](const util::optional<ExecBatch>& batch) -> ControlFlow<int> {
+                std::unique_lock<std::mutex> lock(mutex_);
+                if (!batch || finished_) {
+                  finished_ = true;
+                  return Break(seq);
+                }
+                lock.unlock();
+
+                outputs_[0]->InputReceived(this, seq, *batch);
+                return Continue();
+              },
+              [=](const Status& error) -> ControlFlow<int> {
+                std::unique_lock<std::mutex> lock(mutex_);
+                if (!finished_) {
+                  finished_ = true;
+                  lock.unlock();
+                  // unless we were already finished, push the error to our output
+                  // XXX is this correct? Is it reasonable for a consumer to ignore errors
+                  // from a finished producer?
+                  outputs_[0]->ErrorReceived(this, error);
+                }
+                return Break(seq);
+              });
+        }).Then([&](int seq) {
+          /// XXX this is probably redundant: do we always call InputFinished after
+          /// ErrorReceived or will ErrorRecieved be sufficient?
+          outputs_[0]->InputFinished(this, seq);
+        });
+
+    return Status::OK();
   }
 
-  return Status::OK();
+  void PauseProducing(ExecNode* output) override {}
+
+  void ResumeProducing(ExecNode* output) override {}
+
+  void StopProducing(ExecNode* output) override {
+    DCHECK_EQ(output, outputs_[0]);
+    std::unique_lock<std::mutex> lock(mutex_);
+    finished_ = true;
+  }
+
+  void StopProducing() override { StopProducing(outputs_[0]); }
+
+ private:
+  std::mutex mutex_;
+  bool finished_{false};
+  int next_batch_index_{0};
+  AsyncGenerator<util::optional<ExecBatch>> generator_;
+};
+
+ExecNode* MakeSourceNode(ExecPlan* plan, std::string label,
+                         ExecNode::BatchDescr output_descr,
+                         AsyncGenerator<util::optional<ExecBatch>> generator) {
+  return plan->EmplaceNode<SourceNode>(plan, std::move(label), std::move(output_descr),
+                                       std::move(generator));
+}
+
+struct FilterNode : ExecNode {
+  FilterNode(ExecNode* input, std::string label, Expression filter)
+      : ExecNode(input->plan(), std::move(label), {input}, {"target"},
+                 /*output_descr=*/{input->output_descr()},
+                 /*num_outputs=*/1),
+        filter_(std::move(filter)) {}
+
+  const char* kind_name() override { return "FilterNode"; }
+
+  Result<ExecBatch> DoFilter(const ExecBatch& target) {
+    ARROW_ASSIGN_OR_RAISE(Expression simplified_filter,
+                          SimplifyWithGuarantee(filter_, target.guarantee));
+
+    // XXX get a non-default exec context
+    ARROW_ASSIGN_OR_RAISE(Datum mask, ExecuteScalarExpression(simplified_filter, target));
+
+    if (mask.is_scalar()) {
+      const auto& mask_scalar = mask.scalar_as<BooleanScalar>();
+      if (mask_scalar.is_valid && mask_scalar.value) {
+        return target;
+      }
+
+      return target.Slice(0, 0);
+    }
+
+    auto values = target.values;
+    for (auto& value : values) {
+      if (value.is_scalar()) continue;
+      ARROW_ASSIGN_OR_RAISE(value, Filter(value, mask, FilterOptions::Defaults()));
+    }
+    return ExecBatch::Make(std::move(values));
+  }
+
+  void InputReceived(ExecNode* input, int seq, ExecBatch batch) override {
+    DCHECK_EQ(input, inputs_[0]);
+
+    auto maybe_filtered = DoFilter(std::move(batch));
+    if (!maybe_filtered.ok()) {
+      outputs_[0]->ErrorReceived(this, maybe_filtered.status());
+      inputs_[0]->StopProducing(this);
+      return;
+    }
+
+    maybe_filtered->guarantee = batch.guarantee;
+    outputs_[0]->InputReceived(this, seq, maybe_filtered.MoveValueUnsafe());
+  }
+
+  void ErrorReceived(ExecNode* input, Status error) override {
+    DCHECK_EQ(input, inputs_[0]);
+    outputs_[0]->ErrorReceived(this, std::move(error));
+    inputs_[0]->StopProducing(this);
+  }
+
+  void InputFinished(ExecNode* input, int seq) override {
+    DCHECK_EQ(input, inputs_[0]);
+    outputs_[0]->InputFinished(this, seq);
+    inputs_[0]->StopProducing(this);
+  }
+
+  Status StartProducing() override {
+    // XXX validate inputs_[0]->output_descr() against filter_
+    return Status::OK();
+  }
+
+  void PauseProducing(ExecNode* output) override {}
+
+  void ResumeProducing(ExecNode* output) override {}
+
+  void StopProducing(ExecNode* output) override {
+    DCHECK_EQ(output, outputs_[0]);
+    inputs_[0]->StopProducing(this);
+  }
+
+  void StopProducing() override { StopProducing(outputs_[0]); }
+
+ private:
+  Expression filter_;
+};
+
+ExecNode* MakeFilterNode(ExecNode* input, std::string label, Expression filter) {
+  return input->plan()->EmplaceNode<FilterNode>(input, std::move(label),
+                                                std::move(filter));
+}
+
+struct ProjectNode : ExecNode {
+  ProjectNode(ExecNode* input, std::string label, std::vector<Expression> exprs)
+      : ExecNode(input->plan(), std::move(label), {input}, {"target"},
+                 /*output_descr=*/{input->output_descr()},
+                 /*num_outputs=*/1),
+        exprs_(std::move(exprs)) {}
+
+  const char* kind_name() override { return "ProjectNode"; }
+
+  Result<ExecBatch> DoProject(const ExecBatch& target) {
+    // XXX get a non-default exec context
+    std::vector<Datum> values{exprs_.size()};
+    for (size_t i = 0; i < exprs_.size(); ++i) {
+      ARROW_ASSIGN_OR_RAISE(Expression simplified_expr,
+                            SimplifyWithGuarantee(exprs_[i], target.guarantee));
+
+      ARROW_ASSIGN_OR_RAISE(values[i], ExecuteScalarExpression(simplified_expr, target));
+    }
+    return ExecBatch::Make(std::move(values));
+  }
+
+  void InputReceived(ExecNode* input, int seq, ExecBatch batch) override {
+    DCHECK_EQ(input, inputs_[0]);
+
+    auto maybe_projected = DoProject(std::move(batch));
+    if (!maybe_projected.ok()) {
+      outputs_[0]->ErrorReceived(this, maybe_projected.status());
+      inputs_[0]->StopProducing(this);
+      return;
+    }
+
+    maybe_projected->guarantee = batch.guarantee;
+    outputs_[0]->InputReceived(this, seq, maybe_projected.MoveValueUnsafe());
+  }
+
+  void ErrorReceived(ExecNode* input, Status error) override {
+    DCHECK_EQ(input, inputs_[0]);
+    outputs_[0]->ErrorReceived(this, std::move(error));
+    inputs_[0]->StopProducing(this);
+  }
+
+  void InputFinished(ExecNode* input, int seq) override {
+    DCHECK_EQ(input, inputs_[0]);
+    outputs_[0]->InputFinished(this, seq);
+    inputs_[0]->StopProducing(this);
+  }
+
+  Status StartProducing() override {
+    // XXX validate inputs_[0]->output_descr() against filter_
+    return Status::OK();
+  }
+
+  void PauseProducing(ExecNode* output) override {}
+
+  void ResumeProducing(ExecNode* output) override {}
+
+  void StopProducing(ExecNode* output) override {
+    DCHECK_EQ(output, outputs_[0]);
+    inputs_[0]->StopProducing(this);
+  }
+
+  void StopProducing() override { StopProducing(outputs_[0]); }
+
+ private:
+  std::vector<Expression> exprs_;
+};
+
+ExecNode* MakeProjectNode(ExecNode* input, std::string label,
+                          std::vector<Expression> exprs) {
+  return input->plan()->EmplaceNode<ProjectNode>(input, std::move(label),
+                                                 std::move(exprs));
+}
+
+struct SinkNode : ExecNode {
+  SinkNode(ExecNode* input, std::string label,
+           AsyncGenerator<util::optional<ExecBatch>>* generator)
+      : ExecNode(input->plan(), std::move(label), {input}, {"collected"}, {},
+                 /*num_outputs=*/0),
+        producer_(MakeProducer(generator)) {}
+
+  static PushGenerator<util::optional<ExecBatch>>::Producer MakeProducer(
+      AsyncGenerator<util::optional<ExecBatch>>* out_gen) {
+    PushGenerator<util::optional<ExecBatch>> gen;
+    auto out = gen.producer();
+    *out_gen = std::move(gen);
+    return out;
+  }
+
+  const char* kind_name() override { return "SinkNode"; }
+
+  Status StartProducing() override { return Status::OK(); }
+
+  // sink nodes have no outputs from which to feel backpressure
+  static void NoOutputs() { DCHECK(false) << "no outputs; this should never be called"; }
+  void ResumeProducing(ExecNode* output) override { NoOutputs(); }
+  void PauseProducing(ExecNode* output) override { NoOutputs(); }
+  void StopProducing(ExecNode* output) override { NoOutputs(); }
+
+  void StopProducing() override {
+    std::unique_lock<std::mutex> lock(mutex_);
+    StopProducingUnlocked();
+  }
+
+  void InputReceived(ExecNode* input, int seq_num, ExecBatch exec_batch) override {
+    std::unique_lock<std::mutex> lock(mutex_);
+    if (stopped_) return;
+
+    // TODO would be nice to factor this out in a ReorderQueue
+    if (seq_num <= static_cast<int>(received_batches_.size())) {
+      received_batches_.resize(seq_num + 1);
+      emitted_.resize(seq_num + 1, false);
+    }
+    received_batches_[seq_num] = std::move(exec_batch);
+    ++num_received_;
+
+    if (seq_num != num_emitted_) {
+      // Cannot emit yet as there is a hole at `num_emitted_`
+      DCHECK_GT(seq_num, num_emitted_);
+      return;
+    }
+
+    if (num_received_ == emit_stop_) {
+      StopProducingUnlocked();
+    }
+
+    // Emit batches in order as far as possible
+    // First collect these batches, then unlock before producing.
+    const auto seq_start = seq_num;
+    while (seq_num < static_cast<int>(emitted_.size()) && !emitted_[seq_num]) {
+      emitted_[seq_num] = true;
+      ++seq_num;
+    }
+    DCHECK_GT(seq_num, seq_start);
+    // By moving the values now, we make sure another thread won't emit the same values
+    // below
+    std::vector<ExecBatch> to_emit(
+        std::make_move_iterator(received_batches_.begin() + seq_start),
+        std::make_move_iterator(received_batches_.begin() + seq_num));
+
+    lock.unlock();
+    for (auto&& batch : to_emit) {
+      producer_.Push(std::move(batch));
+    }
+    lock.lock();
+
+    DCHECK_EQ(seq_start, num_emitted_);  // num_emitted_ wasn't bumped in the meantime
+    num_emitted_ = seq_num;

Review comment:
       Ah, I see, you are saying that @lidavidm 's original concern is invalid.  I misread your comment, my mistake.  I agree that it wouldn't be an error as David described.  However, I still think there could be a problem at the end with the current code.  A thread could arrive to deliver the final item (let's say seq_num=100), see that `seq_num != num_emitted_` and return early.  Then the first thread finishes delivering the item with seq_num=99 and sets `num_emitted_` to 100.  At this point `StopProducing` will have already been called, no one will call `InputReceived` anymore, and the last item will be lost.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] bkietz commented on a change in pull request #10397: ARROW-11930: [C++][Dataset][Compute] Use an ExecPlan for dataset scans

Posted by GitBox <gi...@apache.org>.
bkietz commented on a change in pull request #10397:
URL: https://github.com/apache/arrow/pull/10397#discussion_r644291736



##########
File path: cpp/src/arrow/dataset/scanner.h
##########
@@ -25,6 +25,7 @@
 #include <utility>
 #include <vector>
 
+#include "arrow/compute/exec/exec_plan.h"

Review comment:
       I'll ensure we can use a `type_fwd.h` here instead




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] westonpace commented on a change in pull request #10397: ARROW-11930: [C++][Dataset][Compute] Use an ExecPlan for dataset scans

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #10397:
URL: https://github.com/apache/arrow/pull/10397#discussion_r644167250



##########
File path: cpp/src/arrow/compute/exec/exec_plan.cc
##########
@@ -170,48 +165,409 @@ Status ExecPlan::Validate() { return ToDerived(this)->Validate(); }
 
 Status ExecPlan::StartProducing() { return ToDerived(this)->StartProducing(); }
 
-ExecNode::ExecNode(ExecPlan* plan, std::string label,
-                   std::vector<BatchDescr> input_descrs,
+ExecNode::ExecNode(ExecPlan* plan, std::string label, NodeVector inputs,
                    std::vector<std::string> input_labels, BatchDescr output_descr,
                    int num_outputs)
     : plan_(plan),
       label_(std::move(label)),
-      input_descrs_(std::move(input_descrs)),
+      inputs_(std::move(inputs)),
       input_labels_(std::move(input_labels)),
       output_descr_(std::move(output_descr)),
-      num_outputs_(num_outputs) {}
+      num_outputs_(num_outputs) {
+  for (auto input : inputs_) {
+    input->outputs_.push_back(this);
+  }
+}
 
 Status ExecNode::Validate() const {
-  if (inputs_.size() != input_descrs_.size()) {
+  if (inputs_.size() != input_labels_.size()) {
     return Status::Invalid("Invalid number of inputs for '", label(), "' (expected ",
-                           num_inputs(), ", actual ", inputs_.size(), ")");
+                           num_inputs(), ", actual ", input_labels_.size(), ")");
   }
 
   if (static_cast<int>(outputs_.size()) != num_outputs_) {
     return Status::Invalid("Invalid number of outputs for '", label(), "' (expected ",
                            num_outputs(), ", actual ", outputs_.size(), ")");
   }
 
-  DCHECK_EQ(input_descrs_.size(), input_labels_.size());
-
   for (auto out : outputs_) {
     auto input_index = GetNodeIndex(out->inputs(), this);
     if (!input_index) {
       return Status::Invalid("Node '", label(), "' outputs to node '", out->label(),
                              "' but is not listed as an input.");
     }
+  }
 
-    const auto& in_descr = out->input_descrs_[*input_index];
-    if (in_descr != output_descr_) {
-      return Status::Invalid(
-          "Node '", label(), "' (bound to input ", input_labels_[*input_index],
-          ") produces batches with type '", ValueDescr::ToString(output_descr_),
-          "' inconsistent with consumer '", out->label(), "' which accepts '",
-          ValueDescr::ToString(in_descr), "'");
+  return Status::OK();
+}
+
+struct SourceNode : ExecNode {
+  SourceNode(ExecPlan* plan, std::string label, ExecNode::BatchDescr output_descr,
+             AsyncGenerator<util::optional<ExecBatch>> generator)
+      : ExecNode(plan, std::move(label), {}, {}, std::move(output_descr),
+                 /*num_outputs=*/1),
+        generator_(std::move(generator)) {}
+
+  const char* kind_name() override { return "SourceNode"; }
+
+  static void NoInputs() { DCHECK(false) << "no inputs; this should never be called"; }
+  void InputReceived(ExecNode*, int, ExecBatch) override { NoInputs(); }
+  void ErrorReceived(ExecNode*, Status) override { NoInputs(); }
+  void InputFinished(ExecNode*, int) override { NoInputs(); }
+
+  Status StartProducing() override {
+    if (finished_) {
+      return Status::Invalid("Restarted SourceNode '", label(), "'");
     }
+
+    auto gen = std::move(generator_);
+
+    /// XXX should we wait on this future anywhere? In StopProducing() maybe?
+    auto done_fut =
+        Loop([gen, this] {
+          std::unique_lock<std::mutex> lock(mutex_);
+          int seq = next_batch_index_++;
+          if (finished_) {
+            return Future<ControlFlow<int>>::MakeFinished(Break(seq));
+          }
+          lock.unlock();
+
+          return gen().Then(
+              [=](const util::optional<ExecBatch>& batch) -> ControlFlow<int> {
+                std::unique_lock<std::mutex> lock(mutex_);
+                if (!batch || finished_) {
+                  finished_ = true;
+                  return Break(seq);
+                }
+                lock.unlock();
+
+                outputs_[0]->InputReceived(this, seq, *batch);
+                return Continue();
+              },
+              [=](const Status& error) -> ControlFlow<int> {
+                std::unique_lock<std::mutex> lock(mutex_);
+                if (!finished_) {
+                  finished_ = true;
+                  lock.unlock();
+                  // unless we were already finished, push the error to our output
+                  // XXX is this correct? Is it reasonable for a consumer to ignore errors
+                  // from a finished producer?
+                  outputs_[0]->ErrorReceived(this, error);
+                }
+                return Break(seq);
+              });
+        }).Then([&](int seq) {
+          /// XXX this is probably redundant: do we always call InputFinished after
+          /// ErrorReceived or will ErrorRecieved be sufficient?
+          outputs_[0]->InputFinished(this, seq);
+        });
+
+    return Status::OK();
   }
 
-  return Status::OK();
+  void PauseProducing(ExecNode* output) override {}
+
+  void ResumeProducing(ExecNode* output) override {}
+
+  void StopProducing(ExecNode* output) override {
+    DCHECK_EQ(output, outputs_[0]);
+    std::unique_lock<std::mutex> lock(mutex_);
+    finished_ = true;
+  }
+
+  void StopProducing() override { StopProducing(outputs_[0]); }
+
+ private:
+  std::mutex mutex_;
+  bool finished_{false};
+  int next_batch_index_{0};
+  AsyncGenerator<util::optional<ExecBatch>> generator_;
+};
+
+ExecNode* MakeSourceNode(ExecPlan* plan, std::string label,
+                         ExecNode::BatchDescr output_descr,
+                         AsyncGenerator<util::optional<ExecBatch>> generator) {
+  return plan->EmplaceNode<SourceNode>(plan, std::move(label), std::move(output_descr),
+                                       std::move(generator));
+}
+
+struct FilterNode : ExecNode {
+  FilterNode(ExecNode* input, std::string label, Expression filter)
+      : ExecNode(input->plan(), std::move(label), {input}, {"target"},
+                 /*output_descr=*/{input->output_descr()},
+                 /*num_outputs=*/1),
+        filter_(std::move(filter)) {}
+
+  const char* kind_name() override { return "FilterNode"; }
+
+  Result<ExecBatch> DoFilter(const ExecBatch& target) {
+    ARROW_ASSIGN_OR_RAISE(Expression simplified_filter,
+                          SimplifyWithGuarantee(filter_, target.guarantee));
+
+    // XXX get a non-default exec context
+    ARROW_ASSIGN_OR_RAISE(Datum mask, ExecuteScalarExpression(simplified_filter, target));
+
+    if (mask.is_scalar()) {
+      const auto& mask_scalar = mask.scalar_as<BooleanScalar>();
+      if (mask_scalar.is_valid && mask_scalar.value) {
+        return target;
+      }
+
+      return target.Slice(0, 0);
+    }
+
+    auto values = target.values;
+    for (auto& value : values) {
+      if (value.is_scalar()) continue;
+      ARROW_ASSIGN_OR_RAISE(value, Filter(value, mask, FilterOptions::Defaults()));
+    }
+    return ExecBatch::Make(std::move(values));
+  }
+
+  void InputReceived(ExecNode* input, int seq, ExecBatch batch) override {
+    DCHECK_EQ(input, inputs_[0]);
+
+    auto maybe_filtered = DoFilter(std::move(batch));
+    if (!maybe_filtered.ok()) {
+      outputs_[0]->ErrorReceived(this, maybe_filtered.status());
+      inputs_[0]->StopProducing(this);
+      return;
+    }
+
+    maybe_filtered->guarantee = batch.guarantee;
+    outputs_[0]->InputReceived(this, seq, maybe_filtered.MoveValueUnsafe());
+  }
+
+  void ErrorReceived(ExecNode* input, Status error) override {
+    DCHECK_EQ(input, inputs_[0]);
+    outputs_[0]->ErrorReceived(this, std::move(error));
+    inputs_[0]->StopProducing(this);
+  }
+
+  void InputFinished(ExecNode* input, int seq) override {
+    DCHECK_EQ(input, inputs_[0]);
+    outputs_[0]->InputFinished(this, seq);
+    inputs_[0]->StopProducing(this);
+  }
+
+  Status StartProducing() override {
+    // XXX validate inputs_[0]->output_descr() against filter_
+    return Status::OK();
+  }
+
+  void PauseProducing(ExecNode* output) override {}
+
+  void ResumeProducing(ExecNode* output) override {}
+
+  void StopProducing(ExecNode* output) override {
+    DCHECK_EQ(output, outputs_[0]);
+    inputs_[0]->StopProducing(this);
+  }
+
+  void StopProducing() override { StopProducing(outputs_[0]); }
+
+ private:
+  Expression filter_;
+};
+
+ExecNode* MakeFilterNode(ExecNode* input, std::string label, Expression filter) {
+  return input->plan()->EmplaceNode<FilterNode>(input, std::move(label),
+                                                std::move(filter));
+}
+
+struct ProjectNode : ExecNode {
+  ProjectNode(ExecNode* input, std::string label, std::vector<Expression> exprs)
+      : ExecNode(input->plan(), std::move(label), {input}, {"target"},
+                 /*output_descr=*/{input->output_descr()},
+                 /*num_outputs=*/1),
+        exprs_(std::move(exprs)) {}
+
+  const char* kind_name() override { return "ProjectNode"; }
+
+  Result<ExecBatch> DoProject(const ExecBatch& target) {
+    // XXX get a non-default exec context
+    std::vector<Datum> values{exprs_.size()};
+    for (size_t i = 0; i < exprs_.size(); ++i) {
+      ARROW_ASSIGN_OR_RAISE(Expression simplified_expr,
+                            SimplifyWithGuarantee(exprs_[i], target.guarantee));
+
+      ARROW_ASSIGN_OR_RAISE(values[i], ExecuteScalarExpression(simplified_expr, target));
+    }
+    return ExecBatch::Make(std::move(values));
+  }
+
+  void InputReceived(ExecNode* input, int seq, ExecBatch batch) override {
+    DCHECK_EQ(input, inputs_[0]);
+
+    auto maybe_projected = DoProject(std::move(batch));
+    if (!maybe_projected.ok()) {
+      outputs_[0]->ErrorReceived(this, maybe_projected.status());
+      inputs_[0]->StopProducing(this);
+      return;
+    }
+
+    maybe_projected->guarantee = batch.guarantee;
+    outputs_[0]->InputReceived(this, seq, maybe_projected.MoveValueUnsafe());
+  }
+
+  void ErrorReceived(ExecNode* input, Status error) override {
+    DCHECK_EQ(input, inputs_[0]);
+    outputs_[0]->ErrorReceived(this, std::move(error));
+    inputs_[0]->StopProducing(this);
+  }
+
+  void InputFinished(ExecNode* input, int seq) override {
+    DCHECK_EQ(input, inputs_[0]);
+    outputs_[0]->InputFinished(this, seq);
+    inputs_[0]->StopProducing(this);
+  }
+
+  Status StartProducing() override {
+    // XXX validate inputs_[0]->output_descr() against filter_
+    return Status::OK();
+  }
+
+  void PauseProducing(ExecNode* output) override {}
+
+  void ResumeProducing(ExecNode* output) override {}
+
+  void StopProducing(ExecNode* output) override {
+    DCHECK_EQ(output, outputs_[0]);
+    inputs_[0]->StopProducing(this);
+  }
+
+  void StopProducing() override { StopProducing(outputs_[0]); }
+
+ private:
+  std::vector<Expression> exprs_;
+};
+
+ExecNode* MakeProjectNode(ExecNode* input, std::string label,
+                          std::vector<Expression> exprs) {
+  return input->plan()->EmplaceNode<ProjectNode>(input, std::move(label),
+                                                 std::move(exprs));
+}
+
+struct SinkNode : ExecNode {
+  SinkNode(ExecNode* input, std::string label,
+           AsyncGenerator<util::optional<ExecBatch>>* generator)
+      : ExecNode(input->plan(), std::move(label), {input}, {"collected"}, {},
+                 /*num_outputs=*/0),
+        producer_(MakeProducer(generator)) {}
+
+  static PushGenerator<util::optional<ExecBatch>>::Producer MakeProducer(
+      AsyncGenerator<util::optional<ExecBatch>>* out_gen) {
+    PushGenerator<util::optional<ExecBatch>> gen;
+    auto out = gen.producer();
+    *out_gen = std::move(gen);
+    return out;
+  }
+
+  const char* kind_name() override { return "SinkNode"; }
+
+  Status StartProducing() override { return Status::OK(); }
+
+  // sink nodes have no outputs from which to feel backpressure
+  static void NoOutputs() { DCHECK(false) << "no outputs; this should never be called"; }
+  void ResumeProducing(ExecNode* output) override { NoOutputs(); }
+  void PauseProducing(ExecNode* output) override { NoOutputs(); }
+  void StopProducing(ExecNode* output) override { NoOutputs(); }
+
+  void StopProducing() override {
+    std::unique_lock<std::mutex> lock(mutex_);
+    StopProducingUnlocked();
+  }
+
+  void InputReceived(ExecNode* input, int seq_num, ExecBatch exec_batch) override {

Review comment:
       Is sorting always going to be a property of a sink node?  Can an `ORDER BY` operation ever cause a sort node to appear in the middle of a DAG?  If so, should sorting not be its own node?  Also, since we are starting with the "if no other sort is specified then there is a natural ordering based on the sequence number" assumption I feel we will end up needing to maintain the assumption forever.  Is this something we want?




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] bkietz commented on a change in pull request #10397: ARROW-11930: [C++][Dataset][Compute] Use an ExecPlan for dataset scans

Posted by GitBox <gi...@apache.org>.
bkietz commented on a change in pull request #10397:
URL: https://github.com/apache/arrow/pull/10397#discussion_r660110832



##########
File path: cpp/src/arrow/compute/exec/plan_test.cc
##########
@@ -20,27 +20,46 @@
 #include <functional>
 #include <memory>
 
+#include "arrow/compute/exec.h"
 #include "arrow/compute/exec/exec_plan.h"
+#include "arrow/compute/exec/expression.h"
 #include "arrow/compute/exec/test_util.h"
 #include "arrow/record_batch.h"
 #include "arrow/testing/future_util.h"
 #include "arrow/testing/gtest_util.h"
 #include "arrow/testing/random.h"
 #include "arrow/util/logging.h"
 #include "arrow/util/thread_pool.h"
+#include "arrow/util/vector.h"
 
-namespace arrow {
+using testing::ElementsAre;
+using testing::HasSubstr;
+using testing::UnorderedElementsAreArray;
 
-using internal::Executor;
+namespace arrow {
 
 namespace compute {
 
-void AssertBatchesEqual(const RecordBatchVector& expected,
-                        const RecordBatchVector& actual) {
-  ASSERT_EQ(expected.size(), actual.size());
-  for (size_t i = 0; i < expected.size(); ++i) {
-    AssertBatchesEqual(*expected[i], *actual[i]);
+ExecBatch ExecBatchFromJSON(const std::vector<ValueDescr>& descrs,

Review comment:
       I'll put it in arrow/compute/exec/test_util.h




-- 
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-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] bkietz commented on a change in pull request #10397: ARROW-11930: [C++][Dataset][Compute] Use an ExecPlan for dataset scans

Posted by GitBox <gi...@apache.org>.
bkietz commented on a change in pull request #10397:
URL: https://github.com/apache/arrow/pull/10397#discussion_r643448100



##########
File path: cpp/src/arrow/compute/exec/expression.h
##########
@@ -207,11 +218,22 @@ Result<Expression> SimplifyWithGuarantee(Expression,
 
 // Execution
 
-/// Execute a scalar expression against the provided state and input Datum. This
+/// Ensure that a RecordBatch (which may have missing or incorrectly ordered columns)

Review comment:
       will do, thanks




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] bkietz commented on a change in pull request #10397: ARROW-11930: [C++][Dataset][Compute] Use an ExecPlan for dataset scans

Posted by GitBox <gi...@apache.org>.
bkietz commented on a change in pull request #10397:
URL: https://github.com/apache/arrow/pull/10397#discussion_r640908529



##########
File path: cpp/src/arrow/compute/exec/exec_plan.cc
##########
@@ -170,48 +165,409 @@ Status ExecPlan::Validate() { return ToDerived(this)->Validate(); }
 
 Status ExecPlan::StartProducing() { return ToDerived(this)->StartProducing(); }
 
-ExecNode::ExecNode(ExecPlan* plan, std::string label,
-                   std::vector<BatchDescr> input_descrs,
+ExecNode::ExecNode(ExecPlan* plan, std::string label, NodeVector inputs,
                    std::vector<std::string> input_labels, BatchDescr output_descr,
                    int num_outputs)
     : plan_(plan),
       label_(std::move(label)),
-      input_descrs_(std::move(input_descrs)),
+      inputs_(std::move(inputs)),
       input_labels_(std::move(input_labels)),
       output_descr_(std::move(output_descr)),
-      num_outputs_(num_outputs) {}
+      num_outputs_(num_outputs) {
+  for (auto input : inputs_) {
+    input->outputs_.push_back(this);
+  }
+}
 
 Status ExecNode::Validate() const {
-  if (inputs_.size() != input_descrs_.size()) {
+  if (inputs_.size() != input_labels_.size()) {
     return Status::Invalid("Invalid number of inputs for '", label(), "' (expected ",
-                           num_inputs(), ", actual ", inputs_.size(), ")");
+                           num_inputs(), ", actual ", input_labels_.size(), ")");
   }
 
   if (static_cast<int>(outputs_.size()) != num_outputs_) {
     return Status::Invalid("Invalid number of outputs for '", label(), "' (expected ",
                            num_outputs(), ", actual ", outputs_.size(), ")");
   }
 
-  DCHECK_EQ(input_descrs_.size(), input_labels_.size());
-
   for (auto out : outputs_) {
     auto input_index = GetNodeIndex(out->inputs(), this);
     if (!input_index) {
       return Status::Invalid("Node '", label(), "' outputs to node '", out->label(),
                              "' but is not listed as an input.");
     }
+  }
 
-    const auto& in_descr = out->input_descrs_[*input_index];
-    if (in_descr != output_descr_) {
-      return Status::Invalid(
-          "Node '", label(), "' (bound to input ", input_labels_[*input_index],
-          ") produces batches with type '", ValueDescr::ToString(output_descr_),
-          "' inconsistent with consumer '", out->label(), "' which accepts '",
-          ValueDescr::ToString(in_descr), "'");
+  return Status::OK();
+}
+
+struct SourceNode : ExecNode {
+  SourceNode(ExecPlan* plan, std::string label, ExecNode::BatchDescr output_descr,
+             AsyncGenerator<util::optional<ExecBatch>> generator)
+      : ExecNode(plan, std::move(label), {}, {}, std::move(output_descr),
+                 /*num_outputs=*/1),
+        generator_(std::move(generator)) {}
+
+  const char* kind_name() override { return "SourceNode"; }
+
+  static void NoInputs() { DCHECK(false) << "no inputs; this should never be called"; }
+  void InputReceived(ExecNode*, int, ExecBatch) override { NoInputs(); }
+  void ErrorReceived(ExecNode*, Status) override { NoInputs(); }
+  void InputFinished(ExecNode*, int) override { NoInputs(); }
+
+  Status StartProducing() override {
+    if (finished_) {
+      return Status::Invalid("Restarted SourceNode '", label(), "'");
     }
+
+    auto gen = std::move(generator_);
+
+    /// XXX should we wait on this future anywhere? In StopProducing() maybe?
+    auto done_fut =
+        Loop([gen, this] {
+          std::unique_lock<std::mutex> lock(mutex_);
+          int seq = next_batch_index_++;
+          if (finished_) {
+            return Future<ControlFlow<int>>::MakeFinished(Break(seq));
+          }
+          lock.unlock();
+
+          return gen().Then(
+              [=](const util::optional<ExecBatch>& batch) -> ControlFlow<int> {
+                std::unique_lock<std::mutex> lock(mutex_);
+                if (!batch || finished_) {
+                  finished_ = true;
+                  return Break(seq);
+                }
+                lock.unlock();
+
+                outputs_[0]->InputReceived(this, seq, *batch);
+                return Continue();
+              },
+              [=](const Status& error) -> ControlFlow<int> {
+                std::unique_lock<std::mutex> lock(mutex_);
+                if (!finished_) {
+                  finished_ = true;
+                  lock.unlock();
+                  // unless we were already finished, push the error to our output
+                  // XXX is this correct? Is it reasonable for a consumer to ignore errors
+                  // from a finished producer?
+                  outputs_[0]->ErrorReceived(this, error);
+                }
+                return Break(seq);
+              });
+        }).Then([&](int seq) {
+          /// XXX this is probably redundant: do we always call InputFinished after
+          /// ErrorReceived or will ErrorRecieved be sufficient?
+          outputs_[0]->InputFinished(this, seq);
+        });
+
+    return Status::OK();
   }
 
-  return Status::OK();
+  void PauseProducing(ExecNode* output) override {}
+
+  void ResumeProducing(ExecNode* output) override {}
+
+  void StopProducing(ExecNode* output) override {
+    DCHECK_EQ(output, outputs_[0]);
+    std::unique_lock<std::mutex> lock(mutex_);
+    finished_ = true;
+  }
+
+  void StopProducing() override { StopProducing(outputs_[0]); }
+
+ private:
+  std::mutex mutex_;
+  bool finished_{false};
+  int next_batch_index_{0};
+  AsyncGenerator<util::optional<ExecBatch>> generator_;
+};
+
+ExecNode* MakeSourceNode(ExecPlan* plan, std::string label,
+                         ExecNode::BatchDescr output_descr,
+                         AsyncGenerator<util::optional<ExecBatch>> generator) {
+  return plan->EmplaceNode<SourceNode>(plan, std::move(label), std::move(output_descr),
+                                       std::move(generator));
+}
+
+struct FilterNode : ExecNode {
+  FilterNode(ExecNode* input, std::string label, Expression filter)
+      : ExecNode(input->plan(), std::move(label), {input}, {"target"},
+                 /*output_descr=*/{input->output_descr()},
+                 /*num_outputs=*/1),
+        filter_(std::move(filter)) {}
+
+  const char* kind_name() override { return "FilterNode"; }
+
+  Result<ExecBatch> DoFilter(const ExecBatch& target) {
+    ARROW_ASSIGN_OR_RAISE(Expression simplified_filter,
+                          SimplifyWithGuarantee(filter_, target.guarantee));
+
+    // XXX get a non-default exec context
+    ARROW_ASSIGN_OR_RAISE(Datum mask, ExecuteScalarExpression(simplified_filter, target));
+
+    if (mask.is_scalar()) {
+      const auto& mask_scalar = mask.scalar_as<BooleanScalar>();
+      if (mask_scalar.is_valid && mask_scalar.value) {
+        return target;
+      }
+
+      return target.Slice(0, 0);
+    }
+
+    auto values = target.values;
+    for (auto& value : values) {
+      if (value.is_scalar()) continue;
+      ARROW_ASSIGN_OR_RAISE(value, Filter(value, mask, FilterOptions::Defaults()));
+    }
+    return ExecBatch::Make(std::move(values));
+  }
+
+  void InputReceived(ExecNode* input, int seq, ExecBatch batch) override {
+    DCHECK_EQ(input, inputs_[0]);
+
+    auto maybe_filtered = DoFilter(std::move(batch));
+    if (!maybe_filtered.ok()) {
+      outputs_[0]->ErrorReceived(this, maybe_filtered.status());
+      inputs_[0]->StopProducing(this);
+      return;
+    }
+
+    maybe_filtered->guarantee = batch.guarantee;
+    outputs_[0]->InputReceived(this, seq, maybe_filtered.MoveValueUnsafe());
+  }
+
+  void ErrorReceived(ExecNode* input, Status error) override {
+    DCHECK_EQ(input, inputs_[0]);
+    outputs_[0]->ErrorReceived(this, std::move(error));
+    inputs_[0]->StopProducing(this);
+  }
+
+  void InputFinished(ExecNode* input, int seq) override {
+    DCHECK_EQ(input, inputs_[0]);
+    outputs_[0]->InputFinished(this, seq);
+    inputs_[0]->StopProducing(this);
+  }
+
+  Status StartProducing() override {
+    // XXX validate inputs_[0]->output_descr() against filter_
+    return Status::OK();
+  }
+
+  void PauseProducing(ExecNode* output) override {}
+
+  void ResumeProducing(ExecNode* output) override {}
+
+  void StopProducing(ExecNode* output) override {
+    DCHECK_EQ(output, outputs_[0]);
+    inputs_[0]->StopProducing(this);
+  }
+
+  void StopProducing() override { StopProducing(outputs_[0]); }
+
+ private:
+  Expression filter_;
+};
+
+ExecNode* MakeFilterNode(ExecNode* input, std::string label, Expression filter) {
+  return input->plan()->EmplaceNode<FilterNode>(input, std::move(label),
+                                                std::move(filter));
+}
+
+struct ProjectNode : ExecNode {
+  ProjectNode(ExecNode* input, std::string label, std::vector<Expression> exprs)
+      : ExecNode(input->plan(), std::move(label), {input}, {"target"},
+                 /*output_descr=*/{input->output_descr()},
+                 /*num_outputs=*/1),
+        exprs_(std::move(exprs)) {}
+
+  const char* kind_name() override { return "ProjectNode"; }
+
+  Result<ExecBatch> DoProject(const ExecBatch& target) {
+    // XXX get a non-default exec context
+    std::vector<Datum> values{exprs_.size()};
+    for (size_t i = 0; i < exprs_.size(); ++i) {
+      ARROW_ASSIGN_OR_RAISE(Expression simplified_expr,
+                            SimplifyWithGuarantee(exprs_[i], target.guarantee));
+
+      ARROW_ASSIGN_OR_RAISE(values[i], ExecuteScalarExpression(simplified_expr, target));
+    }
+    return ExecBatch::Make(std::move(values));
+  }
+
+  void InputReceived(ExecNode* input, int seq, ExecBatch batch) override {
+    DCHECK_EQ(input, inputs_[0]);
+
+    auto maybe_projected = DoProject(std::move(batch));
+    if (!maybe_projected.ok()) {
+      outputs_[0]->ErrorReceived(this, maybe_projected.status());
+      inputs_[0]->StopProducing(this);
+      return;
+    }
+
+    maybe_projected->guarantee = batch.guarantee;
+    outputs_[0]->InputReceived(this, seq, maybe_projected.MoveValueUnsafe());
+  }
+
+  void ErrorReceived(ExecNode* input, Status error) override {
+    DCHECK_EQ(input, inputs_[0]);
+    outputs_[0]->ErrorReceived(this, std::move(error));
+    inputs_[0]->StopProducing(this);
+  }
+
+  void InputFinished(ExecNode* input, int seq) override {
+    DCHECK_EQ(input, inputs_[0]);
+    outputs_[0]->InputFinished(this, seq);
+    inputs_[0]->StopProducing(this);
+  }
+
+  Status StartProducing() override {
+    // XXX validate inputs_[0]->output_descr() against filter_
+    return Status::OK();
+  }
+
+  void PauseProducing(ExecNode* output) override {}
+
+  void ResumeProducing(ExecNode* output) override {}
+
+  void StopProducing(ExecNode* output) override {
+    DCHECK_EQ(output, outputs_[0]);
+    inputs_[0]->StopProducing(this);
+  }
+
+  void StopProducing() override { StopProducing(outputs_[0]); }
+
+ private:
+  std::vector<Expression> exprs_;
+};
+
+ExecNode* MakeProjectNode(ExecNode* input, std::string label,
+                          std::vector<Expression> exprs) {
+  return input->plan()->EmplaceNode<ProjectNode>(input, std::move(label),
+                                                 std::move(exprs));
+}
+
+struct SinkNode : ExecNode {
+  SinkNode(ExecNode* input, std::string label,
+           AsyncGenerator<util::optional<ExecBatch>>* generator)
+      : ExecNode(input->plan(), std::move(label), {input}, {"collected"}, {},
+                 /*num_outputs=*/0),
+        producer_(MakeProducer(generator)) {}
+
+  static PushGenerator<util::optional<ExecBatch>>::Producer MakeProducer(
+      AsyncGenerator<util::optional<ExecBatch>>* out_gen) {
+    PushGenerator<util::optional<ExecBatch>> gen;
+    auto out = gen.producer();
+    *out_gen = std::move(gen);
+    return out;
+  }
+
+  const char* kind_name() override { return "SinkNode"; }
+
+  Status StartProducing() override { return Status::OK(); }
+
+  // sink nodes have no outputs from which to feel backpressure
+  static void NoOutputs() { DCHECK(false) << "no outputs; this should never be called"; }
+  void ResumeProducing(ExecNode* output) override { NoOutputs(); }
+  void PauseProducing(ExecNode* output) override { NoOutputs(); }
+  void StopProducing(ExecNode* output) override { NoOutputs(); }
+
+  void StopProducing() override {
+    std::unique_lock<std::mutex> lock(mutex_);
+    StopProducingUnlocked();
+  }
+
+  void InputReceived(ExecNode* input, int seq_num, ExecBatch exec_batch) override {
+    std::unique_lock<std::mutex> lock(mutex_);
+    if (stopped_) return;
+
+    // TODO would be nice to factor this out in a ReorderQueue
+    if (seq_num <= static_cast<int>(received_batches_.size())) {
+      received_batches_.resize(seq_num + 1);
+      emitted_.resize(seq_num + 1, false);
+    }
+    received_batches_[seq_num] = std::move(exec_batch);
+    ++num_received_;
+
+    if (seq_num != num_emitted_) {
+      // Cannot emit yet as there is a hole at `num_emitted_`
+      DCHECK_GT(seq_num, num_emitted_);
+      return;
+    }
+
+    if (num_received_ == emit_stop_) {
+      StopProducingUnlocked();
+    }
+
+    // Emit batches in order as far as possible
+    // First collect these batches, then unlock before producing.
+    const auto seq_start = seq_num;
+    while (seq_num < static_cast<int>(emitted_.size()) && !emitted_[seq_num]) {
+      emitted_[seq_num] = true;
+      ++seq_num;
+    }
+    DCHECK_GT(seq_num, seq_start);
+    // By moving the values now, we make sure another thread won't emit the same values
+    // below
+    std::vector<ExecBatch> to_emit(
+        std::make_move_iterator(received_batches_.begin() + seq_start),
+        std::make_move_iterator(received_batches_.begin() + seq_num));
+
+    lock.unlock();
+    for (auto&& batch : to_emit) {
+      producer_.Push(std::move(batch));
+    }
+    lock.lock();
+
+    DCHECK_EQ(seq_start, num_emitted_);  // num_emitted_ wasn't bumped in the meantime
+    num_emitted_ = seq_num;

Review comment:
       I'll rewrite this using SequencingGenerator




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] bkietz commented on a change in pull request #10397: ARROW-11930: [C++][Dataset][Compute] Use an ExecPlan for dataset scans

Posted by GitBox <gi...@apache.org>.
bkietz commented on a change in pull request #10397:
URL: https://github.com/apache/arrow/pull/10397#discussion_r660117770



##########
File path: cpp/src/arrow/type.cc
##########
@@ -1195,6 +1195,10 @@ std::string FieldRef::ToString() const {
 }
 
 std::vector<FieldPath> FieldRef::FindAll(const Schema& schema) const {
+  if (auto name = this->name()) {
+    return internal::MapVector([](int i) { return FieldPath{i}; },
+                               schema.GetAllFieldIndices(*name));
+  }

Review comment:
       This is tested https://github.com/bkietz/arrow/blob/1b8bedcbf4b794858228c85644248300b64ce5a4/cpp/src/arrow/type_test.cc#L391-L392




-- 
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-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] bkietz commented on a change in pull request #10397: ARROW-11930: [C++][Dataset][Compute] Use an ExecPlan for dataset scans

Posted by GitBox <gi...@apache.org>.
bkietz commented on a change in pull request #10397:
URL: https://github.com/apache/arrow/pull/10397#discussion_r643578392



##########
File path: cpp/src/arrow/compute/exec/expression.cc
##########
@@ -61,13 +61,22 @@ Expression call(std::string function, std::vector<Expression> arguments,
   call.function_name = std::move(function);
   call.arguments = std::move(arguments);
   call.options = std::move(options);
+
+  call.hash = std::hash<std::string>{}(call.function_name);
+  for (const auto& arg : call.arguments) {
+    call.hash ^= arg.hash();
+  }
   return Expression(std::move(call));
 }
 
 const Datum* Expression::literal() const { return util::get_if<Datum>(impl_.get()); }
 
+const Expression::Parameter* Expression::parameter() const {

Review comment:
       a Parameter is a {field_ref, type, index} (with the last two properties only available after `Bind`). During simplification against guarantees (which are unbound), parameters must be compared by their field_refs




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] bkietz commented on pull request #10397: ARROW-11930: [C++][Dataset][Compute] Use an ExecPlan for dataset scans

Posted by GitBox <gi...@apache.org>.
bkietz commented on pull request #10397:
URL: https://github.com/apache/arrow/pull/10397#issuecomment-865269157


   @pitrou PTAL


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] bkietz commented on a change in pull request #10397: ARROW-11930: [C++][Dataset][Compute] Use an ExecPlan for dataset scans

Posted by GitBox <gi...@apache.org>.
bkietz commented on a change in pull request #10397:
URL: https://github.com/apache/arrow/pull/10397#discussion_r644265881



##########
File path: cpp/src/arrow/util/iterator.h
##########
@@ -66,6 +66,12 @@ bool IsIterationEnd(const T& val) {
   return IterationTraits<T>::IsEnd(val);
 }
 
+template <typename T>
+bool IsIterationEnd(const Result<T>& maybe_val) {

Review comment:
       removed.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] westonpace commented on a change in pull request #10397: ARROW-11930: [C++][Dataset][Compute] Use an ExecPlan for dataset scans

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #10397:
URL: https://github.com/apache/arrow/pull/10397#discussion_r643353334



##########
File path: cpp/src/arrow/compute/exec/exec_plan.h
##########
@@ -225,22 +212,43 @@ class ARROW_EXPORT ExecNode {
   virtual void StopProducing() = 0;
 
  protected:
-  ExecNode(ExecPlan* plan, std::string label, std::vector<BatchDescr> input_descrs,
+  ExecNode(ExecPlan*, std::string label, NodeVector inputs,

Review comment:
       Nit: Seems odd to have no name for `ExecPlan` but then have a name for everything else.

##########
File path: cpp/src/arrow/compute/exec/expression.cc
##########
@@ -61,13 +61,22 @@ Expression call(std::string function, std::vector<Expression> arguments,
   call.function_name = std::move(function);
   call.arguments = std::move(arguments);
   call.options = std::move(options);
+
+  call.hash = std::hash<std::string>{}(call.function_name);
+  for (const auto& arg : call.arguments) {
+    call.hash ^= arg.hash();
+  }
   return Expression(std::move(call));
 }
 
 const Datum* Expression::literal() const { return util::get_if<Datum>(impl_.get()); }
 
+const Expression::Parameter* Expression::parameter() const {

Review comment:
       Checking my knowledge.  A parameter is an index + an expected type?

##########
File path: cpp/src/arrow/compute/exec/expression.cc
##########
@@ -613,6 +639,22 @@ std::vector<FieldRef> FieldsInExpression(const Expression& expr) {
   return fields;
 }
 
+std::vector<int> ParametersInExpression(const Expression& expr) {

Review comment:
       It seems this could return duplicate indices (e.g. in something like `x < 5 && x > 0`).  Is that a problem?

##########
File path: cpp/src/arrow/dataset/dataset_internal.h
##########
@@ -204,5 +204,35 @@ arrow::Result<std::shared_ptr<T>> GetFragmentScanOptions(
   return internal::checked_pointer_cast<T>(source);
 }
 
+class FragmentDataset : public Dataset {

Review comment:
       Should this be a base type of `InMemoryDataset`?

##########
File path: cpp/src/arrow/compute/exec/test_util.cc
##########
@@ -124,277 +130,42 @@ struct DummyNode : ExecNode {
   bool started_ = false;
 };
 
-struct RecordBatchReaderNode : ExecNode {
-  RecordBatchReaderNode(ExecPlan* plan, std::string label,
-                        std::shared_ptr<RecordBatchReader> reader, Executor* io_executor)
-      : ExecNode(plan, std::move(label), {}, {},
-                 DescrFromSchemaColumns(*reader->schema()), /*num_outputs=*/1),
-        schema_(reader->schema()),
-        reader_(std::move(reader)),
-        io_executor_(io_executor) {}
-
-  RecordBatchReaderNode(ExecPlan* plan, std::string label, std::shared_ptr<Schema> schema,
-                        RecordBatchGenerator generator, Executor* io_executor)
-      : ExecNode(plan, std::move(label), {}, {}, DescrFromSchemaColumns(*schema),
-                 /*num_outputs=*/1),
-        schema_(std::move(schema)),
-        generator_(std::move(generator)),
-        io_executor_(io_executor) {}
-
-  const char* kind_name() override { return "RecordBatchReader"; }
-
-  void InputReceived(ExecNode* input, int seq_num, compute::ExecBatch batch) override {}
-
-  void ErrorReceived(ExecNode* input, Status error) override {}
-
-  void InputFinished(ExecNode* input, int seq_stop) override {}
-
-  Status StartProducing() override {
-    next_batch_index_ = 0;
-    if (!generator_) {
-      auto it = MakeIteratorFromReader(reader_);
-      ARROW_ASSIGN_OR_RAISE(generator_,
-                            MakeBackgroundGenerator(std::move(it), io_executor_));
-    }
-    GenerateOne(std::unique_lock<std::mutex>{mutex_});
-    return Status::OK();
-  }
-
-  void PauseProducing(ExecNode* output) override {}
-
-  void ResumeProducing(ExecNode* output) override {}
-
-  void StopProducing(ExecNode* output) override {
-    ASSERT_EQ(output, outputs_[0]);
-    std::unique_lock<std::mutex> lock(mutex_);
-    generator_ = nullptr;  // null function
-  }
-
-  void StopProducing() override { StopProducing(outputs_[0]); }
-
- private:
-  void GenerateOne(std::unique_lock<std::mutex>&& lock) {
-    if (!generator_) {
-      // Stopped
-      return;
-    }
-    auto plan = this->plan()->shared_from_this();
-    auto fut = generator_();
-    const auto batch_index = next_batch_index_++;
-
-    lock.unlock();
-    // TODO we want to transfer always here
-    io_executor_->Transfer(std::move(fut))
-        .AddCallback(
-            [plan, batch_index, this](const Result<std::shared_ptr<RecordBatch>>& res) {
-              std::unique_lock<std::mutex> lock(mutex_);
-              if (!res.ok()) {
-                for (auto out : outputs_) {
-                  out->ErrorReceived(this, res.status());
-                }
-                return;
-              }
-              const auto& batch = *res;
-              if (IsIterationEnd(batch)) {
-                lock.unlock();
-                for (auto out : outputs_) {
-                  out->InputFinished(this, batch_index);
-                }
-              } else {
-                lock.unlock();
-                for (auto out : outputs_) {
-                  out->InputReceived(this, batch_index, compute::ExecBatch(*batch));
-                }
-                lock.lock();
-                GenerateOne(std::move(lock));
-              }
-            });
-  }
-
-  std::mutex mutex_;
-  const std::shared_ptr<Schema> schema_;
-  const std::shared_ptr<RecordBatchReader> reader_;
-  RecordBatchGenerator generator_;
-  int next_batch_index_;
-
-  Executor* const io_executor_;
-};
-
-struct RecordBatchCollectNodeImpl : public RecordBatchCollectNode {
-  RecordBatchCollectNodeImpl(ExecPlan* plan, std::string label,
-                             std::shared_ptr<Schema> schema)
-      : RecordBatchCollectNode(plan, std::move(label), {DescrFromSchemaColumns(*schema)},
-                               {"batches_to_collect"}, {}, 0),
-        schema_(std::move(schema)) {}
-
-  RecordBatchGenerator generator() override { return generator_; }
-
-  const char* kind_name() override { return "RecordBatchReader"; }
-
-  Status StartProducing() override {
-    num_received_ = 0;
-    num_emitted_ = 0;
-    emit_stop_ = -1;
-    stopped_ = false;
-    producer_.emplace(generator_.producer());
-    return Status::OK();
-  }
-
-  // sink nodes have no outputs from which to feel backpressure
-  void ResumeProducing(ExecNode* output) override {
-    FAIL() << "no outputs; this should never be called";
-  }
-  void PauseProducing(ExecNode* output) override {
-    FAIL() << "no outputs; this should never be called";
-  }
-  void StopProducing(ExecNode* output) override {
-    FAIL() << "no outputs; this should never be called";
-  }
-
-  void StopProducing() override {
-    std::unique_lock<std::mutex> lock(mutex_);
-    StopProducingUnlocked();
-  }
-
-  void InputReceived(ExecNode* input, int seq_num,
-                     compute::ExecBatch exec_batch) override {
-    std::unique_lock<std::mutex> lock(mutex_);
-    if (stopped_) {
-      return;
-    }
-    auto maybe_batch = MakeBatch(std::move(exec_batch));
-    if (!maybe_batch.ok()) {
-      lock.unlock();
-      producer_->Push(std::move(maybe_batch));
-      return;
-    }
-
-    // TODO would be nice to factor this out in a ReorderQueue
-    auto batch = *std::move(maybe_batch);
-    if (seq_num <= static_cast<int>(received_batches_.size())) {
-      received_batches_.resize(seq_num + 1, nullptr);
-    }
-    DCHECK_EQ(received_batches_[seq_num], nullptr);
-    received_batches_[seq_num] = std::move(batch);
-    ++num_received_;
-
-    if (seq_num != num_emitted_) {
-      // Cannot emit yet as there is a hole at `num_emitted_`
-      DCHECK_GT(seq_num, num_emitted_);
-      DCHECK_EQ(received_batches_[num_emitted_], nullptr);
-      return;
-    }
-    if (num_received_ == emit_stop_) {
-      StopProducingUnlocked();
-    }
-
-    // Emit batches in order as far as possible
-    // First collect these batches, then unlock before producing.
-    const auto seq_start = seq_num;
-    while (seq_num < static_cast<int>(received_batches_.size()) &&
-           received_batches_[seq_num] != nullptr) {
-      ++seq_num;
-    }
-    DCHECK_GT(seq_num, seq_start);
-    // By moving the values now, we make sure another thread won't emit the same values
-    // below
-    RecordBatchVector to_emit(
-        std::make_move_iterator(received_batches_.begin() + seq_start),
-        std::make_move_iterator(received_batches_.begin() + seq_num));
-
-    lock.unlock();
-    for (auto&& batch : to_emit) {
-      producer_->Push(std::move(batch));
-    }
-    lock.lock();
-
-    DCHECK_EQ(seq_start, num_emitted_);  // num_emitted_ wasn't bumped in the meantime
-    num_emitted_ = seq_num;
-  }
-
-  void ErrorReceived(ExecNode* input, Status error) override {
-    // XXX do we care about properly sequencing the error?
-    producer_->Push(std::move(error));
-    std::unique_lock<std::mutex> lock(mutex_);
-    StopProducingUnlocked();
-  }
-
-  void InputFinished(ExecNode* input, int seq_stop) override {
-    std::unique_lock<std::mutex> lock(mutex_);
-    DCHECK_GE(seq_stop, static_cast<int>(received_batches_.size()));
-    received_batches_.reserve(seq_stop);
-    emit_stop_ = seq_stop;
-    if (emit_stop_ == num_received_) {
-      DCHECK_EQ(emit_stop_, num_emitted_);
-      StopProducingUnlocked();
-    }
-  }
-
- private:
-  void StopProducingUnlocked() {
-    if (!stopped_) {
-      stopped_ = true;
-      producer_->Close();
-      inputs_[0]->StopProducing(this);
-    }
-  }
-
-  // TODO factor this out as ExecBatch::ToRecordBatch()?
-  Result<std::shared_ptr<RecordBatch>> MakeBatch(compute::ExecBatch&& exec_batch) {
-    ArrayDataVector columns;
-    columns.reserve(exec_batch.values.size());
-    for (auto&& value : exec_batch.values) {
-      if (!value.is_array()) {
-        return Status::TypeError("Expected array input");
-      }
-      columns.push_back(std::move(value).array());
-    }
-    return RecordBatch::Make(schema_, exec_batch.length, std::move(columns));
-  }
-
-  const std::shared_ptr<Schema> schema_;
-
-  std::mutex mutex_;
-  RecordBatchVector received_batches_;
-  int num_received_;
-  int num_emitted_;
-  int emit_stop_;
-  bool stopped_;
-
-  PushGenerator<std::shared_ptr<RecordBatch>> generator_;
-  util::optional<PushGenerator<std::shared_ptr<RecordBatch>>::Producer> producer_;
-};
+AsyncGenerator<util::optional<ExecBatch>> Wrap(RecordBatchGenerator gen,
+                                               ::arrow::internal::Executor* io_executor) {
+  return MakeMappedGenerator(
+      MakeTransferredGenerator(std::move(gen), io_executor),
+      [](const std::shared_ptr<RecordBatch>& batch) -> util::optional<ExecBatch> {
+        return ExecBatch(*batch);
+      });
+}
 
 }  // namespace
 
 ExecNode* MakeRecordBatchReaderNode(ExecPlan* plan, std::string label,
-                                    std::shared_ptr<RecordBatchReader> reader,
-                                    Executor* io_executor) {
-  return plan->EmplaceNode<RecordBatchReaderNode>(plan, std::move(label),
-                                                  std::move(reader), io_executor);
+                                    const std::shared_ptr<Schema>& schema,
+                                    RecordBatchGenerator generator,
+                                    ::arrow::internal::Executor* io_executor) {
+  return MakeSourceNode(plan, std::move(label), DescrFromSchemaColumns(*schema),
+                        Wrap(std::move(generator), io_executor));
 }
 
 ExecNode* MakeRecordBatchReaderNode(ExecPlan* plan, std::string label,
-                                    std::shared_ptr<Schema> schema,
-                                    RecordBatchGenerator generator,
-                                    ::arrow::internal::Executor* io_executor) {
-  return plan->EmplaceNode<RecordBatchReaderNode>(
-      plan, std::move(label), std::move(schema), std::move(generator), io_executor);
+                                    const std::shared_ptr<RecordBatchReader>& reader,
+                                    Executor* io_executor) {
+  auto gen =
+      MakeBackgroundGenerator(MakeIteratorFromReader(reader), io_executor).ValueOrDie();

Review comment:
       Same question, why `io_executor`?

##########
File path: cpp/src/arrow/compute/exec/expression.cc
##########
@@ -510,7 +475,67 @@ Result<Expression> Expression::Bind(const Schema& in_schema,
   return Bind(ValueDescr::Array(struct_(in_schema.fields())), exec_context);
 }
 
-Result<Datum> ExecuteScalarExpression(const Expression& expr, const Datum& input,
+Result<ExecBatch> MakeExecBatch(const Schema& full_schema, const Datum& partial) {
+  ExecBatch out;
+
+  if (partial.kind() == Datum::RECORD_BATCH) {
+    const auto& partial_batch = *partial.record_batch();
+    out.length = partial_batch.num_rows();
+
+    for (const auto& field : full_schema.fields()) {
+      ARROW_ASSIGN_OR_RAISE(auto column,
+                            FieldRef(field->name()).GetOneOrNone(partial_batch));
+
+      if (column) {
+        if (!column->type()->Equals(field->type())) {
+          // Referenced field was present but didn't have the expected type.
+          // This *should* be handled by readers, and will just be an error in the future.
+          ARROW_ASSIGN_OR_RAISE(
+              auto converted,
+              compute::Cast(column, field->type(), compute::CastOptions::Safe()));
+          column = converted.make_array();
+        }
+        out.values.emplace_back(std::move(column));
+      } else {
+        out.values.emplace_back(MakeNullScalar(field->type()));
+      }
+    }
+    return out;
+  }
+
+  // wasteful but useful for testing:
+  if (partial.type()->id() == Type::STRUCT) {
+    if (partial.is_array()) {
+      ARROW_ASSIGN_OR_RAISE(auto partial_batch,
+                            RecordBatch::FromStructArray(partial.make_array()));
+
+      return MakeExecBatch(full_schema, partial_batch);
+    }
+
+    if (partial.is_scalar()) {
+      ARROW_ASSIGN_OR_RAISE(auto partial_array,
+                            MakeArrayFromScalar(*partial.scalar(), 1));
+      ARROW_ASSIGN_OR_RAISE(auto out, MakeExecBatch(full_schema, partial_array));
+
+      for (Datum& value : out.values) {
+        if (value.is_scalar()) continue;
+        ARROW_ASSIGN_OR_RAISE(value, value.make_array()->GetScalar(0));
+      }

Review comment:
       I'm not sure what is going on here (though that is likely my own problem).  If the value is a scalar record batch you want to end up with one each value being a scalar.  Can you not just grab the first item from each column of `partial_array`?  Why do you need to go back in and patch things up?

##########
File path: cpp/src/arrow/compute/exec/expression.cc
##########
@@ -510,7 +475,67 @@ Result<Expression> Expression::Bind(const Schema& in_schema,
   return Bind(ValueDescr::Array(struct_(in_schema.fields())), exec_context);
 }
 
-Result<Datum> ExecuteScalarExpression(const Expression& expr, const Datum& input,
+Result<ExecBatch> MakeExecBatch(const Schema& full_schema, const Datum& partial) {
+  ExecBatch out;
+
+  if (partial.kind() == Datum::RECORD_BATCH) {
+    const auto& partial_batch = *partial.record_batch();
+    out.length = partial_batch.num_rows();
+
+    for (const auto& field : full_schema.fields()) {
+      ARROW_ASSIGN_OR_RAISE(auto column,
+                            FieldRef(field->name()).GetOneOrNone(partial_batch));

Review comment:
       Will this fail if the column names are not unique?

##########
File path: cpp/src/arrow/util/async_generator.h
##########
@@ -258,43 +258,27 @@ class MappingGenerator {
 /// Note: Errors returned from the `map` function will be propagated
 ///
 /// If the source generator is async-reentrant then this generator will be also
-template <typename T, typename V>
-AsyncGenerator<V> MakeMappedGenerator(AsyncGenerator<T> source_generator,
-                                      std::function<Result<V>(const T&)> map) {
-  std::function<Future<V>(const T&)> future_map = [map](const T& val) -> Future<V> {
-    return Future<V>::MakeFinished(map(val));
-  };
-  return MappingGenerator<T, V>(std::move(source_generator), std::move(future_map));
-}
-template <typename T, typename V>
-AsyncGenerator<V> MakeMappedGenerator(AsyncGenerator<T> source_generator,
-                                      std::function<V(const T&)> map) {
-  std::function<Future<V>(const T&)> maybe_future_map = [map](const T& val) -> Future<V> {
-    return Future<V>::MakeFinished(map(val));
-  };
-  return MappingGenerator<T, V>(std::move(source_generator), std::move(maybe_future_map));
-}
-template <typename T, typename V>
-AsyncGenerator<V> MakeMappedGenerator(AsyncGenerator<T> source_generator,
-                                      std::function<Future<V>(const T&)> map) {
-  return MappingGenerator<T, V>(std::move(source_generator), std::move(map));
-}
-
-template <typename V, typename T, typename MapFunc>
-AsyncGenerator<V> MakeMappedGenerator(AsyncGenerator<T> source_generator, MapFunc map) {
+template <typename T, typename MapFn,

Review comment:
       Thanks for figuring this out!

##########
File path: cpp/src/arrow/compute/exec/expression.h
##########
@@ -207,11 +218,22 @@ Result<Expression> SimplifyWithGuarantee(Expression,
 
 // Execution
 
-/// Execute a scalar expression against the provided state and input Datum. This
+/// Ensure that a RecordBatch (which may have missing or incorrectly ordered columns)

Review comment:
       This reads like an explanatory NB more than a description of what the function does.  Maybe...
   
   ```
   Converts a RecordBatch to an ExecBatch
   
   Arrays will be reordered according to schema ordering.  Missing fields will be replaced with null scalars.  This is necessary when executing expressions since we look up fields by index.
   ```
   
   Also, it feels a bit like a catch-22, though this is more of an observation than a complaint.  "We need to change expressions to use indices so they will work on exec batches." and "We need to convert record batches to exec batches because expressions work on indices."
   
   Maybe just remove the `This is necessary...` statement.  By this point the user is already making an exec batch so presumably they have a reason for it.

##########
File path: cpp/src/arrow/compute/exec/exec_plan.h
##########
@@ -225,22 +212,43 @@ class ARROW_EXPORT ExecNode {
   virtual void StopProducing() = 0;
 
  protected:
-  ExecNode(ExecPlan* plan, std::string label, std::vector<BatchDescr> input_descrs,
+  ExecNode(ExecPlan*, std::string label, NodeVector inputs,
            std::vector<std::string> input_labels, BatchDescr output_descr,
            int num_outputs);
 
   ExecPlan* plan_;
-
   std::string label_;
 
-  std::vector<BatchDescr> input_descrs_;
-  std::vector<std::string> input_labels_;
   NodeVector inputs_;
+  std::vector<std::string> input_labels_;
 
   BatchDescr output_descr_;
   int num_outputs_;
   NodeVector outputs_;
 };
 
+/// \brief Adapt an AsyncGenerator<ExecBatch> as a source node
+ARROW_EXPORT
+ExecNode* MakeSourceNode(ExecPlan*, std::string label, ExecNode::BatchDescr output_descr,
+                         AsyncGenerator<util::optional<ExecBatch>>);
+
+/// \brief Add a sink node which forwards to an AsyncGenerator<ExecBatch>
+ARROW_EXPORT
+AsyncGenerator<util::optional<ExecBatch>> MakeSinkNode(ExecNode* input,
+                                                       std::string label);
+
+/// \brief Make a node which excludes some rows from batches passed through it
+///
+/// filter Expression must be bound; no field references will be looked up by name
+ARROW_EXPORT
+ExecNode* MakeFilterNode(ExecNode* input, std::string label, Expression filter);
+
+/// \brief Make a node which executes expressions on input batches, producing new batches.
+///
+/// Expressions must be bound; no field references will be looked up by name
+ARROW_EXPORT
+ExecNode* MakeProjectNode(ExecNode* input, std::string label,

Review comment:
       I maybe understand from the meaning of `Project` what is going on here but if someone were not as familiar with the domain they might not.  It's not clear that the output will have one output column per expression.  From the description this sounds like it might be more of a generic "map" operation that maps input batch to output batch.

##########
File path: cpp/src/arrow/compute/exec/expression_test.cc
##########
@@ -165,6 +165,56 @@ TEST(ExpressionUtils, StripOrderPreservingCasts) {
   Expect(cast(field_ref("i32"), uint64()), no_change);
 }
 
+TEST(ExpressionUtils, MakeExecBatch) {
+  auto Expect = [](std::shared_ptr<RecordBatch> partial_batch) {
+    SCOPED_TRACE(partial_batch->ToString());
+    ASSERT_OK_AND_ASSIGN(auto batch, MakeExecBatch(*kBoringSchema, partial_batch));
+
+    ASSERT_EQ(batch.num_values(), kBoringSchema->num_fields());
+    for (int i = 0; i < kBoringSchema->num_fields(); ++i) {
+      const auto& field = *kBoringSchema->field(i);
+
+      SCOPED_TRACE("Field#" + std::to_string(i) + " " + field.ToString());
+
+      EXPECT_TRUE(batch[i].type()->Equals(field.type()))
+          << "Incorrect type " << batch[i].type()->ToString();
+
+      ASSERT_OK_AND_ASSIGN(auto col, FieldRef(field.name()).GetOneOrNone(*partial_batch));

Review comment:
       Why not `partial_batch->GetFieldByName(field.name())`?

##########
File path: cpp/src/arrow/util/iterator.h
##########
@@ -66,6 +66,12 @@ bool IsIterationEnd(const T& val) {
   return IterationTraits<T>::IsEnd(val);
 }
 
+template <typename T>
+bool IsIterationEnd(const Result<T>& maybe_val) {

Review comment:
       Where is this used?

##########
File path: cpp/src/arrow/compute/exec/test_util.cc
##########
@@ -124,277 +130,42 @@ struct DummyNode : ExecNode {
   bool started_ = false;
 };
 
-struct RecordBatchReaderNode : ExecNode {
-  RecordBatchReaderNode(ExecPlan* plan, std::string label,
-                        std::shared_ptr<RecordBatchReader> reader, Executor* io_executor)
-      : ExecNode(plan, std::move(label), {}, {},
-                 DescrFromSchemaColumns(*reader->schema()), /*num_outputs=*/1),
-        schema_(reader->schema()),
-        reader_(std::move(reader)),
-        io_executor_(io_executor) {}
-
-  RecordBatchReaderNode(ExecPlan* plan, std::string label, std::shared_ptr<Schema> schema,
-                        RecordBatchGenerator generator, Executor* io_executor)
-      : ExecNode(plan, std::move(label), {}, {}, DescrFromSchemaColumns(*schema),
-                 /*num_outputs=*/1),
-        schema_(std::move(schema)),
-        generator_(std::move(generator)),
-        io_executor_(io_executor) {}
-
-  const char* kind_name() override { return "RecordBatchReader"; }
-
-  void InputReceived(ExecNode* input, int seq_num, compute::ExecBatch batch) override {}
-
-  void ErrorReceived(ExecNode* input, Status error) override {}
-
-  void InputFinished(ExecNode* input, int seq_stop) override {}
-
-  Status StartProducing() override {
-    next_batch_index_ = 0;
-    if (!generator_) {
-      auto it = MakeIteratorFromReader(reader_);
-      ARROW_ASSIGN_OR_RAISE(generator_,
-                            MakeBackgroundGenerator(std::move(it), io_executor_));
-    }
-    GenerateOne(std::unique_lock<std::mutex>{mutex_});
-    return Status::OK();
-  }
-
-  void PauseProducing(ExecNode* output) override {}
-
-  void ResumeProducing(ExecNode* output) override {}
-
-  void StopProducing(ExecNode* output) override {
-    ASSERT_EQ(output, outputs_[0]);
-    std::unique_lock<std::mutex> lock(mutex_);
-    generator_ = nullptr;  // null function
-  }
-
-  void StopProducing() override { StopProducing(outputs_[0]); }
-
- private:
-  void GenerateOne(std::unique_lock<std::mutex>&& lock) {
-    if (!generator_) {
-      // Stopped
-      return;
-    }
-    auto plan = this->plan()->shared_from_this();
-    auto fut = generator_();
-    const auto batch_index = next_batch_index_++;
-
-    lock.unlock();
-    // TODO we want to transfer always here
-    io_executor_->Transfer(std::move(fut))
-        .AddCallback(
-            [plan, batch_index, this](const Result<std::shared_ptr<RecordBatch>>& res) {
-              std::unique_lock<std::mutex> lock(mutex_);
-              if (!res.ok()) {
-                for (auto out : outputs_) {
-                  out->ErrorReceived(this, res.status());
-                }
-                return;
-              }
-              const auto& batch = *res;
-              if (IsIterationEnd(batch)) {
-                lock.unlock();
-                for (auto out : outputs_) {
-                  out->InputFinished(this, batch_index);
-                }
-              } else {
-                lock.unlock();
-                for (auto out : outputs_) {
-                  out->InputReceived(this, batch_index, compute::ExecBatch(*batch));
-                }
-                lock.lock();
-                GenerateOne(std::move(lock));
-              }
-            });
-  }
-
-  std::mutex mutex_;
-  const std::shared_ptr<Schema> schema_;
-  const std::shared_ptr<RecordBatchReader> reader_;
-  RecordBatchGenerator generator_;
-  int next_batch_index_;
-
-  Executor* const io_executor_;
-};
-
-struct RecordBatchCollectNodeImpl : public RecordBatchCollectNode {
-  RecordBatchCollectNodeImpl(ExecPlan* plan, std::string label,
-                             std::shared_ptr<Schema> schema)
-      : RecordBatchCollectNode(plan, std::move(label), {DescrFromSchemaColumns(*schema)},
-                               {"batches_to_collect"}, {}, 0),
-        schema_(std::move(schema)) {}
-
-  RecordBatchGenerator generator() override { return generator_; }
-
-  const char* kind_name() override { return "RecordBatchReader"; }
-
-  Status StartProducing() override {
-    num_received_ = 0;
-    num_emitted_ = 0;
-    emit_stop_ = -1;
-    stopped_ = false;
-    producer_.emplace(generator_.producer());
-    return Status::OK();
-  }
-
-  // sink nodes have no outputs from which to feel backpressure
-  void ResumeProducing(ExecNode* output) override {
-    FAIL() << "no outputs; this should never be called";
-  }
-  void PauseProducing(ExecNode* output) override {
-    FAIL() << "no outputs; this should never be called";
-  }
-  void StopProducing(ExecNode* output) override {
-    FAIL() << "no outputs; this should never be called";
-  }
-
-  void StopProducing() override {
-    std::unique_lock<std::mutex> lock(mutex_);
-    StopProducingUnlocked();
-  }
-
-  void InputReceived(ExecNode* input, int seq_num,
-                     compute::ExecBatch exec_batch) override {
-    std::unique_lock<std::mutex> lock(mutex_);
-    if (stopped_) {
-      return;
-    }
-    auto maybe_batch = MakeBatch(std::move(exec_batch));
-    if (!maybe_batch.ok()) {
-      lock.unlock();
-      producer_->Push(std::move(maybe_batch));
-      return;
-    }
-
-    // TODO would be nice to factor this out in a ReorderQueue
-    auto batch = *std::move(maybe_batch);
-    if (seq_num <= static_cast<int>(received_batches_.size())) {
-      received_batches_.resize(seq_num + 1, nullptr);
-    }
-    DCHECK_EQ(received_batches_[seq_num], nullptr);
-    received_batches_[seq_num] = std::move(batch);
-    ++num_received_;
-
-    if (seq_num != num_emitted_) {
-      // Cannot emit yet as there is a hole at `num_emitted_`
-      DCHECK_GT(seq_num, num_emitted_);
-      DCHECK_EQ(received_batches_[num_emitted_], nullptr);
-      return;
-    }
-    if (num_received_ == emit_stop_) {
-      StopProducingUnlocked();
-    }
-
-    // Emit batches in order as far as possible
-    // First collect these batches, then unlock before producing.
-    const auto seq_start = seq_num;
-    while (seq_num < static_cast<int>(received_batches_.size()) &&
-           received_batches_[seq_num] != nullptr) {
-      ++seq_num;
-    }
-    DCHECK_GT(seq_num, seq_start);
-    // By moving the values now, we make sure another thread won't emit the same values
-    // below
-    RecordBatchVector to_emit(
-        std::make_move_iterator(received_batches_.begin() + seq_start),
-        std::make_move_iterator(received_batches_.begin() + seq_num));
-
-    lock.unlock();
-    for (auto&& batch : to_emit) {
-      producer_->Push(std::move(batch));
-    }
-    lock.lock();
-
-    DCHECK_EQ(seq_start, num_emitted_);  // num_emitted_ wasn't bumped in the meantime
-    num_emitted_ = seq_num;
-  }
-
-  void ErrorReceived(ExecNode* input, Status error) override {
-    // XXX do we care about properly sequencing the error?
-    producer_->Push(std::move(error));
-    std::unique_lock<std::mutex> lock(mutex_);
-    StopProducingUnlocked();
-  }
-
-  void InputFinished(ExecNode* input, int seq_stop) override {
-    std::unique_lock<std::mutex> lock(mutex_);
-    DCHECK_GE(seq_stop, static_cast<int>(received_batches_.size()));
-    received_batches_.reserve(seq_stop);
-    emit_stop_ = seq_stop;
-    if (emit_stop_ == num_received_) {
-      DCHECK_EQ(emit_stop_, num_emitted_);
-      StopProducingUnlocked();
-    }
-  }
-
- private:
-  void StopProducingUnlocked() {
-    if (!stopped_) {
-      stopped_ = true;
-      producer_->Close();
-      inputs_[0]->StopProducing(this);
-    }
-  }
-
-  // TODO factor this out as ExecBatch::ToRecordBatch()?
-  Result<std::shared_ptr<RecordBatch>> MakeBatch(compute::ExecBatch&& exec_batch) {
-    ArrayDataVector columns;
-    columns.reserve(exec_batch.values.size());
-    for (auto&& value : exec_batch.values) {
-      if (!value.is_array()) {
-        return Status::TypeError("Expected array input");
-      }
-      columns.push_back(std::move(value).array());
-    }
-    return RecordBatch::Make(schema_, exec_batch.length, std::move(columns));
-  }
-
-  const std::shared_ptr<Schema> schema_;
-
-  std::mutex mutex_;
-  RecordBatchVector received_batches_;
-  int num_received_;
-  int num_emitted_;
-  int emit_stop_;
-  bool stopped_;
-
-  PushGenerator<std::shared_ptr<RecordBatch>> generator_;
-  util::optional<PushGenerator<std::shared_ptr<RecordBatch>>::Producer> producer_;
-};
+AsyncGenerator<util::optional<ExecBatch>> Wrap(RecordBatchGenerator gen,
+                                               ::arrow::internal::Executor* io_executor) {
+  return MakeMappedGenerator(
+      MakeTransferredGenerator(std::move(gen), io_executor),

Review comment:
       Why are you transferring onto the `io_executor` 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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] github-actions[bot] commented on pull request #10397: ARROW-11930: [C++][Dataset][Compute] Use an ExecPlan for dataset scans

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #10397:
URL: https://github.com/apache/arrow/pull/10397#issuecomment-848264046


   https://issues.apache.org/jira/browse/ARROW-11930


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] bkietz commented on a change in pull request #10397: ARROW-11930: [C++][Dataset][Compute] Use an ExecPlan for dataset scans

Posted by GitBox <gi...@apache.org>.
bkietz commented on a change in pull request #10397:
URL: https://github.com/apache/arrow/pull/10397#discussion_r643577382



##########
File path: cpp/src/arrow/compute/exec/expression.cc
##########
@@ -510,7 +475,67 @@ Result<Expression> Expression::Bind(const Schema& in_schema,
   return Bind(ValueDescr::Array(struct_(in_schema.fields())), exec_context);
 }
 
-Result<Datum> ExecuteScalarExpression(const Expression& expr, const Datum& input,
+Result<ExecBatch> MakeExecBatch(const Schema& full_schema, const Datum& partial) {
+  ExecBatch out;
+
+  if (partial.kind() == Datum::RECORD_BATCH) {
+    const auto& partial_batch = *partial.record_batch();
+    out.length = partial_batch.num_rows();
+
+    for (const auto& field : full_schema.fields()) {
+      ARROW_ASSIGN_OR_RAISE(auto column,
+                            FieldRef(field->name()).GetOneOrNone(partial_batch));
+
+      if (column) {
+        if (!column->type()->Equals(field->type())) {
+          // Referenced field was present but didn't have the expected type.
+          // This *should* be handled by readers, and will just be an error in the future.
+          ARROW_ASSIGN_OR_RAISE(
+              auto converted,
+              compute::Cast(column, field->type(), compute::CastOptions::Safe()));
+          column = converted.make_array();
+        }
+        out.values.emplace_back(std::move(column));
+      } else {
+        out.values.emplace_back(MakeNullScalar(field->type()));
+      }
+    }
+    return out;
+  }
+
+  // wasteful but useful for testing:
+  if (partial.type()->id() == Type::STRUCT) {
+    if (partial.is_array()) {
+      ARROW_ASSIGN_OR_RAISE(auto partial_batch,
+                            RecordBatch::FromStructArray(partial.make_array()));
+
+      return MakeExecBatch(full_schema, partial_batch);
+    }
+
+    if (partial.is_scalar()) {
+      ARROW_ASSIGN_OR_RAISE(auto partial_array,
+                            MakeArrayFromScalar(*partial.scalar(), 1));
+      ARROW_ASSIGN_OR_RAISE(auto out, MakeExecBatch(full_schema, partial_array));
+
+      for (Datum& value : out.values) {
+        if (value.is_scalar()) continue;
+        ARROW_ASSIGN_OR_RAISE(value, value.make_array()->GetScalar(0));
+      }

Review comment:
       This was as compact as I could write this case; if you see a way to compress/simplify it then I'll take it but the scalar/array cases are really just for testing purposes




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] pitrou commented on a change in pull request #10397: ARROW-11930: [C++][Dataset][Compute] Use an ExecPlan for dataset scans

Posted by GitBox <gi...@apache.org>.
pitrou commented on a change in pull request #10397:
URL: https://github.com/apache/arrow/pull/10397#discussion_r657029167



##########
File path: cpp/src/arrow/compute/exec/doc/exec_node.md
##########
@@ -0,0 +1,147 @@
+<!---
+  Licensed to the Apache Software Foundation (ASF) under one
+  or more contributor license agreements.  See the NOTICE file
+  distributed with this work for additional information
+  regarding copyright ownership.  The ASF licenses this file
+  to you under the Apache License, Version 2.0 (the
+  "License"); you may not use this file except in compliance
+  with the License.  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing,
+  software distributed under the License is distributed on an
+  "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+  KIND, either express or implied.  See the License for the
+  specific language governing permissions and limitations
+  under the License.
+-->
+
+# ExecNodes and logical operators

Review comment:
       I'm not sure I understand the status of this document. If this is meant to be a persistent document, then can it be part of the Sphinx development docs?

##########
File path: cpp/src/arrow/compute/exec/expression.cc
##########
@@ -61,13 +61,22 @@ Expression call(std::string function, std::vector<Expression> arguments,
   call.function_name = std::move(function);
   call.arguments = std::move(arguments);
   call.options = std::move(options);
+
+  call.hash = std::hash<std::string>{}(call.function_name);
+  for (const auto& arg : call.arguments) {
+    call.hash ^= arg.hash();

Review comment:
       Note that simple XORing is suboptimal (`a,b` will produce the same hash as `b,a`, for example).
   May want to use something like `hash_combine` from `arrow/util/hash_util.h`.
   

##########
File path: cpp/src/arrow/compute/exec/plan_test.cc
##########
@@ -20,27 +20,46 @@
 #include <functional>
 #include <memory>
 
+#include "arrow/compute/exec.h"
 #include "arrow/compute/exec/exec_plan.h"
+#include "arrow/compute/exec/expression.h"
 #include "arrow/compute/exec/test_util.h"
 #include "arrow/record_batch.h"
 #include "arrow/testing/future_util.h"
 #include "arrow/testing/gtest_util.h"
 #include "arrow/testing/random.h"
 #include "arrow/util/logging.h"
 #include "arrow/util/thread_pool.h"
+#include "arrow/util/vector.h"
 
-namespace arrow {
+using testing::ElementsAre;
+using testing::HasSubstr;
+using testing::UnorderedElementsAreArray;
 
-using internal::Executor;
+namespace arrow {
 
 namespace compute {
 
-void AssertBatchesEqual(const RecordBatchVector& expected,
-                        const RecordBatchVector& actual) {
-  ASSERT_EQ(expected.size(), actual.size());
-  for (size_t i = 0; i < expected.size(); ++i) {
-    AssertBatchesEqual(*expected[i], *actual[i]);
+ExecBatch ExecBatchFromJSON(const std::vector<ValueDescr>& descrs,

Review comment:
       Is this something we want to expose in `arrow/testing` or `arrow/compute/test_util.h` perhaps?

##########
File path: cpp/src/arrow/type.cc
##########
@@ -1195,6 +1195,10 @@ std::string FieldRef::ToString() const {
 }
 
 std::vector<FieldPath> FieldRef::FindAll(const Schema& schema) const {
+  if (auto name = this->name()) {
+    return internal::MapVector([](int i) { return FieldPath{i}; },
+                               schema.GetAllFieldIndices(*name));
+  }

Review comment:
       Can you add a test for this?

##########
File path: cpp/src/arrow/util/vector.h
##########
@@ -84,27 +84,49 @@ std::vector<T> FilterVector(std::vector<T> values, Predicate&& predicate) {
   return values;
 }
 
-/// \brief Like MapVector, but where the function can fail.
-template <typename Fn, typename From = internal::call_traits::argument_type<0, Fn>,
-          typename To = typename internal::call_traits::return_type<Fn>::ValueType>
-Result<std::vector<To>> MaybeMapVector(Fn&& map, const std::vector<From>& src) {
+template <typename Fn, typename From,
+          typename To = decltype(std::declval<Fn>()(std::declval<From>()))>
+std::vector<To> MapVector(Fn&& map, const std::vector<From>& source) {
   std::vector<To> out;
-  out.reserve(src.size());
-  ARROW_RETURN_NOT_OK(MaybeTransform(src.begin(), src.end(), std::back_inserter(out),
-                                     std::forward<Fn>(map)));
-  return std::move(out);
+  out.reserve(source.size());
+  std::transform(source.begin(), source.end(), std::back_inserter(out),
+                 std::forward<Fn>(map));
+  return out;
 }
 
 template <typename Fn, typename From,
           typename To = decltype(std::declval<Fn>()(std::declval<From>()))>
-std::vector<To> MapVector(Fn&& map, const std::vector<From>& source) {
+std::vector<To> MapVector(Fn&& map, std::vector<From>&& source) {
   std::vector<To> out;
   out.reserve(source.size());
-  std::transform(source.begin(), source.end(), std::back_inserter(out),
+  std::transform(std::make_move_iterator(source.begin()),
+                 std::make_move_iterator(source.end()), std::back_inserter(out),
                  std::forward<Fn>(map));
   return out;
 }
 
+/// \brief Like MapVector, but where the function can fail.
+template <typename Fn, typename From = internal::call_traits::argument_type<0, Fn>,
+          typename To = typename internal::call_traits::return_type<Fn>::ValueType>

Review comment:
       Why not use the `decltype(declval)` pattern here as well?

##########
File path: cpp/src/arrow/util/thread_pool_test.cc
##########
@@ -395,6 +395,22 @@ TEST_F(TestThreadPool, StressSpawn) {
   SpawnAdds(pool.get(), 1000, task_add<int>);
 }
 
+TEST_F(TestThreadPool, OwnsCurrentThread) {
+  auto pool = this->MakeThreadPool(30);
+  std::atomic<bool> one_failed{false};
+
+  for (int i = 0; i < 1000; ++i) {
+    ASSERT_OK(pool->Spawn([&] {
+      if (pool->OwnsThisThread()) return;
+
+      one_failed = true;
+    }));
+  }
+
+  ASSERT_OK(pool->Shutdown());
+  ASSERT_FALSE(one_failed);

Review comment:
       Also call `pool->OwnsThisThread()` from here?

##########
File path: cpp/src/arrow/compute/exec.h
##########
@@ -28,6 +28,7 @@
 #include <vector>
 
 #include "arrow/array/data.h"
+#include "arrow/compute/exec/expression.h"

Review comment:
       Still a concern probably.

##########
File path: cpp/src/arrow/util/future_test.cc
##########
@@ -1704,5 +1704,45 @@ TEST(FnOnceTest, MoveOnlyDataType) {
   ASSERT_EQ(i0.moves, 0);
   ASSERT_EQ(i1.moves, 0);
 }
+
+TEST(FutureTest, MatcherExamples) {
+  EXPECT_THAT(Future<int>::MakeFinished(Status::Invalid("arbitrary error")),
+              Raises(StatusCode::Invalid));
+
+  EXPECT_THAT(Future<int>::MakeFinished(Status::Invalid("arbitrary error")),
+              Raises(StatusCode::Invalid, testing::HasSubstr("arbitrary")));

Review comment:
       I would find it slightly nicer if `Future` used separate matchers (e.g. `FinishesWith`, `Fails`...). But not a big deal either.

##########
File path: cpp/src/arrow/testing/gtest_util.h
##########
@@ -28,6 +28,7 @@
 #include <utility>
 #include <vector>
 
+#include <gmock/gmock-matchers.h>

Review comment:
       Usual concern: does it blow up compile times?
   (`gtest.h` is already a large inclusion that gets processed for every test file)

##########
File path: cpp/src/arrow/util/future.h
##########
@@ -976,4 +979,43 @@ Future<BreakValueType> Loop(Iterate iterate) {
   return break_fut;
 }
 
+template <typename T>
+struct EnsureFuture {
+  using type = Future<T>;
+};
+
+template <typename T>
+struct EnsureFuture<Result<T>> {
+  using type = Future<T>;
+};
+
+template <typename T>
+struct EnsureFuture<Future<T>> {
+  using type = Future<T>;
+};
+
+template <>
+struct EnsureFuture<Status> {
+  using type = Future<>;
+};

Review comment:
       Should it be simply:
   ```c++
   template <typename T>
   struct EnsureFuture {
     using type = decltype(ToFuture(std::declval<T>()));
   };
   ```




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] bkietz commented on pull request #10397: ARROW-11930: [C++][Dataset][Compute] Use an ExecPlan for dataset scans

Posted by GitBox <gi...@apache.org>.
bkietz commented on pull request #10397:
URL: https://github.com/apache/arrow/pull/10397#issuecomment-872195970


   +1, merging


-- 
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-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] bkietz closed pull request #10397: ARROW-11930: [C++][Dataset][Compute] Use an ExecPlan for dataset scans

Posted by GitBox <gi...@apache.org>.
bkietz closed pull request #10397:
URL: https://github.com/apache/arrow/pull/10397


   


-- 
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-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] pitrou commented on a change in pull request #10397: ARROW-11930: [C++][Dataset][Compute] Use an ExecPlan for dataset scans

Posted by GitBox <gi...@apache.org>.
pitrou commented on a change in pull request #10397:
URL: https://github.com/apache/arrow/pull/10397#discussion_r644137804



##########
File path: cpp/src/arrow/compute/exec/exec_plan.cc
##########
@@ -170,48 +165,409 @@ Status ExecPlan::Validate() { return ToDerived(this)->Validate(); }
 
 Status ExecPlan::StartProducing() { return ToDerived(this)->StartProducing(); }
 
-ExecNode::ExecNode(ExecPlan* plan, std::string label,
-                   std::vector<BatchDescr> input_descrs,
+ExecNode::ExecNode(ExecPlan* plan, std::string label, NodeVector inputs,
                    std::vector<std::string> input_labels, BatchDescr output_descr,
                    int num_outputs)
     : plan_(plan),
       label_(std::move(label)),
-      input_descrs_(std::move(input_descrs)),
+      inputs_(std::move(inputs)),
       input_labels_(std::move(input_labels)),
       output_descr_(std::move(output_descr)),
-      num_outputs_(num_outputs) {}
+      num_outputs_(num_outputs) {
+  for (auto input : inputs_) {
+    input->outputs_.push_back(this);
+  }
+}
 
 Status ExecNode::Validate() const {
-  if (inputs_.size() != input_descrs_.size()) {
+  if (inputs_.size() != input_labels_.size()) {
     return Status::Invalid("Invalid number of inputs for '", label(), "' (expected ",
-                           num_inputs(), ", actual ", inputs_.size(), ")");
+                           num_inputs(), ", actual ", input_labels_.size(), ")");
   }
 
   if (static_cast<int>(outputs_.size()) != num_outputs_) {
     return Status::Invalid("Invalid number of outputs for '", label(), "' (expected ",
                            num_outputs(), ", actual ", outputs_.size(), ")");
   }
 
-  DCHECK_EQ(input_descrs_.size(), input_labels_.size());
-
   for (auto out : outputs_) {
     auto input_index = GetNodeIndex(out->inputs(), this);
     if (!input_index) {
       return Status::Invalid("Node '", label(), "' outputs to node '", out->label(),
                              "' but is not listed as an input.");
     }
+  }
 
-    const auto& in_descr = out->input_descrs_[*input_index];
-    if (in_descr != output_descr_) {
-      return Status::Invalid(
-          "Node '", label(), "' (bound to input ", input_labels_[*input_index],
-          ") produces batches with type '", ValueDescr::ToString(output_descr_),
-          "' inconsistent with consumer '", out->label(), "' which accepts '",
-          ValueDescr::ToString(in_descr), "'");
+  return Status::OK();
+}
+
+struct SourceNode : ExecNode {
+  SourceNode(ExecPlan* plan, std::string label, ExecNode::BatchDescr output_descr,
+             AsyncGenerator<util::optional<ExecBatch>> generator)
+      : ExecNode(plan, std::move(label), {}, {}, std::move(output_descr),
+                 /*num_outputs=*/1),
+        generator_(std::move(generator)) {}
+
+  const char* kind_name() override { return "SourceNode"; }
+
+  static void NoInputs() { DCHECK(false) << "no inputs; this should never be called"; }
+  void InputReceived(ExecNode*, int, ExecBatch) override { NoInputs(); }
+  void ErrorReceived(ExecNode*, Status) override { NoInputs(); }
+  void InputFinished(ExecNode*, int) override { NoInputs(); }
+
+  Status StartProducing() override {
+    if (finished_) {
+      return Status::Invalid("Restarted SourceNode '", label(), "'");
     }
+
+    auto gen = std::move(generator_);
+
+    /// XXX should we wait on this future anywhere? In StopProducing() maybe?
+    auto done_fut =
+        Loop([gen, this] {
+          std::unique_lock<std::mutex> lock(mutex_);
+          int seq = next_batch_index_++;
+          if (finished_) {
+            return Future<ControlFlow<int>>::MakeFinished(Break(seq));
+          }
+          lock.unlock();
+
+          return gen().Then(
+              [=](const util::optional<ExecBatch>& batch) -> ControlFlow<int> {
+                std::unique_lock<std::mutex> lock(mutex_);
+                if (!batch || finished_) {
+                  finished_ = true;
+                  return Break(seq);
+                }
+                lock.unlock();
+
+                outputs_[0]->InputReceived(this, seq, *batch);
+                return Continue();
+              },
+              [=](const Status& error) -> ControlFlow<int> {
+                std::unique_lock<std::mutex> lock(mutex_);
+                if (!finished_) {
+                  finished_ = true;
+                  lock.unlock();
+                  // unless we were already finished, push the error to our output
+                  // XXX is this correct? Is it reasonable for a consumer to ignore errors
+                  // from a finished producer?
+                  outputs_[0]->ErrorReceived(this, error);
+                }
+                return Break(seq);
+              });
+        }).Then([&](int seq) {
+          /// XXX this is probably redundant: do we always call InputFinished after
+          /// ErrorReceived or will ErrorRecieved be sufficient?
+          outputs_[0]->InputFinished(this, seq);
+        });
+
+    return Status::OK();
   }
 
-  return Status::OK();
+  void PauseProducing(ExecNode* output) override {}
+
+  void ResumeProducing(ExecNode* output) override {}

Review comment:
       Oh, you're right, my bad.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] bkietz commented on a change in pull request #10397: ARROW-11930: [C++][Dataset][Compute] Use an ExecPlan for dataset scans

Posted by GitBox <gi...@apache.org>.
bkietz commented on a change in pull request #10397:
URL: https://github.com/apache/arrow/pull/10397#discussion_r644302347



##########
File path: cpp/src/arrow/compute/exec/exec_plan.cc
##########
@@ -170,48 +165,409 @@ Status ExecPlan::Validate() { return ToDerived(this)->Validate(); }
 
 Status ExecPlan::StartProducing() { return ToDerived(this)->StartProducing(); }
 
-ExecNode::ExecNode(ExecPlan* plan, std::string label,
-                   std::vector<BatchDescr> input_descrs,
+ExecNode::ExecNode(ExecPlan* plan, std::string label, NodeVector inputs,
                    std::vector<std::string> input_labels, BatchDescr output_descr,
                    int num_outputs)
     : plan_(plan),
       label_(std::move(label)),
-      input_descrs_(std::move(input_descrs)),
+      inputs_(std::move(inputs)),
       input_labels_(std::move(input_labels)),
       output_descr_(std::move(output_descr)),
-      num_outputs_(num_outputs) {}
+      num_outputs_(num_outputs) {
+  for (auto input : inputs_) {
+    input->outputs_.push_back(this);
+  }
+}
 
 Status ExecNode::Validate() const {
-  if (inputs_.size() != input_descrs_.size()) {
+  if (inputs_.size() != input_labels_.size()) {
     return Status::Invalid("Invalid number of inputs for '", label(), "' (expected ",
-                           num_inputs(), ", actual ", inputs_.size(), ")");
+                           num_inputs(), ", actual ", input_labels_.size(), ")");
   }
 
   if (static_cast<int>(outputs_.size()) != num_outputs_) {
     return Status::Invalid("Invalid number of outputs for '", label(), "' (expected ",
                            num_outputs(), ", actual ", outputs_.size(), ")");
   }
 
-  DCHECK_EQ(input_descrs_.size(), input_labels_.size());
-
   for (auto out : outputs_) {
     auto input_index = GetNodeIndex(out->inputs(), this);
     if (!input_index) {
       return Status::Invalid("Node '", label(), "' outputs to node '", out->label(),
                              "' but is not listed as an input.");
     }
+  }
 
-    const auto& in_descr = out->input_descrs_[*input_index];
-    if (in_descr != output_descr_) {
-      return Status::Invalid(
-          "Node '", label(), "' (bound to input ", input_labels_[*input_index],
-          ") produces batches with type '", ValueDescr::ToString(output_descr_),
-          "' inconsistent with consumer '", out->label(), "' which accepts '",
-          ValueDescr::ToString(in_descr), "'");
+  return Status::OK();
+}
+
+struct SourceNode : ExecNode {
+  SourceNode(ExecPlan* plan, std::string label, ExecNode::BatchDescr output_descr,
+             AsyncGenerator<util::optional<ExecBatch>> generator)
+      : ExecNode(plan, std::move(label), {}, {}, std::move(output_descr),
+                 /*num_outputs=*/1),
+        generator_(std::move(generator)) {}
+
+  const char* kind_name() override { return "SourceNode"; }
+
+  static void NoInputs() { DCHECK(false) << "no inputs; this should never be called"; }
+  void InputReceived(ExecNode*, int, ExecBatch) override { NoInputs(); }
+  void ErrorReceived(ExecNode*, Status) override { NoInputs(); }
+  void InputFinished(ExecNode*, int) override { NoInputs(); }
+
+  Status StartProducing() override {
+    if (finished_) {
+      return Status::Invalid("Restarted SourceNode '", label(), "'");
     }
+
+    auto gen = std::move(generator_);
+
+    /// XXX should we wait on this future anywhere? In StopProducing() maybe?
+    auto done_fut =
+        Loop([gen, this] {
+          std::unique_lock<std::mutex> lock(mutex_);
+          int seq = next_batch_index_++;
+          if (finished_) {
+            return Future<ControlFlow<int>>::MakeFinished(Break(seq));
+          }
+          lock.unlock();
+
+          return gen().Then(
+              [=](const util::optional<ExecBatch>& batch) -> ControlFlow<int> {
+                std::unique_lock<std::mutex> lock(mutex_);
+                if (!batch || finished_) {
+                  finished_ = true;
+                  return Break(seq);
+                }
+                lock.unlock();
+
+                outputs_[0]->InputReceived(this, seq, *batch);
+                return Continue();
+              },
+              [=](const Status& error) -> ControlFlow<int> {
+                std::unique_lock<std::mutex> lock(mutex_);
+                if (!finished_) {
+                  finished_ = true;
+                  lock.unlock();
+                  // unless we were already finished, push the error to our output
+                  // XXX is this correct? Is it reasonable for a consumer to ignore errors
+                  // from a finished producer?
+                  outputs_[0]->ErrorReceived(this, error);
+                }
+                return Break(seq);
+              });
+        }).Then([&](int seq) {
+          /// XXX this is probably redundant: do we always call InputFinished after
+          /// ErrorReceived or will ErrorRecieved be sufficient?
+          outputs_[0]->InputFinished(this, seq);
+        });
+
+    return Status::OK();
   }
 
-  return Status::OK();
+  void PauseProducing(ExecNode* output) override {}
+
+  void ResumeProducing(ExecNode* output) override {}
+
+  void StopProducing(ExecNode* output) override {
+    DCHECK_EQ(output, outputs_[0]);
+    std::unique_lock<std::mutex> lock(mutex_);
+    finished_ = true;
+  }
+
+  void StopProducing() override { StopProducing(outputs_[0]); }
+
+ private:
+  std::mutex mutex_;
+  bool finished_{false};
+  int next_batch_index_{0};
+  AsyncGenerator<util::optional<ExecBatch>> generator_;
+};
+
+ExecNode* MakeSourceNode(ExecPlan* plan, std::string label,
+                         ExecNode::BatchDescr output_descr,
+                         AsyncGenerator<util::optional<ExecBatch>> generator) {
+  return plan->EmplaceNode<SourceNode>(plan, std::move(label), std::move(output_descr),
+                                       std::move(generator));
+}
+
+struct FilterNode : ExecNode {
+  FilterNode(ExecNode* input, std::string label, Expression filter)
+      : ExecNode(input->plan(), std::move(label), {input}, {"target"},
+                 /*output_descr=*/{input->output_descr()},
+                 /*num_outputs=*/1),
+        filter_(std::move(filter)) {}
+
+  const char* kind_name() override { return "FilterNode"; }
+
+  Result<ExecBatch> DoFilter(const ExecBatch& target) {
+    ARROW_ASSIGN_OR_RAISE(Expression simplified_filter,
+                          SimplifyWithGuarantee(filter_, target.guarantee));
+
+    // XXX get a non-default exec context
+    ARROW_ASSIGN_OR_RAISE(Datum mask, ExecuteScalarExpression(simplified_filter, target));
+
+    if (mask.is_scalar()) {
+      const auto& mask_scalar = mask.scalar_as<BooleanScalar>();
+      if (mask_scalar.is_valid && mask_scalar.value) {
+        return target;
+      }
+
+      return target.Slice(0, 0);
+    }
+
+    auto values = target.values;
+    for (auto& value : values) {
+      if (value.is_scalar()) continue;
+      ARROW_ASSIGN_OR_RAISE(value, Filter(value, mask, FilterOptions::Defaults()));
+    }
+    return ExecBatch::Make(std::move(values));
+  }
+
+  void InputReceived(ExecNode* input, int seq, ExecBatch batch) override {
+    DCHECK_EQ(input, inputs_[0]);
+
+    auto maybe_filtered = DoFilter(std::move(batch));
+    if (!maybe_filtered.ok()) {
+      outputs_[0]->ErrorReceived(this, maybe_filtered.status());
+      inputs_[0]->StopProducing(this);
+      return;
+    }
+
+    maybe_filtered->guarantee = batch.guarantee;
+    outputs_[0]->InputReceived(this, seq, maybe_filtered.MoveValueUnsafe());
+  }
+
+  void ErrorReceived(ExecNode* input, Status error) override {
+    DCHECK_EQ(input, inputs_[0]);
+    outputs_[0]->ErrorReceived(this, std::move(error));
+    inputs_[0]->StopProducing(this);
+  }
+
+  void InputFinished(ExecNode* input, int seq) override {
+    DCHECK_EQ(input, inputs_[0]);
+    outputs_[0]->InputFinished(this, seq);
+    inputs_[0]->StopProducing(this);
+  }
+
+  Status StartProducing() override {
+    // XXX validate inputs_[0]->output_descr() against filter_
+    return Status::OK();
+  }
+
+  void PauseProducing(ExecNode* output) override {}
+
+  void ResumeProducing(ExecNode* output) override {}
+
+  void StopProducing(ExecNode* output) override {
+    DCHECK_EQ(output, outputs_[0]);
+    inputs_[0]->StopProducing(this);
+  }
+
+  void StopProducing() override { StopProducing(outputs_[0]); }
+
+ private:
+  Expression filter_;
+};
+
+ExecNode* MakeFilterNode(ExecNode* input, std::string label, Expression filter) {
+  return input->plan()->EmplaceNode<FilterNode>(input, std::move(label),
+                                                std::move(filter));
+}
+
+struct ProjectNode : ExecNode {
+  ProjectNode(ExecNode* input, std::string label, std::vector<Expression> exprs)
+      : ExecNode(input->plan(), std::move(label), {input}, {"target"},
+                 /*output_descr=*/{input->output_descr()},
+                 /*num_outputs=*/1),
+        exprs_(std::move(exprs)) {}
+
+  const char* kind_name() override { return "ProjectNode"; }
+
+  Result<ExecBatch> DoProject(const ExecBatch& target) {
+    // XXX get a non-default exec context
+    std::vector<Datum> values{exprs_.size()};
+    for (size_t i = 0; i < exprs_.size(); ++i) {
+      ARROW_ASSIGN_OR_RAISE(Expression simplified_expr,
+                            SimplifyWithGuarantee(exprs_[i], target.guarantee));
+
+      ARROW_ASSIGN_OR_RAISE(values[i], ExecuteScalarExpression(simplified_expr, target));
+    }
+    return ExecBatch::Make(std::move(values));
+  }
+
+  void InputReceived(ExecNode* input, int seq, ExecBatch batch) override {
+    DCHECK_EQ(input, inputs_[0]);
+
+    auto maybe_projected = DoProject(std::move(batch));
+    if (!maybe_projected.ok()) {
+      outputs_[0]->ErrorReceived(this, maybe_projected.status());
+      inputs_[0]->StopProducing(this);
+      return;
+    }
+
+    maybe_projected->guarantee = batch.guarantee;
+    outputs_[0]->InputReceived(this, seq, maybe_projected.MoveValueUnsafe());
+  }
+
+  void ErrorReceived(ExecNode* input, Status error) override {
+    DCHECK_EQ(input, inputs_[0]);
+    outputs_[0]->ErrorReceived(this, std::move(error));
+    inputs_[0]->StopProducing(this);
+  }
+
+  void InputFinished(ExecNode* input, int seq) override {
+    DCHECK_EQ(input, inputs_[0]);
+    outputs_[0]->InputFinished(this, seq);
+    inputs_[0]->StopProducing(this);
+  }
+
+  Status StartProducing() override {
+    // XXX validate inputs_[0]->output_descr() against filter_
+    return Status::OK();
+  }
+
+  void PauseProducing(ExecNode* output) override {}
+
+  void ResumeProducing(ExecNode* output) override {}
+
+  void StopProducing(ExecNode* output) override {
+    DCHECK_EQ(output, outputs_[0]);
+    inputs_[0]->StopProducing(this);
+  }
+
+  void StopProducing() override { StopProducing(outputs_[0]); }
+
+ private:
+  std::vector<Expression> exprs_;
+};
+
+ExecNode* MakeProjectNode(ExecNode* input, std::string label,
+                          std::vector<Expression> exprs) {
+  return input->plan()->EmplaceNode<ProjectNode>(input, std::move(label),
+                                                 std::move(exprs));
+}
+
+struct SinkNode : ExecNode {
+  SinkNode(ExecNode* input, std::string label,
+           AsyncGenerator<util::optional<ExecBatch>>* generator)
+      : ExecNode(input->plan(), std::move(label), {input}, {"collected"}, {},
+                 /*num_outputs=*/0),
+        producer_(MakeProducer(generator)) {}
+
+  static PushGenerator<util::optional<ExecBatch>>::Producer MakeProducer(
+      AsyncGenerator<util::optional<ExecBatch>>* out_gen) {
+    PushGenerator<util::optional<ExecBatch>> gen;
+    auto out = gen.producer();
+    *out_gen = std::move(gen);
+    return out;
+  }
+
+  const char* kind_name() override { return "SinkNode"; }
+
+  Status StartProducing() override { return Status::OK(); }
+
+  // sink nodes have no outputs from which to feel backpressure
+  static void NoOutputs() { DCHECK(false) << "no outputs; this should never be called"; }
+  void ResumeProducing(ExecNode* output) override { NoOutputs(); }
+  void PauseProducing(ExecNode* output) override { NoOutputs(); }
+  void StopProducing(ExecNode* output) override { NoOutputs(); }
+
+  void StopProducing() override {
+    std::unique_lock<std::mutex> lock(mutex_);
+    StopProducingUnlocked();
+  }
+
+  void InputReceived(ExecNode* input, int seq_num, ExecBatch exec_batch) override {
+    std::unique_lock<std::mutex> lock(mutex_);
+    if (stopped_) return;
+
+    // TODO would be nice to factor this out in a ReorderQueue
+    if (seq_num <= static_cast<int>(received_batches_.size())) {
+      received_batches_.resize(seq_num + 1);
+      emitted_.resize(seq_num + 1, false);
+    }
+    received_batches_[seq_num] = std::move(exec_batch);
+    ++num_received_;
+
+    if (seq_num != num_emitted_) {
+      // Cannot emit yet as there is a hole at `num_emitted_`
+      DCHECK_GT(seq_num, num_emitted_);
+      return;
+    }
+
+    if (num_received_ == emit_stop_) {
+      StopProducingUnlocked();
+    }
+
+    // Emit batches in order as far as possible
+    // First collect these batches, then unlock before producing.
+    const auto seq_start = seq_num;
+    while (seq_num < static_cast<int>(emitted_.size()) && !emitted_[seq_num]) {
+      emitted_[seq_num] = true;
+      ++seq_num;
+    }
+    DCHECK_GT(seq_num, seq_start);
+    // By moving the values now, we make sure another thread won't emit the same values
+    // below
+    std::vector<ExecBatch> to_emit(
+        std::make_move_iterator(received_batches_.begin() + seq_start),
+        std::make_move_iterator(received_batches_.begin() + seq_num));
+
+    lock.unlock();
+    for (auto&& batch : to_emit) {
+      producer_.Push(std::move(batch));
+    }
+    lock.lock();
+
+    DCHECK_EQ(seq_start, num_emitted_);  // num_emitted_ wasn't bumped in the meantime
+    num_emitted_ = seq_num;
+  }
+
+  void ErrorReceived(ExecNode* input, Status error) override {
+    // XXX do we care about properly sequencing the error?

Review comment:
       I'll remove this comment. (See also https://github.com/apache/arrow/pull/10397#discussion_r640799446 ). I think for now it's acceptable to consider an error in one node an error for all subsequent nodes, regardless of sequencing.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] westonpace commented on a change in pull request #10397: ARROW-11930: [C++][Dataset][Compute] Use an ExecPlan for dataset scans

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #10397:
URL: https://github.com/apache/arrow/pull/10397#discussion_r644127798



##########
File path: cpp/src/arrow/compute/exec/exec_plan.cc
##########
@@ -170,48 +165,409 @@ Status ExecPlan::Validate() { return ToDerived(this)->Validate(); }
 
 Status ExecPlan::StartProducing() { return ToDerived(this)->StartProducing(); }
 
-ExecNode::ExecNode(ExecPlan* plan, std::string label,
-                   std::vector<BatchDescr> input_descrs,
+ExecNode::ExecNode(ExecPlan* plan, std::string label, NodeVector inputs,
                    std::vector<std::string> input_labels, BatchDescr output_descr,
                    int num_outputs)
     : plan_(plan),
       label_(std::move(label)),
-      input_descrs_(std::move(input_descrs)),
+      inputs_(std::move(inputs)),
       input_labels_(std::move(input_labels)),
       output_descr_(std::move(output_descr)),
-      num_outputs_(num_outputs) {}
+      num_outputs_(num_outputs) {
+  for (auto input : inputs_) {
+    input->outputs_.push_back(this);
+  }
+}
 
 Status ExecNode::Validate() const {
-  if (inputs_.size() != input_descrs_.size()) {
+  if (inputs_.size() != input_labels_.size()) {
     return Status::Invalid("Invalid number of inputs for '", label(), "' (expected ",
-                           num_inputs(), ", actual ", inputs_.size(), ")");
+                           num_inputs(), ", actual ", input_labels_.size(), ")");
   }
 
   if (static_cast<int>(outputs_.size()) != num_outputs_) {
     return Status::Invalid("Invalid number of outputs for '", label(), "' (expected ",
                            num_outputs(), ", actual ", outputs_.size(), ")");
   }
 
-  DCHECK_EQ(input_descrs_.size(), input_labels_.size());
-
   for (auto out : outputs_) {
     auto input_index = GetNodeIndex(out->inputs(), this);
     if (!input_index) {
       return Status::Invalid("Node '", label(), "' outputs to node '", out->label(),
                              "' but is not listed as an input.");
     }
+  }
 
-    const auto& in_descr = out->input_descrs_[*input_index];
-    if (in_descr != output_descr_) {
-      return Status::Invalid(
-          "Node '", label(), "' (bound to input ", input_labels_[*input_index],
-          ") produces batches with type '", ValueDescr::ToString(output_descr_),
-          "' inconsistent with consumer '", out->label(), "' which accepts '",
-          ValueDescr::ToString(in_descr), "'");
+  return Status::OK();
+}
+
+struct SourceNode : ExecNode {
+  SourceNode(ExecPlan* plan, std::string label, ExecNode::BatchDescr output_descr,
+             AsyncGenerator<util::optional<ExecBatch>> generator)
+      : ExecNode(plan, std::move(label), {}, {}, std::move(output_descr),
+                 /*num_outputs=*/1),
+        generator_(std::move(generator)) {}
+
+  const char* kind_name() override { return "SourceNode"; }
+
+  static void NoInputs() { DCHECK(false) << "no inputs; this should never be called"; }
+  void InputReceived(ExecNode*, int, ExecBatch) override { NoInputs(); }
+  void ErrorReceived(ExecNode*, Status) override { NoInputs(); }
+  void InputFinished(ExecNode*, int) override { NoInputs(); }
+
+  Status StartProducing() override {
+    if (finished_) {
+      return Status::Invalid("Restarted SourceNode '", label(), "'");
     }
+
+    auto gen = std::move(generator_);
+
+    /// XXX should we wait on this future anywhere? In StopProducing() maybe?
+    auto done_fut =
+        Loop([gen, this] {
+          std::unique_lock<std::mutex> lock(mutex_);
+          int seq = next_batch_index_++;
+          if (finished_) {
+            return Future<ControlFlow<int>>::MakeFinished(Break(seq));
+          }
+          lock.unlock();
+
+          return gen().Then(
+              [=](const util::optional<ExecBatch>& batch) -> ControlFlow<int> {
+                std::unique_lock<std::mutex> lock(mutex_);
+                if (!batch || finished_) {
+                  finished_ = true;
+                  return Break(seq);
+                }
+                lock.unlock();
+
+                outputs_[0]->InputReceived(this, seq, *batch);
+                return Continue();
+              },
+              [=](const Status& error) -> ControlFlow<int> {
+                std::unique_lock<std::mutex> lock(mutex_);
+                if (!finished_) {
+                  finished_ = true;
+                  lock.unlock();
+                  // unless we were already finished, push the error to our output
+                  // XXX is this correct? Is it reasonable for a consumer to ignore errors
+                  // from a finished producer?
+                  outputs_[0]->ErrorReceived(this, error);
+                }
+                return Break(seq);
+              });
+        }).Then([&](int seq) {
+          /// XXX this is probably redundant: do we always call InputFinished after
+          /// ErrorReceived or will ErrorRecieved be sufficient?
+          outputs_[0]->InputFinished(this, seq);
+        });
+
+    return Status::OK();
   }
 
-  return Status::OK();
+  void PauseProducing(ExecNode* output) override {}
+
+  void ResumeProducing(ExecNode* output) override {}
+
+  void StopProducing(ExecNode* output) override {
+    DCHECK_EQ(output, outputs_[0]);
+    std::unique_lock<std::mutex> lock(mutex_);
+    finished_ = true;
+  }
+
+  void StopProducing() override { StopProducing(outputs_[0]); }
+
+ private:
+  std::mutex mutex_;
+  bool finished_{false};
+  int next_batch_index_{0};
+  AsyncGenerator<util::optional<ExecBatch>> generator_;
+};
+
+ExecNode* MakeSourceNode(ExecPlan* plan, std::string label,
+                         ExecNode::BatchDescr output_descr,
+                         AsyncGenerator<util::optional<ExecBatch>> generator) {
+  return plan->EmplaceNode<SourceNode>(plan, std::move(label), std::move(output_descr),
+                                       std::move(generator));
+}
+
+struct FilterNode : ExecNode {
+  FilterNode(ExecNode* input, std::string label, Expression filter)
+      : ExecNode(input->plan(), std::move(label), {input}, {"target"},
+                 /*output_descr=*/{input->output_descr()},
+                 /*num_outputs=*/1),
+        filter_(std::move(filter)) {}
+
+  const char* kind_name() override { return "FilterNode"; }
+
+  Result<ExecBatch> DoFilter(const ExecBatch& target) {
+    ARROW_ASSIGN_OR_RAISE(Expression simplified_filter,
+                          SimplifyWithGuarantee(filter_, target.guarantee));
+
+    // XXX get a non-default exec context
+    ARROW_ASSIGN_OR_RAISE(Datum mask, ExecuteScalarExpression(simplified_filter, target));
+
+    if (mask.is_scalar()) {
+      const auto& mask_scalar = mask.scalar_as<BooleanScalar>();
+      if (mask_scalar.is_valid && mask_scalar.value) {
+        return target;
+      }
+
+      return target.Slice(0, 0);
+    }
+
+    auto values = target.values;
+    for (auto& value : values) {
+      if (value.is_scalar()) continue;
+      ARROW_ASSIGN_OR_RAISE(value, Filter(value, mask, FilterOptions::Defaults()));
+    }
+    return ExecBatch::Make(std::move(values));
+  }
+
+  void InputReceived(ExecNode* input, int seq, ExecBatch batch) override {
+    DCHECK_EQ(input, inputs_[0]);
+
+    auto maybe_filtered = DoFilter(std::move(batch));
+    if (!maybe_filtered.ok()) {
+      outputs_[0]->ErrorReceived(this, maybe_filtered.status());
+      inputs_[0]->StopProducing(this);
+      return;
+    }
+
+    maybe_filtered->guarantee = batch.guarantee;
+    outputs_[0]->InputReceived(this, seq, maybe_filtered.MoveValueUnsafe());
+  }
+
+  void ErrorReceived(ExecNode* input, Status error) override {
+    DCHECK_EQ(input, inputs_[0]);
+    outputs_[0]->ErrorReceived(this, std::move(error));
+    inputs_[0]->StopProducing(this);
+  }
+
+  void InputFinished(ExecNode* input, int seq) override {
+    DCHECK_EQ(input, inputs_[0]);
+    outputs_[0]->InputFinished(this, seq);
+    inputs_[0]->StopProducing(this);
+  }
+
+  Status StartProducing() override {
+    // XXX validate inputs_[0]->output_descr() against filter_
+    return Status::OK();
+  }
+
+  void PauseProducing(ExecNode* output) override {}
+
+  void ResumeProducing(ExecNode* output) override {}
+
+  void StopProducing(ExecNode* output) override {
+    DCHECK_EQ(output, outputs_[0]);
+    inputs_[0]->StopProducing(this);
+  }
+
+  void StopProducing() override { StopProducing(outputs_[0]); }
+
+ private:
+  Expression filter_;
+};
+
+ExecNode* MakeFilterNode(ExecNode* input, std::string label, Expression filter) {
+  return input->plan()->EmplaceNode<FilterNode>(input, std::move(label),
+                                                std::move(filter));
+}
+
+struct ProjectNode : ExecNode {
+  ProjectNode(ExecNode* input, std::string label, std::vector<Expression> exprs)
+      : ExecNode(input->plan(), std::move(label), {input}, {"target"},
+                 /*output_descr=*/{input->output_descr()},
+                 /*num_outputs=*/1),
+        exprs_(std::move(exprs)) {}
+
+  const char* kind_name() override { return "ProjectNode"; }
+
+  Result<ExecBatch> DoProject(const ExecBatch& target) {
+    // XXX get a non-default exec context
+    std::vector<Datum> values{exprs_.size()};
+    for (size_t i = 0; i < exprs_.size(); ++i) {
+      ARROW_ASSIGN_OR_RAISE(Expression simplified_expr,
+                            SimplifyWithGuarantee(exprs_[i], target.guarantee));
+
+      ARROW_ASSIGN_OR_RAISE(values[i], ExecuteScalarExpression(simplified_expr, target));
+    }
+    return ExecBatch::Make(std::move(values));
+  }
+
+  void InputReceived(ExecNode* input, int seq, ExecBatch batch) override {
+    DCHECK_EQ(input, inputs_[0]);
+
+    auto maybe_projected = DoProject(std::move(batch));
+    if (!maybe_projected.ok()) {
+      outputs_[0]->ErrorReceived(this, maybe_projected.status());
+      inputs_[0]->StopProducing(this);
+      return;
+    }
+
+    maybe_projected->guarantee = batch.guarantee;
+    outputs_[0]->InputReceived(this, seq, maybe_projected.MoveValueUnsafe());
+  }
+
+  void ErrorReceived(ExecNode* input, Status error) override {
+    DCHECK_EQ(input, inputs_[0]);
+    outputs_[0]->ErrorReceived(this, std::move(error));
+    inputs_[0]->StopProducing(this);
+  }
+
+  void InputFinished(ExecNode* input, int seq) override {
+    DCHECK_EQ(input, inputs_[0]);
+    outputs_[0]->InputFinished(this, seq);
+    inputs_[0]->StopProducing(this);
+  }
+
+  Status StartProducing() override {
+    // XXX validate inputs_[0]->output_descr() against filter_
+    return Status::OK();
+  }
+
+  void PauseProducing(ExecNode* output) override {}
+
+  void ResumeProducing(ExecNode* output) override {}
+
+  void StopProducing(ExecNode* output) override {
+    DCHECK_EQ(output, outputs_[0]);
+    inputs_[0]->StopProducing(this);
+  }
+
+  void StopProducing() override { StopProducing(outputs_[0]); }
+
+ private:
+  std::vector<Expression> exprs_;
+};
+
+ExecNode* MakeProjectNode(ExecNode* input, std::string label,
+                          std::vector<Expression> exprs) {
+  return input->plan()->EmplaceNode<ProjectNode>(input, std::move(label),
+                                                 std::move(exprs));
+}
+
+struct SinkNode : ExecNode {
+  SinkNode(ExecNode* input, std::string label,
+           AsyncGenerator<util::optional<ExecBatch>>* generator)
+      : ExecNode(input->plan(), std::move(label), {input}, {"collected"}, {},
+                 /*num_outputs=*/0),
+        producer_(MakeProducer(generator)) {}
+
+  static PushGenerator<util::optional<ExecBatch>>::Producer MakeProducer(
+      AsyncGenerator<util::optional<ExecBatch>>* out_gen) {
+    PushGenerator<util::optional<ExecBatch>> gen;
+    auto out = gen.producer();
+    *out_gen = std::move(gen);
+    return out;
+  }
+
+  const char* kind_name() override { return "SinkNode"; }
+
+  Status StartProducing() override { return Status::OK(); }
+
+  // sink nodes have no outputs from which to feel backpressure
+  static void NoOutputs() { DCHECK(false) << "no outputs; this should never be called"; }
+  void ResumeProducing(ExecNode* output) override { NoOutputs(); }
+  void PauseProducing(ExecNode* output) override { NoOutputs(); }
+  void StopProducing(ExecNode* output) override { NoOutputs(); }
+
+  void StopProducing() override {
+    std::unique_lock<std::mutex> lock(mutex_);
+    StopProducingUnlocked();
+  }
+
+  void InputReceived(ExecNode* input, int seq_num, ExecBatch exec_batch) override {
+    std::unique_lock<std::mutex> lock(mutex_);
+    if (stopped_) return;
+
+    // TODO would be nice to factor this out in a ReorderQueue
+    if (seq_num <= static_cast<int>(received_batches_.size())) {
+      received_batches_.resize(seq_num + 1);
+      emitted_.resize(seq_num + 1, false);
+    }
+    received_batches_[seq_num] = std::move(exec_batch);
+    ++num_received_;
+
+    if (seq_num != num_emitted_) {
+      // Cannot emit yet as there is a hole at `num_emitted_`
+      DCHECK_GT(seq_num, num_emitted_);
+      return;
+    }
+
+    if (num_received_ == emit_stop_) {
+      StopProducingUnlocked();
+    }
+
+    // Emit batches in order as far as possible
+    // First collect these batches, then unlock before producing.
+    const auto seq_start = seq_num;
+    while (seq_num < static_cast<int>(emitted_.size()) && !emitted_[seq_num]) {
+      emitted_[seq_num] = true;
+      ++seq_num;
+    }
+    DCHECK_GT(seq_num, seq_start);
+    // By moving the values now, we make sure another thread won't emit the same values
+    // below
+    std::vector<ExecBatch> to_emit(
+        std::make_move_iterator(received_batches_.begin() + seq_start),
+        std::make_move_iterator(received_batches_.begin() + seq_num));
+
+    lock.unlock();
+    for (auto&& batch : to_emit) {
+      producer_.Push(std::move(batch));
+    }
+    lock.lock();
+
+    DCHECK_EQ(seq_start, num_emitted_);  // num_emitted_ wasn't bumped in the meantime
+    num_emitted_ = seq_num;

Review comment:
       I suspect the problem with putting this statement inside the lock is that (as an example) a future thread could come in while the first thread is emitting items 10 through 20 and then decide to emit item 21 before the previous thread finished.  The sequencing generator is a good workaround as it would apply the sequencing at pull time and not at push time and the generator rules prevent two threads being in that logic at once.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] bkietz commented on a change in pull request #10397: ARROW-11930: [C++][Dataset][Compute] Use an ExecPlan for dataset scans

Posted by GitBox <gi...@apache.org>.
bkietz commented on a change in pull request #10397:
URL: https://github.com/apache/arrow/pull/10397#discussion_r643447263



##########
File path: cpp/src/arrow/compute/exec/expression.cc
##########
@@ -510,7 +475,67 @@ Result<Expression> Expression::Bind(const Schema& in_schema,
   return Bind(ValueDescr::Array(struct_(in_schema.fields())), exec_context);
 }
 
-Result<Datum> ExecuteScalarExpression(const Expression& expr, const Datum& input,
+Result<ExecBatch> MakeExecBatch(const Schema& full_schema, const Datum& partial) {
+  ExecBatch out;
+
+  if (partial.kind() == Datum::RECORD_BATCH) {
+    const auto& partial_batch = *partial.record_batch();
+    out.length = partial_batch.num_rows();
+
+    for (const auto& field : full_schema.fields()) {
+      ARROW_ASSIGN_OR_RAISE(auto column,
+                            FieldRef(field->name()).GetOneOrNone(partial_batch));

Review comment:
       It will




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] westonpace commented on a change in pull request #10397: ARROW-11930: [C++][Dataset][Compute] Use an ExecPlan for dataset scans

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #10397:
URL: https://github.com/apache/arrow/pull/10397#discussion_r644175511



##########
File path: cpp/src/arrow/compute/exec/exec_plan.cc
##########
@@ -170,48 +165,409 @@ Status ExecPlan::Validate() { return ToDerived(this)->Validate(); }
 
 Status ExecPlan::StartProducing() { return ToDerived(this)->StartProducing(); }
 
-ExecNode::ExecNode(ExecPlan* plan, std::string label,
-                   std::vector<BatchDescr> input_descrs,
+ExecNode::ExecNode(ExecPlan* plan, std::string label, NodeVector inputs,
                    std::vector<std::string> input_labels, BatchDescr output_descr,
                    int num_outputs)
     : plan_(plan),
       label_(std::move(label)),
-      input_descrs_(std::move(input_descrs)),
+      inputs_(std::move(inputs)),
       input_labels_(std::move(input_labels)),
       output_descr_(std::move(output_descr)),
-      num_outputs_(num_outputs) {}
+      num_outputs_(num_outputs) {
+  for (auto input : inputs_) {
+    input->outputs_.push_back(this);
+  }
+}
 
 Status ExecNode::Validate() const {
-  if (inputs_.size() != input_descrs_.size()) {
+  if (inputs_.size() != input_labels_.size()) {
     return Status::Invalid("Invalid number of inputs for '", label(), "' (expected ",
-                           num_inputs(), ", actual ", inputs_.size(), ")");
+                           num_inputs(), ", actual ", input_labels_.size(), ")");
   }
 
   if (static_cast<int>(outputs_.size()) != num_outputs_) {
     return Status::Invalid("Invalid number of outputs for '", label(), "' (expected ",
                            num_outputs(), ", actual ", outputs_.size(), ")");
   }
 
-  DCHECK_EQ(input_descrs_.size(), input_labels_.size());
-
   for (auto out : outputs_) {
     auto input_index = GetNodeIndex(out->inputs(), this);
     if (!input_index) {
       return Status::Invalid("Node '", label(), "' outputs to node '", out->label(),
                              "' but is not listed as an input.");
     }
+  }
 
-    const auto& in_descr = out->input_descrs_[*input_index];
-    if (in_descr != output_descr_) {
-      return Status::Invalid(
-          "Node '", label(), "' (bound to input ", input_labels_[*input_index],
-          ") produces batches with type '", ValueDescr::ToString(output_descr_),
-          "' inconsistent with consumer '", out->label(), "' which accepts '",
-          ValueDescr::ToString(in_descr), "'");
+  return Status::OK();
+}
+
+struct SourceNode : ExecNode {
+  SourceNode(ExecPlan* plan, std::string label, ExecNode::BatchDescr output_descr,
+             AsyncGenerator<util::optional<ExecBatch>> generator)
+      : ExecNode(plan, std::move(label), {}, {}, std::move(output_descr),
+                 /*num_outputs=*/1),
+        generator_(std::move(generator)) {}
+
+  const char* kind_name() override { return "SourceNode"; }
+
+  static void NoInputs() { DCHECK(false) << "no inputs; this should never be called"; }
+  void InputReceived(ExecNode*, int, ExecBatch) override { NoInputs(); }
+  void ErrorReceived(ExecNode*, Status) override { NoInputs(); }
+  void InputFinished(ExecNode*, int) override { NoInputs(); }
+
+  Status StartProducing() override {
+    if (finished_) {
+      return Status::Invalid("Restarted SourceNode '", label(), "'");
     }
+
+    auto gen = std::move(generator_);
+
+    /// XXX should we wait on this future anywhere? In StopProducing() maybe?
+    auto done_fut =
+        Loop([gen, this] {
+          std::unique_lock<std::mutex> lock(mutex_);
+          int seq = next_batch_index_++;
+          if (finished_) {
+            return Future<ControlFlow<int>>::MakeFinished(Break(seq));
+          }
+          lock.unlock();
+
+          return gen().Then(
+              [=](const util::optional<ExecBatch>& batch) -> ControlFlow<int> {
+                std::unique_lock<std::mutex> lock(mutex_);
+                if (!batch || finished_) {
+                  finished_ = true;
+                  return Break(seq);
+                }
+                lock.unlock();
+
+                outputs_[0]->InputReceived(this, seq, *batch);
+                return Continue();
+              },
+              [=](const Status& error) -> ControlFlow<int> {
+                std::unique_lock<std::mutex> lock(mutex_);
+                if (!finished_) {
+                  finished_ = true;
+                  lock.unlock();
+                  // unless we were already finished, push the error to our output
+                  // XXX is this correct? Is it reasonable for a consumer to ignore errors
+                  // from a finished producer?
+                  outputs_[0]->ErrorReceived(this, error);
+                }
+                return Break(seq);
+              });
+        }).Then([&](int seq) {
+          /// XXX this is probably redundant: do we always call InputFinished after
+          /// ErrorReceived or will ErrorRecieved be sufficient?
+          outputs_[0]->InputFinished(this, seq);
+        });
+
+    return Status::OK();
   }
 
-  return Status::OK();
+  void PauseProducing(ExecNode* output) override {}
+
+  void ResumeProducing(ExecNode* output) override {}
+
+  void StopProducing(ExecNode* output) override {
+    DCHECK_EQ(output, outputs_[0]);
+    std::unique_lock<std::mutex> lock(mutex_);
+    finished_ = true;
+  }
+
+  void StopProducing() override { StopProducing(outputs_[0]); }
+
+ private:
+  std::mutex mutex_;
+  bool finished_{false};
+  int next_batch_index_{0};
+  AsyncGenerator<util::optional<ExecBatch>> generator_;
+};
+
+ExecNode* MakeSourceNode(ExecPlan* plan, std::string label,
+                         ExecNode::BatchDescr output_descr,
+                         AsyncGenerator<util::optional<ExecBatch>> generator) {
+  return plan->EmplaceNode<SourceNode>(plan, std::move(label), std::move(output_descr),
+                                       std::move(generator));
+}
+
+struct FilterNode : ExecNode {
+  FilterNode(ExecNode* input, std::string label, Expression filter)
+      : ExecNode(input->plan(), std::move(label), {input}, {"target"},
+                 /*output_descr=*/{input->output_descr()},
+                 /*num_outputs=*/1),
+        filter_(std::move(filter)) {}
+
+  const char* kind_name() override { return "FilterNode"; }
+
+  Result<ExecBatch> DoFilter(const ExecBatch& target) {
+    ARROW_ASSIGN_OR_RAISE(Expression simplified_filter,
+                          SimplifyWithGuarantee(filter_, target.guarantee));
+
+    // XXX get a non-default exec context
+    ARROW_ASSIGN_OR_RAISE(Datum mask, ExecuteScalarExpression(simplified_filter, target));
+
+    if (mask.is_scalar()) {
+      const auto& mask_scalar = mask.scalar_as<BooleanScalar>();
+      if (mask_scalar.is_valid && mask_scalar.value) {
+        return target;
+      }
+
+      return target.Slice(0, 0);
+    }
+
+    auto values = target.values;
+    for (auto& value : values) {
+      if (value.is_scalar()) continue;
+      ARROW_ASSIGN_OR_RAISE(value, Filter(value, mask, FilterOptions::Defaults()));
+    }
+    return ExecBatch::Make(std::move(values));
+  }
+
+  void InputReceived(ExecNode* input, int seq, ExecBatch batch) override {
+    DCHECK_EQ(input, inputs_[0]);
+
+    auto maybe_filtered = DoFilter(std::move(batch));
+    if (!maybe_filtered.ok()) {
+      outputs_[0]->ErrorReceived(this, maybe_filtered.status());
+      inputs_[0]->StopProducing(this);
+      return;
+    }
+
+    maybe_filtered->guarantee = batch.guarantee;
+    outputs_[0]->InputReceived(this, seq, maybe_filtered.MoveValueUnsafe());
+  }
+
+  void ErrorReceived(ExecNode* input, Status error) override {
+    DCHECK_EQ(input, inputs_[0]);
+    outputs_[0]->ErrorReceived(this, std::move(error));
+    inputs_[0]->StopProducing(this);
+  }
+
+  void InputFinished(ExecNode* input, int seq) override {
+    DCHECK_EQ(input, inputs_[0]);
+    outputs_[0]->InputFinished(this, seq);
+    inputs_[0]->StopProducing(this);
+  }
+
+  Status StartProducing() override {
+    // XXX validate inputs_[0]->output_descr() against filter_
+    return Status::OK();
+  }
+
+  void PauseProducing(ExecNode* output) override {}
+
+  void ResumeProducing(ExecNode* output) override {}
+
+  void StopProducing(ExecNode* output) override {
+    DCHECK_EQ(output, outputs_[0]);
+    inputs_[0]->StopProducing(this);
+  }
+
+  void StopProducing() override { StopProducing(outputs_[0]); }
+
+ private:
+  Expression filter_;
+};
+
+ExecNode* MakeFilterNode(ExecNode* input, std::string label, Expression filter) {
+  return input->plan()->EmplaceNode<FilterNode>(input, std::move(label),
+                                                std::move(filter));
+}
+
+struct ProjectNode : ExecNode {
+  ProjectNode(ExecNode* input, std::string label, std::vector<Expression> exprs)
+      : ExecNode(input->plan(), std::move(label), {input}, {"target"},
+                 /*output_descr=*/{input->output_descr()},
+                 /*num_outputs=*/1),
+        exprs_(std::move(exprs)) {}
+
+  const char* kind_name() override { return "ProjectNode"; }
+
+  Result<ExecBatch> DoProject(const ExecBatch& target) {
+    // XXX get a non-default exec context
+    std::vector<Datum> values{exprs_.size()};
+    for (size_t i = 0; i < exprs_.size(); ++i) {
+      ARROW_ASSIGN_OR_RAISE(Expression simplified_expr,
+                            SimplifyWithGuarantee(exprs_[i], target.guarantee));
+
+      ARROW_ASSIGN_OR_RAISE(values[i], ExecuteScalarExpression(simplified_expr, target));
+    }
+    return ExecBatch::Make(std::move(values));
+  }
+
+  void InputReceived(ExecNode* input, int seq, ExecBatch batch) override {
+    DCHECK_EQ(input, inputs_[0]);
+
+    auto maybe_projected = DoProject(std::move(batch));
+    if (!maybe_projected.ok()) {
+      outputs_[0]->ErrorReceived(this, maybe_projected.status());
+      inputs_[0]->StopProducing(this);
+      return;
+    }
+
+    maybe_projected->guarantee = batch.guarantee;
+    outputs_[0]->InputReceived(this, seq, maybe_projected.MoveValueUnsafe());
+  }
+
+  void ErrorReceived(ExecNode* input, Status error) override {
+    DCHECK_EQ(input, inputs_[0]);
+    outputs_[0]->ErrorReceived(this, std::move(error));
+    inputs_[0]->StopProducing(this);
+  }
+
+  void InputFinished(ExecNode* input, int seq) override {
+    DCHECK_EQ(input, inputs_[0]);
+    outputs_[0]->InputFinished(this, seq);
+    inputs_[0]->StopProducing(this);
+  }
+
+  Status StartProducing() override {
+    // XXX validate inputs_[0]->output_descr() against filter_
+    return Status::OK();
+  }
+
+  void PauseProducing(ExecNode* output) override {}
+
+  void ResumeProducing(ExecNode* output) override {}
+
+  void StopProducing(ExecNode* output) override {
+    DCHECK_EQ(output, outputs_[0]);
+    inputs_[0]->StopProducing(this);
+  }
+
+  void StopProducing() override { StopProducing(outputs_[0]); }
+
+ private:
+  std::vector<Expression> exprs_;
+};
+
+ExecNode* MakeProjectNode(ExecNode* input, std::string label,
+                          std::vector<Expression> exprs) {
+  return input->plan()->EmplaceNode<ProjectNode>(input, std::move(label),
+                                                 std::move(exprs));
+}
+
+struct SinkNode : ExecNode {

Review comment:
       I'm pondering how back pressure would be applied.  I think there would be a new argument added to this `SinkNode` for `max_items_queued` or something like that.  However, we could not naively apply that limit to `received_batches_` because of the resequencing.
   
   Since we are delivering to a pull-based model I think the appropriate way to apply back pressure would be to have the `PushGenerator` keep track of how many undelivered items it has.  Then there would need to be a check in this code and, after pushing, if the `PushGenerator` is full, then apply back pressure to the inputs.  The `PushGenerator` would also need some way of signalling back into the `SinkNode` that the pressure has been relieved and it is ready for more items.
   
   I don't think this has to be implemented now, but does that sound reasonable?




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] bkietz commented on a change in pull request #10397: ARROW-11930: [C++][Dataset][Compute] Use an ExecPlan for dataset scans

Posted by GitBox <gi...@apache.org>.
bkietz commented on a change in pull request #10397:
URL: https://github.com/apache/arrow/pull/10397#discussion_r643575745



##########
File path: cpp/src/arrow/dataset/dataset_internal.h
##########
@@ -204,5 +204,35 @@ arrow::Result<std::shared_ptr<T>> GetFragmentScanOptions(
   return internal::checked_pointer_cast<T>(source);
 }
 
+class FragmentDataset : public Dataset {

Review comment:
       Whether or not the component fragments are in memory is up to the fragments, so I don't think it's appropriate to inherit that 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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] westonpace commented on a change in pull request #10397: ARROW-11930: [C++][Dataset][Compute] Use an ExecPlan for dataset scans

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #10397:
URL: https://github.com/apache/arrow/pull/10397#discussion_r644118992



##########
File path: cpp/src/arrow/compute/exec/exec_plan.cc
##########
@@ -170,48 +165,409 @@ Status ExecPlan::Validate() { return ToDerived(this)->Validate(); }
 
 Status ExecPlan::StartProducing() { return ToDerived(this)->StartProducing(); }
 
-ExecNode::ExecNode(ExecPlan* plan, std::string label,
-                   std::vector<BatchDescr> input_descrs,
+ExecNode::ExecNode(ExecPlan* plan, std::string label, NodeVector inputs,
                    std::vector<std::string> input_labels, BatchDescr output_descr,
                    int num_outputs)
     : plan_(plan),
       label_(std::move(label)),
-      input_descrs_(std::move(input_descrs)),
+      inputs_(std::move(inputs)),
       input_labels_(std::move(input_labels)),
       output_descr_(std::move(output_descr)),
-      num_outputs_(num_outputs) {}
+      num_outputs_(num_outputs) {
+  for (auto input : inputs_) {
+    input->outputs_.push_back(this);
+  }
+}
 
 Status ExecNode::Validate() const {
-  if (inputs_.size() != input_descrs_.size()) {
+  if (inputs_.size() != input_labels_.size()) {
     return Status::Invalid("Invalid number of inputs for '", label(), "' (expected ",
-                           num_inputs(), ", actual ", inputs_.size(), ")");
+                           num_inputs(), ", actual ", input_labels_.size(), ")");
   }
 
   if (static_cast<int>(outputs_.size()) != num_outputs_) {
     return Status::Invalid("Invalid number of outputs for '", label(), "' (expected ",
                            num_outputs(), ", actual ", outputs_.size(), ")");
   }
 
-  DCHECK_EQ(input_descrs_.size(), input_labels_.size());
-
   for (auto out : outputs_) {
     auto input_index = GetNodeIndex(out->inputs(), this);
     if (!input_index) {
       return Status::Invalid("Node '", label(), "' outputs to node '", out->label(),
                              "' but is not listed as an input.");
     }
+  }
 
-    const auto& in_descr = out->input_descrs_[*input_index];
-    if (in_descr != output_descr_) {
-      return Status::Invalid(
-          "Node '", label(), "' (bound to input ", input_labels_[*input_index],
-          ") produces batches with type '", ValueDescr::ToString(output_descr_),
-          "' inconsistent with consumer '", out->label(), "' which accepts '",
-          ValueDescr::ToString(in_descr), "'");
+  return Status::OK();
+}
+
+struct SourceNode : ExecNode {
+  SourceNode(ExecPlan* plan, std::string label, ExecNode::BatchDescr output_descr,
+             AsyncGenerator<util::optional<ExecBatch>> generator)
+      : ExecNode(plan, std::move(label), {}, {}, std::move(output_descr),
+                 /*num_outputs=*/1),
+        generator_(std::move(generator)) {}
+
+  const char* kind_name() override { return "SourceNode"; }
+
+  static void NoInputs() { DCHECK(false) << "no inputs; this should never be called"; }
+  void InputReceived(ExecNode*, int, ExecBatch) override { NoInputs(); }
+  void ErrorReceived(ExecNode*, Status) override { NoInputs(); }
+  void InputFinished(ExecNode*, int) override { NoInputs(); }
+
+  Status StartProducing() override {
+    if (finished_) {
+      return Status::Invalid("Restarted SourceNode '", label(), "'");
     }
+
+    auto gen = std::move(generator_);
+
+    /// XXX should we wait on this future anywhere? In StopProducing() maybe?
+    auto done_fut =
+        Loop([gen, this] {
+          std::unique_lock<std::mutex> lock(mutex_);
+          int seq = next_batch_index_++;
+          if (finished_) {
+            return Future<ControlFlow<int>>::MakeFinished(Break(seq));
+          }
+          lock.unlock();
+
+          return gen().Then(
+              [=](const util::optional<ExecBatch>& batch) -> ControlFlow<int> {
+                std::unique_lock<std::mutex> lock(mutex_);
+                if (!batch || finished_) {
+                  finished_ = true;
+                  return Break(seq);
+                }
+                lock.unlock();
+
+                outputs_[0]->InputReceived(this, seq, *batch);
+                return Continue();
+              },
+              [=](const Status& error) -> ControlFlow<int> {
+                std::unique_lock<std::mutex> lock(mutex_);
+                if (!finished_) {
+                  finished_ = true;
+                  lock.unlock();
+                  // unless we were already finished, push the error to our output
+                  // XXX is this correct? Is it reasonable for a consumer to ignore errors
+                  // from a finished producer?
+                  outputs_[0]->ErrorReceived(this, error);
+                }
+                return Break(seq);
+              });
+        }).Then([&](int seq) {
+          /// XXX this is probably redundant: do we always call InputFinished after
+          /// ErrorReceived or will ErrorRecieved be sufficient?
+          outputs_[0]->InputFinished(this, seq);
+        });
+
+    return Status::OK();
   }
 
-  return Status::OK();
+  void PauseProducing(ExecNode* output) override {}
+
+  void ResumeProducing(ExecNode* output) override {}

Review comment:
       Pull-based models (e.g. generators) apply backpressure by default, you have to specifically ask for each item of data that you want.  Although it's turned around so maybe it's right to just call it pressure.  If we want to apply it here it could be done by adding a flag in the loop (perhaps near `if (finished_)`) that looks something like...
   
   ```
   if (pause_future_) {
     return pause_future_;
   }
   ```
   
   Then `PauseProducing` becomes:
   
   ```
   pause_future_ = Future<>::Make();
   ```
   
   and `ResumeProducing` becomes:
   
   ```
   pause_future_.MarkFinished();
   pause_future_ = Future<>(); // Maybe we need a `Reset` or `MakeInvalid`
   ```




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] bkietz commented on a change in pull request #10397: ARROW-11930: [C++][Dataset][Compute] Use an ExecPlan for dataset scans

Posted by GitBox <gi...@apache.org>.
bkietz commented on a change in pull request #10397:
URL: https://github.com/apache/arrow/pull/10397#discussion_r640897501



##########
File path: cpp/src/arrow/compute/exec/expression.cc
##########
@@ -492,9 +448,18 @@ Result<Expression> Expression::Bind(ValueDescr in,
   if (literal()) return *this;
 
   if (auto ref = field_ref()) {
-    ARROW_ASSIGN_OR_RAISE(auto field, ref->GetOneOrNone(*in.type));
-    auto descr = field ? ValueDescr{field->type(), in.shape} : ValueDescr::Scalar(null());
-    return Expression{Parameter{*ref, std::move(descr)}};
+    if (ref->IsNested()) {
+      return Status::NotImplemented("nested field references");
+    }
+
+    ARROW_ASSIGN_OR_RAISE(auto path, ref->FindOne(*in.type));

Review comment:
       Extracting indices is necessitated by working with ExecBatches, which don't have named fields (so by-name lookup can't work). What quadratic behavior do you see?




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] lidavidm commented on a change in pull request #10397: ARROW-11930: [C++][Dataset][Compute] Use an ExecPlan for dataset scans

Posted by GitBox <gi...@apache.org>.
lidavidm commented on a change in pull request #10397:
URL: https://github.com/apache/arrow/pull/10397#discussion_r640799446



##########
File path: cpp/src/arrow/compute/exec/exec_plan.cc
##########
@@ -170,48 +165,409 @@ Status ExecPlan::Validate() { return ToDerived(this)->Validate(); }
 
 Status ExecPlan::StartProducing() { return ToDerived(this)->StartProducing(); }
 
-ExecNode::ExecNode(ExecPlan* plan, std::string label,
-                   std::vector<BatchDescr> input_descrs,
+ExecNode::ExecNode(ExecPlan* plan, std::string label, NodeVector inputs,
                    std::vector<std::string> input_labels, BatchDescr output_descr,
                    int num_outputs)
     : plan_(plan),
       label_(std::move(label)),
-      input_descrs_(std::move(input_descrs)),
+      inputs_(std::move(inputs)),
       input_labels_(std::move(input_labels)),
       output_descr_(std::move(output_descr)),
-      num_outputs_(num_outputs) {}
+      num_outputs_(num_outputs) {
+  for (auto input : inputs_) {
+    input->outputs_.push_back(this);
+  }
+}
 
 Status ExecNode::Validate() const {
-  if (inputs_.size() != input_descrs_.size()) {
+  if (inputs_.size() != input_labels_.size()) {
     return Status::Invalid("Invalid number of inputs for '", label(), "' (expected ",
-                           num_inputs(), ", actual ", inputs_.size(), ")");
+                           num_inputs(), ", actual ", input_labels_.size(), ")");
   }
 
   if (static_cast<int>(outputs_.size()) != num_outputs_) {
     return Status::Invalid("Invalid number of outputs for '", label(), "' (expected ",
                            num_outputs(), ", actual ", outputs_.size(), ")");
   }
 
-  DCHECK_EQ(input_descrs_.size(), input_labels_.size());
-
   for (auto out : outputs_) {
     auto input_index = GetNodeIndex(out->inputs(), this);
     if (!input_index) {
       return Status::Invalid("Node '", label(), "' outputs to node '", out->label(),
                              "' but is not listed as an input.");
     }
+  }
 
-    const auto& in_descr = out->input_descrs_[*input_index];
-    if (in_descr != output_descr_) {
-      return Status::Invalid(
-          "Node '", label(), "' (bound to input ", input_labels_[*input_index],
-          ") produces batches with type '", ValueDescr::ToString(output_descr_),
-          "' inconsistent with consumer '", out->label(), "' which accepts '",
-          ValueDescr::ToString(in_descr), "'");
+  return Status::OK();
+}
+
+struct SourceNode : ExecNode {
+  SourceNode(ExecPlan* plan, std::string label, ExecNode::BatchDescr output_descr,
+             AsyncGenerator<util::optional<ExecBatch>> generator)
+      : ExecNode(plan, std::move(label), {}, {}, std::move(output_descr),
+                 /*num_outputs=*/1),
+        generator_(std::move(generator)) {}
+
+  const char* kind_name() override { return "SourceNode"; }
+
+  static void NoInputs() { DCHECK(false) << "no inputs; this should never be called"; }
+  void InputReceived(ExecNode*, int, ExecBatch) override { NoInputs(); }
+  void ErrorReceived(ExecNode*, Status) override { NoInputs(); }
+  void InputFinished(ExecNode*, int) override { NoInputs(); }
+
+  Status StartProducing() override {
+    if (finished_) {
+      return Status::Invalid("Restarted SourceNode '", label(), "'");
     }
+
+    auto gen = std::move(generator_);
+
+    /// XXX should we wait on this future anywhere? In StopProducing() maybe?
+    auto done_fut =
+        Loop([gen, this] {
+          std::unique_lock<std::mutex> lock(mutex_);
+          int seq = next_batch_index_++;
+          if (finished_) {
+            return Future<ControlFlow<int>>::MakeFinished(Break(seq));
+          }
+          lock.unlock();
+
+          return gen().Then(
+              [=](const util::optional<ExecBatch>& batch) -> ControlFlow<int> {
+                std::unique_lock<std::mutex> lock(mutex_);
+                if (!batch || finished_) {
+                  finished_ = true;
+                  return Break(seq);
+                }
+                lock.unlock();
+
+                outputs_[0]->InputReceived(this, seq, *batch);
+                return Continue();
+              },
+              [=](const Status& error) -> ControlFlow<int> {
+                std::unique_lock<std::mutex> lock(mutex_);
+                if (!finished_) {
+                  finished_ = true;
+                  lock.unlock();
+                  // unless we were already finished, push the error to our output
+                  // XXX is this correct? Is it reasonable for a consumer to ignore errors
+                  // from a finished producer?

Review comment:
       Given that stopping a producer doesn't necessarily immediately terminate everything, the consumer needs to be prepared to get and handle/ignore an error anyways.

##########
File path: cpp/src/arrow/compute/exec/expression.cc
##########
@@ -510,7 +475,67 @@ Result<Expression> Expression::Bind(const Schema& in_schema,
   return Bind(ValueDescr::Array(struct_(in_schema.fields())), exec_context);
 }
 
-Result<Datum> ExecuteScalarExpression(const Expression& expr, const Datum& input,
+Result<ExecBatch> MakeExecBatch(const Schema& full_schema, const Datum& partial) {
+  ExecBatch out;
+
+  if (partial.kind() == Datum::RECORD_BATCH) {
+    const auto& partial_batch = *partial.record_batch();
+    out.length = partial_batch.num_rows();
+
+    for (const auto& field : full_schema.fields()) {
+      ARROW_ASSIGN_OR_RAISE(auto column,
+                            FieldRef(field->name()).GetOneOrNone(partial_batch));

Review comment:
       On the other hand, this will still be quadratic.

##########
File path: cpp/src/arrow/compute/exec/expression.h
##########
@@ -64,8 +64,11 @@ class ARROW_EXPORT Expression {
   /// Bind this expression to the given input type, looking up Kernels and field types.
   /// Some expression simplification may be performed and implicit casts will be inserted.
   /// Any state necessary for execution will be initialized and returned.
-  Result<Expression> Bind(ValueDescr in, compute::ExecContext* = NULLPTR) const;
-  Result<Expression> Bind(const Schema& in_schema, compute::ExecContext* = NULLPTR) const;
+  Result<Expression> Bind(ValueDescr in, ExecContext* = NULLPTR) const;
+  Result<Expression> Bind(const Schema& in_schema, ExecContext* = NULLPTR) const;
+
+  Result<Expression> BindFlattened(ValueDescr in, ExecContext* = NULLPTR) const;

Review comment:
       Are these implemented?

##########
File path: cpp/src/arrow/compute/exec/exec_plan.cc
##########
@@ -170,48 +165,409 @@ Status ExecPlan::Validate() { return ToDerived(this)->Validate(); }
 
 Status ExecPlan::StartProducing() { return ToDerived(this)->StartProducing(); }
 
-ExecNode::ExecNode(ExecPlan* plan, std::string label,
-                   std::vector<BatchDescr> input_descrs,
+ExecNode::ExecNode(ExecPlan* plan, std::string label, NodeVector inputs,
                    std::vector<std::string> input_labels, BatchDescr output_descr,
                    int num_outputs)
     : plan_(plan),
       label_(std::move(label)),
-      input_descrs_(std::move(input_descrs)),
+      inputs_(std::move(inputs)),
       input_labels_(std::move(input_labels)),
       output_descr_(std::move(output_descr)),
-      num_outputs_(num_outputs) {}
+      num_outputs_(num_outputs) {
+  for (auto input : inputs_) {
+    input->outputs_.push_back(this);
+  }
+}
 
 Status ExecNode::Validate() const {
-  if (inputs_.size() != input_descrs_.size()) {
+  if (inputs_.size() != input_labels_.size()) {
     return Status::Invalid("Invalid number of inputs for '", label(), "' (expected ",
-                           num_inputs(), ", actual ", inputs_.size(), ")");
+                           num_inputs(), ", actual ", input_labels_.size(), ")");
   }
 
   if (static_cast<int>(outputs_.size()) != num_outputs_) {
     return Status::Invalid("Invalid number of outputs for '", label(), "' (expected ",
                            num_outputs(), ", actual ", outputs_.size(), ")");
   }
 
-  DCHECK_EQ(input_descrs_.size(), input_labels_.size());
-
   for (auto out : outputs_) {
     auto input_index = GetNodeIndex(out->inputs(), this);
     if (!input_index) {
       return Status::Invalid("Node '", label(), "' outputs to node '", out->label(),
                              "' but is not listed as an input.");
     }
+  }
 
-    const auto& in_descr = out->input_descrs_[*input_index];
-    if (in_descr != output_descr_) {
-      return Status::Invalid(
-          "Node '", label(), "' (bound to input ", input_labels_[*input_index],
-          ") produces batches with type '", ValueDescr::ToString(output_descr_),
-          "' inconsistent with consumer '", out->label(), "' which accepts '",
-          ValueDescr::ToString(in_descr), "'");
+  return Status::OK();
+}
+
+struct SourceNode : ExecNode {
+  SourceNode(ExecPlan* plan, std::string label, ExecNode::BatchDescr output_descr,
+             AsyncGenerator<util::optional<ExecBatch>> generator)
+      : ExecNode(plan, std::move(label), {}, {}, std::move(output_descr),
+                 /*num_outputs=*/1),
+        generator_(std::move(generator)) {}
+
+  const char* kind_name() override { return "SourceNode"; }
+
+  static void NoInputs() { DCHECK(false) << "no inputs; this should never be called"; }
+  void InputReceived(ExecNode*, int, ExecBatch) override { NoInputs(); }
+  void ErrorReceived(ExecNode*, Status) override { NoInputs(); }
+  void InputFinished(ExecNode*, int) override { NoInputs(); }
+
+  Status StartProducing() override {
+    if (finished_) {
+      return Status::Invalid("Restarted SourceNode '", label(), "'");
     }
+
+    auto gen = std::move(generator_);
+
+    /// XXX should we wait on this future anywhere? In StopProducing() maybe?
+    auto done_fut =
+        Loop([gen, this] {
+          std::unique_lock<std::mutex> lock(mutex_);
+          int seq = next_batch_index_++;
+          if (finished_) {
+            return Future<ControlFlow<int>>::MakeFinished(Break(seq));
+          }
+          lock.unlock();
+
+          return gen().Then(
+              [=](const util::optional<ExecBatch>& batch) -> ControlFlow<int> {
+                std::unique_lock<std::mutex> lock(mutex_);
+                if (!batch || finished_) {
+                  finished_ = true;
+                  return Break(seq);
+                }
+                lock.unlock();
+
+                outputs_[0]->InputReceived(this, seq, *batch);
+                return Continue();
+              },
+              [=](const Status& error) -> ControlFlow<int> {
+                std::unique_lock<std::mutex> lock(mutex_);
+                if (!finished_) {
+                  finished_ = true;
+                  lock.unlock();
+                  // unless we were already finished, push the error to our output
+                  // XXX is this correct? Is it reasonable for a consumer to ignore errors
+                  // from a finished producer?
+                  outputs_[0]->ErrorReceived(this, error);
+                }
+                return Break(seq);
+              });
+        }).Then([&](int seq) {
+          /// XXX this is probably redundant: do we always call InputFinished after
+          /// ErrorReceived or will ErrorRecieved be sufficient?
+          outputs_[0]->InputFinished(this, seq);
+        });
+
+    return Status::OK();
   }
 
-  return Status::OK();
+  void PauseProducing(ExecNode* output) override {}
+
+  void ResumeProducing(ExecNode* output) override {}
+
+  void StopProducing(ExecNode* output) override {
+    DCHECK_EQ(output, outputs_[0]);
+    std::unique_lock<std::mutex> lock(mutex_);
+    finished_ = true;
+  }
+
+  void StopProducing() override { StopProducing(outputs_[0]); }
+
+ private:
+  std::mutex mutex_;
+  bool finished_{false};
+  int next_batch_index_{0};
+  AsyncGenerator<util::optional<ExecBatch>> generator_;
+};
+
+ExecNode* MakeSourceNode(ExecPlan* plan, std::string label,
+                         ExecNode::BatchDescr output_descr,
+                         AsyncGenerator<util::optional<ExecBatch>> generator) {
+  return plan->EmplaceNode<SourceNode>(plan, std::move(label), std::move(output_descr),
+                                       std::move(generator));
+}
+
+struct FilterNode : ExecNode {
+  FilterNode(ExecNode* input, std::string label, Expression filter)
+      : ExecNode(input->plan(), std::move(label), {input}, {"target"},
+                 /*output_descr=*/{input->output_descr()},
+                 /*num_outputs=*/1),
+        filter_(std::move(filter)) {}
+
+  const char* kind_name() override { return "FilterNode"; }
+
+  Result<ExecBatch> DoFilter(const ExecBatch& target) {
+    ARROW_ASSIGN_OR_RAISE(Expression simplified_filter,
+                          SimplifyWithGuarantee(filter_, target.guarantee));
+
+    // XXX get a non-default exec context
+    ARROW_ASSIGN_OR_RAISE(Datum mask, ExecuteScalarExpression(simplified_filter, target));
+
+    if (mask.is_scalar()) {
+      const auto& mask_scalar = mask.scalar_as<BooleanScalar>();
+      if (mask_scalar.is_valid && mask_scalar.value) {
+        return target;
+      }
+
+      return target.Slice(0, 0);
+    }
+
+    auto values = target.values;
+    for (auto& value : values) {
+      if (value.is_scalar()) continue;
+      ARROW_ASSIGN_OR_RAISE(value, Filter(value, mask, FilterOptions::Defaults()));
+    }
+    return ExecBatch::Make(std::move(values));
+  }
+
+  void InputReceived(ExecNode* input, int seq, ExecBatch batch) override {
+    DCHECK_EQ(input, inputs_[0]);
+
+    auto maybe_filtered = DoFilter(std::move(batch));
+    if (!maybe_filtered.ok()) {
+      outputs_[0]->ErrorReceived(this, maybe_filtered.status());
+      inputs_[0]->StopProducing(this);
+      return;
+    }
+
+    maybe_filtered->guarantee = batch.guarantee;
+    outputs_[0]->InputReceived(this, seq, maybe_filtered.MoveValueUnsafe());
+  }
+
+  void ErrorReceived(ExecNode* input, Status error) override {
+    DCHECK_EQ(input, inputs_[0]);
+    outputs_[0]->ErrorReceived(this, std::move(error));
+    inputs_[0]->StopProducing(this);
+  }
+
+  void InputFinished(ExecNode* input, int seq) override {
+    DCHECK_EQ(input, inputs_[0]);
+    outputs_[0]->InputFinished(this, seq);
+    inputs_[0]->StopProducing(this);
+  }
+
+  Status StartProducing() override {
+    // XXX validate inputs_[0]->output_descr() against filter_
+    return Status::OK();
+  }
+
+  void PauseProducing(ExecNode* output) override {}
+
+  void ResumeProducing(ExecNode* output) override {}
+
+  void StopProducing(ExecNode* output) override {
+    DCHECK_EQ(output, outputs_[0]);
+    inputs_[0]->StopProducing(this);
+  }
+
+  void StopProducing() override { StopProducing(outputs_[0]); }
+
+ private:
+  Expression filter_;
+};
+
+ExecNode* MakeFilterNode(ExecNode* input, std::string label, Expression filter) {
+  return input->plan()->EmplaceNode<FilterNode>(input, std::move(label),
+                                                std::move(filter));
+}
+
+struct ProjectNode : ExecNode {
+  ProjectNode(ExecNode* input, std::string label, std::vector<Expression> exprs)
+      : ExecNode(input->plan(), std::move(label), {input}, {"target"},
+                 /*output_descr=*/{input->output_descr()},
+                 /*num_outputs=*/1),
+        exprs_(std::move(exprs)) {}
+
+  const char* kind_name() override { return "ProjectNode"; }
+
+  Result<ExecBatch> DoProject(const ExecBatch& target) {
+    // XXX get a non-default exec context
+    std::vector<Datum> values{exprs_.size()};
+    for (size_t i = 0; i < exprs_.size(); ++i) {
+      ARROW_ASSIGN_OR_RAISE(Expression simplified_expr,
+                            SimplifyWithGuarantee(exprs_[i], target.guarantee));
+
+      ARROW_ASSIGN_OR_RAISE(values[i], ExecuteScalarExpression(simplified_expr, target));
+    }
+    return ExecBatch::Make(std::move(values));
+  }
+
+  void InputReceived(ExecNode* input, int seq, ExecBatch batch) override {
+    DCHECK_EQ(input, inputs_[0]);
+
+    auto maybe_projected = DoProject(std::move(batch));
+    if (!maybe_projected.ok()) {
+      outputs_[0]->ErrorReceived(this, maybe_projected.status());
+      inputs_[0]->StopProducing(this);
+      return;
+    }
+
+    maybe_projected->guarantee = batch.guarantee;
+    outputs_[0]->InputReceived(this, seq, maybe_projected.MoveValueUnsafe());
+  }
+
+  void ErrorReceived(ExecNode* input, Status error) override {
+    DCHECK_EQ(input, inputs_[0]);
+    outputs_[0]->ErrorReceived(this, std::move(error));
+    inputs_[0]->StopProducing(this);
+  }
+
+  void InputFinished(ExecNode* input, int seq) override {
+    DCHECK_EQ(input, inputs_[0]);
+    outputs_[0]->InputFinished(this, seq);
+    inputs_[0]->StopProducing(this);
+  }
+
+  Status StartProducing() override {
+    // XXX validate inputs_[0]->output_descr() against filter_
+    return Status::OK();
+  }
+
+  void PauseProducing(ExecNode* output) override {}
+
+  void ResumeProducing(ExecNode* output) override {}
+
+  void StopProducing(ExecNode* output) override {
+    DCHECK_EQ(output, outputs_[0]);
+    inputs_[0]->StopProducing(this);
+  }
+
+  void StopProducing() override { StopProducing(outputs_[0]); }
+
+ private:
+  std::vector<Expression> exprs_;
+};
+
+ExecNode* MakeProjectNode(ExecNode* input, std::string label,
+                          std::vector<Expression> exprs) {
+  return input->plan()->EmplaceNode<ProjectNode>(input, std::move(label),
+                                                 std::move(exprs));
+}
+
+struct SinkNode : ExecNode {
+  SinkNode(ExecNode* input, std::string label,
+           AsyncGenerator<util::optional<ExecBatch>>* generator)
+      : ExecNode(input->plan(), std::move(label), {input}, {"collected"}, {},
+                 /*num_outputs=*/0),
+        producer_(MakeProducer(generator)) {}
+
+  static PushGenerator<util::optional<ExecBatch>>::Producer MakeProducer(
+      AsyncGenerator<util::optional<ExecBatch>>* out_gen) {
+    PushGenerator<util::optional<ExecBatch>> gen;
+    auto out = gen.producer();
+    *out_gen = std::move(gen);
+    return out;
+  }
+
+  const char* kind_name() override { return "SinkNode"; }
+
+  Status StartProducing() override { return Status::OK(); }
+
+  // sink nodes have no outputs from which to feel backpressure
+  static void NoOutputs() { DCHECK(false) << "no outputs; this should never be called"; }
+  void ResumeProducing(ExecNode* output) override { NoOutputs(); }
+  void PauseProducing(ExecNode* output) override { NoOutputs(); }
+  void StopProducing(ExecNode* output) override { NoOutputs(); }
+
+  void StopProducing() override {
+    std::unique_lock<std::mutex> lock(mutex_);
+    StopProducingUnlocked();
+  }
+
+  void InputReceived(ExecNode* input, int seq_num, ExecBatch exec_batch) override {
+    std::unique_lock<std::mutex> lock(mutex_);
+    if (stopped_) return;
+
+    // TODO would be nice to factor this out in a ReorderQueue
+    if (seq_num <= static_cast<int>(received_batches_.size())) {
+      received_batches_.resize(seq_num + 1);
+      emitted_.resize(seq_num + 1, false);
+    }
+    received_batches_[seq_num] = std::move(exec_batch);
+    ++num_received_;
+
+    if (seq_num != num_emitted_) {
+      // Cannot emit yet as there is a hole at `num_emitted_`
+      DCHECK_GT(seq_num, num_emitted_);
+      return;
+    }
+
+    if (num_received_ == emit_stop_) {
+      StopProducingUnlocked();
+    }
+
+    // Emit batches in order as far as possible
+    // First collect these batches, then unlock before producing.
+    const auto seq_start = seq_num;
+    while (seq_num < static_cast<int>(emitted_.size()) && !emitted_[seq_num]) {
+      emitted_[seq_num] = true;
+      ++seq_num;
+    }
+    DCHECK_GT(seq_num, seq_start);
+    // By moving the values now, we make sure another thread won't emit the same values
+    // below
+    std::vector<ExecBatch> to_emit(
+        std::make_move_iterator(received_batches_.begin() + seq_start),
+        std::make_move_iterator(received_batches_.begin() + seq_num));
+
+    lock.unlock();
+    for (auto&& batch : to_emit) {
+      producer_.Push(std::move(batch));
+    }
+    lock.lock();
+
+    DCHECK_EQ(seq_start, num_emitted_);  // num_emitted_ wasn't bumped in the meantime
+    num_emitted_ = seq_num;

Review comment:
       Why not just put this inside the lock? Also, if one thread were to run everything up to here, unlock the lock, and get unscheduled before the assignment, a subsequent call could feasibly update num_emitted_ before the first thread gets scheduled again.

##########
File path: cpp/src/arrow/compute/exec/expression.cc
##########
@@ -492,9 +448,18 @@ Result<Expression> Expression::Bind(ValueDescr in,
   if (literal()) return *this;
 
   if (auto ref = field_ref()) {
-    ARROW_ASSIGN_OR_RAISE(auto field, ref->GetOneOrNone(*in.type));
-    auto descr = field ? ValueDescr{field->type(), in.shape} : ValueDescr::Scalar(null());
-    return Expression{Parameter{*ref, std::move(descr)}};
+    if (ref->IsNested()) {
+      return Status::NotImplemented("nested field references");
+    }
+
+    ARROW_ASSIGN_OR_RAISE(auto path, ref->FindOne(*in.type));

Review comment:
       Is the change to using indices meant to be an optimization (for the wide schema case)?
   
   This will still be quadratic but at least binding happens only once.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] bkietz commented on a change in pull request #10397: ARROW-11930: [C++][Dataset][Compute] Use an ExecPlan for dataset scans

Posted by GitBox <gi...@apache.org>.
bkietz commented on a change in pull request #10397:
URL: https://github.com/apache/arrow/pull/10397#discussion_r640917034



##########
File path: cpp/src/arrow/compute/exec/expression.cc
##########
@@ -492,9 +448,18 @@ Result<Expression> Expression::Bind(ValueDescr in,
   if (literal()) return *this;
 
   if (auto ref = field_ref()) {
-    ARROW_ASSIGN_OR_RAISE(auto field, ref->GetOneOrNone(*in.type));
-    auto descr = field ? ValueDescr{field->type(), in.shape} : ValueDescr::Scalar(null());
-    return Expression{Parameter{*ref, std::move(descr)}};
+    if (ref->IsNested()) {
+      return Status::NotImplemented("nested field references");
+    }
+
+    ARROW_ASSIGN_OR_RAISE(auto path, ref->FindOne(*in.type));

Review comment:
       For now, I've added a hot path for Schema since that already has a hash table from name -> index




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] bkietz commented on a change in pull request #10397: ARROW-11930: [C++][Dataset][Compute] Use an ExecPlan for dataset scans

Posted by GitBox <gi...@apache.org>.
bkietz commented on a change in pull request #10397:
URL: https://github.com/apache/arrow/pull/10397#discussion_r644278039



##########
File path: cpp/src/arrow/util/async_generator.h
##########
@@ -258,43 +258,27 @@ class MappingGenerator {
 /// Note: Errors returned from the `map` function will be propagated
 ///
 /// If the source generator is async-reentrant then this generator will be also
-template <typename T, typename V>
-AsyncGenerator<V> MakeMappedGenerator(AsyncGenerator<T> source_generator,
-                                      std::function<Result<V>(const T&)> map) {
-  std::function<Future<V>(const T&)> future_map = [map](const T& val) -> Future<V> {
-    return Future<V>::MakeFinished(map(val));
-  };
-  return MappingGenerator<T, V>(std::move(source_generator), std::move(future_map));
-}
-template <typename T, typename V>
-AsyncGenerator<V> MakeMappedGenerator(AsyncGenerator<T> source_generator,
-                                      std::function<V(const T&)> map) {
-  std::function<Future<V>(const T&)> maybe_future_map = [map](const T& val) -> Future<V> {
-    return Future<V>::MakeFinished(map(val));
-  };
-  return MappingGenerator<T, V>(std::move(source_generator), std::move(maybe_future_map));
-}
-template <typename T, typename V>
-AsyncGenerator<V> MakeMappedGenerator(AsyncGenerator<T> source_generator,
-                                      std::function<Future<V>(const T&)> map) {
-  return MappingGenerator<T, V>(std::move(source_generator), std::move(map));
-}
-
-template <typename V, typename T, typename MapFunc>
-AsyncGenerator<V> MakeMappedGenerator(AsyncGenerator<T> source_generator, MapFunc map) {
+template <typename T, typename MapFn,
+          typename Mapped = detail::result_of_t<MapFn(const T&)>,
+          typename V = typename EnsureFuture<Mapped>::type::ValueType>
+AsyncGenerator<V> MakeMappedGenerator(AsyncGenerator<T> source_generator, MapFn map) {
   struct MapCallback {
-    MapFunc map;
+    MapFn map_;
 
-    Future<V> operator()(const T& val) { return EnsureFuture(map(val)); }
+    Future<V> operator()(const T& val) { return EnsureFuture(map_(val)); }
+
+    Future<V> EnsureFuture(V mapped) {
+      return Future<V>::MakeFinished(std::move(mapped));
+    }
 
-    Future<V> EnsureFuture(Result<V> val) {
-      return Future<V>::MakeFinished(std::move(val));
+    Future<V> EnsureFuture(Result<V> mapped) {
+      return Future<V>::MakeFinished(std::move(mapped));
     }
-    Future<V> EnsureFuture(V val) { return Future<V>::MakeFinished(std::move(val)); }
-    Future<V> EnsureFuture(Future<V> val) { return val; }
+
+    Future<V> EnsureFuture(Future<V> mapped) { return mapped; }

Review comment:
       I'll extract a helper 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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] bkietz commented on a change in pull request #10397: ARROW-11930: [C++][Dataset][Compute] Use an ExecPlan for dataset scans

Posted by GitBox <gi...@apache.org>.
bkietz commented on a change in pull request #10397:
URL: https://github.com/apache/arrow/pull/10397#discussion_r643447263



##########
File path: cpp/src/arrow/compute/exec/expression.cc
##########
@@ -510,7 +475,67 @@ Result<Expression> Expression::Bind(const Schema& in_schema,
   return Bind(ValueDescr::Array(struct_(in_schema.fields())), exec_context);
 }
 
-Result<Datum> ExecuteScalarExpression(const Expression& expr, const Datum& input,
+Result<ExecBatch> MakeExecBatch(const Schema& full_schema, const Datum& partial) {
+  ExecBatch out;
+
+  if (partial.kind() == Datum::RECORD_BATCH) {
+    const auto& partial_batch = *partial.record_batch();
+    out.length = partial_batch.num_rows();
+
+    for (const auto& field : full_schema.fields()) {
+      ARROW_ASSIGN_OR_RAISE(auto column,
+                            FieldRef(field->name()).GetOneOrNone(partial_batch));

Review comment:
       It will raise Status::Invalid




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] bkietz commented on pull request #10397: ARROW-11930: [C++][Dataset][Compute] Use an ExecPlan for dataset scans

Posted by GitBox <gi...@apache.org>.
bkietz commented on pull request #10397:
URL: https://github.com/apache/arrow/pull/10397#issuecomment-853380645


   >  How do you see things evolving? Do you think the various operations achieved by a scanner today will be achieved by an execution plan? For example, will ScanBatches, CountRows, etc. create and execute an execution plan instead of maintaining the dual paths?
   
   I'd like the ExecPlan to be usable enough to replace all filtering and projection currently in Scanner. So for example ScanBatches could assemble an ExecPlan to handle filtering and projection then receive and reorder batches; never needing to explicitly evaluate an expression.
   
   Ultimately, I'm not positive we'll keep Scanner. It's possible we could simplify the dataset module to a factory for source/sink nodes. In that case, anything which currently builds a Scanner would instead produce an ExecPlan. We'll see


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] bkietz commented on a change in pull request #10397: ARROW-11930: [C++][Dataset][Compute] Use an ExecPlan for dataset scans

Posted by GitBox <gi...@apache.org>.
bkietz commented on a change in pull request #10397:
URL: https://github.com/apache/arrow/pull/10397#discussion_r644292253



##########
File path: cpp/src/arrow/compute/exec/exec_plan.h
##########
@@ -23,6 +23,7 @@
 
 #include "arrow/compute/type_fwd.h"
 #include "arrow/type_fwd.h"
+#include "arrow/util/async_generator.h"

Review comment:
       Alright




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] bkietz commented on a change in pull request #10397: ARROW-11930: [C++][Dataset][Compute] Use an ExecPlan for dataset scans

Posted by GitBox <gi...@apache.org>.
bkietz commented on a change in pull request #10397:
URL: https://github.com/apache/arrow/pull/10397#discussion_r644278145



##########
File path: cpp/src/arrow/util/future.h
##########
@@ -937,4 +940,19 @@ Future<BreakValueType> Loop(Iterate iterate) {
   return break_fut;
 }
 
+template <typename T>
+struct EnsureFuture {
+  using type = Future<T>;
+};
+
+template <typename T>
+struct EnsureFuture<Result<T>> {

Review comment:
       sure




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] bkietz commented on a change in pull request #10397: ARROW-11930: [C++][Dataset][Compute] Use an ExecPlan for dataset scans

Posted by GitBox <gi...@apache.org>.
bkietz commented on a change in pull request #10397:
URL: https://github.com/apache/arrow/pull/10397#discussion_r644836145



##########
File path: cpp/src/arrow/dataset/dataset_internal.h
##########
@@ -204,5 +204,35 @@ arrow::Result<std::shared_ptr<T>> GetFragmentScanOptions(
   return internal::checked_pointer_cast<T>(source);
 }
 
+class FragmentDataset : public Dataset {

Review comment:
       Added https://issues.apache.org/jira/browse/ARROW-12945 to track this improvement.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] bkietz commented on a change in pull request #10397: ARROW-11930: [C++][Dataset][Compute] Use an ExecPlan for dataset scans

Posted by GitBox <gi...@apache.org>.
bkietz commented on a change in pull request #10397:
URL: https://github.com/apache/arrow/pull/10397#discussion_r644307351



##########
File path: cpp/src/arrow/compute/exec/exec_plan.cc
##########
@@ -170,48 +165,409 @@ Status ExecPlan::Validate() { return ToDerived(this)->Validate(); }
 
 Status ExecPlan::StartProducing() { return ToDerived(this)->StartProducing(); }
 
-ExecNode::ExecNode(ExecPlan* plan, std::string label,
-                   std::vector<BatchDescr> input_descrs,
+ExecNode::ExecNode(ExecPlan* plan, std::string label, NodeVector inputs,
                    std::vector<std::string> input_labels, BatchDescr output_descr,
                    int num_outputs)
     : plan_(plan),
       label_(std::move(label)),
-      input_descrs_(std::move(input_descrs)),
+      inputs_(std::move(inputs)),
       input_labels_(std::move(input_labels)),
       output_descr_(std::move(output_descr)),
-      num_outputs_(num_outputs) {}
+      num_outputs_(num_outputs) {
+  for (auto input : inputs_) {
+    input->outputs_.push_back(this);
+  }
+}
 
 Status ExecNode::Validate() const {
-  if (inputs_.size() != input_descrs_.size()) {
+  if (inputs_.size() != input_labels_.size()) {
     return Status::Invalid("Invalid number of inputs for '", label(), "' (expected ",
-                           num_inputs(), ", actual ", inputs_.size(), ")");
+                           num_inputs(), ", actual ", input_labels_.size(), ")");
   }
 
   if (static_cast<int>(outputs_.size()) != num_outputs_) {
     return Status::Invalid("Invalid number of outputs for '", label(), "' (expected ",
                            num_outputs(), ", actual ", outputs_.size(), ")");
   }
 
-  DCHECK_EQ(input_descrs_.size(), input_labels_.size());
-
   for (auto out : outputs_) {
     auto input_index = GetNodeIndex(out->inputs(), this);
     if (!input_index) {
       return Status::Invalid("Node '", label(), "' outputs to node '", out->label(),
                              "' but is not listed as an input.");
     }
+  }
 
-    const auto& in_descr = out->input_descrs_[*input_index];
-    if (in_descr != output_descr_) {
-      return Status::Invalid(
-          "Node '", label(), "' (bound to input ", input_labels_[*input_index],
-          ") produces batches with type '", ValueDescr::ToString(output_descr_),
-          "' inconsistent with consumer '", out->label(), "' which accepts '",
-          ValueDescr::ToString(in_descr), "'");
+  return Status::OK();
+}
+
+struct SourceNode : ExecNode {
+  SourceNode(ExecPlan* plan, std::string label, ExecNode::BatchDescr output_descr,
+             AsyncGenerator<util::optional<ExecBatch>> generator)
+      : ExecNode(plan, std::move(label), {}, {}, std::move(output_descr),
+                 /*num_outputs=*/1),
+        generator_(std::move(generator)) {}
+
+  const char* kind_name() override { return "SourceNode"; }
+
+  static void NoInputs() { DCHECK(false) << "no inputs; this should never be called"; }
+  void InputReceived(ExecNode*, int, ExecBatch) override { NoInputs(); }
+  void ErrorReceived(ExecNode*, Status) override { NoInputs(); }
+  void InputFinished(ExecNode*, int) override { NoInputs(); }
+
+  Status StartProducing() override {
+    if (finished_) {
+      return Status::Invalid("Restarted SourceNode '", label(), "'");
     }
+
+    auto gen = std::move(generator_);
+
+    /// XXX should we wait on this future anywhere? In StopProducing() maybe?
+    auto done_fut =
+        Loop([gen, this] {
+          std::unique_lock<std::mutex> lock(mutex_);
+          int seq = next_batch_index_++;
+          if (finished_) {
+            return Future<ControlFlow<int>>::MakeFinished(Break(seq));
+          }
+          lock.unlock();
+
+          return gen().Then(
+              [=](const util::optional<ExecBatch>& batch) -> ControlFlow<int> {
+                std::unique_lock<std::mutex> lock(mutex_);
+                if (!batch || finished_) {
+                  finished_ = true;
+                  return Break(seq);
+                }
+                lock.unlock();
+
+                outputs_[0]->InputReceived(this, seq, *batch);
+                return Continue();
+              },
+              [=](const Status& error) -> ControlFlow<int> {
+                std::unique_lock<std::mutex> lock(mutex_);
+                if (!finished_) {
+                  finished_ = true;
+                  lock.unlock();
+                  // unless we were already finished, push the error to our output
+                  // XXX is this correct? Is it reasonable for a consumer to ignore errors
+                  // from a finished producer?
+                  outputs_[0]->ErrorReceived(this, error);
+                }
+                return Break(seq);
+              });
+        }).Then([&](int seq) {
+          /// XXX this is probably redundant: do we always call InputFinished after
+          /// ErrorReceived or will ErrorRecieved be sufficient?
+          outputs_[0]->InputFinished(this, seq);
+        });
+
+    return Status::OK();
   }
 
-  return Status::OK();
+  void PauseProducing(ExecNode* output) override {}
+
+  void ResumeProducing(ExecNode* output) override {}
+
+  void StopProducing(ExecNode* output) override {
+    DCHECK_EQ(output, outputs_[0]);
+    std::unique_lock<std::mutex> lock(mutex_);
+    finished_ = true;
+  }
+
+  void StopProducing() override { StopProducing(outputs_[0]); }
+
+ private:
+  std::mutex mutex_;
+  bool finished_{false};
+  int next_batch_index_{0};
+  AsyncGenerator<util::optional<ExecBatch>> generator_;
+};
+
+ExecNode* MakeSourceNode(ExecPlan* plan, std::string label,
+                         ExecNode::BatchDescr output_descr,
+                         AsyncGenerator<util::optional<ExecBatch>> generator) {
+  return plan->EmplaceNode<SourceNode>(plan, std::move(label), std::move(output_descr),
+                                       std::move(generator));
+}
+
+struct FilterNode : ExecNode {
+  FilterNode(ExecNode* input, std::string label, Expression filter)
+      : ExecNode(input->plan(), std::move(label), {input}, {"target"},
+                 /*output_descr=*/{input->output_descr()},
+                 /*num_outputs=*/1),
+        filter_(std::move(filter)) {}
+
+  const char* kind_name() override { return "FilterNode"; }
+
+  Result<ExecBatch> DoFilter(const ExecBatch& target) {
+    ARROW_ASSIGN_OR_RAISE(Expression simplified_filter,
+                          SimplifyWithGuarantee(filter_, target.guarantee));
+
+    // XXX get a non-default exec context
+    ARROW_ASSIGN_OR_RAISE(Datum mask, ExecuteScalarExpression(simplified_filter, target));
+
+    if (mask.is_scalar()) {
+      const auto& mask_scalar = mask.scalar_as<BooleanScalar>();
+      if (mask_scalar.is_valid && mask_scalar.value) {
+        return target;
+      }
+
+      return target.Slice(0, 0);
+    }
+
+    auto values = target.values;
+    for (auto& value : values) {
+      if (value.is_scalar()) continue;
+      ARROW_ASSIGN_OR_RAISE(value, Filter(value, mask, FilterOptions::Defaults()));
+    }
+    return ExecBatch::Make(std::move(values));
+  }
+
+  void InputReceived(ExecNode* input, int seq, ExecBatch batch) override {
+    DCHECK_EQ(input, inputs_[0]);
+
+    auto maybe_filtered = DoFilter(std::move(batch));
+    if (!maybe_filtered.ok()) {
+      outputs_[0]->ErrorReceived(this, maybe_filtered.status());
+      inputs_[0]->StopProducing(this);
+      return;
+    }
+
+    maybe_filtered->guarantee = batch.guarantee;
+    outputs_[0]->InputReceived(this, seq, maybe_filtered.MoveValueUnsafe());
+  }
+
+  void ErrorReceived(ExecNode* input, Status error) override {
+    DCHECK_EQ(input, inputs_[0]);
+    outputs_[0]->ErrorReceived(this, std::move(error));
+    inputs_[0]->StopProducing(this);
+  }
+
+  void InputFinished(ExecNode* input, int seq) override {
+    DCHECK_EQ(input, inputs_[0]);
+    outputs_[0]->InputFinished(this, seq);
+    inputs_[0]->StopProducing(this);
+  }
+
+  Status StartProducing() override {
+    // XXX validate inputs_[0]->output_descr() against filter_
+    return Status::OK();
+  }
+
+  void PauseProducing(ExecNode* output) override {}
+
+  void ResumeProducing(ExecNode* output) override {}
+
+  void StopProducing(ExecNode* output) override {
+    DCHECK_EQ(output, outputs_[0]);
+    inputs_[0]->StopProducing(this);
+  }
+
+  void StopProducing() override { StopProducing(outputs_[0]); }
+
+ private:
+  Expression filter_;
+};
+
+ExecNode* MakeFilterNode(ExecNode* input, std::string label, Expression filter) {
+  return input->plan()->EmplaceNode<FilterNode>(input, std::move(label),
+                                                std::move(filter));
+}
+
+struct ProjectNode : ExecNode {
+  ProjectNode(ExecNode* input, std::string label, std::vector<Expression> exprs)
+      : ExecNode(input->plan(), std::move(label), {input}, {"target"},
+                 /*output_descr=*/{input->output_descr()},
+                 /*num_outputs=*/1),
+        exprs_(std::move(exprs)) {}
+
+  const char* kind_name() override { return "ProjectNode"; }
+
+  Result<ExecBatch> DoProject(const ExecBatch& target) {
+    // XXX get a non-default exec context
+    std::vector<Datum> values{exprs_.size()};
+    for (size_t i = 0; i < exprs_.size(); ++i) {
+      ARROW_ASSIGN_OR_RAISE(Expression simplified_expr,
+                            SimplifyWithGuarantee(exprs_[i], target.guarantee));
+
+      ARROW_ASSIGN_OR_RAISE(values[i], ExecuteScalarExpression(simplified_expr, target));
+    }
+    return ExecBatch::Make(std::move(values));
+  }
+
+  void InputReceived(ExecNode* input, int seq, ExecBatch batch) override {
+    DCHECK_EQ(input, inputs_[0]);
+
+    auto maybe_projected = DoProject(std::move(batch));
+    if (!maybe_projected.ok()) {
+      outputs_[0]->ErrorReceived(this, maybe_projected.status());
+      inputs_[0]->StopProducing(this);
+      return;
+    }
+
+    maybe_projected->guarantee = batch.guarantee;
+    outputs_[0]->InputReceived(this, seq, maybe_projected.MoveValueUnsafe());
+  }
+
+  void ErrorReceived(ExecNode* input, Status error) override {
+    DCHECK_EQ(input, inputs_[0]);
+    outputs_[0]->ErrorReceived(this, std::move(error));
+    inputs_[0]->StopProducing(this);
+  }
+
+  void InputFinished(ExecNode* input, int seq) override {
+    DCHECK_EQ(input, inputs_[0]);
+    outputs_[0]->InputFinished(this, seq);
+    inputs_[0]->StopProducing(this);
+  }
+
+  Status StartProducing() override {
+    // XXX validate inputs_[0]->output_descr() against filter_
+    return Status::OK();
+  }
+
+  void PauseProducing(ExecNode* output) override {}
+
+  void ResumeProducing(ExecNode* output) override {}
+
+  void StopProducing(ExecNode* output) override {
+    DCHECK_EQ(output, outputs_[0]);
+    inputs_[0]->StopProducing(this);
+  }
+
+  void StopProducing() override { StopProducing(outputs_[0]); }
+
+ private:
+  std::vector<Expression> exprs_;
+};
+
+ExecNode* MakeProjectNode(ExecNode* input, std::string label,
+                          std::vector<Expression> exprs) {
+  return input->plan()->EmplaceNode<ProjectNode>(input, std::move(label),
+                                                 std::move(exprs));
+}
+
+struct SinkNode : ExecNode {
+  SinkNode(ExecNode* input, std::string label,
+           AsyncGenerator<util::optional<ExecBatch>>* generator)
+      : ExecNode(input->plan(), std::move(label), {input}, {"collected"}, {},
+                 /*num_outputs=*/0),
+        producer_(MakeProducer(generator)) {}
+
+  static PushGenerator<util::optional<ExecBatch>>::Producer MakeProducer(
+      AsyncGenerator<util::optional<ExecBatch>>* out_gen) {
+    PushGenerator<util::optional<ExecBatch>> gen;
+    auto out = gen.producer();
+    *out_gen = std::move(gen);
+    return out;
+  }
+
+  const char* kind_name() override { return "SinkNode"; }
+
+  Status StartProducing() override { return Status::OK(); }
+
+  // sink nodes have no outputs from which to feel backpressure
+  static void NoOutputs() { DCHECK(false) << "no outputs; this should never be called"; }
+  void ResumeProducing(ExecNode* output) override { NoOutputs(); }
+  void PauseProducing(ExecNode* output) override { NoOutputs(); }
+  void StopProducing(ExecNode* output) override { NoOutputs(); }
+
+  void StopProducing() override {
+    std::unique_lock<std::mutex> lock(mutex_);
+    StopProducingUnlocked();
+  }
+
+  void InputReceived(ExecNode* input, int seq_num, ExecBatch exec_batch) override {

Review comment:
       I'll remove sorting from SinkNode. It's only used for testing purposes at the moment. In actual usage, ordering will be undefined unless explicitly specified - including the ToTable case, which will be handled by augmenting batches with fragment/batch index.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] pitrou commented on a change in pull request #10397: ARROW-11930: [C++][Dataset][Compute] Use an ExecPlan for dataset scans

Posted by GitBox <gi...@apache.org>.
pitrou commented on a change in pull request #10397:
URL: https://github.com/apache/arrow/pull/10397#discussion_r644140991



##########
File path: cpp/src/arrow/compute/exec/exec_plan.cc
##########
@@ -170,48 +165,409 @@ Status ExecPlan::Validate() { return ToDerived(this)->Validate(); }
 
 Status ExecPlan::StartProducing() { return ToDerived(this)->StartProducing(); }
 
-ExecNode::ExecNode(ExecPlan* plan, std::string label,
-                   std::vector<BatchDescr> input_descrs,
+ExecNode::ExecNode(ExecPlan* plan, std::string label, NodeVector inputs,
                    std::vector<std::string> input_labels, BatchDescr output_descr,
                    int num_outputs)
     : plan_(plan),
       label_(std::move(label)),
-      input_descrs_(std::move(input_descrs)),
+      inputs_(std::move(inputs)),
       input_labels_(std::move(input_labels)),
       output_descr_(std::move(output_descr)),
-      num_outputs_(num_outputs) {}
+      num_outputs_(num_outputs) {
+  for (auto input : inputs_) {
+    input->outputs_.push_back(this);
+  }
+}
 
 Status ExecNode::Validate() const {
-  if (inputs_.size() != input_descrs_.size()) {
+  if (inputs_.size() != input_labels_.size()) {
     return Status::Invalid("Invalid number of inputs for '", label(), "' (expected ",
-                           num_inputs(), ", actual ", inputs_.size(), ")");
+                           num_inputs(), ", actual ", input_labels_.size(), ")");
   }
 
   if (static_cast<int>(outputs_.size()) != num_outputs_) {
     return Status::Invalid("Invalid number of outputs for '", label(), "' (expected ",
                            num_outputs(), ", actual ", outputs_.size(), ")");
   }
 
-  DCHECK_EQ(input_descrs_.size(), input_labels_.size());
-
   for (auto out : outputs_) {
     auto input_index = GetNodeIndex(out->inputs(), this);
     if (!input_index) {
       return Status::Invalid("Node '", label(), "' outputs to node '", out->label(),
                              "' but is not listed as an input.");
     }
+  }
 
-    const auto& in_descr = out->input_descrs_[*input_index];
-    if (in_descr != output_descr_) {
-      return Status::Invalid(
-          "Node '", label(), "' (bound to input ", input_labels_[*input_index],
-          ") produces batches with type '", ValueDescr::ToString(output_descr_),
-          "' inconsistent with consumer '", out->label(), "' which accepts '",
-          ValueDescr::ToString(in_descr), "'");
+  return Status::OK();
+}
+
+struct SourceNode : ExecNode {
+  SourceNode(ExecPlan* plan, std::string label, ExecNode::BatchDescr output_descr,
+             AsyncGenerator<util::optional<ExecBatch>> generator)
+      : ExecNode(plan, std::move(label), {}, {}, std::move(output_descr),
+                 /*num_outputs=*/1),
+        generator_(std::move(generator)) {}
+
+  const char* kind_name() override { return "SourceNode"; }
+
+  static void NoInputs() { DCHECK(false) << "no inputs; this should never be called"; }
+  void InputReceived(ExecNode*, int, ExecBatch) override { NoInputs(); }
+  void ErrorReceived(ExecNode*, Status) override { NoInputs(); }
+  void InputFinished(ExecNode*, int) override { NoInputs(); }
+
+  Status StartProducing() override {
+    if (finished_) {
+      return Status::Invalid("Restarted SourceNode '", label(), "'");
     }
+
+    auto gen = std::move(generator_);
+
+    /// XXX should we wait on this future anywhere? In StopProducing() maybe?
+    auto done_fut =
+        Loop([gen, this] {
+          std::unique_lock<std::mutex> lock(mutex_);
+          int seq = next_batch_index_++;
+          if (finished_) {
+            return Future<ControlFlow<int>>::MakeFinished(Break(seq));
+          }
+          lock.unlock();
+
+          return gen().Then(
+              [=](const util::optional<ExecBatch>& batch) -> ControlFlow<int> {
+                std::unique_lock<std::mutex> lock(mutex_);
+                if (!batch || finished_) {
+                  finished_ = true;
+                  return Break(seq);
+                }
+                lock.unlock();
+
+                outputs_[0]->InputReceived(this, seq, *batch);
+                return Continue();
+              },
+              [=](const Status& error) -> ControlFlow<int> {
+                std::unique_lock<std::mutex> lock(mutex_);
+                if (!finished_) {
+                  finished_ = true;
+                  lock.unlock();
+                  // unless we were already finished, push the error to our output
+                  // XXX is this correct? Is it reasonable for a consumer to ignore errors
+                  // from a finished producer?
+                  outputs_[0]->ErrorReceived(this, error);
+                }
+                return Break(seq);
+              });
+        }).Then([&](int seq) {
+          /// XXX this is probably redundant: do we always call InputFinished after
+          /// ErrorReceived or will ErrorRecieved be sufficient?
+          outputs_[0]->InputFinished(this, seq);
+        });
+
+    return Status::OK();
   }
 
-  return Status::OK();
+  void PauseProducing(ExecNode* output) override {}
+
+  void ResumeProducing(ExecNode* output) override {}
+
+  void StopProducing(ExecNode* output) override {
+    DCHECK_EQ(output, outputs_[0]);
+    std::unique_lock<std::mutex> lock(mutex_);
+    finished_ = true;
+  }
+
+  void StopProducing() override { StopProducing(outputs_[0]); }
+
+ private:
+  std::mutex mutex_;
+  bool finished_{false};
+  int next_batch_index_{0};
+  AsyncGenerator<util::optional<ExecBatch>> generator_;
+};
+
+ExecNode* MakeSourceNode(ExecPlan* plan, std::string label,
+                         ExecNode::BatchDescr output_descr,
+                         AsyncGenerator<util::optional<ExecBatch>> generator) {
+  return plan->EmplaceNode<SourceNode>(plan, std::move(label), std::move(output_descr),
+                                       std::move(generator));
+}
+
+struct FilterNode : ExecNode {
+  FilterNode(ExecNode* input, std::string label, Expression filter)
+      : ExecNode(input->plan(), std::move(label), {input}, {"target"},
+                 /*output_descr=*/{input->output_descr()},
+                 /*num_outputs=*/1),
+        filter_(std::move(filter)) {}
+
+  const char* kind_name() override { return "FilterNode"; }
+
+  Result<ExecBatch> DoFilter(const ExecBatch& target) {
+    ARROW_ASSIGN_OR_RAISE(Expression simplified_filter,
+                          SimplifyWithGuarantee(filter_, target.guarantee));
+
+    // XXX get a non-default exec context
+    ARROW_ASSIGN_OR_RAISE(Datum mask, ExecuteScalarExpression(simplified_filter, target));
+
+    if (mask.is_scalar()) {
+      const auto& mask_scalar = mask.scalar_as<BooleanScalar>();
+      if (mask_scalar.is_valid && mask_scalar.value) {
+        return target;
+      }
+
+      return target.Slice(0, 0);
+    }
+
+    auto values = target.values;
+    for (auto& value : values) {
+      if (value.is_scalar()) continue;
+      ARROW_ASSIGN_OR_RAISE(value, Filter(value, mask, FilterOptions::Defaults()));
+    }
+    return ExecBatch::Make(std::move(values));
+  }
+
+  void InputReceived(ExecNode* input, int seq, ExecBatch batch) override {
+    DCHECK_EQ(input, inputs_[0]);
+
+    auto maybe_filtered = DoFilter(std::move(batch));
+    if (!maybe_filtered.ok()) {
+      outputs_[0]->ErrorReceived(this, maybe_filtered.status());
+      inputs_[0]->StopProducing(this);
+      return;
+    }
+
+    maybe_filtered->guarantee = batch.guarantee;
+    outputs_[0]->InputReceived(this, seq, maybe_filtered.MoveValueUnsafe());
+  }
+
+  void ErrorReceived(ExecNode* input, Status error) override {
+    DCHECK_EQ(input, inputs_[0]);
+    outputs_[0]->ErrorReceived(this, std::move(error));
+    inputs_[0]->StopProducing(this);
+  }
+
+  void InputFinished(ExecNode* input, int seq) override {
+    DCHECK_EQ(input, inputs_[0]);
+    outputs_[0]->InputFinished(this, seq);
+    inputs_[0]->StopProducing(this);
+  }
+
+  Status StartProducing() override {
+    // XXX validate inputs_[0]->output_descr() against filter_
+    return Status::OK();
+  }
+
+  void PauseProducing(ExecNode* output) override {}
+
+  void ResumeProducing(ExecNode* output) override {}
+
+  void StopProducing(ExecNode* output) override {
+    DCHECK_EQ(output, outputs_[0]);
+    inputs_[0]->StopProducing(this);
+  }
+
+  void StopProducing() override { StopProducing(outputs_[0]); }
+
+ private:
+  Expression filter_;
+};
+
+ExecNode* MakeFilterNode(ExecNode* input, std::string label, Expression filter) {
+  return input->plan()->EmplaceNode<FilterNode>(input, std::move(label),
+                                                std::move(filter));
+}
+
+struct ProjectNode : ExecNode {
+  ProjectNode(ExecNode* input, std::string label, std::vector<Expression> exprs)
+      : ExecNode(input->plan(), std::move(label), {input}, {"target"},
+                 /*output_descr=*/{input->output_descr()},
+                 /*num_outputs=*/1),
+        exprs_(std::move(exprs)) {}
+
+  const char* kind_name() override { return "ProjectNode"; }
+
+  Result<ExecBatch> DoProject(const ExecBatch& target) {
+    // XXX get a non-default exec context
+    std::vector<Datum> values{exprs_.size()};
+    for (size_t i = 0; i < exprs_.size(); ++i) {
+      ARROW_ASSIGN_OR_RAISE(Expression simplified_expr,
+                            SimplifyWithGuarantee(exprs_[i], target.guarantee));
+
+      ARROW_ASSIGN_OR_RAISE(values[i], ExecuteScalarExpression(simplified_expr, target));
+    }
+    return ExecBatch::Make(std::move(values));
+  }
+
+  void InputReceived(ExecNode* input, int seq, ExecBatch batch) override {
+    DCHECK_EQ(input, inputs_[0]);
+
+    auto maybe_projected = DoProject(std::move(batch));
+    if (!maybe_projected.ok()) {
+      outputs_[0]->ErrorReceived(this, maybe_projected.status());
+      inputs_[0]->StopProducing(this);
+      return;
+    }
+
+    maybe_projected->guarantee = batch.guarantee;
+    outputs_[0]->InputReceived(this, seq, maybe_projected.MoveValueUnsafe());
+  }
+
+  void ErrorReceived(ExecNode* input, Status error) override {
+    DCHECK_EQ(input, inputs_[0]);
+    outputs_[0]->ErrorReceived(this, std::move(error));
+    inputs_[0]->StopProducing(this);
+  }
+
+  void InputFinished(ExecNode* input, int seq) override {
+    DCHECK_EQ(input, inputs_[0]);
+    outputs_[0]->InputFinished(this, seq);
+    inputs_[0]->StopProducing(this);
+  }
+
+  Status StartProducing() override {
+    // XXX validate inputs_[0]->output_descr() against filter_
+    return Status::OK();
+  }
+
+  void PauseProducing(ExecNode* output) override {}
+
+  void ResumeProducing(ExecNode* output) override {}
+
+  void StopProducing(ExecNode* output) override {
+    DCHECK_EQ(output, outputs_[0]);
+    inputs_[0]->StopProducing(this);
+  }
+
+  void StopProducing() override { StopProducing(outputs_[0]); }
+
+ private:
+  std::vector<Expression> exprs_;
+};
+
+ExecNode* MakeProjectNode(ExecNode* input, std::string label,
+                          std::vector<Expression> exprs) {
+  return input->plan()->EmplaceNode<ProjectNode>(input, std::move(label),
+                                                 std::move(exprs));
+}
+
+struct SinkNode : ExecNode {
+  SinkNode(ExecNode* input, std::string label,
+           AsyncGenerator<util::optional<ExecBatch>>* generator)
+      : ExecNode(input->plan(), std::move(label), {input}, {"collected"}, {},
+                 /*num_outputs=*/0),
+        producer_(MakeProducer(generator)) {}
+
+  static PushGenerator<util::optional<ExecBatch>>::Producer MakeProducer(
+      AsyncGenerator<util::optional<ExecBatch>>* out_gen) {
+    PushGenerator<util::optional<ExecBatch>> gen;
+    auto out = gen.producer();
+    *out_gen = std::move(gen);
+    return out;
+  }
+
+  const char* kind_name() override { return "SinkNode"; }
+
+  Status StartProducing() override { return Status::OK(); }
+
+  // sink nodes have no outputs from which to feel backpressure
+  static void NoOutputs() { DCHECK(false) << "no outputs; this should never be called"; }
+  void ResumeProducing(ExecNode* output) override { NoOutputs(); }
+  void PauseProducing(ExecNode* output) override { NoOutputs(); }
+  void StopProducing(ExecNode* output) override { NoOutputs(); }
+
+  void StopProducing() override {
+    std::unique_lock<std::mutex> lock(mutex_);
+    StopProducingUnlocked();
+  }
+
+  void InputReceived(ExecNode* input, int seq_num, ExecBatch exec_batch) override {
+    std::unique_lock<std::mutex> lock(mutex_);
+    if (stopped_) return;
+
+    // TODO would be nice to factor this out in a ReorderQueue
+    if (seq_num <= static_cast<int>(received_batches_.size())) {
+      received_batches_.resize(seq_num + 1);
+      emitted_.resize(seq_num + 1, false);
+    }
+    received_batches_[seq_num] = std::move(exec_batch);
+    ++num_received_;
+
+    if (seq_num != num_emitted_) {
+      // Cannot emit yet as there is a hole at `num_emitted_`
+      DCHECK_GT(seq_num, num_emitted_);
+      return;
+    }
+
+    if (num_received_ == emit_stop_) {
+      StopProducingUnlocked();
+    }
+
+    // Emit batches in order as far as possible
+    // First collect these batches, then unlock before producing.
+    const auto seq_start = seq_num;
+    while (seq_num < static_cast<int>(emitted_.size()) && !emitted_[seq_num]) {
+      emitted_[seq_num] = true;
+      ++seq_num;
+    }
+    DCHECK_GT(seq_num, seq_start);
+    // By moving the values now, we make sure another thread won't emit the same values
+    // below
+    std::vector<ExecBatch> to_emit(
+        std::make_move_iterator(received_batches_.begin() + seq_start),
+        std::make_move_iterator(received_batches_.begin() + seq_num));
+
+    lock.unlock();
+    for (auto&& batch : to_emit) {
+      producer_.Push(std::move(batch));
+    }
+    lock.lock();
+
+    DCHECK_EQ(seq_start, num_emitted_);  // num_emitted_ wasn't bumped in the meantime
+    num_emitted_ = seq_num;

Review comment:
       In your example, `num_emitted_` is 10 until the first thread all items through 20. The second thread would simply see that 21 is larger than `num_emitted_` and return early.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] bkietz commented on a change in pull request #10397: ARROW-11930: [C++][Dataset][Compute] Use an ExecPlan for dataset scans

Posted by GitBox <gi...@apache.org>.
bkietz commented on a change in pull request #10397:
URL: https://github.com/apache/arrow/pull/10397#discussion_r660110496



##########
File path: cpp/src/arrow/testing/gtest_util.h
##########
@@ -28,6 +28,7 @@
 #include <utility>
 #include <vector>
 
+#include <gmock/gmock-matchers.h>

Review comment:
       I'll move these to `arrow/testing/matchers.h`, then




-- 
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-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] westonpace commented on a change in pull request #10397: ARROW-11930: [C++][Dataset][Compute] Use an ExecPlan for dataset scans

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #10397:
URL: https://github.com/apache/arrow/pull/10397#discussion_r644133965



##########
File path: cpp/src/arrow/compute/exec/exec_plan.cc
##########
@@ -170,48 +165,409 @@ Status ExecPlan::Validate() { return ToDerived(this)->Validate(); }
 
 Status ExecPlan::StartProducing() { return ToDerived(this)->StartProducing(); }
 
-ExecNode::ExecNode(ExecPlan* plan, std::string label,
-                   std::vector<BatchDescr> input_descrs,
+ExecNode::ExecNode(ExecPlan* plan, std::string label, NodeVector inputs,
                    std::vector<std::string> input_labels, BatchDescr output_descr,
                    int num_outputs)
     : plan_(plan),
       label_(std::move(label)),
-      input_descrs_(std::move(input_descrs)),
+      inputs_(std::move(inputs)),
       input_labels_(std::move(input_labels)),
       output_descr_(std::move(output_descr)),
-      num_outputs_(num_outputs) {}
+      num_outputs_(num_outputs) {
+  for (auto input : inputs_) {
+    input->outputs_.push_back(this);
+  }
+}
 
 Status ExecNode::Validate() const {
-  if (inputs_.size() != input_descrs_.size()) {
+  if (inputs_.size() != input_labels_.size()) {
     return Status::Invalid("Invalid number of inputs for '", label(), "' (expected ",
-                           num_inputs(), ", actual ", inputs_.size(), ")");
+                           num_inputs(), ", actual ", input_labels_.size(), ")");
   }
 
   if (static_cast<int>(outputs_.size()) != num_outputs_) {
     return Status::Invalid("Invalid number of outputs for '", label(), "' (expected ",
                            num_outputs(), ", actual ", outputs_.size(), ")");
   }
 
-  DCHECK_EQ(input_descrs_.size(), input_labels_.size());
-
   for (auto out : outputs_) {
     auto input_index = GetNodeIndex(out->inputs(), this);
     if (!input_index) {
       return Status::Invalid("Node '", label(), "' outputs to node '", out->label(),
                              "' but is not listed as an input.");
     }
+  }
 
-    const auto& in_descr = out->input_descrs_[*input_index];
-    if (in_descr != output_descr_) {
-      return Status::Invalid(
-          "Node '", label(), "' (bound to input ", input_labels_[*input_index],
-          ") produces batches with type '", ValueDescr::ToString(output_descr_),
-          "' inconsistent with consumer '", out->label(), "' which accepts '",
-          ValueDescr::ToString(in_descr), "'");
+  return Status::OK();
+}
+
+struct SourceNode : ExecNode {
+  SourceNode(ExecPlan* plan, std::string label, ExecNode::BatchDescr output_descr,
+             AsyncGenerator<util::optional<ExecBatch>> generator)
+      : ExecNode(plan, std::move(label), {}, {}, std::move(output_descr),
+                 /*num_outputs=*/1),
+        generator_(std::move(generator)) {}
+
+  const char* kind_name() override { return "SourceNode"; }
+
+  static void NoInputs() { DCHECK(false) << "no inputs; this should never be called"; }
+  void InputReceived(ExecNode*, int, ExecBatch) override { NoInputs(); }
+  void ErrorReceived(ExecNode*, Status) override { NoInputs(); }
+  void InputFinished(ExecNode*, int) override { NoInputs(); }
+
+  Status StartProducing() override {
+    if (finished_) {
+      return Status::Invalid("Restarted SourceNode '", label(), "'");
     }
+
+    auto gen = std::move(generator_);
+
+    /// XXX should we wait on this future anywhere? In StopProducing() maybe?
+    auto done_fut =
+        Loop([gen, this] {
+          std::unique_lock<std::mutex> lock(mutex_);
+          int seq = next_batch_index_++;
+          if (finished_) {
+            return Future<ControlFlow<int>>::MakeFinished(Break(seq));
+          }
+          lock.unlock();
+
+          return gen().Then(
+              [=](const util::optional<ExecBatch>& batch) -> ControlFlow<int> {
+                std::unique_lock<std::mutex> lock(mutex_);
+                if (!batch || finished_) {
+                  finished_ = true;
+                  return Break(seq);
+                }
+                lock.unlock();
+
+                outputs_[0]->InputReceived(this, seq, *batch);
+                return Continue();
+              },
+              [=](const Status& error) -> ControlFlow<int> {
+                std::unique_lock<std::mutex> lock(mutex_);
+                if (!finished_) {
+                  finished_ = true;
+                  lock.unlock();
+                  // unless we were already finished, push the error to our output
+                  // XXX is this correct? Is it reasonable for a consumer to ignore errors
+                  // from a finished producer?
+                  outputs_[0]->ErrorReceived(this, error);
+                }
+                return Break(seq);
+              });
+        }).Then([&](int seq) {
+          /// XXX this is probably redundant: do we always call InputFinished after
+          /// ErrorReceived or will ErrorRecieved be sufficient?
+          outputs_[0]->InputFinished(this, seq);
+        });
+
+    return Status::OK();
   }
 
-  return Status::OK();
+  void PauseProducing(ExecNode* output) override {}
+
+  void ResumeProducing(ExecNode* output) override {}
+
+  void StopProducing(ExecNode* output) override {
+    DCHECK_EQ(output, outputs_[0]);
+    std::unique_lock<std::mutex> lock(mutex_);
+    finished_ = true;
+  }
+
+  void StopProducing() override { StopProducing(outputs_[0]); }
+
+ private:
+  std::mutex mutex_;
+  bool finished_{false};
+  int next_batch_index_{0};
+  AsyncGenerator<util::optional<ExecBatch>> generator_;
+};
+
+ExecNode* MakeSourceNode(ExecPlan* plan, std::string label,
+                         ExecNode::BatchDescr output_descr,
+                         AsyncGenerator<util::optional<ExecBatch>> generator) {
+  return plan->EmplaceNode<SourceNode>(plan, std::move(label), std::move(output_descr),
+                                       std::move(generator));
+}
+
+struct FilterNode : ExecNode {
+  FilterNode(ExecNode* input, std::string label, Expression filter)
+      : ExecNode(input->plan(), std::move(label), {input}, {"target"},
+                 /*output_descr=*/{input->output_descr()},
+                 /*num_outputs=*/1),
+        filter_(std::move(filter)) {}
+
+  const char* kind_name() override { return "FilterNode"; }
+
+  Result<ExecBatch> DoFilter(const ExecBatch& target) {
+    ARROW_ASSIGN_OR_RAISE(Expression simplified_filter,
+                          SimplifyWithGuarantee(filter_, target.guarantee));
+
+    // XXX get a non-default exec context
+    ARROW_ASSIGN_OR_RAISE(Datum mask, ExecuteScalarExpression(simplified_filter, target));
+
+    if (mask.is_scalar()) {
+      const auto& mask_scalar = mask.scalar_as<BooleanScalar>();
+      if (mask_scalar.is_valid && mask_scalar.value) {
+        return target;
+      }
+
+      return target.Slice(0, 0);
+    }
+
+    auto values = target.values;
+    for (auto& value : values) {
+      if (value.is_scalar()) continue;
+      ARROW_ASSIGN_OR_RAISE(value, Filter(value, mask, FilterOptions::Defaults()));
+    }
+    return ExecBatch::Make(std::move(values));
+  }
+
+  void InputReceived(ExecNode* input, int seq, ExecBatch batch) override {
+    DCHECK_EQ(input, inputs_[0]);
+
+    auto maybe_filtered = DoFilter(std::move(batch));
+    if (!maybe_filtered.ok()) {
+      outputs_[0]->ErrorReceived(this, maybe_filtered.status());
+      inputs_[0]->StopProducing(this);
+      return;
+    }
+
+    maybe_filtered->guarantee = batch.guarantee;
+    outputs_[0]->InputReceived(this, seq, maybe_filtered.MoveValueUnsafe());
+  }
+
+  void ErrorReceived(ExecNode* input, Status error) override {
+    DCHECK_EQ(input, inputs_[0]);
+    outputs_[0]->ErrorReceived(this, std::move(error));
+    inputs_[0]->StopProducing(this);
+  }
+
+  void InputFinished(ExecNode* input, int seq) override {
+    DCHECK_EQ(input, inputs_[0]);
+    outputs_[0]->InputFinished(this, seq);
+    inputs_[0]->StopProducing(this);
+  }
+
+  Status StartProducing() override {
+    // XXX validate inputs_[0]->output_descr() against filter_
+    return Status::OK();
+  }
+
+  void PauseProducing(ExecNode* output) override {}
+
+  void ResumeProducing(ExecNode* output) override {}
+
+  void StopProducing(ExecNode* output) override {
+    DCHECK_EQ(output, outputs_[0]);
+    inputs_[0]->StopProducing(this);
+  }
+
+  void StopProducing() override { StopProducing(outputs_[0]); }
+
+ private:
+  Expression filter_;
+};
+
+ExecNode* MakeFilterNode(ExecNode* input, std::string label, Expression filter) {
+  return input->plan()->EmplaceNode<FilterNode>(input, std::move(label),
+                                                std::move(filter));
+}
+
+struct ProjectNode : ExecNode {
+  ProjectNode(ExecNode* input, std::string label, std::vector<Expression> exprs)
+      : ExecNode(input->plan(), std::move(label), {input}, {"target"},
+                 /*output_descr=*/{input->output_descr()},
+                 /*num_outputs=*/1),
+        exprs_(std::move(exprs)) {}
+
+  const char* kind_name() override { return "ProjectNode"; }
+
+  Result<ExecBatch> DoProject(const ExecBatch& target) {
+    // XXX get a non-default exec context
+    std::vector<Datum> values{exprs_.size()};
+    for (size_t i = 0; i < exprs_.size(); ++i) {
+      ARROW_ASSIGN_OR_RAISE(Expression simplified_expr,
+                            SimplifyWithGuarantee(exprs_[i], target.guarantee));
+
+      ARROW_ASSIGN_OR_RAISE(values[i], ExecuteScalarExpression(simplified_expr, target));
+    }
+    return ExecBatch::Make(std::move(values));
+  }
+
+  void InputReceived(ExecNode* input, int seq, ExecBatch batch) override {
+    DCHECK_EQ(input, inputs_[0]);
+
+    auto maybe_projected = DoProject(std::move(batch));
+    if (!maybe_projected.ok()) {
+      outputs_[0]->ErrorReceived(this, maybe_projected.status());
+      inputs_[0]->StopProducing(this);
+      return;
+    }
+
+    maybe_projected->guarantee = batch.guarantee;
+    outputs_[0]->InputReceived(this, seq, maybe_projected.MoveValueUnsafe());
+  }
+
+  void ErrorReceived(ExecNode* input, Status error) override {
+    DCHECK_EQ(input, inputs_[0]);
+    outputs_[0]->ErrorReceived(this, std::move(error));
+    inputs_[0]->StopProducing(this);
+  }
+
+  void InputFinished(ExecNode* input, int seq) override {
+    DCHECK_EQ(input, inputs_[0]);
+    outputs_[0]->InputFinished(this, seq);
+    inputs_[0]->StopProducing(this);
+  }
+
+  Status StartProducing() override {
+    // XXX validate inputs_[0]->output_descr() against filter_
+    return Status::OK();
+  }
+
+  void PauseProducing(ExecNode* output) override {}
+
+  void ResumeProducing(ExecNode* output) override {}
+
+  void StopProducing(ExecNode* output) override {
+    DCHECK_EQ(output, outputs_[0]);
+    inputs_[0]->StopProducing(this);
+  }
+
+  void StopProducing() override { StopProducing(outputs_[0]); }
+
+ private:
+  std::vector<Expression> exprs_;
+};
+
+ExecNode* MakeProjectNode(ExecNode* input, std::string label,
+                          std::vector<Expression> exprs) {
+  return input->plan()->EmplaceNode<ProjectNode>(input, std::move(label),
+                                                 std::move(exprs));
+}
+
+struct SinkNode : ExecNode {
+  SinkNode(ExecNode* input, std::string label,
+           AsyncGenerator<util::optional<ExecBatch>>* generator)
+      : ExecNode(input->plan(), std::move(label), {input}, {"collected"}, {},
+                 /*num_outputs=*/0),
+        producer_(MakeProducer(generator)) {}
+
+  static PushGenerator<util::optional<ExecBatch>>::Producer MakeProducer(
+      AsyncGenerator<util::optional<ExecBatch>>* out_gen) {
+    PushGenerator<util::optional<ExecBatch>> gen;
+    auto out = gen.producer();
+    *out_gen = std::move(gen);
+    return out;
+  }
+
+  const char* kind_name() override { return "SinkNode"; }
+
+  Status StartProducing() override { return Status::OK(); }
+
+  // sink nodes have no outputs from which to feel backpressure
+  static void NoOutputs() { DCHECK(false) << "no outputs; this should never be called"; }
+  void ResumeProducing(ExecNode* output) override { NoOutputs(); }
+  void PauseProducing(ExecNode* output) override { NoOutputs(); }
+  void StopProducing(ExecNode* output) override { NoOutputs(); }
+
+  void StopProducing() override {
+    std::unique_lock<std::mutex> lock(mutex_);
+    StopProducingUnlocked();
+  }
+
+  void InputReceived(ExecNode* input, int seq_num, ExecBatch exec_batch) override {
+    std::unique_lock<std::mutex> lock(mutex_);
+    if (stopped_) return;
+
+    // TODO would be nice to factor this out in a ReorderQueue
+    if (seq_num <= static_cast<int>(received_batches_.size())) {
+      received_batches_.resize(seq_num + 1);
+      emitted_.resize(seq_num + 1, false);
+    }
+    received_batches_[seq_num] = std::move(exec_batch);
+    ++num_received_;
+
+    if (seq_num != num_emitted_) {
+      // Cannot emit yet as there is a hole at `num_emitted_`
+      DCHECK_GT(seq_num, num_emitted_);
+      return;
+    }
+
+    if (num_received_ == emit_stop_) {
+      StopProducingUnlocked();
+    }
+
+    // Emit batches in order as far as possible
+    // First collect these batches, then unlock before producing.
+    const auto seq_start = seq_num;
+    while (seq_num < static_cast<int>(emitted_.size()) && !emitted_[seq_num]) {
+      emitted_[seq_num] = true;
+      ++seq_num;
+    }
+    DCHECK_GT(seq_num, seq_start);
+    // By moving the values now, we make sure another thread won't emit the same values
+    // below
+    std::vector<ExecBatch> to_emit(
+        std::make_move_iterator(received_batches_.begin() + seq_start),
+        std::make_move_iterator(received_batches_.begin() + seq_num));
+
+    lock.unlock();
+    for (auto&& batch : to_emit) {
+      producer_.Push(std::move(batch));
+    }
+    lock.lock();
+
+    DCHECK_EQ(seq_start, num_emitted_);  // num_emitted_ wasn't bumped in the meantime
+    num_emitted_ = seq_num;
+  }
+
+  void ErrorReceived(ExecNode* input, Status error) override {
+    // XXX do we care about properly sequencing the error?

Review comment:
       I think we should not try and sequence the error in any fashion.  A similar question comes up with generators which is "when an error arrives, should we drain the queue of valid results first?  Or simply return the error as quickly as possible?"  I went with the "purge all valid results and return error as quickly as possible" approach.  However, I think @lidavidm and I disagreed on this point but neither of us felt too strongly about it.  A counterpoint that I think David brought up at the time was the case of someone trying to inspect or work with a broken dataset by using a limit operation to read as much as they could before the broken part.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] bkietz commented on a change in pull request #10397: ARROW-11930: [C++][Dataset][Compute] Use an ExecPlan for dataset scans

Posted by GitBox <gi...@apache.org>.
bkietz commented on a change in pull request #10397:
URL: https://github.com/apache/arrow/pull/10397#discussion_r644303096



##########
File path: cpp/src/arrow/compute/exec/exec_plan.cc
##########
@@ -170,48 +165,409 @@ Status ExecPlan::Validate() { return ToDerived(this)->Validate(); }
 
 Status ExecPlan::StartProducing() { return ToDerived(this)->StartProducing(); }
 
-ExecNode::ExecNode(ExecPlan* plan, std::string label,
-                   std::vector<BatchDescr> input_descrs,
+ExecNode::ExecNode(ExecPlan* plan, std::string label, NodeVector inputs,
                    std::vector<std::string> input_labels, BatchDescr output_descr,
                    int num_outputs)
     : plan_(plan),
       label_(std::move(label)),
-      input_descrs_(std::move(input_descrs)),
+      inputs_(std::move(inputs)),
       input_labels_(std::move(input_labels)),
       output_descr_(std::move(output_descr)),
-      num_outputs_(num_outputs) {}
+      num_outputs_(num_outputs) {
+  for (auto input : inputs_) {
+    input->outputs_.push_back(this);
+  }
+}
 
 Status ExecNode::Validate() const {
-  if (inputs_.size() != input_descrs_.size()) {
+  if (inputs_.size() != input_labels_.size()) {
     return Status::Invalid("Invalid number of inputs for '", label(), "' (expected ",
-                           num_inputs(), ", actual ", inputs_.size(), ")");
+                           num_inputs(), ", actual ", input_labels_.size(), ")");
   }
 
   if (static_cast<int>(outputs_.size()) != num_outputs_) {
     return Status::Invalid("Invalid number of outputs for '", label(), "' (expected ",
                            num_outputs(), ", actual ", outputs_.size(), ")");
   }
 
-  DCHECK_EQ(input_descrs_.size(), input_labels_.size());
-
   for (auto out : outputs_) {
     auto input_index = GetNodeIndex(out->inputs(), this);
     if (!input_index) {
       return Status::Invalid("Node '", label(), "' outputs to node '", out->label(),
                              "' but is not listed as an input.");
     }
+  }
 
-    const auto& in_descr = out->input_descrs_[*input_index];
-    if (in_descr != output_descr_) {
-      return Status::Invalid(
-          "Node '", label(), "' (bound to input ", input_labels_[*input_index],
-          ") produces batches with type '", ValueDescr::ToString(output_descr_),
-          "' inconsistent with consumer '", out->label(), "' which accepts '",
-          ValueDescr::ToString(in_descr), "'");
+  return Status::OK();
+}
+
+struct SourceNode : ExecNode {
+  SourceNode(ExecPlan* plan, std::string label, ExecNode::BatchDescr output_descr,
+             AsyncGenerator<util::optional<ExecBatch>> generator)
+      : ExecNode(plan, std::move(label), {}, {}, std::move(output_descr),
+                 /*num_outputs=*/1),
+        generator_(std::move(generator)) {}
+
+  const char* kind_name() override { return "SourceNode"; }
+
+  static void NoInputs() { DCHECK(false) << "no inputs; this should never be called"; }
+  void InputReceived(ExecNode*, int, ExecBatch) override { NoInputs(); }
+  void ErrorReceived(ExecNode*, Status) override { NoInputs(); }
+  void InputFinished(ExecNode*, int) override { NoInputs(); }
+
+  Status StartProducing() override {
+    if (finished_) {
+      return Status::Invalid("Restarted SourceNode '", label(), "'");
     }
+
+    auto gen = std::move(generator_);
+
+    /// XXX should we wait on this future anywhere? In StopProducing() maybe?
+    auto done_fut =
+        Loop([gen, this] {
+          std::unique_lock<std::mutex> lock(mutex_);
+          int seq = next_batch_index_++;
+          if (finished_) {
+            return Future<ControlFlow<int>>::MakeFinished(Break(seq));
+          }
+          lock.unlock();
+
+          return gen().Then(
+              [=](const util::optional<ExecBatch>& batch) -> ControlFlow<int> {
+                std::unique_lock<std::mutex> lock(mutex_);
+                if (!batch || finished_) {
+                  finished_ = true;
+                  return Break(seq);
+                }
+                lock.unlock();
+
+                outputs_[0]->InputReceived(this, seq, *batch);
+                return Continue();
+              },
+              [=](const Status& error) -> ControlFlow<int> {
+                std::unique_lock<std::mutex> lock(mutex_);
+                if (!finished_) {
+                  finished_ = true;
+                  lock.unlock();
+                  // unless we were already finished, push the error to our output
+                  // XXX is this correct? Is it reasonable for a consumer to ignore errors
+                  // from a finished producer?
+                  outputs_[0]->ErrorReceived(this, error);
+                }
+                return Break(seq);
+              });
+        }).Then([&](int seq) {
+          /// XXX this is probably redundant: do we always call InputFinished after
+          /// ErrorReceived or will ErrorRecieved be sufficient?
+          outputs_[0]->InputFinished(this, seq);
+        });
+
+    return Status::OK();
   }
 
-  return Status::OK();
+  void PauseProducing(ExecNode* output) override {}
+
+  void ResumeProducing(ExecNode* output) override {}
+
+  void StopProducing(ExecNode* output) override {
+    DCHECK_EQ(output, outputs_[0]);
+    std::unique_lock<std::mutex> lock(mutex_);
+    finished_ = true;
+  }
+
+  void StopProducing() override { StopProducing(outputs_[0]); }
+
+ private:
+  std::mutex mutex_;
+  bool finished_{false};
+  int next_batch_index_{0};
+  AsyncGenerator<util::optional<ExecBatch>> generator_;
+};
+
+ExecNode* MakeSourceNode(ExecPlan* plan, std::string label,
+                         ExecNode::BatchDescr output_descr,
+                         AsyncGenerator<util::optional<ExecBatch>> generator) {
+  return plan->EmplaceNode<SourceNode>(plan, std::move(label), std::move(output_descr),
+                                       std::move(generator));
+}
+
+struct FilterNode : ExecNode {
+  FilterNode(ExecNode* input, std::string label, Expression filter)
+      : ExecNode(input->plan(), std::move(label), {input}, {"target"},
+                 /*output_descr=*/{input->output_descr()},
+                 /*num_outputs=*/1),
+        filter_(std::move(filter)) {}
+
+  const char* kind_name() override { return "FilterNode"; }
+
+  Result<ExecBatch> DoFilter(const ExecBatch& target) {
+    ARROW_ASSIGN_OR_RAISE(Expression simplified_filter,
+                          SimplifyWithGuarantee(filter_, target.guarantee));
+
+    // XXX get a non-default exec context
+    ARROW_ASSIGN_OR_RAISE(Datum mask, ExecuteScalarExpression(simplified_filter, target));
+
+    if (mask.is_scalar()) {
+      const auto& mask_scalar = mask.scalar_as<BooleanScalar>();
+      if (mask_scalar.is_valid && mask_scalar.value) {
+        return target;
+      }
+
+      return target.Slice(0, 0);
+    }
+
+    auto values = target.values;
+    for (auto& value : values) {
+      if (value.is_scalar()) continue;
+      ARROW_ASSIGN_OR_RAISE(value, Filter(value, mask, FilterOptions::Defaults()));
+    }
+    return ExecBatch::Make(std::move(values));
+  }
+
+  void InputReceived(ExecNode* input, int seq, ExecBatch batch) override {
+    DCHECK_EQ(input, inputs_[0]);
+
+    auto maybe_filtered = DoFilter(std::move(batch));
+    if (!maybe_filtered.ok()) {
+      outputs_[0]->ErrorReceived(this, maybe_filtered.status());
+      inputs_[0]->StopProducing(this);
+      return;
+    }
+
+    maybe_filtered->guarantee = batch.guarantee;
+    outputs_[0]->InputReceived(this, seq, maybe_filtered.MoveValueUnsafe());
+  }
+
+  void ErrorReceived(ExecNode* input, Status error) override {
+    DCHECK_EQ(input, inputs_[0]);
+    outputs_[0]->ErrorReceived(this, std::move(error));
+    inputs_[0]->StopProducing(this);
+  }
+
+  void InputFinished(ExecNode* input, int seq) override {
+    DCHECK_EQ(input, inputs_[0]);
+    outputs_[0]->InputFinished(this, seq);
+    inputs_[0]->StopProducing(this);
+  }
+
+  Status StartProducing() override {
+    // XXX validate inputs_[0]->output_descr() against filter_
+    return Status::OK();
+  }
+
+  void PauseProducing(ExecNode* output) override {}
+
+  void ResumeProducing(ExecNode* output) override {}
+
+  void StopProducing(ExecNode* output) override {
+    DCHECK_EQ(output, outputs_[0]);
+    inputs_[0]->StopProducing(this);
+  }
+
+  void StopProducing() override { StopProducing(outputs_[0]); }
+
+ private:
+  Expression filter_;
+};
+
+ExecNode* MakeFilterNode(ExecNode* input, std::string label, Expression filter) {
+  return input->plan()->EmplaceNode<FilterNode>(input, std::move(label),
+                                                std::move(filter));
+}
+
+struct ProjectNode : ExecNode {
+  ProjectNode(ExecNode* input, std::string label, std::vector<Expression> exprs)
+      : ExecNode(input->plan(), std::move(label), {input}, {"target"},
+                 /*output_descr=*/{input->output_descr()},
+                 /*num_outputs=*/1),
+        exprs_(std::move(exprs)) {}
+
+  const char* kind_name() override { return "ProjectNode"; }
+
+  Result<ExecBatch> DoProject(const ExecBatch& target) {
+    // XXX get a non-default exec context
+    std::vector<Datum> values{exprs_.size()};
+    for (size_t i = 0; i < exprs_.size(); ++i) {
+      ARROW_ASSIGN_OR_RAISE(Expression simplified_expr,
+                            SimplifyWithGuarantee(exprs_[i], target.guarantee));
+
+      ARROW_ASSIGN_OR_RAISE(values[i], ExecuteScalarExpression(simplified_expr, target));
+    }
+    return ExecBatch::Make(std::move(values));
+  }
+
+  void InputReceived(ExecNode* input, int seq, ExecBatch batch) override {
+    DCHECK_EQ(input, inputs_[0]);
+
+    auto maybe_projected = DoProject(std::move(batch));
+    if (!maybe_projected.ok()) {
+      outputs_[0]->ErrorReceived(this, maybe_projected.status());
+      inputs_[0]->StopProducing(this);
+      return;
+    }
+
+    maybe_projected->guarantee = batch.guarantee;
+    outputs_[0]->InputReceived(this, seq, maybe_projected.MoveValueUnsafe());
+  }
+
+  void ErrorReceived(ExecNode* input, Status error) override {
+    DCHECK_EQ(input, inputs_[0]);
+    outputs_[0]->ErrorReceived(this, std::move(error));
+    inputs_[0]->StopProducing(this);
+  }
+
+  void InputFinished(ExecNode* input, int seq) override {
+    DCHECK_EQ(input, inputs_[0]);
+    outputs_[0]->InputFinished(this, seq);
+    inputs_[0]->StopProducing(this);
+  }
+
+  Status StartProducing() override {
+    // XXX validate inputs_[0]->output_descr() against filter_
+    return Status::OK();
+  }
+
+  void PauseProducing(ExecNode* output) override {}

Review comment:
       These are currently stubs due to lack of support for pausing in any source node. For now, I'll remove these and add a follow up to support pause/resume




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] bkietz commented on a change in pull request #10397: ARROW-11930: [C++][Dataset][Compute] Use an ExecPlan for dataset scans

Posted by GitBox <gi...@apache.org>.
bkietz commented on a change in pull request #10397:
URL: https://github.com/apache/arrow/pull/10397#discussion_r644836145



##########
File path: cpp/src/arrow/dataset/dataset_internal.h
##########
@@ -204,5 +204,35 @@ arrow::Result<std::shared_ptr<T>> GetFragmentScanOptions(
   return internal::checked_pointer_cast<T>(source);
 }
 
+class FragmentDataset : public Dataset {

Review comment:
       Added https://issues.apache.org/jira/browse/ARROW-12945 to track this improvement.

##########
File path: cpp/src/arrow/compute/exec/exec_plan.cc
##########
@@ -170,48 +165,409 @@ Status ExecPlan::Validate() { return ToDerived(this)->Validate(); }
 
 Status ExecPlan::StartProducing() { return ToDerived(this)->StartProducing(); }
 
-ExecNode::ExecNode(ExecPlan* plan, std::string label,
-                   std::vector<BatchDescr> input_descrs,
+ExecNode::ExecNode(ExecPlan* plan, std::string label, NodeVector inputs,
                    std::vector<std::string> input_labels, BatchDescr output_descr,
                    int num_outputs)
     : plan_(plan),
       label_(std::move(label)),
-      input_descrs_(std::move(input_descrs)),
+      inputs_(std::move(inputs)),
       input_labels_(std::move(input_labels)),
       output_descr_(std::move(output_descr)),
-      num_outputs_(num_outputs) {}
+      num_outputs_(num_outputs) {
+  for (auto input : inputs_) {
+    input->outputs_.push_back(this);
+  }
+}
 
 Status ExecNode::Validate() const {
-  if (inputs_.size() != input_descrs_.size()) {
+  if (inputs_.size() != input_labels_.size()) {
     return Status::Invalid("Invalid number of inputs for '", label(), "' (expected ",
-                           num_inputs(), ", actual ", inputs_.size(), ")");
+                           num_inputs(), ", actual ", input_labels_.size(), ")");
   }
 
   if (static_cast<int>(outputs_.size()) != num_outputs_) {
     return Status::Invalid("Invalid number of outputs for '", label(), "' (expected ",
                            num_outputs(), ", actual ", outputs_.size(), ")");
   }
 
-  DCHECK_EQ(input_descrs_.size(), input_labels_.size());
-
   for (auto out : outputs_) {
     auto input_index = GetNodeIndex(out->inputs(), this);
     if (!input_index) {
       return Status::Invalid("Node '", label(), "' outputs to node '", out->label(),
                              "' but is not listed as an input.");
     }
+  }
 
-    const auto& in_descr = out->input_descrs_[*input_index];
-    if (in_descr != output_descr_) {
-      return Status::Invalid(
-          "Node '", label(), "' (bound to input ", input_labels_[*input_index],
-          ") produces batches with type '", ValueDescr::ToString(output_descr_),
-          "' inconsistent with consumer '", out->label(), "' which accepts '",
-          ValueDescr::ToString(in_descr), "'");
+  return Status::OK();
+}
+
+struct SourceNode : ExecNode {
+  SourceNode(ExecPlan* plan, std::string label, ExecNode::BatchDescr output_descr,
+             AsyncGenerator<util::optional<ExecBatch>> generator)
+      : ExecNode(plan, std::move(label), {}, {}, std::move(output_descr),
+                 /*num_outputs=*/1),
+        generator_(std::move(generator)) {}
+
+  const char* kind_name() override { return "SourceNode"; }
+
+  static void NoInputs() { DCHECK(false) << "no inputs; this should never be called"; }
+  void InputReceived(ExecNode*, int, ExecBatch) override { NoInputs(); }
+  void ErrorReceived(ExecNode*, Status) override { NoInputs(); }
+  void InputFinished(ExecNode*, int) override { NoInputs(); }
+
+  Status StartProducing() override {
+    if (finished_) {
+      return Status::Invalid("Restarted SourceNode '", label(), "'");
     }
+
+    auto gen = std::move(generator_);
+
+    /// XXX should we wait on this future anywhere? In StopProducing() maybe?

Review comment:
       Perhaps in `StopProducing()`? Then we append to the ExecNode contract that StopProducing must be called on a producing node before it is destroyed?




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] bkietz commented on a change in pull request #10397: ARROW-11930: [C++][Dataset][Compute] Use an ExecPlan for dataset scans

Posted by GitBox <gi...@apache.org>.
bkietz commented on a change in pull request #10397:
URL: https://github.com/apache/arrow/pull/10397#discussion_r643578583



##########
File path: cpp/src/arrow/compute/exec/expression.cc
##########
@@ -613,6 +639,22 @@ std::vector<FieldRef> FieldsInExpression(const Expression& expr) {
   return fields;
 }
 
+std::vector<int> ParametersInExpression(const Expression& expr) {

Review comment:
       I may remove this, I don't think I'm using it anymore




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] westonpace commented on a change in pull request #10397: ARROW-11930: [C++][Dataset][Compute] Use an ExecPlan for dataset scans

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #10397:
URL: https://github.com/apache/arrow/pull/10397#discussion_r644150584



##########
File path: cpp/src/arrow/compute/exec/exec_plan.cc
##########
@@ -170,48 +165,409 @@ Status ExecPlan::Validate() { return ToDerived(this)->Validate(); }
 
 Status ExecPlan::StartProducing() { return ToDerived(this)->StartProducing(); }
 
-ExecNode::ExecNode(ExecPlan* plan, std::string label,
-                   std::vector<BatchDescr> input_descrs,
+ExecNode::ExecNode(ExecPlan* plan, std::string label, NodeVector inputs,
                    std::vector<std::string> input_labels, BatchDescr output_descr,
                    int num_outputs)
     : plan_(plan),
       label_(std::move(label)),
-      input_descrs_(std::move(input_descrs)),
+      inputs_(std::move(inputs)),
       input_labels_(std::move(input_labels)),
       output_descr_(std::move(output_descr)),
-      num_outputs_(num_outputs) {}
+      num_outputs_(num_outputs) {
+  for (auto input : inputs_) {
+    input->outputs_.push_back(this);
+  }
+}
 
 Status ExecNode::Validate() const {
-  if (inputs_.size() != input_descrs_.size()) {
+  if (inputs_.size() != input_labels_.size()) {
     return Status::Invalid("Invalid number of inputs for '", label(), "' (expected ",
-                           num_inputs(), ", actual ", inputs_.size(), ")");
+                           num_inputs(), ", actual ", input_labels_.size(), ")");
   }
 
   if (static_cast<int>(outputs_.size()) != num_outputs_) {
     return Status::Invalid("Invalid number of outputs for '", label(), "' (expected ",
                            num_outputs(), ", actual ", outputs_.size(), ")");
   }
 
-  DCHECK_EQ(input_descrs_.size(), input_labels_.size());
-
   for (auto out : outputs_) {
     auto input_index = GetNodeIndex(out->inputs(), this);
     if (!input_index) {
       return Status::Invalid("Node '", label(), "' outputs to node '", out->label(),
                              "' but is not listed as an input.");
     }
+  }
 
-    const auto& in_descr = out->input_descrs_[*input_index];
-    if (in_descr != output_descr_) {
-      return Status::Invalid(
-          "Node '", label(), "' (bound to input ", input_labels_[*input_index],
-          ") produces batches with type '", ValueDescr::ToString(output_descr_),
-          "' inconsistent with consumer '", out->label(), "' which accepts '",
-          ValueDescr::ToString(in_descr), "'");
+  return Status::OK();
+}
+
+struct SourceNode : ExecNode {
+  SourceNode(ExecPlan* plan, std::string label, ExecNode::BatchDescr output_descr,
+             AsyncGenerator<util::optional<ExecBatch>> generator)
+      : ExecNode(plan, std::move(label), {}, {}, std::move(output_descr),
+                 /*num_outputs=*/1),
+        generator_(std::move(generator)) {}
+
+  const char* kind_name() override { return "SourceNode"; }
+
+  static void NoInputs() { DCHECK(false) << "no inputs; this should never be called"; }
+  void InputReceived(ExecNode*, int, ExecBatch) override { NoInputs(); }
+  void ErrorReceived(ExecNode*, Status) override { NoInputs(); }
+  void InputFinished(ExecNode*, int) override { NoInputs(); }
+
+  Status StartProducing() override {
+    if (finished_) {
+      return Status::Invalid("Restarted SourceNode '", label(), "'");
     }
+
+    auto gen = std::move(generator_);
+
+    /// XXX should we wait on this future anywhere? In StopProducing() maybe?
+    auto done_fut =
+        Loop([gen, this] {
+          std::unique_lock<std::mutex> lock(mutex_);
+          int seq = next_batch_index_++;
+          if (finished_) {
+            return Future<ControlFlow<int>>::MakeFinished(Break(seq));
+          }
+          lock.unlock();
+
+          return gen().Then(
+              [=](const util::optional<ExecBatch>& batch) -> ControlFlow<int> {
+                std::unique_lock<std::mutex> lock(mutex_);
+                if (!batch || finished_) {
+                  finished_ = true;
+                  return Break(seq);
+                }
+                lock.unlock();
+
+                outputs_[0]->InputReceived(this, seq, *batch);
+                return Continue();
+              },
+              [=](const Status& error) -> ControlFlow<int> {
+                std::unique_lock<std::mutex> lock(mutex_);
+                if (!finished_) {
+                  finished_ = true;
+                  lock.unlock();
+                  // unless we were already finished, push the error to our output
+                  // XXX is this correct? Is it reasonable for a consumer to ignore errors
+                  // from a finished producer?

Review comment:
       I would agree to keep the logic as simple as possible.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] bkietz commented on a change in pull request #10397: ARROW-11930: [C++][Dataset][Compute] Use an ExecPlan for dataset scans

Posted by GitBox <gi...@apache.org>.
bkietz commented on a change in pull request #10397:
URL: https://github.com/apache/arrow/pull/10397#discussion_r660116713



##########
File path: cpp/src/arrow/util/vector.h
##########
@@ -84,27 +84,49 @@ std::vector<T> FilterVector(std::vector<T> values, Predicate&& predicate) {
   return values;
 }
 
-/// \brief Like MapVector, but where the function can fail.
-template <typename Fn, typename From = internal::call_traits::argument_type<0, Fn>,
-          typename To = typename internal::call_traits::return_type<Fn>::ValueType>
-Result<std::vector<To>> MaybeMapVector(Fn&& map, const std::vector<From>& src) {
+template <typename Fn, typename From,
+          typename To = decltype(std::declval<Fn>()(std::declval<From>()))>
+std::vector<To> MapVector(Fn&& map, const std::vector<From>& source) {
   std::vector<To> out;
-  out.reserve(src.size());
-  ARROW_RETURN_NOT_OK(MaybeTransform(src.begin(), src.end(), std::back_inserter(out),
-                                     std::forward<Fn>(map)));
-  return std::move(out);
+  out.reserve(source.size());
+  std::transform(source.begin(), source.end(), std::back_inserter(out),
+                 std::forward<Fn>(map));
+  return out;
 }
 
 template <typename Fn, typename From,
           typename To = decltype(std::declval<Fn>()(std::declval<From>()))>
-std::vector<To> MapVector(Fn&& map, const std::vector<From>& source) {
+std::vector<To> MapVector(Fn&& map, std::vector<From>&& source) {
   std::vector<To> out;
   out.reserve(source.size());
-  std::transform(source.begin(), source.end(), std::back_inserter(out),
+  std::transform(std::make_move_iterator(source.begin()),
+                 std::make_move_iterator(source.end()), std::back_inserter(out),
                  std::forward<Fn>(map));
   return out;
 }
 
+/// \brief Like MapVector, but where the function can fail.
+template <typename Fn, typename From = internal::call_traits::argument_type<0, Fn>,
+          typename To = typename internal::call_traits::return_type<Fn>::ValueType>

Review comment:
       there's not a good reason; just uniformity with getting `From` from call_traits.




-- 
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-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] bkietz commented on a change in pull request #10397: ARROW-11930: [C++][Dataset][Compute] Use an ExecPlan for dataset scans

Posted by GitBox <gi...@apache.org>.
bkietz commented on a change in pull request #10397:
URL: https://github.com/apache/arrow/pull/10397#discussion_r660110832



##########
File path: cpp/src/arrow/compute/exec/plan_test.cc
##########
@@ -20,27 +20,46 @@
 #include <functional>
 #include <memory>
 
+#include "arrow/compute/exec.h"
 #include "arrow/compute/exec/exec_plan.h"
+#include "arrow/compute/exec/expression.h"
 #include "arrow/compute/exec/test_util.h"
 #include "arrow/record_batch.h"
 #include "arrow/testing/future_util.h"
 #include "arrow/testing/gtest_util.h"
 #include "arrow/testing/random.h"
 #include "arrow/util/logging.h"
 #include "arrow/util/thread_pool.h"
+#include "arrow/util/vector.h"
 
-namespace arrow {
+using testing::ElementsAre;
+using testing::HasSubstr;
+using testing::UnorderedElementsAreArray;
 
-using internal::Executor;
+namespace arrow {
 
 namespace compute {
 
-void AssertBatchesEqual(const RecordBatchVector& expected,
-                        const RecordBatchVector& actual) {
-  ASSERT_EQ(expected.size(), actual.size());
-  for (size_t i = 0; i < expected.size(); ++i) {
-    AssertBatchesEqual(*expected[i], *actual[i]);
+ExecBatch ExecBatchFromJSON(const std::vector<ValueDescr>& descrs,

Review comment:
       I'll put it in arrow/compute/test_util.h




-- 
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-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] westonpace commented on a change in pull request #10397: ARROW-11930: [C++][Dataset][Compute] Use an ExecPlan for dataset scans

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #10397:
URL: https://github.com/apache/arrow/pull/10397#discussion_r644285918



##########
File path: cpp/src/arrow/compute/exec/test_util.cc
##########
@@ -124,277 +130,42 @@ struct DummyNode : ExecNode {
   bool started_ = false;
 };
 
-struct RecordBatchReaderNode : ExecNode {
-  RecordBatchReaderNode(ExecPlan* plan, std::string label,
-                        std::shared_ptr<RecordBatchReader> reader, Executor* io_executor)
-      : ExecNode(plan, std::move(label), {}, {},
-                 DescrFromSchemaColumns(*reader->schema()), /*num_outputs=*/1),
-        schema_(reader->schema()),
-        reader_(std::move(reader)),
-        io_executor_(io_executor) {}
-
-  RecordBatchReaderNode(ExecPlan* plan, std::string label, std::shared_ptr<Schema> schema,
-                        RecordBatchGenerator generator, Executor* io_executor)
-      : ExecNode(plan, std::move(label), {}, {}, DescrFromSchemaColumns(*schema),
-                 /*num_outputs=*/1),
-        schema_(std::move(schema)),
-        generator_(std::move(generator)),
-        io_executor_(io_executor) {}
-
-  const char* kind_name() override { return "RecordBatchReader"; }
-
-  void InputReceived(ExecNode* input, int seq_num, compute::ExecBatch batch) override {}
-
-  void ErrorReceived(ExecNode* input, Status error) override {}
-
-  void InputFinished(ExecNode* input, int seq_stop) override {}
-
-  Status StartProducing() override {
-    next_batch_index_ = 0;
-    if (!generator_) {
-      auto it = MakeIteratorFromReader(reader_);
-      ARROW_ASSIGN_OR_RAISE(generator_,
-                            MakeBackgroundGenerator(std::move(it), io_executor_));
-    }
-    GenerateOne(std::unique_lock<std::mutex>{mutex_});
-    return Status::OK();
-  }
-
-  void PauseProducing(ExecNode* output) override {}
-
-  void ResumeProducing(ExecNode* output) override {}
-
-  void StopProducing(ExecNode* output) override {
-    ASSERT_EQ(output, outputs_[0]);
-    std::unique_lock<std::mutex> lock(mutex_);
-    generator_ = nullptr;  // null function
-  }
-
-  void StopProducing() override { StopProducing(outputs_[0]); }
-
- private:
-  void GenerateOne(std::unique_lock<std::mutex>&& lock) {
-    if (!generator_) {
-      // Stopped
-      return;
-    }
-    auto plan = this->plan()->shared_from_this();
-    auto fut = generator_();
-    const auto batch_index = next_batch_index_++;
-
-    lock.unlock();
-    // TODO we want to transfer always here
-    io_executor_->Transfer(std::move(fut))
-        .AddCallback(
-            [plan, batch_index, this](const Result<std::shared_ptr<RecordBatch>>& res) {
-              std::unique_lock<std::mutex> lock(mutex_);
-              if (!res.ok()) {
-                for (auto out : outputs_) {
-                  out->ErrorReceived(this, res.status());
-                }
-                return;
-              }
-              const auto& batch = *res;
-              if (IsIterationEnd(batch)) {
-                lock.unlock();
-                for (auto out : outputs_) {
-                  out->InputFinished(this, batch_index);
-                }
-              } else {
-                lock.unlock();
-                for (auto out : outputs_) {
-                  out->InputReceived(this, batch_index, compute::ExecBatch(*batch));
-                }
-                lock.lock();
-                GenerateOne(std::move(lock));
-              }
-            });
-  }
-
-  std::mutex mutex_;
-  const std::shared_ptr<Schema> schema_;
-  const std::shared_ptr<RecordBatchReader> reader_;
-  RecordBatchGenerator generator_;
-  int next_batch_index_;
-
-  Executor* const io_executor_;
-};
-
-struct RecordBatchCollectNodeImpl : public RecordBatchCollectNode {
-  RecordBatchCollectNodeImpl(ExecPlan* plan, std::string label,
-                             std::shared_ptr<Schema> schema)
-      : RecordBatchCollectNode(plan, std::move(label), {DescrFromSchemaColumns(*schema)},
-                               {"batches_to_collect"}, {}, 0),
-        schema_(std::move(schema)) {}
-
-  RecordBatchGenerator generator() override { return generator_; }
-
-  const char* kind_name() override { return "RecordBatchReader"; }
-
-  Status StartProducing() override {
-    num_received_ = 0;
-    num_emitted_ = 0;
-    emit_stop_ = -1;
-    stopped_ = false;
-    producer_.emplace(generator_.producer());
-    return Status::OK();
-  }
-
-  // sink nodes have no outputs from which to feel backpressure
-  void ResumeProducing(ExecNode* output) override {
-    FAIL() << "no outputs; this should never be called";
-  }
-  void PauseProducing(ExecNode* output) override {
-    FAIL() << "no outputs; this should never be called";
-  }
-  void StopProducing(ExecNode* output) override {
-    FAIL() << "no outputs; this should never be called";
-  }
-
-  void StopProducing() override {
-    std::unique_lock<std::mutex> lock(mutex_);
-    StopProducingUnlocked();
-  }
-
-  void InputReceived(ExecNode* input, int seq_num,
-                     compute::ExecBatch exec_batch) override {
-    std::unique_lock<std::mutex> lock(mutex_);
-    if (stopped_) {
-      return;
-    }
-    auto maybe_batch = MakeBatch(std::move(exec_batch));
-    if (!maybe_batch.ok()) {
-      lock.unlock();
-      producer_->Push(std::move(maybe_batch));
-      return;
-    }
-
-    // TODO would be nice to factor this out in a ReorderQueue
-    auto batch = *std::move(maybe_batch);
-    if (seq_num <= static_cast<int>(received_batches_.size())) {
-      received_batches_.resize(seq_num + 1, nullptr);
-    }
-    DCHECK_EQ(received_batches_[seq_num], nullptr);
-    received_batches_[seq_num] = std::move(batch);
-    ++num_received_;
-
-    if (seq_num != num_emitted_) {
-      // Cannot emit yet as there is a hole at `num_emitted_`
-      DCHECK_GT(seq_num, num_emitted_);
-      DCHECK_EQ(received_batches_[num_emitted_], nullptr);
-      return;
-    }
-    if (num_received_ == emit_stop_) {
-      StopProducingUnlocked();
-    }
-
-    // Emit batches in order as far as possible
-    // First collect these batches, then unlock before producing.
-    const auto seq_start = seq_num;
-    while (seq_num < static_cast<int>(received_batches_.size()) &&
-           received_batches_[seq_num] != nullptr) {
-      ++seq_num;
-    }
-    DCHECK_GT(seq_num, seq_start);
-    // By moving the values now, we make sure another thread won't emit the same values
-    // below
-    RecordBatchVector to_emit(
-        std::make_move_iterator(received_batches_.begin() + seq_start),
-        std::make_move_iterator(received_batches_.begin() + seq_num));
-
-    lock.unlock();
-    for (auto&& batch : to_emit) {
-      producer_->Push(std::move(batch));
-    }
-    lock.lock();
-
-    DCHECK_EQ(seq_start, num_emitted_);  // num_emitted_ wasn't bumped in the meantime
-    num_emitted_ = seq_num;
-  }
-
-  void ErrorReceived(ExecNode* input, Status error) override {
-    // XXX do we care about properly sequencing the error?
-    producer_->Push(std::move(error));
-    std::unique_lock<std::mutex> lock(mutex_);
-    StopProducingUnlocked();
-  }
-
-  void InputFinished(ExecNode* input, int seq_stop) override {
-    std::unique_lock<std::mutex> lock(mutex_);
-    DCHECK_GE(seq_stop, static_cast<int>(received_batches_.size()));
-    received_batches_.reserve(seq_stop);
-    emit_stop_ = seq_stop;
-    if (emit_stop_ == num_received_) {
-      DCHECK_EQ(emit_stop_, num_emitted_);
-      StopProducingUnlocked();
-    }
-  }
-
- private:
-  void StopProducingUnlocked() {
-    if (!stopped_) {
-      stopped_ = true;
-      producer_->Close();
-      inputs_[0]->StopProducing(this);
-    }
-  }
-
-  // TODO factor this out as ExecBatch::ToRecordBatch()?
-  Result<std::shared_ptr<RecordBatch>> MakeBatch(compute::ExecBatch&& exec_batch) {
-    ArrayDataVector columns;
-    columns.reserve(exec_batch.values.size());
-    for (auto&& value : exec_batch.values) {
-      if (!value.is_array()) {
-        return Status::TypeError("Expected array input");
-      }
-      columns.push_back(std::move(value).array());
-    }
-    return RecordBatch::Make(schema_, exec_batch.length, std::move(columns));
-  }
-
-  const std::shared_ptr<Schema> schema_;
-
-  std::mutex mutex_;
-  RecordBatchVector received_batches_;
-  int num_received_;
-  int num_emitted_;
-  int emit_stop_;
-  bool stopped_;
-
-  PushGenerator<std::shared_ptr<RecordBatch>> generator_;
-  util::optional<PushGenerator<std::shared_ptr<RecordBatch>>::Producer> producer_;
-};
+AsyncGenerator<util::optional<ExecBatch>> Wrap(RecordBatchGenerator gen,
+                                               ::arrow::internal::Executor* io_executor) {
+  return MakeMappedGenerator(
+      MakeTransferredGenerator(std::move(gen), io_executor),

Review comment:
       Ah, but there is no corresponding transfer back to the CPU pool?  Is that the half-baked part (i.e. you just don't have that piece written yet?)




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] bkietz commented on pull request #10397: ARROW-11930: [C++][Dataset][Compute] Use an ExecPlan for dataset scans

Posted by GitBox <gi...@apache.org>.
bkietz commented on pull request #10397:
URL: https://github.com/apache/arrow/pull/10397#issuecomment-853380645


   >  How do you see things evolving? Do you think the various operations achieved by a scanner today will be achieved by an execution plan? For example, will ScanBatches, CountRows, etc. create and execute an execution plan instead of maintaining the dual paths?
   
   I'd like the ExecPlan to be usable enough to replace all filtering and projection currently in Scanner. So for example ScanBatches could assemble an ExecPlan to handle filtering and projection then receive and reorder batches; never needing to explicitly evaluate an expression.
   
   Ultimately, I'm not positive we'll keep Scanner. It's possible we could simplify the dataset module to a factory for source/sink nodes. In that case, anything which currently builds a Scanner would instead produce an ExecPlan. We'll see


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] bkietz commented on a change in pull request #10397: ARROW-11930: [C++][Dataset][Compute] Use an ExecPlan for dataset scans

Posted by GitBox <gi...@apache.org>.
bkietz commented on a change in pull request #10397:
URL: https://github.com/apache/arrow/pull/10397#discussion_r661903748



##########
File path: cpp/src/arrow/compute/exec/doc/exec_node.md
##########
@@ -0,0 +1,147 @@
+<!---
+  Licensed to the Apache Software Foundation (ASF) under one
+  or more contributor license agreements.  See the NOTICE file
+  distributed with this work for additional information
+  regarding copyright ownership.  The ASF licenses this file
+  to you under the Apache License, Version 2.0 (the
+  "License"); you may not use this file except in compliance
+  with the License.  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing,
+  software distributed under the License is distributed on an
+  "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+  KIND, either express or implied.  See the License for the
+  specific language governing permissions and limitations
+  under the License.
+-->
+
+# ExecNodes and logical operators

Review comment:
       I'll promote this to a Sphinx doc in a follow up. https://issues.apache.org/jira/browse/ARROW-13227




-- 
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-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] westonpace commented on a change in pull request #10397: ARROW-11930: [C++][Dataset][Compute] Use an ExecPlan for dataset scans

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #10397:
URL: https://github.com/apache/arrow/pull/10397#discussion_r644162327



##########
File path: cpp/src/arrow/compute/exec/exec_plan.cc
##########
@@ -170,48 +165,409 @@ Status ExecPlan::Validate() { return ToDerived(this)->Validate(); }
 
 Status ExecPlan::StartProducing() { return ToDerived(this)->StartProducing(); }
 
-ExecNode::ExecNode(ExecPlan* plan, std::string label,
-                   std::vector<BatchDescr> input_descrs,
+ExecNode::ExecNode(ExecPlan* plan, std::string label, NodeVector inputs,
                    std::vector<std::string> input_labels, BatchDescr output_descr,
                    int num_outputs)
     : plan_(plan),
       label_(std::move(label)),
-      input_descrs_(std::move(input_descrs)),
+      inputs_(std::move(inputs)),
       input_labels_(std::move(input_labels)),
       output_descr_(std::move(output_descr)),
-      num_outputs_(num_outputs) {}
+      num_outputs_(num_outputs) {
+  for (auto input : inputs_) {
+    input->outputs_.push_back(this);
+  }
+}
 
 Status ExecNode::Validate() const {
-  if (inputs_.size() != input_descrs_.size()) {
+  if (inputs_.size() != input_labels_.size()) {
     return Status::Invalid("Invalid number of inputs for '", label(), "' (expected ",
-                           num_inputs(), ", actual ", inputs_.size(), ")");
+                           num_inputs(), ", actual ", input_labels_.size(), ")");
   }
 
   if (static_cast<int>(outputs_.size()) != num_outputs_) {
     return Status::Invalid("Invalid number of outputs for '", label(), "' (expected ",
                            num_outputs(), ", actual ", outputs_.size(), ")");
   }
 
-  DCHECK_EQ(input_descrs_.size(), input_labels_.size());
-
   for (auto out : outputs_) {
     auto input_index = GetNodeIndex(out->inputs(), this);
     if (!input_index) {
       return Status::Invalid("Node '", label(), "' outputs to node '", out->label(),
                              "' but is not listed as an input.");
     }
+  }
 
-    const auto& in_descr = out->input_descrs_[*input_index];
-    if (in_descr != output_descr_) {
-      return Status::Invalid(
-          "Node '", label(), "' (bound to input ", input_labels_[*input_index],
-          ") produces batches with type '", ValueDescr::ToString(output_descr_),
-          "' inconsistent with consumer '", out->label(), "' which accepts '",
-          ValueDescr::ToString(in_descr), "'");
+  return Status::OK();
+}
+
+struct SourceNode : ExecNode {
+  SourceNode(ExecPlan* plan, std::string label, ExecNode::BatchDescr output_descr,
+             AsyncGenerator<util::optional<ExecBatch>> generator)
+      : ExecNode(plan, std::move(label), {}, {}, std::move(output_descr),
+                 /*num_outputs=*/1),
+        generator_(std::move(generator)) {}
+
+  const char* kind_name() override { return "SourceNode"; }
+
+  static void NoInputs() { DCHECK(false) << "no inputs; this should never be called"; }
+  void InputReceived(ExecNode*, int, ExecBatch) override { NoInputs(); }
+  void ErrorReceived(ExecNode*, Status) override { NoInputs(); }
+  void InputFinished(ExecNode*, int) override { NoInputs(); }
+
+  Status StartProducing() override {
+    if (finished_) {
+      return Status::Invalid("Restarted SourceNode '", label(), "'");
     }
+
+    auto gen = std::move(generator_);
+
+    /// XXX should we wait on this future anywhere? In StopProducing() maybe?
+    auto done_fut =
+        Loop([gen, this] {
+          std::unique_lock<std::mutex> lock(mutex_);
+          int seq = next_batch_index_++;
+          if (finished_) {
+            return Future<ControlFlow<int>>::MakeFinished(Break(seq));
+          }
+          lock.unlock();
+
+          return gen().Then(
+              [=](const util::optional<ExecBatch>& batch) -> ControlFlow<int> {
+                std::unique_lock<std::mutex> lock(mutex_);
+                if (!batch || finished_) {
+                  finished_ = true;
+                  return Break(seq);
+                }
+                lock.unlock();
+
+                outputs_[0]->InputReceived(this, seq, *batch);
+                return Continue();
+              },
+              [=](const Status& error) -> ControlFlow<int> {
+                std::unique_lock<std::mutex> lock(mutex_);
+                if (!finished_) {
+                  finished_ = true;
+                  lock.unlock();
+                  // unless we were already finished, push the error to our output
+                  // XXX is this correct? Is it reasonable for a consumer to ignore errors
+                  // from a finished producer?
+                  outputs_[0]->ErrorReceived(this, error);
+                }
+                return Break(seq);
+              });
+        }).Then([&](int seq) {
+          /// XXX this is probably redundant: do we always call InputFinished after
+          /// ErrorReceived or will ErrorRecieved be sufficient?
+          outputs_[0]->InputFinished(this, seq);
+        });
+
+    return Status::OK();
   }
 
-  return Status::OK();
+  void PauseProducing(ExecNode* output) override {}
+
+  void ResumeProducing(ExecNode* output) override {}
+
+  void StopProducing(ExecNode* output) override {
+    DCHECK_EQ(output, outputs_[0]);
+    std::unique_lock<std::mutex> lock(mutex_);
+    finished_ = true;
+  }
+
+  void StopProducing() override { StopProducing(outputs_[0]); }
+
+ private:
+  std::mutex mutex_;
+  bool finished_{false};
+  int next_batch_index_{0};
+  AsyncGenerator<util::optional<ExecBatch>> generator_;
+};
+
+ExecNode* MakeSourceNode(ExecPlan* plan, std::string label,
+                         ExecNode::BatchDescr output_descr,
+                         AsyncGenerator<util::optional<ExecBatch>> generator) {
+  return plan->EmplaceNode<SourceNode>(plan, std::move(label), std::move(output_descr),
+                                       std::move(generator));
+}
+
+struct FilterNode : ExecNode {
+  FilterNode(ExecNode* input, std::string label, Expression filter)
+      : ExecNode(input->plan(), std::move(label), {input}, {"target"},
+                 /*output_descr=*/{input->output_descr()},
+                 /*num_outputs=*/1),
+        filter_(std::move(filter)) {}
+
+  const char* kind_name() override { return "FilterNode"; }
+
+  Result<ExecBatch> DoFilter(const ExecBatch& target) {
+    ARROW_ASSIGN_OR_RAISE(Expression simplified_filter,
+                          SimplifyWithGuarantee(filter_, target.guarantee));
+
+    // XXX get a non-default exec context
+    ARROW_ASSIGN_OR_RAISE(Datum mask, ExecuteScalarExpression(simplified_filter, target));
+
+    if (mask.is_scalar()) {
+      const auto& mask_scalar = mask.scalar_as<BooleanScalar>();
+      if (mask_scalar.is_valid && mask_scalar.value) {
+        return target;
+      }
+
+      return target.Slice(0, 0);
+    }
+
+    auto values = target.values;
+    for (auto& value : values) {
+      if (value.is_scalar()) continue;
+      ARROW_ASSIGN_OR_RAISE(value, Filter(value, mask, FilterOptions::Defaults()));
+    }
+    return ExecBatch::Make(std::move(values));
+  }
+
+  void InputReceived(ExecNode* input, int seq, ExecBatch batch) override {
+    DCHECK_EQ(input, inputs_[0]);
+
+    auto maybe_filtered = DoFilter(std::move(batch));
+    if (!maybe_filtered.ok()) {
+      outputs_[0]->ErrorReceived(this, maybe_filtered.status());
+      inputs_[0]->StopProducing(this);
+      return;
+    }
+
+    maybe_filtered->guarantee = batch.guarantee;
+    outputs_[0]->InputReceived(this, seq, maybe_filtered.MoveValueUnsafe());
+  }
+
+  void ErrorReceived(ExecNode* input, Status error) override {
+    DCHECK_EQ(input, inputs_[0]);
+    outputs_[0]->ErrorReceived(this, std::move(error));
+    inputs_[0]->StopProducing(this);
+  }
+
+  void InputFinished(ExecNode* input, int seq) override {
+    DCHECK_EQ(input, inputs_[0]);
+    outputs_[0]->InputFinished(this, seq);
+    inputs_[0]->StopProducing(this);
+  }
+
+  Status StartProducing() override {
+    // XXX validate inputs_[0]->output_descr() against filter_
+    return Status::OK();
+  }
+
+  void PauseProducing(ExecNode* output) override {}
+
+  void ResumeProducing(ExecNode* output) override {}
+
+  void StopProducing(ExecNode* output) override {
+    DCHECK_EQ(output, outputs_[0]);
+    inputs_[0]->StopProducing(this);
+  }
+
+  void StopProducing() override { StopProducing(outputs_[0]); }
+
+ private:
+  Expression filter_;
+};
+
+ExecNode* MakeFilterNode(ExecNode* input, std::string label, Expression filter) {
+  return input->plan()->EmplaceNode<FilterNode>(input, std::move(label),
+                                                std::move(filter));
+}
+
+struct ProjectNode : ExecNode {
+  ProjectNode(ExecNode* input, std::string label, std::vector<Expression> exprs)
+      : ExecNode(input->plan(), std::move(label), {input}, {"target"},
+                 /*output_descr=*/{input->output_descr()},
+                 /*num_outputs=*/1),
+        exprs_(std::move(exprs)) {}
+
+  const char* kind_name() override { return "ProjectNode"; }
+
+  Result<ExecBatch> DoProject(const ExecBatch& target) {
+    // XXX get a non-default exec context
+    std::vector<Datum> values{exprs_.size()};
+    for (size_t i = 0; i < exprs_.size(); ++i) {
+      ARROW_ASSIGN_OR_RAISE(Expression simplified_expr,
+                            SimplifyWithGuarantee(exprs_[i], target.guarantee));
+
+      ARROW_ASSIGN_OR_RAISE(values[i], ExecuteScalarExpression(simplified_expr, target));
+    }
+    return ExecBatch::Make(std::move(values));
+  }
+
+  void InputReceived(ExecNode* input, int seq, ExecBatch batch) override {
+    DCHECK_EQ(input, inputs_[0]);
+
+    auto maybe_projected = DoProject(std::move(batch));
+    if (!maybe_projected.ok()) {
+      outputs_[0]->ErrorReceived(this, maybe_projected.status());
+      inputs_[0]->StopProducing(this);
+      return;
+    }
+
+    maybe_projected->guarantee = batch.guarantee;
+    outputs_[0]->InputReceived(this, seq, maybe_projected.MoveValueUnsafe());
+  }
+
+  void ErrorReceived(ExecNode* input, Status error) override {
+    DCHECK_EQ(input, inputs_[0]);
+    outputs_[0]->ErrorReceived(this, std::move(error));
+    inputs_[0]->StopProducing(this);
+  }
+
+  void InputFinished(ExecNode* input, int seq) override {
+    DCHECK_EQ(input, inputs_[0]);
+    outputs_[0]->InputFinished(this, seq);
+    inputs_[0]->StopProducing(this);
+  }
+
+  Status StartProducing() override {
+    // XXX validate inputs_[0]->output_descr() against filter_
+    return Status::OK();
+  }
+
+  void PauseProducing(ExecNode* output) override {}
+
+  void ResumeProducing(ExecNode* output) override {}
+
+  void StopProducing(ExecNode* output) override {
+    DCHECK_EQ(output, outputs_[0]);
+    inputs_[0]->StopProducing(this);
+  }
+
+  void StopProducing() override { StopProducing(outputs_[0]); }
+
+ private:
+  std::vector<Expression> exprs_;
+};
+
+ExecNode* MakeProjectNode(ExecNode* input, std::string label,
+                          std::vector<Expression> exprs) {
+  return input->plan()->EmplaceNode<ProjectNode>(input, std::move(label),
+                                                 std::move(exprs));
+}
+
+struct SinkNode : ExecNode {
+  SinkNode(ExecNode* input, std::string label,
+           AsyncGenerator<util::optional<ExecBatch>>* generator)
+      : ExecNode(input->plan(), std::move(label), {input}, {"collected"}, {},
+                 /*num_outputs=*/0),
+        producer_(MakeProducer(generator)) {}
+
+  static PushGenerator<util::optional<ExecBatch>>::Producer MakeProducer(
+      AsyncGenerator<util::optional<ExecBatch>>* out_gen) {
+    PushGenerator<util::optional<ExecBatch>> gen;
+    auto out = gen.producer();
+    *out_gen = std::move(gen);
+    return out;
+  }
+
+  const char* kind_name() override { return "SinkNode"; }
+
+  Status StartProducing() override { return Status::OK(); }
+
+  // sink nodes have no outputs from which to feel backpressure
+  static void NoOutputs() { DCHECK(false) << "no outputs; this should never be called"; }
+  void ResumeProducing(ExecNode* output) override { NoOutputs(); }
+  void PauseProducing(ExecNode* output) override { NoOutputs(); }
+  void StopProducing(ExecNode* output) override { NoOutputs(); }
+
+  void StopProducing() override {
+    std::unique_lock<std::mutex> lock(mutex_);
+    StopProducingUnlocked();
+  }
+
+  void InputReceived(ExecNode* input, int seq_num, ExecBatch exec_batch) override {
+    std::unique_lock<std::mutex> lock(mutex_);
+    if (stopped_) return;
+
+    // TODO would be nice to factor this out in a ReorderQueue
+    if (seq_num <= static_cast<int>(received_batches_.size())) {
+      received_batches_.resize(seq_num + 1);
+      emitted_.resize(seq_num + 1, false);
+    }
+    received_batches_[seq_num] = std::move(exec_batch);
+    ++num_received_;
+
+    if (seq_num != num_emitted_) {
+      // Cannot emit yet as there is a hole at `num_emitted_`
+      DCHECK_GT(seq_num, num_emitted_);
+      return;
+    }
+
+    if (num_received_ == emit_stop_) {
+      StopProducingUnlocked();
+    }
+
+    // Emit batches in order as far as possible
+    // First collect these batches, then unlock before producing.
+    const auto seq_start = seq_num;
+    while (seq_num < static_cast<int>(emitted_.size()) && !emitted_[seq_num]) {
+      emitted_[seq_num] = true;
+      ++seq_num;
+    }
+    DCHECK_GT(seq_num, seq_start);
+    // By moving the values now, we make sure another thread won't emit the same values
+    // below
+    std::vector<ExecBatch> to_emit(
+        std::make_move_iterator(received_batches_.begin() + seq_start),
+        std::make_move_iterator(received_batches_.begin() + seq_num));
+
+    lock.unlock();
+    for (auto&& batch : to_emit) {
+      producer_.Push(std::move(batch));
+    }
+    lock.lock();
+
+    DCHECK_EQ(seq_start, num_emitted_);  // num_emitted_ wasn't bumped in the meantime
+    num_emitted_ = seq_num;

Review comment:
       Also, even when it isn't the final item, it could lead to unnecessary delays.  If item 10 arrives while item 9 is being delivered then it might get stuck in the buffer until item 11 arrives to unclog things.
   
   I suppose the entire thing could be wrapped in a `while` loop that only breaks out if the next item to deliver isn't present (and the lock is held).




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] bkietz commented on a change in pull request #10397: ARROW-11930: [C++][Dataset][Compute] Use an ExecPlan for dataset scans

Posted by GitBox <gi...@apache.org>.
bkietz commented on a change in pull request #10397:
URL: https://github.com/apache/arrow/pull/10397#discussion_r643576195



##########
File path: cpp/src/arrow/compute/exec/expression_test.cc
##########
@@ -165,6 +165,56 @@ TEST(ExpressionUtils, StripOrderPreservingCasts) {
   Expect(cast(field_ref("i32"), uint64()), no_change);
 }
 
+TEST(ExpressionUtils, MakeExecBatch) {
+  auto Expect = [](std::shared_ptr<RecordBatch> partial_batch) {
+    SCOPED_TRACE(partial_batch->ToString());
+    ASSERT_OK_AND_ASSIGN(auto batch, MakeExecBatch(*kBoringSchema, partial_batch));
+
+    ASSERT_EQ(batch.num_values(), kBoringSchema->num_fields());
+    for (int i = 0; i < kBoringSchema->num_fields(); ++i) {
+      const auto& field = *kBoringSchema->field(i);
+
+      SCOPED_TRACE("Field#" + std::to_string(i) + " " + field.ToString());
+
+      EXPECT_TRUE(batch[i].type()->Equals(field.type()))
+          << "Incorrect type " << batch[i].type()->ToString();
+
+      ASSERT_OK_AND_ASSIGN(auto col, FieldRef(field.name()).GetOneOrNone(*partial_batch));

Review comment:
       GetOneOrNone raises a descriptive error if duplicate field names are found, whereas GetFieldByName will just return null IIRC




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] bkietz commented on a change in pull request #10397: ARROW-11930: [C++][Dataset][Compute] Use an ExecPlan for dataset scans

Posted by GitBox <gi...@apache.org>.
bkietz commented on a change in pull request #10397:
URL: https://github.com/apache/arrow/pull/10397#discussion_r644299278



##########
File path: cpp/src/arrow/compute/exec/exec_plan.cc
##########
@@ -105,18 +107,11 @@ struct ExecPlanImpl : public ExecPlan {
           return Status::OK();
         }
 
-        auto it_success = visiting.insert(node);
-        if (!it_success.second) {
-          // Insertion failed => node is already being visited
-          return Status::Invalid("Cycle detected in execution plan");
-        }

Review comment:
       Since nodes are now constructed with their inputs it's impossible to produce a cycle




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] westonpace commented on a change in pull request #10397: ARROW-11930: [C++][Dataset][Compute] Use an ExecPlan for dataset scans

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #10397:
URL: https://github.com/apache/arrow/pull/10397#discussion_r643637072



##########
File path: cpp/src/arrow/dataset/dataset_internal.h
##########
@@ -204,5 +204,35 @@ arrow::Result<std::shared_ptr<T>> GetFragmentScanOptions(
   return internal::checked_pointer_cast<T>(source);
 }
 
+class FragmentDataset : public Dataset {

Review comment:
       Right, sorry, I meant the other way around.  `InMemoryDataset : public FragmentDataset`.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] bkietz commented on a change in pull request #10397: ARROW-11930: [C++][Dataset][Compute] Use an ExecPlan for dataset scans

Posted by GitBox <gi...@apache.org>.
bkietz commented on a change in pull request #10397:
URL: https://github.com/apache/arrow/pull/10397#discussion_r644838129



##########
File path: cpp/src/arrow/compute/exec/exec_plan.cc
##########
@@ -170,48 +165,409 @@ Status ExecPlan::Validate() { return ToDerived(this)->Validate(); }
 
 Status ExecPlan::StartProducing() { return ToDerived(this)->StartProducing(); }
 
-ExecNode::ExecNode(ExecPlan* plan, std::string label,
-                   std::vector<BatchDescr> input_descrs,
+ExecNode::ExecNode(ExecPlan* plan, std::string label, NodeVector inputs,
                    std::vector<std::string> input_labels, BatchDescr output_descr,
                    int num_outputs)
     : plan_(plan),
       label_(std::move(label)),
-      input_descrs_(std::move(input_descrs)),
+      inputs_(std::move(inputs)),
       input_labels_(std::move(input_labels)),
       output_descr_(std::move(output_descr)),
-      num_outputs_(num_outputs) {}
+      num_outputs_(num_outputs) {
+  for (auto input : inputs_) {
+    input->outputs_.push_back(this);
+  }
+}
 
 Status ExecNode::Validate() const {
-  if (inputs_.size() != input_descrs_.size()) {
+  if (inputs_.size() != input_labels_.size()) {
     return Status::Invalid("Invalid number of inputs for '", label(), "' (expected ",
-                           num_inputs(), ", actual ", inputs_.size(), ")");
+                           num_inputs(), ", actual ", input_labels_.size(), ")");
   }
 
   if (static_cast<int>(outputs_.size()) != num_outputs_) {
     return Status::Invalid("Invalid number of outputs for '", label(), "' (expected ",
                            num_outputs(), ", actual ", outputs_.size(), ")");
   }
 
-  DCHECK_EQ(input_descrs_.size(), input_labels_.size());
-
   for (auto out : outputs_) {
     auto input_index = GetNodeIndex(out->inputs(), this);
     if (!input_index) {
       return Status::Invalid("Node '", label(), "' outputs to node '", out->label(),
                              "' but is not listed as an input.");
     }
+  }
 
-    const auto& in_descr = out->input_descrs_[*input_index];
-    if (in_descr != output_descr_) {
-      return Status::Invalid(
-          "Node '", label(), "' (bound to input ", input_labels_[*input_index],
-          ") produces batches with type '", ValueDescr::ToString(output_descr_),
-          "' inconsistent with consumer '", out->label(), "' which accepts '",
-          ValueDescr::ToString(in_descr), "'");
+  return Status::OK();
+}
+
+struct SourceNode : ExecNode {
+  SourceNode(ExecPlan* plan, std::string label, ExecNode::BatchDescr output_descr,
+             AsyncGenerator<util::optional<ExecBatch>> generator)
+      : ExecNode(plan, std::move(label), {}, {}, std::move(output_descr),
+                 /*num_outputs=*/1),
+        generator_(std::move(generator)) {}
+
+  const char* kind_name() override { return "SourceNode"; }
+
+  static void NoInputs() { DCHECK(false) << "no inputs; this should never be called"; }
+  void InputReceived(ExecNode*, int, ExecBatch) override { NoInputs(); }
+  void ErrorReceived(ExecNode*, Status) override { NoInputs(); }
+  void InputFinished(ExecNode*, int) override { NoInputs(); }
+
+  Status StartProducing() override {
+    if (finished_) {
+      return Status::Invalid("Restarted SourceNode '", label(), "'");
     }
+
+    auto gen = std::move(generator_);
+
+    /// XXX should we wait on this future anywhere? In StopProducing() maybe?

Review comment:
       Perhaps in `StopProducing()`? Then we append to the ExecNode contract that StopProducing must be called on a producing node before it is destroyed?




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org