-
Notifications
You must be signed in to change notification settings - Fork 531
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
fix MimirIngesterStuckProcessingRecordsFromKafka
#9855
base: main
Are you sure you want to change the base?
Conversation
The alert `MimirIngesterStuckProcessingRecordsFromKafka` relied on the metric `cortex_ingest_storage_reader_buffered_fetch_records_total ` provided by the Kafka client to identify wether we had stuck buffers or not. Now that we've implemented concurrent fetching from Kafka and bypass the client's polling function we needed an equivalent metric when using concurrent fetching. This PR does that; In addition to that - the metric also takes the client's buffered records In case we do use a mixture of non-concurrent fetching and concurrent fetching.
Signed-off-by: gotjosh <[email protected]>
@@ -226,14 +229,8 @@ type concurrentFetchers struct { | |||
|
|||
// trackCompressedBytes controls whether to calculate MaxBytes for fetch requests based on previous responses' compressed or uncompressed bytes. | |||
trackCompressedBytes bool | |||
} | |||
|
|||
// Stop implements fetcher |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This is just a move as is - these methods above the constructor seem to be oddly placed.
Signed-off-by: gotjosh <[email protected]>
Signed-off-by: gotjosh <[email protected]>
Signed-off-by: gotjosh <[email protected]>
Signed-off-by: gotjosh <[email protected]>
Signed-off-by: gotjosh <[email protected]>
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
You improved an unit test to check that the value is 0 when done, but there's no unit test asserting that this value is actually tracked. Can we add a test running for the 4 concurrency variants that test the value is tracked to the buffered number of records? I think we can implement a mocked "pusher" that doesn't process records (it just hangs) so next records are buffered in the client, then we assert on the metric, and finally we release the pusher to end the test.
Signed-off-by: gotjosh <[email protected]>
Signed-off-by: gotjosh <[email protected]>
Signed-off-by: gotjosh <[email protected]>
|
||
r.committer = newPartitionCommitter(r.kafkaCfg, kadm.NewClient(r.client), r.partitionID, r.consumerGroup, r.logger, r.reg) | ||
r.committer = newPartitionCommitter(r.kafkaCfg, kadm.NewClient(r.getClient()), r.partitionID, r.consumerGroup, r.logger, r.reg) |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I think I don't need to rework these arguments as closures here (e.g. func() *kgo.Client return { r.getClient() }
) as this only happens within start
and from my understanding, start
, stop
and run
are guaranteed to never concurrently - but please let me know if I'm wrong.
Signed-off-by: gotjosh <[email protected]>
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Nice work! LGTM
@@ -496,6 +510,7 @@ func (r *concurrentFetchers) run(ctx context.Context, wants chan fetchWant, logg | |||
attemptSpan.SetTag("attempt", attempt) | |||
|
|||
f := r.fetchSingle(ctx, w) | |||
r.bufferedFetchedRecords.Add(int64(len(f.FetchPartition.Records))) |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
if something calls Stop()
before these records are pushed to orderedFetches
then the atomic counter will always be positive. Should we decrement the counter if we give up on sending these records?
I'm thinking about the cases in this select which end up returning
select {
case <-r.done:
wantSpan.Finish()
attemptSpan.Finish()
close(w.result)
return
case w.result <- f:
previousResult = fetchResult{}
case <-ctx.Done():
default:
if w.startOffset >= w.endOffset {
// We've fetched all we were asked for the whole batch is ready, and we definitely have to wait to send on the channel now.
f.startWaitingForConsumption()
select {
case <-r.done:
wantSpan.Finish()
attemptSpan.Finish()
close(w.result)
return
case w.result <- f:
previousResult = fetchResult{}
case <-ctx.Done():
}
}
}
at this point you can also try to unify all the places which do cleanup actions (like finishing spans, closing channels, now also decrementing the atomic counter)
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
if something calls Stop() before these records are pushed to orderedFetches then the atomic counter will always be positive
I thought about it too, but do we really care?
- If the stop is called at shutdown, we don't care.
- If the stop is called because we're moving from concurrent fetcher at startup to sequential fetchwe when ongoing, we don't care because the fetcher reference is trashed
- If the stop is called because we're updating the concurrent fetcher config (
Update()
) then we care... so what if we simply reset the buffer inStop()
after ther.wg.Wait()
to keep it simple?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
- If the stop is called because we're updating the concurrent fetcher config (
Update()
) then we care... so what if we simply reset the buffer inStop()
after ther.wg.Wait()
to keep it simple?
yeah that's another option 👍
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
can you explain why we need the mutex around the fetcher and the client?
Client is set in the starting function but there's no guarantee metrics couldn't be scraped in the meanwhile, so race condition. Fetcher same, and also could be updated later if we change it between start and ongoing, so race condition. |
😿 go guarantees atomic pointer swaps, so technically we should be safe. Was the race detector complaining? |
Does it? So why |
Whole-word pointer operations are atomic, but their order w/ respect to surrounding code executing is not guaranteed without the atomic stuff. That's what the race detector complains about. Many of the whole-word atomic types boil down to using different assembly instructions that make sure dirty CPU cache entries are invalidated appropriately. |
Signed-off-by: gotjosh <[email protected]>
What this PR does
The alert
MimirIngesterStuckProcessingRecordsFromKafka
relied on the metriccortex_ingest_storage_reader_buffered_fetch_records_total
provided by the Kafka client to identify wether we had stuck buffers or not.Now that we've implemented concurrent fetching from Kafka and bypass the client's polling function we needed an equivalent metric when using concurrent fetching. This PR does that; In addition to that - the metric also takes the client's buffered records In case we do use a mixture of non-concurrent fetching and concurrent fetching.
Which issue(s) this PR fixes or relates to
N/A
Checklist
CHANGELOG.md
updated - the order of entries should be[CHANGE]
,[FEATURE]
,[ENHANCEMENT]
,[BUGFIX]
.about-versioning.md
updated with experimental features.