Skip to content

Conversation

@tchaikov
Copy link
Contributor

@tchaikov tchaikov commented Dec 25, 2025

Fix the double-free bug in iocb pool management that causes bad_alloc during AIO teardown (issue #3169, scylladb/scylladb#17369, scylladb/scylladb#26481).

Root Cause:
The bug occurs when stop() is called while AIO retry operations are in progress. The race condition:

  1. schedule_retry() A is processing iocbs in _aio_retries
  2. stop() calls std::exchange(_pending_aio_retry_fut, ready_future)
  3. New I/Os complete with -EAGAIN, added to _pending_aio_retry
  4. submit_work() sees need_to_retry()=TRUE && !retry_in_progress() (because we exchanged the future) and starts a NEW schedule_retry() B
  5. schedule_retry() A completes, .finally() block runs
  6. .finally() calls reap_completions(false) which returns -EAGAIN iocbs to the pool (since allow_retry=false)
  7. But schedule_retry() B is still running with those same iocbs in _aio_retries!
  8. When B tries to process them and encounters errors, handle_aio_error() calls put_one() again
  9. Double-free: attempting to push to a full 1024-capacity free list, causing boost::container::bad_alloc

The Fix:
Add a _stopping flag that is set at the start of stop(). Check this flag in need_to_retry() to prevent new schedule_retry() loops from starting during shutdown. This eliminates the race window where a second retry loop could start after the future is exchanged.

The existing retry loop completes normally, and stop()'s .finally() block simply waits for all in-flight iocbs to be reaped and returned. No iocbs are leaked, and no double-free can occur.

Additionally, add debug-mode tracking (_iocb_allocated[]) to detect future double-free bugs and provide diagnostic information.

Fixes: #3169

@tchaikov
Copy link
Contributor Author

The Problem

After stop() calls std::exchange(_pending_aio_retry_fut, make_ready_future<>()), the function retry_in_progress() returns FALSE (since it checks !_pending_aio_retry_fut.available()). This allows a second schedule_retry() loop to start while the first one is completing, creating a race condition.

sequenceDiagram
    participant Reactor
    participant ScheduleRetry_A as schedule_retry() A
    participant Stop as stop()
    participant ScheduleRetry_B as schedule_retry() B
    participant Kernel
    participant Pool as iocb_pool

    Note over ScheduleRetry_A: Processing iocbs in _aio_retries
    ScheduleRetry_A->>Kernel: io_submit(iocb_X, ...)
    Note over Kernel: iocb_X in flight

    Reactor->>Stop: stop() called
    Stop->>Stop: exchange(_pending_aio_retry_fut)
    Note over Stop: retry_in_progress() now FALSE

    Kernel-->>Reactor: iocb_Y completes with -EAGAIN
    Reactor->>Reactor: reap_completions(true)
    Note over Reactor: iocb_Y → _pending_aio_retry

    Reactor->>Reactor: submit_work() checks
    Note over Reactor: need_to_retry() = TRUE<br/>retry_in_progress() = FALSE
    Reactor->>ScheduleRetry_B: NEW schedule_retry() B starts!
    ScheduleRetry_B->>ScheduleRetry_B: swap _pending_aio_retry → _aio_retries
    Note over ScheduleRetry_B: iocb_Y now in _aio_retries

    ScheduleRetry_A->>ScheduleRetry_A: Completes (vectors empty)
    ScheduleRetry_A->>Stop: .finally() block runs

    Kernel-->>Stop: iocb_Y completes with -EAGAIN again
    Stop->>Stop: reap_completions(false)
    Note over Stop: allow_retry=FALSE
    Stop->>Pool: put_one(iocb_Y) ✓ FIRST FREE
    Note over Pool: iocb_Y returned to pool

    ScheduleRetry_B->>ScheduleRetry_B: Process iocb_Y
    ScheduleRetry_B->>ScheduleRetry_B: Gets error (EBADF)
    ScheduleRetry_B->>Pool: put_one(iocb_Y) ✗ SECOND FREE
    Note over Pool: CRASH! Double-free detected<br/>boost::container::bad_alloc
Loading

The Fix

sequenceDiagram
    participant Reactor
    participant ScheduleRetry_A as schedule_retry() A
    participant Stop as stop()
    participant SubmitWork as submit_work()
    participant Kernel
    participant Pool as iocb_pool

    Note over ScheduleRetry_A: Processing iocbs in _aio_retries
    ScheduleRetry_A->>Kernel: io_submit(iocb_X, ...)

    Reactor->>Stop: stop() called
    Stop->>Stop:  _stopping = true
    Stop->>Stop: exchange(_pending_aio_retry_fut)

    Kernel-->>Reactor: iocb_Y completes with -EAGAIN
    Reactor->>Reactor: reap_completions(true)
    Note over Reactor: iocb_Y → _pending_aio_retry

    Note over ScheduleRetry_A: Existing loop continues<br/>Processes iocb_Y
    ScheduleRetry_A->>ScheduleRetry_A: swap _pending_aio_retry → _aio_retries
    ScheduleRetry_A->>Kernel: io_submit(iocb_Y, ...)

    Reactor->>SubmitWork: submit_work() checks
    SubmitWork->>SubmitWork: need_to_retry() checks
    Note over SubmitWork: _stopping = TRUE<br/>Returns FALSE
    Note over SubmitWork: No new retry starts!

    Kernel-->>ScheduleRetry_A: iocb_Y completes
    ScheduleRetry_A->>Pool: put_one(iocb_Y) Returned
    ScheduleRetry_A->>ScheduleRetry_A: Both vectors empty

    ScheduleRetry_A->>Stop: .finally() block runs
    Stop->>Stop: Waits for outstanding() = 0
    Note over Stop: All iocbs accounted for
    Note over Pool: No double-free!<br/> No leaks!
Loading

@xemul
Copy link
Contributor

xemul commented Dec 25, 2025

  • schedule_retry() A is processing iocbs in _aio_retries

OK, so in the picture, there's iocb_X that gets into the kernel.

  • stop() calls std::exchange(_pending_aio_retry_fut, ready_future)

  • New I/Os complete with -EAGAIN, added to _pending_aio_retry

In the picture, there's iocb_Y that completes with EAGAIN and gets into _pending_aio_retry

  • submit_work() sees need_to_retry()=TRUE && !retry_in_progress() (because we exchanged the future) and starts a NEW schedule_retry() B

In the picture, it says that iocb_Y gets into _aio_retries

  • schedule_retry() A completes, .finally() block runs

  • .finally() calls reap_completions(false) which returns -EAGAIN iocbs to the pool (since allow_retry=false)

In the picture, this place says that iocb_Y completes with EAGAIN. If iocb_Y completed, it means that it was submitted in previous step by schedule_retry() B, is that correct?

  • But schedule_retry() B is still running with those same iocbs in _aio_retries!

  • When B tries to process them and encounters errors, handle_aio_error() calls put_one() again

But why? The schedule_retry() B calls io_submit(iocb_Y) and gets error (picture says it's EBADF). In that case, how could reap_completions(false) see that iocb, if it wasn't submitted by schedule_retry() (it returned EBADF on submission). At the same time, if iocb_Y got to that place, it means that it was submitted by schedule_retry() B without errors.

  • Double-free: attempting to push to a full 1024-capacity free list, causing boost::container::bad_alloc

@xemul
Copy link
Contributor

xemul commented Dec 25, 2025

I think I got it. The problem is that two schedule_retry()'s are both submitting each iocb from _aio_retry . So they complete both and are handled both.

If that's correct, the problem is that submit_work() has work to do after aio_storage_context::stop() is called and replaced _aio_retry_fut with ready future from under running retry fiber. With the current fix it may happen that schedule_retry() A finishes, then submit_work() puts more stuff into _pending_aio_retry, then reap_completions(false) is called, but pending retry is never cleaned.

Maybe the proper fix should be that aio_storage_context::stop() first "makes sure" submit_work() is not called again (or at best -- instantly completes all it has with ESOMETHING), then replaces retry fut with ready-future and starts it waiting?

@tchaikov
Copy link
Contributor Author

  • schedule_retry() A is processing iocbs in _aio_retries

OK, so in the picture, there's iocb_X that gets into the kernel.

  • stop() calls std::exchange(_pending_aio_retry_fut, ready_future)
  • New I/Os complete with -EAGAIN, added to _pending_aio_retry

In the picture, there's iocb_Y that completes with EAGAIN and gets into _pending_aio_retry

  • submit_work() sees need_to_retry()=TRUE && !retry_in_progress() (because we exchanged the future) and starts a NEW schedule_retry() B

In the picture, it says that iocb_Y gets into _aio_retries

  • schedule_retry() A completes, .finally() block runs
  • .finally() calls reap_completions(false) which returns -EAGAIN iocbs to the pool (since allow_retry=false)

In the picture, this place says that iocb_Y completes with EAGAIN. If iocb_Y completed, it means that it was submitted in previous step by schedule_retry() B, is that correct?

  • But schedule_retry() B is still running with those same iocbs in _aio_retries!
  • When B tries to process them and encounters errors, handle_aio_error() calls put_one() again

But why? The schedule_retry() B calls io_submit(iocb_Y) and gets error (picture says it's EBADF). In that case, how could reap_completions(false) see that iocb, if it wasn't submitted by schedule_retry() (it returned EBADF on submission). At the same time, if iocb_Y got to that place, it means that it was submitted by schedule_retry() B without errors.

@xemul You're absolutely right to question this - the specific sequence I described may not be entirely accurate. Let me clarify just to make sure we are now on the same page:

The Actual Race Condition

The core issue is simpler than my original description: two concurrent schedule_retry() loops can run simultaneously during shutdown, which leads to shared state corruption in _aio_retries and _pending_aio_retry vectors.

Here's how it happens:

  1. schedule_retry() A is running: it has swapped _aio_retries with _pending_aio_retry and is in the middle of submitting iocbs via the thread pool
  2. stop() is called: executes std::exchange(_pending_aio_retry_fut, make_ready_future<>())
  3. submit_work() gets called by the poller (which is still running)
  4. Line 277 check: retry_in_progress() now returns false (because we just exchanged the future), but there are still items in the retry queues
  5. schedule_retry() B starts: now we have TWO loops both manipulating the same _aio_retries and _pending_aio_retry vectors

The Double-Free

The double-free doesn't necessarily happen through the completion path - it happens because:

  • Both loops perform _aio_retries.erase(...) operations (line 323)
  • Both loops perform std::swap(_aio_retries, _pending_aio_retry) (line 294)
  • These operations on shared state can cause the same iocb to be erased/processed twice
  • When iocbs are "consumed" and erased, they get returned to the pool via handle_aio_error() or via the completion handler
  • If both loops think they consumed the same iocb, it gets returned to the pool twice = double-free
  • Double-free: attempting to push to a full 1024-capacity free list, causing boost::container::bad_alloc

@tchaikov
Copy link
Contributor Author

let me update the time sequence diagram to explain the problem:

  sequenceDiagram
      participant Poller as kernel_submit_work_pollfn
      participant SW as submit_work()                                                                    
      participant SR_A as schedule_retry() A
      participant SR_B as schedule_retry() B     
      participant Pool as iocb_pool
      participant Vectors as _aio_retries / _pending_aio_retry

      Note over SR_A: Already running in thread pool
      SR_A->>Vectors: swap(_aio_retries, _pending_aio_retry)
      SR_A->>SR_A: Submitting iocbs from _aio_retries
                                                    
      Note over Poller: stop() is called on reactor
      Poller->>Poller: _stopping = true       
      Poller->>Poller: exchange(_pending_aio_retry_fut, ready)
                                                    
      Note over Poller: Poller still active!
      Poller->>SW: kernel_submit_work()                                                                  
      SW->>SW: Check: need_to_retry() && !retry_in_progress()
      Note over SW: retry_in_progress() = FALSE<br/>(future was exchanged!)
      Note over SW: need_to_retry() = TRUE<br/>(vectors still have items)
                                                                                                         
      SW->>SR_B: schedule_retry()         
                                                                                                         
      rect rgb(100, 100, 100)
          Note over SR_A,SR_B: RACE: Both loops operating on same vectors!

          par schedule_retry() A                                                                                                                                                                                   
              SR_A->>Vectors: _aio_retries.erase(...)                                          
              SR_A->>Pool: handle_aio_error() -> put_one(iocb_X)                                                                                                                                                   
              Note over Pool: iocb_X returned to pool                                  
          and schedule_retry() B
              SR_B->>Vectors: swap(_aio_retries, _pending_aio_retry)                                                                                                                                               
              SR_B->>Vectors: Process same iocbs
              SR_B->>Vectors: _aio_retries.erase(...)                                                                                                                                                              
              SR_B->>Pool: handle_aio_error() -> put_one(iocb_X)
              Note over Pool: DOUBLE FREE!                                                                                                                                                                         
          end                                  
      end

      Pool->>Pool: ❌ Assertion/abort on double-free
Loading

@tchaikov tchaikov force-pushed the aio-teardown-double-free branch from b22504f to eb6e789 Compare December 26, 2025 07:48

bool need_to_retry() const noexcept {
return !_pending_aio_retry.empty() || !_aio_retries.empty();
return !_stopping && (!_pending_aio_retry.empty() || !_aio_retries.empty());
Copy link
Contributor

Choose a reason for hiding this comment

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

Can _stopping be true here? The submit_work doesn't do anything in that case

bool
aio_storage_context::submit_work() {
if (_stopping) [[unlikely]] {
return false; // Don't accept new work during shutdown
Copy link
Contributor

Choose a reason for hiding this comment

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

What if _stopping == true, but there's something in io_sink? Those IOs won't be completed
Maybe something like that:

void submit_work() {
    ...
    if (need_to_retry()) /* as it is before this patch */ {
        if (_stopping) {
            /* walk _pending_aio_retry and complete all with ESOMETHING */
        } else if (!retry_in_progress()) {
            schedule_retry();
        }
    }
}

?

@tchaikov tchaikov force-pushed the aio-teardown-double-free branch 4 times, most recently from a16b72f to d584d39 Compare December 26, 2025 11:30
@tchaikov
Copy link
Contributor Author

v3:

  • do not check for _stopping in need_to_retry()
  • incorporated Pavel's suggestion in aio_storage_context::submit_work()

@tchaikov tchaikov force-pushed the aio-teardown-double-free branch from d584d39 to eb0cc71 Compare December 26, 2025 12:18
@tchaikov tchaikov requested a review from xemul December 28, 2025 14:03
desc->complete_with(-ECANCELED);
return true;
});

Copy link
Member

Choose a reason for hiding this comment

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

Q: why are we even here? If we're shutting down the reactor, nothing should be in progress.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

it was a defensive draining approach, which intended to address the misbehaved applications which fail to drain all in-progress operations. but yes, it seems we don't tolerate this. so i am removing it.

Copy link
Contributor

Choose a reason for hiding this comment

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

The problem this PR fixes happens when there's something in submit_work() after aio_storage_context::stop() has been called.

Still I think it's not great to put such ifs in submit_work() main path. All dangling iocbs should be collected by aio_storage_context::stop() and submit_work() shouldn't put more load into the stopped/ing retry fiber (#3182 (comment))

Copy link
Contributor Author

@tchaikov tchaikov left a comment

Choose a reason for hiding this comment

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

v4:

  • dropped the defensive draining logic in aio_storage_context::stop()
  • guarded the assert of _free_iocbs.size() < max_aio in aio_storage_context::iocb_pool::put_one() with SEASTAR_IOCB_POOL_DEBUG, as it's in the hot path.

@tchaikov tchaikov force-pushed the aio-teardown-double-free branch 2 times, most recently from 4386573 to 7c58150 Compare December 29, 2025 01:46
@tchaikov tchaikov requested a review from avikivity December 29, 2025 01:52
tchaikov added a commit to tchaikov/seastar that referenced this pull request Dec 29, 2025
Fix the double-free bug in iocb pool management that causes bad_alloc
during AIO teardown (issue scylladb#3169, scylladb/scylladb#17369,
scylladb/scylladb#26481).

Root Cause:
The bug occurs when stop() is called while AIO retry operations are
in progress. The race condition happens because two concurrent
schedule_retry() loops can run simultaneously during shutdown:

1. schedule_retry() A is running: has swapped _aio_retries with
   _pending_aio_retry and is submitting iocbs via thread pool
2. stop() is called and executes:
   std::exchange(_pending_aio_retry_fut, make_ready_future<>())
3. kernel_submit_work_pollfn continues polling and calls submit_work()
4. submit_work() checks: need_to_retry() && !retry_in_progress()
   - retry_in_progress() returns FALSE (future was just exchanged)
   - need_to_retry() returns TRUE (vectors still contain items)
5. A second schedule_retry() B starts
6. RACE: Both loops manipulate the same _aio_retries and _pending_aio_retry
   vectors concurrently
7. Both loops perform swap() and erase() operations on shared state
8. Same iocbs get processed/erased by both loops
9. handle_aio_error() or completion handlers call put_one() twice for the
   same iocb
10. Double-free: attempting to push to a full 1024-capacity free list,
    causing boost::container::bad_alloc

The Fix:
Add a _stopping flag that is set at the start of stop() and checked
in need_to_retry() and submit_work(). This prevents new schedule_retry()
loops from starting during shutdown.

Implementation Strategy:
To avoid adding overhead to the frequently-called submit_work() main path,
the draining logic is placed in stop() rather than submit_work(). This
addresses the concern that putting conditional checks in submit_work()'s
main path is not ideal (per review feedback in PR scylladb#3182).

When _stopping is set in stop():
- All dangling iocbs are immediately collected by stop() itself:
  * Items in io_sink (not yet allocated iocbs) are drained and completed
    with -ECANCELED
  * Items in _pending_aio_retry (already allocated iocbs) are returned
    to the pool and completed with -ECANCELED
- need_to_retry() returns false, preventing new retry loops from starting
- submit_work() simply returns early without processing any work
- The single in-flight retry loop (if any) completes safely
- stop()'s do_until loop waits for all outstanding iocbs to be reaped
  and returned to the pool exactly once

This ensures:
- submit_work() doesn't put more work into the stopped/stopping retry fiber
- No iocbs are leaked
- All pending operations receive proper completion notifications
- Minimal overhead on the submit_work() fast path (just a flag check)

Additional debug tracking (_iocb_allocated bitset) helps detect and
diagnose future double-free bugs with clear error messages.

Fixes: scylladb#3169
Signed-off-by: Kefu Chai <k.chai@proxmox.com>
@tchaikov tchaikov force-pushed the aio-teardown-double-free branch from 7c58150 to c68ec57 Compare December 29, 2025 08:50

bool
aio_storage_context::submit_work() {
if (_stopping) [[unlikely]] {
Copy link
Contributor

Choose a reason for hiding this comment

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

Let's allow submit_work() do whatever it prefers and make aio_storage_context::stop() collect whatever appears from it. At most, add extra activity under existing need_retry() check.

Maybe something like that would work? (I'm not sure that putting default-initialized future into _pending_aio_retry_fut is good)

diff --git a/src/core/reactor_backend.cc b/src/core/reactor_backend.cc
index 725f0daf6..4e6dc20bf 100644
--- a/src/core/reactor_backend.cc
+++ b/src/core/reactor_backend.cc
@@ -131,9 +131,10 @@ aio_storage_context::~aio_storage_context() {
 }
 
 future<> aio_storage_context::stop() noexcept {
-    return std::exchange(_pending_aio_retry_fut, make_ready_future<>()).finally([this] {
+    return std::exchange(_pending_aio_retry_fut, future<>() /* not future::available() future */).finally([this] {
         return do_until([this] { return !_iocb_pool.outstanding(); }, [this] {
             reap_completions(false);
+            reap_pending_retries();
             return make_ready_future<>();
         });
     });
@@ -243,7 +244,7 @@ aio_storage_context::submit_work() {
         did_work = true;
     }
 
-    if (need_to_retry() && !retry_in_progress()) {
+    if (need_to_retry()) {
         schedule_retry();
     }
 
@@ -251,6 +252,10 @@ aio_storage_context::submit_work() {
 }
 
 void aio_storage_context::schedule_retry() {
+    if (!_pending_aio_retry_fut.available()) {
+        return;
+    }
+
     // loop until both _pending_aio_retry and _aio_retries are empty.
     // While retrying _aio_retries, new retries may be queued onto _pending_aio_retry.
     _pending_aio_retry_fut = do_until([this] {
@@ -317,6 +322,16 @@ bool aio_storage_context::reap_completions(bool allow_retry)
     return n;
 }
 
+void aio_storage_context::reap_pending_retries()
+{
+    for (auto* iocb : _pending_aio_retry) {
+        auto desc = get_user_data<kernel_completion>(*iocb);
+        desc->complete_with(-ECANCELED);
+        _iocb_pool.put_one(iocb);
+    }
+    _pending_aio_retry.clear();
+}
+
 bool aio_storage_context::can_sleep() const {
     // Because aio depends on polling, it cannot generate events to wake us up, Therefore, sleep
     // is only possible if there are no in-flight aios. If there are, we need to keep polling.
diff --git a/src/core/reactor_backend.hh b/src/core/reactor_backend.hh
index aa240b8c6..3d4efb9c0 100644
--- a/src/core/reactor_backend.hh
+++ b/src/core/reactor_backend.hh
@@ -87,9 +87,7 @@ class aio_storage_context {
         return !_pending_aio_retry.empty() || !_aio_retries.empty();
     }
 
-    bool retry_in_progress() const noexcept {
-        return !_pending_aio_retry_fut.available();
-    }
+    void reap_pending_retries();
 
 public:
     explicit aio_storage_context(reactor& r);

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Thanks Pavel for the suggested patch! This implementation is largely based on your approach, with a few differences:

  1. future<>() vs promise<>.get_future(): because future<>() doesn't have a public default constructor. . The only ways to create futures are:

    • make_ready_future<>() - available immediately
    • promise<>.get_future() - available when promise is fulfilled
    • make_exception_future<>() - available with exception
  2. single reap_pending_retries() vs repeated draining
    The reasoning: after exchange, no new items can enter _pending_aio_retry because:

    1. io_sink already drained → no new iocbs allocated
    2. schedule_retry() returns early → no new retry loops
    3. reap_completions(false) doesn't add to retry queue

    So calling it once is sufficient by design.

  3. .finally() vs .then(): The difference: .then() waits for the in-flight retry loop to complete successfully before draining. With .finally(), if the old future fails, we'd still proceed with cleanup - but the old retry loop should always complete normally (it's a do_until that exits when vectors are empty).

Fix the double-free bug in iocb pool management that causes bad_alloc
during AIO teardown (issue scylladb#3169, scylladb/scylladb#17369,
scylladb/scylladb#26481).

Root Cause:
The bug occurs when stop() is called while AIO retry operations are
in progress. The race condition happens because two concurrent
schedule_retry() loops can run simultaneously during shutdown:

1. schedule_retry() A is running: has swapped _aio_retries with
   _pending_aio_retry and is submitting iocbs via thread pool
2. stop() is called and executes:
   std::exchange(_pending_aio_retry_fut, make_ready_future<>())
3. kernel_submit_work_pollfn continues polling and calls submit_work()
4. submit_work() checks: need_to_retry() && !retry_in_progress()
   - retry_in_progress() returns FALSE (future was just exchanged)
   - need_to_retry() returns TRUE (vectors still contain items)
5. A second schedule_retry() B starts
6. RACE: Both loops manipulate the same _aio_retries and _pending_aio_retry
   vectors concurrently
7. Both loops perform swap() and erase() operations on shared state
8. Same iocbs get processed/erased by both loops
9. handle_aio_error() or completion handlers call put_one() twice for the
   same iocb
10. Double-free: attempting to push to a full 1024-capacity free list,
    causing boost::container::bad_alloc

The Fix:
Use std::exchange to atomically replace _pending_aio_retry_fut with a
never-available future. This prevents new retry loops from starting during
shutdown while avoiding undefined behavior.

Implementation:
1. In schedule_retry():
   - Check !_pending_aio_retry_fut.available() before starting new loop
   - After stop() exchanges the future, available() returns false
   - Prevents new retry loops during shutdown
   - Well-defined behavior (not accessing moved-from future)

2. In stop():
   - Drain io_sink (items without allocated iocbs)
   - Create a promise that is never fulfilled (never-available future)
   - Use std::exchange to swap _pending_aio_retry_fut with this future
   - Wait for the old (in-flight) retry loop to complete
   - Drain pending retry queues with reap_pending_retries()
   - Wait for all outstanding iocbs to be reaped

3. reap_pending_retries() drains both _pending_aio_retry and _aio_retries,
   returning iocbs to pool and completing with -ECANCELED

4. Debug tracking (#ifdef SEASTAR_IOCB_POOL_DEBUG):
   - Bitset to track allocated iocbs
   - Detects double-allocation and double-free
   - Clear error messages with iocb index/pointer

Cannot use std::exchange with make_ready_future<>() because:
- That would make _pending_aio_retry_fut.available() return true
- schedule_retry() would pass the !available() check
- It could start a new retry loop during shutdown (incorrect!)

Key Assumptions (Single-Threaded Reactor Model):
- All operations on shared state run on the reactor thread
- stop() is called once during shutdown by the reactor
- After stop() drains io_sink, no new I/O requests arrive
- The reactor event loop provides happens-before ordering
- No explicit synchronization primitives needed

This ensures:
- No new retry loops start after stop() is called
- Exactly one retry loop runs at a time
- The in-flight retry loop completes safely
- All pending iocbs are properly drained and returned exactly once
- All operations receive proper completion notifications (-ECANCELED for
  items not yet submitted to kernel, actual kernel result for submitted items)
- No iocbs are leaked
- Well-defined behavior (no UB from moved-from futures)
- Simple implementation relying on reactor guarantees

Fixes: scylladb#3169
Signed-off-by: Kefu Chai <k.chai@proxmox.com>
@tchaikov tchaikov force-pushed the aio-teardown-double-free branch from c68ec57 to ac3cbef Compare December 30, 2025 06:30
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.

bad_alloc on AIO teardown

3 participants