You are viewing an old version of this page. View the current version.

Compare with Current View Page History

« Previous Version 22 Next »

Status

Current state: Under discussion

Discussion thread: here

JIRA: here

Please keep the discussion on the mailing list rather than commenting on the wiki (wiki discussions get unwieldy fast).

Motivation

In order for state stores to provide stronger consistency in the future (e.g., RYW consistency) they need to be able to collect record metadata (e.g., offset information).

Today, we already make record metadata available in the ProcessContext (::recordMetadata()), and it is present in the abstract class that implements both ProcessorContext and StateStoreContext, but the method is not currently exposed through the StateStoreContext interface, so it is not available in the state stores.

Public Interfaces

In this KIP we propose to expose recordMetada via the StateStoreContext. The following interface will change:

  • org.apache.kafka.stream.processor.StateStoreContext - adding method recordMetadata()

Proposed Changes

The code segment below shows the actual code change together with the corresponding JavaDoc:

public interface StateStoreContext {
    ...

    /**
     * Return the metadata of the current topic/partition/offset if available.
     * This metadata in a StateStore is defined as the metadata of the last record
     * that is currently been processed by the StreamTask that holds the store.
     * <p>
     * Note that the metadata is not defined during all store interactions, for
     * example, while the StreamTask is running a punctuation.
     */
      Optional<RecordMetadata> recordMetadata();

    ...
}

Note that recordMetadata() returns an Optional which account for the fact that there might not be any recordMetadata available if no records have been processed yet.

The following code snippet illustrates how the above interface can be used within the state store to keep track of the latest offset a store has processed:

public class RocksDBStore implements KeyValueStore<Bytes, byte[]>, BatchWritingStore {
     private StateStoreContext context;
     private Map<String, Map<Integer, Long>> seenOffsets = new HashMap<>();
	 ...

     public synchronized void put(final Bytes key,
        final byte[] value) {
        Objects.requireNonNull(key, "key cannot be null");
        validateStoreOpen();
        dbAccessor.put(key.get(), value);

        if (context != null && context.recordMetadata().isPresent()) {
            final RecordMetadata meta = context.recordMetadata().get();
            //update seen offsets
            seenOffsets.computeIfAbsent(meta.topic(), t -> new HashMap<>())
                .put(meta.partition(), meta.offset());
        }*/
    }

	...
}


Compatibility, Deprecation, and Migration Plan

The change of the StateStoreContext interface does not require any changes to any of the implementations of the interface, it merely exposes (recordMetadata) a method that is already implemented in AbstractProcessorContext. Therefore we do not expect any compatibility issues. Moreover, recordMetadata returns an object of type RecordMetadata which is exclusively read-only, thus, protecting Kafka-internal from being tampered by applications. 

Rejected Alternatives

None.

  • No labels