Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[C++] Backpressure broken in asof join node #35838

Closed
westonpace opened this issue May 31, 2023 · 1 comment · Fixed by #35874 or #35878
Closed

[C++] Backpressure broken in asof join node #35838

westonpace opened this issue May 31, 2023 · 1 comment · Fixed by #35874 or #35878
Assignees
Milestone

Comments

@westonpace
Copy link
Member

Describe the bug, including details regarding any error messages, version, and platform.

Originally the InputState was created with...

  static Result<std::unique_ptr<InputState>> Make(
      size_t index, TolType tolerance, bool must_hash, bool may_rehash,
      KeyHasher* key_hasher, ExecNode* node, ExecNode* output,
      std::atomic<int32_t>& backpressure_counter,
      const std::shared_ptr<arrow::Schema>& schema, const col_index_t time_col_index,
      const std::vector<col_index_t>& key_col_index) {
    constexpr size_t low_threshold = 4, high_threshold = 8;
    std::unique_ptr<BackpressureControl> backpressure_control =
        std::make_unique<BackpressureController>(node, output, backpressure_counter);
...

          InputState::Make(i, tolerance_, must_hash_, may_rehash_, key_hashers_[i].get(),
                           inputs[i], this, backpressure_counter_,
                           inputs[i]->output_schema(), indices_of_on_key_[i],
                           indices_of_by_key_[i]));

The BackpressureController is given node (which is inputs[i]) and output (which is this).

After dcdeab7 the code changed to:

  static Result<std::unique_ptr<InputState>> Make(
      size_t index, TolType tolerance, bool must_hash, bool may_rehash,
      KeyHasher* key_hasher, AsofJoinNode* node, ExecNode* output,
      std::atomic<int32_t>& backpressure_counter,
      const std::shared_ptr<arrow::Schema>& schema, const col_index_t time_col_index,
      const std::vector<col_index_t>& key_col_index) {
    constexpr size_t low_threshold = 4, high_threshold = 8;
    std::unique_ptr<BackpressureControl> backpressure_control =
        std::make_unique<BackpressureController>(node, output, backpressure_counter);

...

          InputState::Make(i, tolerance_, must_hash_, may_rehash_, key_hashers_[i].get(),
                           this, inputs[i], backpressure_counter_,
                           inputs[i]->output_schema(), indices_of_on_key_[i],
                           indices_of_by_key_[i]));

Note that BackpressureControl is now created with node (which is this) and then output (which is inputs[i]). This means, when the asof join node decides to pause it will call PauseProducing on AsofJoinNode which currently does nothing:

  void PauseProducing(ExecNode* output, int32_t counter) override {}
  void ResumeProducing(ExecNode* output, int32_t counter) override {}

Component(s)

C++

@rtpsw
Copy link
Contributor

rtpsw commented Jun 1, 2023

take

rtpsw added a commit to rtpsw/arrow that referenced this issue Jun 1, 2023
icexelloss added a commit to icexelloss/arrow that referenced this issue Jun 1, 2023
icexelloss added a commit that referenced this issue Jun 1, 2023
### Rationale for this change

To fix a bug in asof join backpresure handling.

### What changes are included in this PR?

This is reverting a bug introduced in GH-34391 on this line that breaks asof join backpresure
https://github.com/apache/arrow/pull/34392/files#diff-5493b6ae7ea2a4d5cfb581034c076e9c4be7608382168de6d1301ef67b6c01eeR1410

### Are these changes tested?
No. However code change simply reverts to the state before the bug was introduced in GH-34391 and therefore should be pretty safe (we have tested that the code before GH-34391 is working). In the meantime @ rtpsw is working on adding tests around this but I would like to merge this to unblock internal  issues around this.

### Are there any user-facing changes?

* Closes: #35838

Authored-by: Li Jin <ice.xelloss@gmail.com>
Signed-off-by: Li Jin <ice.xelloss@gmail.com>
@icexelloss icexelloss added this to the 13.0.0 milestone Jun 1, 2023
rtpsw added a commit to rtpsw/arrow that referenced this issue Jun 2, 2023
rtpsw added a commit to westonpace/arrow that referenced this issue Jun 13, 2023
rtpsw added a commit to rtpsw/arrow that referenced this issue Jun 14, 2023
Convert the delaying node to a gated node to demonstrate my original idea
icexelloss pushed a commit that referenced this issue Jun 21, 2023
### What changes are included in this PR?

Passing the correct nodes to the backpressure controller, along with better parameter naming/doc. Also added reusable gate-classes (`Gate`, `GatedNodeOptions` and `GatedNode`) that enable holding all input batches until a gate is released, in order to support more robust backpressure testing in this PR.

### Are these changes tested?

Yes.

### Are there any user-facing changes?

No.

**This PR contains a "Critical Fix".**
* Closes: #35838

Lead-authored-by: Yaron Gvili <rtpsw@hotmail.com>
Co-authored-by: Weston Pace <weston.pace@gmail.com>
Co-authored-by: rtpsw <rtpsw@hotmail.com>
Signed-off-by: Li Jin <ice.xelloss@gmail.com>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Projects
None yet
3 participants