Versions Compared

Key

  • This line was added.
  • This line was removed.
  • Formatting was changed.

...

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):

public static Builder forMaxTimestamp(IsolationLevel isolationLevel) {
  return new Builder(3, ApiKeys.LIST_OFFSETS.latestVersion(), CONSUMER_REPLICA_ID, isolationLevel);
}

Proposed Changes

OffsetSpecs are mapped to Long values for timestamp fetches in org.apache.kafka.clients.admin.KafkaAdminClient:

...

The only behaviour changes created by this KIP are for requests to AdminClient.listOffsets that are looking for a timestamp of -3L. As this is not a valid timestamp there should be no migration, compatibility or deprecation concerns. As mentioned in public interfaces,  this functionality is not backwards compatible and attempt to fetch by max timestamp on earlier brokers will be failed.

Rejected Alternatives

  • 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.