lhotari opened a new pull request, #4728:
URL: https://github.com/apache/bookkeeper/pull/4728

   ## Motivation
   
   While investigating a CI flakiness in `BookieAutoRecoveryTest` (PR #4711) — 
where tests waiting up to 60 s for ledgers to be marked as underreplicated 
would time out — three related race conditions in `Auditor` were found that 
could allow a bookie departure to go undetected until the next periodic audit 
run (default: 86400 s).
   
   Analysis of the surefire logs for the failing CI run showed a ~70-second 
silence from the auditor after a bookie was killed, followed by a new auditor 
election that eventually caught the departure. This pointed to the auditor's 
bookie-change detection path rather than a test-setup issue.
   
   ## Races fixed
   
   ### Race 1 – watcher-provided bookie set was discarded
   
   `watchBookieChanges()` registered `bookies -> submitAuditTask()` as the ZK 
listener, ignoring the current bookie set already delivered by the callback. 
Inside `submitAuditTask()` a fresh `getAvailableBookies()` ZK call was made 
from an executor lambda that could execute well after the watcher fired — for 
example while `AuditorCheckAllLedgersTask` was occupying the single-threaded 
executor (it creates a temporary `BookKeeper` client which registers its own ZK 
watches, and closing it can overlap with the watcher event). In that window the 
two snapshots of the cluster could diverge, causing `bookiesToBeAudited` to 
remain empty and the departure to be silently skipped.
   
   **Fix:** capture `pendingWritableBookies` / `pendingReadOnlyBookies` 
atomically in each watcher callback and consume them in the extracted 
`runAuditTask()`, falling back to `getAvailableBookies()` only when watchers 
have not yet fired (e.g. direct test calls via `submitAuditTask()`).
   
   ### Race 2 – duplicate audit tasks could be queued per change type
   
   Rapid successive watcher callbacks (or a slow executor) could enqueue many 
identical audit tasks, each making its own ZK calls and risking interleaved 
mutations to `knownBookies` / `bookiesToBeAudited`.
   
   **Fix:** separate `AtomicBoolean` queued-flags (`writableAuditTaskQueued`, 
`readOnlyAuditTaskQueued`) ensure at most one task is *queued* per change type 
at a time. The flag is cleared at the *start* of the task (not at submission), 
so watcher callbacks arriving while a task is already running queue exactly one 
follow-up task rather than being silently dropped.
   
   ### Race 3 – `knownBookies` initialised after watcher registration
   
   `start()` called `watchBookieChanges()` before assigning `knownBookies`. A 
watcher callback firing immediately (the ZK registration client fires listeners 
synchronously with a cached bookie set if one is already available) could race 
with the subsequent `knownBookies = admin.getAllBookies()` assignment. Although 
the `synchronized` on `start()` and `submitAuditTask()` prevented the executor 
*lambda* from running before `knownBookies` was set, the field was not 
`volatile`, leaving a visibility gap for the executor thread.
   
   **Fix:** swap the initialisation order so `knownBookies` is populated before 
`watchBookieChanges()` is called, and declare the field `volatile` to guarantee 
cross-thread visibility of the reference.
   
   ## Changes
   
   | What | Change |
   |------|--------|
   | `Auditor.knownBookies` | Add `volatile` |
   | `pendingWritableBookies`, `pendingReadOnlyBookies` | New 
`AtomicReference<Set<String>>` fields holding the latest bookie set per watcher 
type |
   | `writableAuditTaskQueued`, `readOnlyAuditTaskQueued` | New `AtomicBoolean` 
deduplication flags per change type |
   | `watchBookieChanges()` | Populate the pending sets and call 
`submitAuditTaskForBookieChange(boolean)` |
   | `submitAuditTaskForBookieChange(boolean)` | New method: queues a task only 
when no task of that type is already queued; clears its flag at task start |
   | `submitAuditTask()` | Public/test API unchanged; delegates to extracted 
`runAuditTask()` |
   | `runAuditTask()` | Extracted audit logic; uses watcher-captured bookie 
sets, falls back to `getAvailableBookies()` |
   | `start()` | Initialise `knownBookies` before calling 
`watchBookieChanges()` |
   
   ## Testing
   
   The change is purely in production code in `Auditor`. Existing tests in 
`BookieAutoRecoveryTest` exercise all affected paths and continue to pass.


-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to