This page is meant as a template for writing a KIP. To create a KIP choose Tools->Copy on this page and modify with your content and replace the heading with the next KIP number and a description of your issue. Replace anything in italics with your own description.
Status
Current state: "Under Discussion"
Discussion thread: here [Change the link from the KIP proposal email archive to your own email thread]
JIRA: here [Change the link from KAFKA-1 to your own ticket]
Please keep the discussion on the mailing list rather than commenting on the wiki (wiki discussions get unwieldy fast).
Motivation
Tiered Storage provides infinite storage to Kafka. To achieve this Kafka uploads segments, indecies, leader epochs and producer snapshots to remote storage. Producer snapshots started being aligned to segments as part of addressing KIP-405: Kafka Tiered Storage#Upgrade.
in Apache Kafka 2.8.0. This means that if a customer upgrades from a version < 2.8.0 they will have to wait for retention to clean up segments without an associated producer snapshot as detailed in
This, however, is very niche knowledge and in our experience customers expect to be able to immediately enable tiering on a topic once their upgrade is complete. Once they do this, however, they start seeing https://github.com/apache/kafka/blob/trunk/storage/api/src/main/java/org/apache/kafka/server/log/remote/storage/LogSegmentData.java#L61 and no data is uploaded to Tiered Storage.
To provide a smooth upgrade experience we propose to change Kafka to create an empty producer snapshot file and upload it whenever a segment is due to be archived and lacks one.
Public Interfaces
Briefly list any new interfaces that will be introduced as part of this proposal or any existing interfaces that will be removed or changed. The purpose of this section is to concisely call out the public contract that will come along with this feature.
No public interfaces will be changed as part of this KIP.
Proposed Changes
Describe the new thing you want to do in appropriate detail. This may be fairly extensive and have large subsections of its own. Or it may be a few sentences. Use judgement based on the scope of the change.
We propose to create a segment-aligned empty producer snapshot whenever we detect that the value here https://github.com/apache/kafka/blob/trunk/core/src/main/java/kafka/log/remote/RemoteLogManager.java#L581 is null. The rest of Tiered Storage's logic should pick it up for upload from then on.
Compatibility, Deprecation, and Migration Plan
- What impact (if any) will there be on existing users? N/A
- If we are changing behavior how will we phase out the older behavior? N/A
- If we need special migration tools, describe them here. N/A
- When will we remove the existing behavior? N/A
Test Plan
Describe in few sentences how the KIP will be tested. We are mostly interested in system tests (since unit-tests are specific to implementation details). How will we know that the implementation works as expected? How will we know nothing broke?
Unit and integration tests.
Rejected Alternatives
If there are alternative ways of accomplishing the same thing, what were they? The purpose of this section is to motivate why the design is the way it is and not some other way.
- Generate a correct producer snapshot prior to upload - this will be expensive because we either have to reply the log from the beginning or choose an arbitrary earlier segment as the starting point to start calculating said snapshot from.
- Make Kafka wait to upload segments until it has expired all segments lacking a producer snapshot - we think this is just as counter-intuitive (especially if customers aren't well-aware of what producer snapshots are used for) and requiring niche knowledge as what the current approach is.
- Do not allow a topic to have its tiering configuration set to true until all of its segments have an associated producer snapshot - the limitation here is that this condition needs to hold true across all brokers and additional synchronisation will be required.
- Do not archive snapshot files - like indecies, snapshot files can be recreated by rereading the log. However, in the case of Tiered Storage we make the assumption that replaying the whole log will be quite costly.
- Create empty snapshot files on read if a snapshot file is not found - this is a close second runner, but we chose to discard it. It would not be immediately obvious whether a snapshot is not presented because of an upgrade of Kafka version, because there is a bug in the plugin implementation or because the remote storage itself has been tampered with - if we explicitly write an empty file we reduce some of this ambiguity.