Skip to content

KAFKA-19937: Introduced Shared ReaperThread for Persister / NetworkPartitionMetadataClient#21842

Open
rionmonster wants to merge 3 commits intoapache:trunkfrom
rionmonster:KAFKA-19937
Open

KAFKA-19937: Introduced Shared ReaperThread for Persister / NetworkPartitionMetadataClient#21842
rionmonster wants to merge 3 commits intoapache:trunkfrom
rionmonster:KAFKA-19937

Conversation

@rionmonster
Copy link
Copy Markdown
Contributor

@rionmonster rionmonster commented Mar 20, 2026

Description

This pull request addresses the redundant thread usage detailed in
KAFKA-19937
affecting the PersisterStateManager and
NetworkPartitionMetadataClient classes specifically. Presently each
creates/manages its own separate SystemTimerReaper instances, but rely
on identical timers with independent tasks. The changes proposed address
this by introducing a new, sharable instance of the thread to reduce
overhead.

Key Changes

  • Updated BrokerServer to create a single shared SystemTimerReaper
    instance used by both PersisterStateManager and
    NetworkPartitionMetadataClient, with cleanup in the shutdown path
    after both components have been stopped.
  • Moved timer ownership to the caller for the affected classes to the
    caller (e.g., PersisterStateManager.stop() and
    NetworkPartitionMetadataClient.close() no longer close their injected
    timer, as lifecycle is managed by BrokerServer).
    • This specific timer ownership behavior is documented via JavaDocs
      for both PersisterStateManager and NetworkPartitionMetadataClient
  • Added null validation to the SystemTimerReaper constructor
    arguments.

Tests and Verification

Verified that all existing test suites still pass as expected and added
the following to verify new behavior and usage related to the above
changes:

  • Extended SystemTimerReaperTest.java to verify null validity and
    timer-sharing behavior (e.g., two consumers sharing a timer can both
    schedule and expire tasks independently).
  • Updated PersisterStateManagerTest.java to verify that stop() does
    not close the timer, consistent with the new caller-ownership contract.

Reviewer(s)

Tagging @AndrewJSchofield (initial reporter)

Reviewers: Sushant Mahajan smahajan@confluent.io, Andrew Schofield
aschofield@confluent.io

@github-actions github-actions Bot added triage PRs from the community core Kafka Broker and removed triage PRs from the community labels Mar 20, 2026
Copy link
Copy Markdown
Collaborator

@smjn smjn left a comment

Choose a reason for hiding this comment

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

Thanks for the PR, minor changes

@rionmonster
Copy link
Copy Markdown
Contributor Author

@smjn

Thanks for the feedback! I've addressed both the items you mentioned, feel free to take a look whenever convenient.

Copy link
Copy Markdown
Collaborator

@smjn smjn left a comment

Choose a reason for hiding this comment

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

Thanks for the changes, LGTM

Copy link
Copy Markdown
Member

@AndrewJSchofield AndrewJSchofield left a comment

Choose a reason for hiding this comment

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

Thanks for the PR. I wonder if this could be restructured a bit. For example, SystemTimerReaper.close() doesn't really close at all if ownsReaper is set.

});
reaper.awaitShutdown();
timer.close();
if (ownsReaper) {
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

I'm not entirely comfortable with the structure of the code here. There are perhaps two closely related classes here, instead of one class with ownsReaper which makes quite a big difference to the lifecycle of the objects.

Copy link
Copy Markdown
Contributor Author

@rionmonster rionmonster Apr 2, 2026

Choose a reason for hiding this comment

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

Good point, that makes sense. The current wrapper is effectively modeling two different lifecycle/ownership modes in a single type, which makes the semantics a bit muddled.

One option to consider would be splitting this into two distinct classes: one owning and one shared (e.g., SystemTimerReaper could remain the owning variant, while a separate SharedSystemTimerReaper could represent the shared case without conditional lifecycle behavior). This would better isolate the behavioral differences, and the names alone would likely improve clarity.

Another would be to just move the ownership of the timer to the BrokerServer itself (e.g., the components sharing it wouldn't be responsible for closing it at all), which might be an even simpler option.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

@AndrewJSchofield

I revisited this with fresh eyes and have updated the PR accordingly (rebased for clarity). The new approach moves ownership of the shared timer to BrokerServer rather than the child components, which aligns with how I believe some other shared resources are managed in the codebase. This significantly reduces the footprint of the changes compared to the original approach as well (e.g., no extracted separate class, conditional logic, etc.).

Let me know what you think or if you'd like to explore a different direction.

@github-actions github-actions Bot added the small Small PRs label Apr 10, 2026
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

ci-approved core Kafka Broker small Small PRs

Projects

None yet

Development

Successfully merging this pull request may close these issues.

3 participants