Table of Contents |
---|
Status
Current state: Under DiscussionAccepted
Discussion thread: here
JIRA: KAFKA-12541
...
We will add the new offset spec in org.apache.kafka.clients.admin.OffsetSpec:
public class OffsetSpec { public static class EarliestSpec extends OffsetSpec { } public static class LatestSpec extends OffsetSpec { } public static class MaxTimestampSpec extends OffsetSpec { } // this is new .... /** .... |
We will need to bump the ListOffsets API version to do this to ensure that requests made to earlier brokers that do not understand this specification are failed. This would be enforced in ListOffsetsRequest.Builder with something similar to the below (where 3 is the new version):
...
- Create a new API to retrieve the maximum timestamp record information, e.g. AdminClient.getMaxTimestampAndOffset(TopicPartition topicpartition). Rejected as AdminClient.listOffsets is already a good match for this functionality.
- Extend FetchedTimestampAndOffset to include a timestamp type - timestamp type (log append time or creation time) is a useful piece of information to return in any offset fetching method. It was considered to enrich the listOffsets API with this information as part of this KIP. However, this would involve significant changes to the existing API and for all but some corner cases (where the timestamp type used for the topic changes over time) the timestamp type can be fetched using the AdminClient (describeConfigs) at the topic level.