Skip to content

Conversation

@gotocoding-DB
Copy link
Contributor

@gotocoding-DB gotocoding-DB commented Dec 3, 2025

Note: it's not a PR to merge, it's just a report (issue) which needs code examples to show how to repro the problem. Treat it as bug report.

TokioMultiThreadExecutor Deadlock with sync_channel

Problem

read_files() deadlocks when called from a tokio worker thread using current TokioMultiThreadExecutor.

Sequence:

  1. read_files() creates sync_channel(0) and calls task_executor.spawn()
  2. Tokio schedules task to current worker's local queue (optimization when called from worker context)
  3. read_files() returns iterator that blocks on receiver.recv()
  4. Task stuck in local queue - worker is OS-blocked, other workers don't steal from busy worker

Root cause: Tokio's schedule_task() uses with_current() to detect worker context and schedules locally for performance. However, sync_channel.recv() OS-blocks the thread, preventing the local task from executing.

Reproduction

Minimal (pure tokio example)

cargo test --test executor_deadlock_repro test_deadlock_minimal_repro

Demonstrates core issue: spawning from worker thread then blocking on sync primitive.

Full delta-kernel reproduction

cargo test --test executor_deadlock_repro test_tokio_multi_thread_executor_deadlock -- --ignored

Times out after 5 seconds. Marked #[ignore] since it's expected to hang.

Workaround verification

cargo test --test executor_deadlock_repro test_tokio_background_executor_works

Shows TokioBackgroundExecutor avoids the issue by using separate OS thread.

Solutions

Immediate workaround

Use TokioBackgroundExecutor instead of TokioMultiThreadExecutor.

Long-term fixes

Option 1: Replace sync_channel with tokio async channel

  • Requires changing StorageHandler::read_files() to return async iterator
  • Breaking API change

Option 2: Force remote scheduling

  • Spawn to separate runtime/thread pool
  • Avoids local queue optimization
  • In-place different paths depending on the provided Tokio Executor
  • Breaks the contract that we launch all tasks only in provided Executor (when we construct the Engine)
// In read_files() or in TokioMultiThreadExecutor::spawn()
let handle = Arc::clone(&self.handle);
tokio::task::spawn_blocking(move || {
  // Now on blocking thread pool, not a worker
  handle.spawn(actual_task);
  // This spawn() call goes to remote queue
});

Option 3: Use tokio::task::block_in_place

  • Informs tokio runtime that thread will block
  • Runtime can spawn new worker thread
  • Performance overhead:
    • If 100 tasks all call read_files() concurrently
    • Tokio spawns 100 replacement worker threads
    • Now have 104 threads (4 original workers + 100 replacements)
    • High memory overhead, context switching overhead
tokio::task::block_in_place(|| {
  self.receiver.recv().ok()
})

Technical Details

Call stack

filesystem.rs:273     task_executor.spawn()
executor.rs:203       handle.spawn()
tokio/worker.rs:1055  schedule_task()
  ↓ with_current() finds worker context
  ↓ schedule_local() - LOCAL QUEUE
filesystem.rs:314     return receiver.into_iter()
caller                iterator.next() → recv() blocks → DEADLOCK

Tokio scheduling logic

In tokio/src/runtime/scheduler/multi_thread/worker.rs:

pub(super) fn schedule_task(&self, task: Notified, is_yield: bool) {
    with_current(|maybe_cx| {
        if let Some(cx) = maybe_cx {
            if self.ptr_eq(&cx.worker.handle) {
                if let Some(core) = cx.core.borrow_mut().as_mut() {
                    self.schedule_local(core, task, is_yield);  // LOCAL
                    return;
                }
            }
        }
        self.push_remote_task(task);      // REMOTE
        self.notify_parked_remote();      // Wake idle workers
    });
}

Optimization assumes worker continues executing. sync_channel.recv() violates this assumption.

Debugged using:

  • delta-kernel-rs: v0.18.1
  • tokio: 1.47.1

@@ -0,0 +1,119 @@
//! Reproduction for deadlock in TokioMultiThreadExecutor when mixing sync_channel with handle.spawn()
Copy link
Contributor Author

@gotocoding-DB gotocoding-DB Dec 3, 2025

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@nicklan @scovich @zachschuermann @OussamaSaoudi
Hey folks, could you please review my report?
In this PR (treat it as a GH Issue), I tried to explain (in PR summary) why we can't switch to TokioMultiThreadExecutor – we have a deadlock problem, and I don't know how to fix it properly. I discussed about 3 different options; maybe you can suggest better approaches.

@@ -0,0 +1,121 @@
//! Reproduction for deadlock in TokioMultiThreadExecutor when mixing sync_channel with handle.spawn()
//!
//! When read_files() is called from a tokio worker thread:
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I assume you mean something like read_json_files here. So I think this is the key problem. We don't really expect those functions to be called from a worker thread. This would mean calling into kernel from a tokio worker thread.

So what happens is that the current thread is expected to handle both sides of the computation, and since it hits the recv() first the producer side never runs.

A couple of things:

  1. I think @scovich 's change in refactor: separate async handler logic from sync bridge logic #1435 should help here, maybe worth testing on that branch
  2. I can't repro this with a change to use a multi-threaded executor in our example programs. Do you know a way to make the hang more deterministic?

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

2 participants