Table of Contents |
---|
Status
Current state: Under discussionAccepted
Discussion thread: here
Vote thread: https://lists.apache.org/thread/4krwr1p9h71t3qs3kpjr5j1gnomcsn63
JIRA: here
Please keep the discussion on the mailing list rather than commenting on the wiki (wiki discussions get unwieldy fast).
...
State stores access processor context via the StateStoreContext interface. Today, the AbstractProcessorContext class which implements/extends the StateStore interface StateStoreContext already makes record metadata available via recordMetadata(). Unfortunately that method is not part of the StateStoreContext interface, thus, state stores cannot access it.
...
The change of the StateStoreContext interface does not require any changes to any of the implementations of the interface, it merely exposes a method (recordMetadata) that is already implemented in AbstractProcessorContext. Therefore, we do not expect any compatibility issues. Moreover, recordMetadata returns an object of type RecordMetadata which is read-only, thus, protecting Kafka-internals from being tampered by applications.
Rejected Alternatives
Add a new Put
...
method to the KeyValueStore interface
The idea would be to couple data and metadata together in a single Put call.
...
Code Block |
---|
public interface KeyValueStore<K, V> extends StateStore, ReadOnlyKeyValueStore<K, V> { ... void UpdatePositionupdatePosition(String topic, int partition, int offset); ... } |
...