fix: don't block all partitions while one is initializing#118
Merged
novatechflow merged 2 commits intoKafScale:mainfrom Feb 28, 2026
Merged
fix: don't block all partitions while one is initializing#118novatechflow merged 2 commits intoKafScale:mainfrom
novatechflow merged 2 commits intoKafScale:mainfrom
Conversation
85656c7 to
c9ef3d7
Compare
logMu was held as an exclusive lock across etcd and S3 I/O during partition initialization. A single slow RestoreFromS3 (1+2N S3 round-trips for N segments) blocked all produce, fetch, and list-offsets requests broker-wide. Replace sync.Mutex with sync.RWMutex so the fast path (partition already initialized) uses a shared read lock. Move all I/O outside the lock and use singleflight.Group to deduplicate concurrent initialization per partition without blocking other partitions.
c9ef3d7 to
cc9d7f6
Compare
novatechflow
approved these changes
Feb 28, 2026
Collaborator
novatechflow
left a comment
There was a problem hiding this comment.
smart! Thanks again @klaudworks
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Sign up for free
to join this conversation on GitHub.
Already have an account?
Sign in to comment
Add this suggestion to a batch that can be applied as a single commit.This suggestion is invalid because no changes were made to the code.Suggestions cannot be applied while the pull request is closed.Suggestions cannot be applied while viewing a subset of changes.Only one suggestion per line can be applied in a batch.Add this suggestion to a batch that can be applied as a single commit.Applying suggestions on deleted lines is not supported.You must change the existing code in this line in order to create a valid suggestion.Outdated suggestions cannot be applied.This suggestion has been applied or marked resolved.Suggestions cannot be applied from pending reviews.Suggestions cannot be applied on multi-line comments.Suggestions cannot be applied while the pull request is queued to merge.Suggestion cannot be applied right now. Please check back later.
Problem
After a broker restart, partition state is rebuilt from S3 on first access.
getPartitionLogholds a single global lock while doing this — listing segments, downloading footers and indexes. If a partition has many segments, this can take seconds, and during that time every produce, fetch, and list-offsets request on the entire broker is blocked, even for unrelated topics.We hit this during benchmarking: producing to a brand new empty topic hung because another partition was rebuilding its index from ~300k S3 segments.
Fix
singleflightto make sure only one goroutine initializes a given partition, without blocking other partitions from initializing in parallel