cmccabe opened a new pull request, #12776:
URL: https://github.com/apache/kafka/pull/12776

   Unify KRaft metadata snapshot generation between the broker and the 
controller, so that it all flows through the MetadataImage#write code path.
   
   Previously, the broker generated MetadataImage objects inside 
BrokerMetadataListener, and published them to a single publisher, named 
BrokerMetadataPublisher. Now, with this PR, MetadataImage objects are loaded by 
the MetadataLoader, a generic metadata class shared between controller and 
broker. The image objects are now published to a list of publishers, rather 
than just to a single publisher.  Also, publishers can also be dynamically 
added and removed. On the broker side, we use the dynamic addition and removal 
to only add BrokerMetadataPublisher when our metadata has "caught up" to the 
controller's.
   
   With this PR, snapshot generation for both the controller and the broker is 
handled by a metadata publisher named SnapshotGenerator. This replaces 
BrokerMetadataSnapshotter and the controller's SnapshotGenerator. This means 
that the code for generating snapshots from the controller data structures can 
be deleted and we can maintain only one code path here. The new 
SnapshotGenerator implements time-based snapshotting as specified by KIP-876.
   
   This PR also changes how we handle changing the metadata version in 
MetadataDelta and MetadataImage. Previously, we called 
handleMetadataVersionChange on all delta objects whenever we encountered a 
metadata change in the log. This approach was difficult to maintain, however, 
since changing metadata versions can add or remove fields from both deltas and 
images. In practice, we did not correctly implement version changes, which is 
why we had to disable KRaft downgrades in the 3.3 release. The new approach 
here is to perform an image save and load cycle. That way we only have to 
maintain one code path for upgrades and downgrades, in MetadataImage#write. 
Because doing a save and load cycle is somewhat out of scope for MetadataDelta 
itself, it simply throws a MetadataVersionChangedException and lets the 
MetadataLoader handle the change.
   
   On a related note, this PR removes offset and epoch tracking from 
MetadataDelta. This tracking is done by external classes (such as 
MetadataLoader) so it doesn't need to be done in MetadataDelta. It also 
introduces a builder for MetadataDelta so that we can add new constructor 
parameters more easily in the future. (The only new parameter here is 
MetadataVersion.) Finally, there is a new class, MetadataProvenance, which 
encompasses offset, epoch, and timestamp, which identifies the source 
(provenance) of all MetadataImages.


-- 
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: jira-unsubscr...@kafka.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org

Reply via email to