Skip to content

feat(flotilla): Distributed Limit Counter#6942

Open
colin-ho wants to merge 29 commits into
mainfrom
colin/distributed-limit-actor
Open

feat(flotilla): Distributed Limit Counter#6942
colin-ho wants to merge 29 commits into
mainfrom
colin/distributed-limit-actor

Conversation

@colin-ho
Copy link
Copy Markdown
Collaborator

@colin-ho colin-ho commented May 16, 2026

Changes Made

Replaces the materializing two-phase local+global limit with a streaming limit driven by a Ray-actor-backed global counter.

  • LimitCounterActor (daft/execution/ray_distributed_limit.py) holds (remaining_skip, remaining_take). claim(input_id, num_rows) -> (skip, take, done) is atomic; start_task(input_id) refunds a prior attempt's claims so retries see consistent state. await_limit_completion() resolves once the limit is fully claimed and returns the input_ids that actually consumed budget. Actor is pinned to head node.

  • DistributedLimitSink (src/daft-local-execution/src/streaming_sink/distributed_limit.rs) calls claim(input_id, num_rows) on the counter the actor per morsel, and slices the morsel based on the returned (skip, take, done).

  • LimitNode (src/daft-distributed/src/pipeline_node/limit.rs) creates the counter actor, and appends distributed_limit tasks to the input tasks. It awaits the limit completion from the actor, and once done, cancels the scheduling of any subsequent limit tasks. It is aware of which input ids contributed to the limit, and only cancels tasks not with these ids.

  • Scheduler now filters cancelled tasks at schedule_tasks and emits TaskEvent::Cancelled to avoid scheduling new limit tasks.

Ordering note

Across-partition order is no longer preserved — workers race to claim. tests/integration/iceberg/test_partition_pruning.py sorts before limiting (matching the _on_number precedent already in that file); a related dataframe test was loosened similarly.

Related Issues

colin-ho and others added 2 commits May 15, 2026 16:12
…ter actor

Replace the two-phase submit-then-trim Limit with a fusing node that
composes a streaming `DistributedLimit` op into each upstream task
builder. A Ray actor holds the global `(remaining_skip, remaining_take)`
state and serves atomic `claim(task_id, num_rows)` calls; the
per-worker op consults the actor on every batch and finishes when the
actor reports the limit is hit.

This eliminates the materialization boundary that Limit previously
forced: `scan.limit(N).write(s3)` now runs as a single fused task per
partition (scan → limit → write), data flowing worker → S3 directly,
no intermediate object-store round-trip.

The hooks needed to make this work, beyond the obvious counter actor
plumbing:

- Scope `result_tx` to drop as soon as forwarding finishes. Downstream
  batching nodes (e.g. `IntoPartitionsNode`) wait for end-of-stream
  before submitting anything; holding `result_tx` past the forwarding
  loop deadlocks against the drain step.
- Drain all `notify_token`s unconditionally — a `SubmittedTask`
  dropped without being polled closes its notify channel with
  `RecvError`. Breaking on it tears down the actor while workers are
  still calling into it.
- Stamp each forwarded builder with a unique `extend_fingerprint(seq)`.
  The worker caches local pipelines by `plan_fingerprint`; tasks
  sharing a fingerprint share one pipeline, but our streaming sink's
  state becomes Finished after the first task hits the limit, killing
  the pipeline for the rest. Each task needs its own pipeline.
- Pass the SwordfishTask's `task_id` (from `BuilderContext.context`)
  to the local op so the actor's `start_task` is idempotent across
  task retries: retried task ids rewind their prior claims back into
  the global budget.

Co-Authored-By: Claude Opus 4.7 (1M context) <noreply@anthropic.com>
Three audit findings from the post-merge review:

- **Multi-`InputId` could call `start_task` twice with the same `task_id`.**
  `start_task` rewinds prior claims back into the global budget, so a
  second call would corrupt the count. With `max_concurrency=1` the
  streaming-sink machinery still creates a fresh state per `InputId`,
  not one state globally. The previous `started: bool` on `State` only
  guarded per-state, not per-sink. Move `started` to a shared
  `Arc<tokio::sync::OnceCell<()>>` on the sink so the actor is told
  exactly once per task, regardless of how many states the runtime
  creates.

- **`task_claims` HashMap on the actor grew unbounded.** Every task
  past the limit accumulated a `(0, 0)` entry that was never evicted —
  O(partitions) memory tied to one query. Drop the entry when its
  accumulated claim totals to `(0, 0)`; rewinding `(0, 0)` on retry is
  a no-op, so this is information-preserving.

- **Silent `"unknown"` fallback when `task_id` is missing from
  `BuilderContext.context`.** Would collapse every malformed task's
  claims under one key and break retry-rewind without surfacing
  anything. The invariant always holds in practice (the builder
  stamps it), so a missing entry is a real bug — panic instead of
  fall back.

Full `DAFT_RUNNER=ray` limit suite still passes (210 tests).

Co-Authored-By: Claude Opus 4.7 (1M context) <noreply@anthropic.com>
@github-actions
Copy link
Copy Markdown

github-actions Bot commented May 16, 2026

Rust Dependency Diff

Head: 332cce637e3cbbb684bee23e01d74a9acc12de1f vs Base: 36a17445c881babe5fe72be8f81259dae79e2e64.

OK: Within budget.

  • New Crates: 0
  • Removed Crates: 0

@github-actions github-actions Bot added the feat label May 16, 2026
@blacksmith-sh

This comment has been minimized.

colin-ho and others added 4 commits May 15, 2026 17:18
The `estimated_num_rows` helper was added with the distributed Limit
work but the final LimitNode design no longer calls it — only the tests
referenced it. CI failed with `-D dead_code`; remove the method, its
test helpers, and its tests rather than silencing the lint. Also pick
up a pending rustfmt diff in limit.rs.

Co-Authored-By: Claude Opus 4.7 (1M context) <noreply@anthropic.com>
Per simplify review:
- `actor_ready_timeout` duplicated config already reachable via
  `self.config.execution_config.actor_udf_ready_timeout` — read it
  directly at the call site.
- `schema` was captured into the `append_distributed_limit_to_builder`
  closure only to be ignored with `let _ = schema;`. Drop the capture.

Co-Authored-By: Claude Opus 4.7 (1M context) <noreply@anthropic.com>
Two new CI failures from the prior cleanup:
- check-unused-cargo-deps: `daft-stats` was only used by the
  `estimated_num_rows` tests that just got deleted.
- style (no-python feature): `DistributedLimitSink` is gated on
  `feature = "python"`, but the import in `pipeline.rs` was
  unconditional, breaking the non-python build. Move the `use` into
  the gated match arm.

Co-Authored-By: Claude Opus 4.7 (1M context) <noreply@anthropic.com>
Adds direct unit-tests for the distributed-Limit state machine
(_LimitCounterImpl): claim/offset arithmetic, start_task rewind on
retry (refund of skip + take, per-task isolation, double-call
idempotence), and the boundary-task bookkeeping invariant.

Writing the boundary-task test surfaced a real bug: start_task
unconditionally inserted (0, 0), but claim()'s early-return path (taken
when the global limit was already hit) skipped the cleanup branch, so
every post-limit task left a (0, 0) entry behind. task_claims grew
O(total tasks scheduled) instead of O(boundary tasks) as the comment
claimed. Fix: register entries lazily in claim() only when budget is
actually consumed, and pop in start_task on rewind.

Also splits LimitCounterActor into a pure-Python _LimitCounterImpl
class plus a ray.remote()(...)-wrapped Ray actor handle so the
state-machine logic is testable without standing up a Ray cluster.

Co-Authored-By: Claude Opus 4.7 (1M context) <noreply@anthropic.com>
@colin-ho colin-ho marked this pull request as ready for review May 16, 2026 00:52
@colin-ho colin-ho requested a review from a team as a code owner May 16, 2026 00:52
The distributed Limit operator now claims rows from whichever worker
task finishes first, rather than strict partition order, so `limit(10)`
on a 10-partition source may pull from non-consecutive partitions. The
test was over-specifying which 10 values come back; what it actually
cares about is that the async UDF executed in an asyncio context and
produced 10 valid outputs. Assert len == 10, all odd, all in (0, 100],
no duplicates instead.

Co-Authored-By: Claude Opus 4.7 (1M context) <noreply@anthropic.com>
@greptile-apps
Copy link
Copy Markdown
Contributor

greptile-apps Bot commented May 16, 2026

Greptile Summary

Replaces the old speculative two-phase local-limit-then-materialize design with a Ray-actor-backed global counter (LimitCounterActor) that serializes claim(task_id, num_rows) → (skip, take, done) calls across all workers, with retry-rewind semantics so failed tasks refund their budget on the next start_task call.

  • New actor + sink: _LimitCounterImpl / LimitCounterHandle hold (remaining_skip, remaining_take) globally; DistributedLimitSink calls the actor once per micropartition and slices accordingly, returning Finished when done is set.
  • LimitNode rewrite: forwards every input SwordfishTaskBuilder with an appended DistributedLimit plan node and a unique per-task fingerprint extension to prevent pipeline-cache sharing; drains all notify tokens before killing the actor.
  • Ordering trade-off: df.limit(N) no longer returns rows from the lowest partition indices — workers race to claim — reflected in updated test assertions and sort-before-limit fixups in the iceberg integration tests.

Confidence Score: 5/5

Safe to merge; the actor-backed claim/rewind design is correct, retry semantics are well-tested, and the ordering-guarantee change is deliberate and documented.

The global state machine is sound: the Ray actor serialises all claims so there are no races, retry-rewind correctly refunds both skip and take progress, the OnceCell prevents duplicate start_task calls within the same task execution, and the per-task fingerprint extension avoids pipeline-cache sharing.

No files require special attention; the core state machine in ray_distributed_limit.py and the sink in distributed_limit.rs are well-covered by the new unit tests.

Important Files Changed

Filename Overview
daft/execution/ray_distributed_limit.py New Ray-actor-backed global limit counter; state machine logic is correct, retry-rewind semantics are sound, and top-level Ray import is intentional (ruff TID253 constraint).
src/daft-distributed/src/pipeline_node/limit.rs Replaces the old speculative LimitState/LimitStats machinery with actor-based design; per-task fingerprint extension prevents pipeline-cache sharing; teardown drains all notify tokens before killing the actor.
src/daft-local-execution/src/streaming_sink/distributed_limit.rs New DistributedLimitSink that calls the Ray actor once per micropartition; OnceCell guards the single start_task call; debug_assert guards negative skip/take before usize cast.
src/daft-local-plan/src/plan.rs Adds DistributedLimit variant to LocalPhysicalPlan with correct schema, stats_state, and context propagation; consistent with the DistributedActorPoolProject pattern.
src/daft-distributed/src/scheduling/task.rs Removes estimated_num_rows and associated scan-task helpers; no longer needed with the actor-based approach.
tests/test_distributed_limit_actor.py Thorough unit tests of _LimitCounterImpl covering basic claims, offset spanning multiple calls, retry-rewind, per-task isolation, idempotency, and past-limit entry cleanup.
tests/dataframe/test_async_context.py Loosens the deterministic row-value assertion to a set-membership check, accommodating the non-deterministic partition-race ordering of the new distributed limit.
tests/integration/iceberg/test_partition_pruning.py Adds sort before limit in three predicate-pushdown tests so the expected slice is deterministic regardless of manifest/scan-task execution order.
src/daft-local-execution/src/pipeline.rs Adds a DistributedLimit match arm that extracts task_id from BuilderContext and wires the sink into the streaming pipeline.
src/daft-local-execution/src/streaming_sink/mod.rs Exposes the new distributed_limit module under the python feature flag.
src/daft-local-plan/src/lib.rs Re-exports DistributedLimit alongside the other python-feature plan types.

Sequence Diagram

sequenceDiagram
    participant LN as LimitNode (Rust)
    participant RA as LimitCounterActor (Ray/Python)
    participant S1 as DistributedLimitSink Task-1
    participant S2 as DistributedLimitSink Task-2

    LN->>RA: start_limit_counter_actor(limit, offset)
    RA-->>LN: LimitCounterHandle

    par Task-1 execution
        S1->>RA: start_task("t1")
        RA-->>S1: ok
        S1->>RA: claim("t1", num_rows_A)
        RA-->>S1: "(skip, take, done=False)"
        S1->>S1: slice input[skip : skip+take]
        S1->>RA: claim("t1", num_rows_B)
        RA-->>S1: "(0, take2, done=True)"
        S1-->>LN: Finished(output)
    and Task-2 execution
        S2->>RA: start_task("t2")
        RA-->>S2: ok (rewinds prior claim if retry)
        S2->>RA: claim("t2", num_rows_C)
        RA-->>S2: "(0, 0, done=True)"
        S2-->>LN: Finished(empty)
    end

    LN->>LN: drain notify tokens
    LN->>RA: teardown() → ray.kill(actor)
Loading

Reviews (2): Last reviewed commit: "refactor(flotilla): simplify Distributed..." | Re-trigger Greptile

Comment thread daft/execution/ray_distributed_limit.py Outdated
Comment thread src/daft-local-execution/src/streaming_sink/distributed_limit.rs
Comment thread src/daft-distributed/src/pipeline_node/limit.rs Outdated
@codecov
Copy link
Copy Markdown

codecov Bot commented May 16, 2026

Codecov Report

❌ Patch coverage is 28.93258% with 253 lines in your changes missing coverage. Please review.
✅ Project coverage is 75.54%. Comparing base (36a1744) to head (3a3f04e).

Files with missing lines Patch % Lines
...-execution/src/streaming_sink/distributed_limit.rs 0.00% 90 Missing ⚠️
src/daft-distributed/src/pipeline_node/limit.rs 2.40% 81 Missing ⚠️
src/daft-local-plan/src/plan.rs 0.00% 36 Missing ⚠️
daft/execution/ray_distributed_limit.py 68.96% 18 Missing ⚠️
src/daft-local-execution/src/pipeline.rs 0.00% 16 Missing ⚠️
src/daft-distributed/src/scheduling/task.rs 42.85% 4 Missing ⚠️
...ibuted/src/scheduling/scheduler/scheduler_actor.rs 62.50% 3 Missing ⚠️
...aft-distributed/src/scheduling/scheduler/linear.rs 89.47% 2 Missing ⚠️
...buted/src/pipeline_node/join/key_filtering_join.rs 0.00% 1 Missing ⚠️
...ft-distributed/src/scheduling/scheduler/default.rs 95.65% 1 Missing ⚠️
... and 1 more
Additional details and impacted files

Impacted file tree graph

@@            Coverage Diff             @@
##             main    #6942      +/-   ##
==========================================
+ Coverage   75.05%   75.54%   +0.49%     
==========================================
  Files        1135     1137       +2     
  Lines      163301   161151    -2150     
==========================================
- Hits       122562   121749     -813     
+ Misses      40739    39402    -1337     
Files with missing lines Coverage Δ
...c/daft-distributed/src/scheduling/scheduler/mod.rs 86.50% <100.00%> (+0.41%) ⬆️
...ft-local-execution/src/streaming_sink/async_udf.rs 92.96% <100.00%> (ø)
...rc/daft-local-execution/src/streaming_sink/base.rs 87.91% <100.00%> (+0.06%) ⬆️
...c/daft-local-execution/src/streaming_sink/limit.rs 98.75% <100.00%> (ø)
.../src/streaming_sink/monotonically_increasing_id.rs 94.73% <100.00%> (+71.05%) ⬆️
.../daft-local-execution/src/streaming_sink/sample.rs 93.11% <100.00%> (+4.11%) ⬆️
src/daft-local-plan/src/lib.rs 100.00% <ø> (ø)
...buted/src/pipeline_node/join/key_filtering_join.rs 21.29% <0.00%> (ø)
...ft-distributed/src/scheduling/scheduler/default.rs 89.91% <95.65%> (+0.10%) ⬆️
...rc/daft-local-execution/src/streaming_sink/vllm.rs 0.00% <0.00%> (ø)
... and 8 more

... and 37 files with indirect coverage changes

🚀 New features to boost your workflow:
  • ❄️ Test Analytics: Detect flaky tests, report on failures, and find test suite problems.
  • 📦 JS Bundle Analysis: Save yourself from yourself by tracking and limiting bundle sizes in JS merges.

colin-ho and others added 4 commits May 15, 2026 19:44
After adding `DistributedLimit` to this `pub use plan::{...}` block, the
five names fit on one line. rustfmt rewraps under the project's
max-width; commit the rewrap.

Co-Authored-By: Claude Opus 4.7 (1M context) <noreply@anthropic.com>
The distributed Limit operator claims budget in whichever worker order
the actor sees first, not strict partition-index order, so
`df.limit(N)` no longer matches `iceberg_pandas[:N]` row-for-row. The
`_on_number` variant already sorts-before-limit for the same reason
(comment dating from a prior scheduling change); apply the same fix to
the `date`, `timestamp`, and `letter` variants.

Co-Authored-By: Claude Opus 4.7 (1M context) <noreply@anthropic.com>
- eprintln! -> tracing::warn! in LimitActor::teardown.
- debug_assert! the i64 claim values are non-negative before casting
  to usize.
- Replace the no-op `try: import ray; except ImportError: raise` with
  a real user-facing message (wrapping is still required to satisfy
  ruff TID253).

Co-Authored-By: Claude Opus 4.7 (1M context) <noreply@anthropic.com>
Three small cleanups surfaced by a simplification-focused re-review:

- Drop `LimitCounterHandle.is_done` — defined but never called from
  Rust or tests. `_LimitCounterImpl.is_done` stays because the unit
  tests use it.
- Drop `DistributedLimitSinkState`. The struct carried two Arc clones
  that were already on the sink itself, so the state was pure
  boilerplate. Use `type State = ()` and clone from `&self` in
  `execute()` directly. Removes ~10 lines and the `make_state`
  ceremony.
- Rewrite `start_limit_counter_actor`'s readiness wait in terms of
  `asyncio.wait_for` — the manual `asyncio.wait(...)` + non-empty
  check + `asyncio.gather` was reimplementing the same primitive.

Co-Authored-By: Claude Opus 4.7 (1M context) <noreply@anthropic.com>
@colin-ho
Copy link
Copy Markdown
Collaborator Author

@greptileai

colin-ho and others added 3 commits May 15, 2026 21:02
The `LimitActor` enum + `get_or_start`/`teardown` impl was lazy-init
machinery for an actor that gets started on the first forwarded task
anyway. Collapse it:

- Replace with free `start_limit_counter_actor` /
  `teardown_limit_counter_actor` helpers. Start eagerly at the top of
  `execution_loop_fused`; the only case where eager init wastes work
  is an empty input stream, which is rare for a LimitNode the planner
  just inserted.
- Drop the `{ result_tx }` scope block in favour of explicit
  `drop(result_tx)`; same behaviour, less indentation.
- Rename `append_distributed_limit_to_builder` to
  `wrap_with_distributed_limit` and tighten the inline comments.

Net: 224 -> 197 lines and the loop body is no longer interleaved with
state-machine plumbing.

Co-Authored-By: Claude Opus 4.7 (1M context) <noreply@anthropic.com>
The old comment said "sharing one pipeline lets the streaming sink hit
Finished once and kill the pipeline for everyone after it" — confusing
phrasing, and it omitted the more fundamental reason: the
DistributedLimitSink also bakes in the task's task_id at construction,
which the actor uses as the bookkeeping key for retries. Two tasks
sharing one cached sink would also share one task_id at the actor.

Also rename `execution_loop_fused` -> `limit_execution_loop` to match
the surrounding node-name convention; nothing was being "fused" anyway.

Co-Authored-By: Claude Opus 4.7 (1M context) <noreply@anthropic.com>
…erprint

`DistributedLimitSink` previously baked the SwordfishTask's task_id in
at sink construction so the actor could attribute per-task claims for
retry-rewind. That forced `LimitNode` to stamp a unique fingerprint per
task to defeat the worker's pipeline cache — every task built its own
pipeline.

Use the framework's own per-input identity instead: extend the
`StreamingSink::make_state` signature with `input_id: InputId`, which
matches the SwordfishTask's task_id in flotilla mode (asserted at
`run.rs:447-451`). The distributed-limit state derives
`task_id = input_id.to_string()` and owns its own `OnceCell<()>` for
the `start_task` guard, so:

- The sink itself is task-agnostic — no `task_id` field, no shared
  `Arc<OnceCell>`.
- Workers cache one compiled pipeline per `plan_fingerprint`. The
  seq-based fingerprint stamping in `LimitNode` is gone.
- Each new SwordfishTask landing on a worker gets a fresh state with
  its own task_id and OnceCell; retry-rewind correctness is preserved.

Other `StreamingSink` impls (Limit, Sample, AsyncUdf, VLLM,
MonotonicallyIncreasingId) just accept the new `_input_id` parameter
and ignore it.

Co-Authored-By: Claude Opus 4.7 (1M context) <noreply@anthropic.com>
@blacksmith-sh

This comment has been minimized.

colin-ho and others added 6 commits May 15, 2026 21:36
…LimitSinkState

OnceCell was protecting against a race that can't happen — the
streaming-sink framework owns the state and hands it to one execute
call at a time. A plain `bool` (with `mut state` in execute) captures
the same one-shot intent without the locking primitive or async-init
ceremony.

Co-Authored-By: Claude Opus 4.7 (1M context) <noreply@anthropic.com>
The `make_state` plumbing is documented at the sink itself; restating
it at the call site is noise.

Co-Authored-By: Claude Opus 4.7 (1M context) <noreply@anthropic.com>
`LimitNode` previously kept forwarding builders even after the actor's
budget was exhausted; each downstream worker would then spawn, do its
`start_task`+`claim` round trip, get `done=True`, and tear down — small
per-task waste, but multiplied across many partitions.

Restructure `limit_execution_loop` as a `tokio::select!` between the
upstream task stream and the running-tasks `JoinSet`. Whenever a
forwarded task completes (notify_token fires), ask the actor
`is_done()`. If true, stop forwarding — `result_tx` is dropped, the
input stream closes for downstream, and any in-flight tasks
short-circuit naturally on their next `claim`.

No cancel-token plumbing needed: the existing flow is already cheap
enough once the actor returns `done=True`. The win is purely "don't
hand out more builders to workers that would do nothing."

Also inlines the `wrap_with_distributed_limit` helper and drops the
result_tx scoping comment.

Co-Authored-By: Claude Opus 4.7 (1M context) <noreply@anthropic.com>
`SchedulerSender` is unbounded, so by the time the LimitNode forwards a
builder it's already been submitted into the scheduler queue and the
`SubmittedTask` is held by `materialize::task_materializer`. Dropping
`result_tx` only stops *future* forwards — the queued ones still get
dispatched.

Wire a shared cancel-token through:
- Add `SwordfishTaskBuilder::with_cancel_token` (`Option<CancellationToken>`,
  consumed at `build()` to bind to the resulting `SubmittedTask`).
- `LimitNode` owns a `parent_cancel`; every forwarded builder gets a
  child token. When the actor reports `is_done`, drop `result_tx` and
  cancel the parent, which propagates to every in-flight + queued
  `SubmittedTask` via the existing `TaskResultAwaiter` cancel path.

Co-Authored-By: Claude Opus 4.7 (1M context) <noreply@anthropic.com>
Once `parent_cancel.cancel()` fires, every in-flight `SubmittedTask` gets
`TaskStatus::Cancelled` via `TaskResultAwaiter`'s biased select on the
cancel branch (`task.rs:561-569`), and `materialize::task_materializer`
treats Cancelled as a normal termination — not a query error. A worker
that races with `ray.kill(actor)` on a `claim` RPC just surfaces the
RPC error along the same cancel path; nothing leaks to the query.

The drain loop was a defensive wait for scheduler-side resolution, but
it only weakly approximates "workers stopped calling the actor" anyway
(Ray's cancel is async). Removing it for simplicity; ray-runner
limit/offset tests (200) still pass.

Co-Authored-By: Claude Opus 4.7 (1M context) <noreply@anthropic.com>
`RemoteFlotillaRunner` is pinned to the head node (`flotilla.py:592-599`),
and every `claim`/`done` RPC against the LimitCounterActor originates
inside the LimitNode running in that runner. Defaulting the actor's
placement to "DEFAULT" let Ray put it anywhere, paying a cross-node hop
per RPC. Pin to the runner's current node via
`NodeAffinitySchedulingStrategy(node_id=current, soft=False)` so the
actor lives on the same node and every RPC stays local.

Co-Authored-By: Claude Opus 4.7 (1M context) <noreply@anthropic.com>
colin-ho and others added 5 commits May 15, 2026 22:27
The `with_cancel_token` method already documents the contract.

Co-Authored-By: Claude Opus 4.7 (1M context) <noreply@anthropic.com>
Both `DefaultScheduler` and `LinearScheduler` previously dispatched
cancelled tasks all the way to a worker before the cancel propagated
back through `TaskResultAwaiter`. Filter at both entry points:

- `enqueue_tasks` drops already-cancelled tasks before they hit the
  pending queue.
- `schedule_tasks` skips cancelled entries it pops from the queue
  (drops `result_tx`, which signals the SubmittedTask's `Err` path —
  treated as a normal cancellation by `task_materializer`).
- `pending_tasks_in_priority_order` filters cancelled tasks so the
  autoscaling demand doesn't include resources we'll never use.

Also trim the `DistributedLimitSinkState` doc-comments and drop the
redundant doc on `SwordfishTaskBuilder::with_cancel_token`.

Co-Authored-By: Claude Opus 4.7 (1M context) <noreply@anthropic.com>
The actor's bookkeeping key is sourced from the streaming-sink
framework's `InputId` (cast to a string), not from a SwordfishTask-side
`task_id` constructed at sink-build time. The two happen to coincide in
flotilla mode (asserted at `run.rs:447-451`), but calling the parameter
`task_id` everywhere obscures where the value actually comes from.
Rename to `input_id` in:

- `_LimitCounterImpl.{start_task, claim}` and `LimitCounterHandle`.
- `_LimitCounterImpl.task_claims` -> `input_claims`.
- `DistributedLimitSinkState.task_id` -> `input_id`.
- `tests/test_distributed_limit_actor.py`.

Public Python method names (`start_task`, `claim`) are unchanged.

Co-Authored-By: Claude Opus 4.7 (1M context) <noreply@anthropic.com>
clippy::unwrap-or-default fires on
`self.cancel_token.unwrap_or_else(CancellationToken::new)`. `CancellationToken`
already implements `Default` as a fresh independent token; switch to
`unwrap_or_default()`.

Co-Authored-By: Claude Opus 4.7 (1M context) <noreply@anthropic.com>
When a downstream node batches its input — e.g. `IntoPartitionsNode`
used by `df.into_batches(...)` — forwarded builders are buffered and
only submitted after the LimitNode's `result_tx` closes. Cancelling
`parent_cancel` on the natural-exhaustion exit path would then drop
those just-submitted builders via the scheduler's is_cancelled filter,
silently returning zero rows from `df.limit(N).into_batches(M)` and
similar chains.

Only cancel when the loop broke because the actor reported `is_done`.
Natural exhaustion of the input stream means everything was already
forwarded for legitimate processing; no cancellation needed.

Co-Authored-By: Claude Opus 4.7 (1M context) <noreply@anthropic.com>
@codspeed-hq
Copy link
Copy Markdown

codspeed-hq Bot commented May 16, 2026

Merging this PR will not alter performance

✅ 40 untouched benchmarks
⏩ 10 skipped benchmarks1


Comparing colin/distributed-limit-actor (3a3f04e) with main (36a1744)

Open in CodSpeed

Footnotes

  1. 10 benchmarks were skipped, so the baseline results were used instead. If they were deleted from the codebase, click here and archive them to remove them from the performance reports.

colin-ho and others added 4 commits May 16, 2026 00:52
`StreamingSinkOutput::Finished` from `DistributedLimitSink` terminated the
entire streaming-sink node (see base.rs:326), killing the cached pipeline on
a flotilla worker. Other SwordfishTasks sharing the cached pipeline as
separate input_ids died with "Plan execution task has died", silently
truncating queries like `df.filter(...).limit(N).count_rows()` and
`df.limit(N).into_batches(M)`.

Polling `actor.is_done()` after each notify also conflated "limit reached"
with "safe to cancel". Cancelling the moment the actor reports done could
kill an in-flight contributor whose data wasn't yet materialized, losing
limit rows.

Rework:

- Sink always returns `NeedMoreInput`. The cached pipeline stays alive
  across input_ids; per-input streams drain naturally via finalize.
- Actor exposes `wait_for_contributors()`: awaits `is_done`, then returns
  the input_ids that consumed budget (`take > 0`).
- Notify token payload changes from `usize` (row count) to `TaskID`, so
  the LimitNode learns *which* tasks completed.
- LimitNode awaits `wait_for_contributors` and the notify_tokens; it only
  cancels `parent_cancel` once every contributing input_id has appeared in
  `completed_ids`. The scheduler's is_cancelled filter then drops pending
  tasks; in-flight tasks that get killed are non-contributors (their data
  is 0), so cancellation never loses limit rows.

Co-Authored-By: Claude Opus 4.7 (1M context) <noreply@anthropic.com>
The `is_cancelled` filter at the scheduler (added so `LimitNode` can drop
pending tasks once the limit is satisfied) was silently dropping cancelled
tasks. This broke two invariants:

- `dropping_result_stream_still_finalizes_operators` lifecycle test: every
  `OperatorStart` is supposed to be paired with `OperatorEnd`, but tasks
  cancelled at the scheduler emitted neither a `TaskStatus::Cancelled` nor
  a `TaskEvent::Cancelled`, so the lifecycle gate never closed.
- UI/observability: nodes whose tasks were all filtered before dispatch
  would show as perpetually "running."

Change `Scheduler::{enqueue_tasks, schedule_tasks}` to return the
cancelled-pending-tasks they dropped. `SchedulerLoop` emits
`TaskEvent::Cancelled` for each and then drops them (closing their
`result_tx`, which unblocks awaiters with `Ok(None)`).

`test_scheduler_actor_cancelled_task` previously assumed a cancelled task
always reached a worker so its `cancel_callback` would fire the notifier.
With the scheduler-side filter that's no longer guaranteed — accept either
path (callback fired *or* notifier dropped) as valid cancellation.

Co-Authored-By: Claude Opus 4.7 (1M context) <noreply@anthropic.com>
- Move all cancel-token filtering to `schedule_tasks` — `enqueue_tasks`
  keeps its old signature and accepts everything. Tasks that were already
  cancelled when enqueued sit briefly in the queue, then get filtered (and
  Cancelled events emitted) on the next schedule cycle.
- Rename `wait_for_contributors` → `await_limit_completion`.
- Trim verbose comments.
- Drop verbose docstrings in `_LimitCounterImpl` / `LimitCounterHandle` /
  `start_limit_counter_actor`; the method bodies and types are clear on
  their own.
- Replace the `loop { if done { break; }; select! { ... } }` pattern in
  `LimitNode::limit_execution_loop` with a `while` whose match returns
  "still pending."
@colin-ho colin-ho requested a review from srilman May 16, 2026 16:53
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

Projects

None yet

Development

Successfully merging this pull request may close these issues.

1 participant