Table of Contents |
---|
Status
Current state: "Under Discussion"Implemented and merged
Discussion thread: here [Change the link from the KIP proposal email archive to your own email thread]
JIRA: here
Please keep the discussion on the mailing list rather than commenting on the wiki (wiki discussions get unwieldy fast).
Motivation
KIP-478, a the new strongly-typed Processor API, brings the option to reconsider the abstractions for around custom processing in the Kafka Streams DSL.
Currently, multiple Transformers
and a final Processor
operation are the options available to implement custom processing with access to the Record context; including record metadata (incle.g. topic, partition, offset), timestamp, and headers; and reference to state stores.
There have been discussions on how to refactor these APIs to minimize/simplify into a single processor:
- https://issues.apache.org/jira/browse/KAFKA-8396
- https://issues.apache.org/jira/browse/KAFKA-8410
- https://issues.apache.org/jira/browse/KAFKA-10603
Transformers
are currently limited to the old ProcessorContext
, and there have been multiple extensions to support value processing without repartitioningand one-to-many record processing. With the addition of the new Processor
API, KStream
can be extended to offer access to the newnewer, chainable API that can typed API and effectively replace most of the operations transformers *Transforms
offer at the moment, with more open control to forward records.
This KIP could be considered as a step towards deprecating Transforms
, though that should be discussed in a follow-up KIP.
Public Interfaces
...
Modified methods
KStream<KOut,VOut> KStream#process(ProcessorSupplier<K, V, KOut, VOut> processorSupplier, String... stateStoreNames)
from void KStream#process(
ProcessorSupplier<K, V, Void, Void> processorSupplier
, ...)
KStream<KOut,VOut> KStream#process(ProcessorSupplier<K, V, KOut, VOut> processorSupplier, Named named, String... stateStoreNames)
from void KStream#process(
ProcessorSupplier<K, V, Void, Void> processorSupplier
, ...)
New methods
Processors without forcing repartitioning:
KStream<K,VOut> KStream#processValues(FixedKeyProcessorSupplier<K, V, K, VOut> processorSupplier, String... stateStoreNames)
KStream<K,VOut> KStream#processValues(ProcessorSupplier<KFixedKeyProcessorSupplier<K, V, K, VOut> processorSupplier, Named named, String... stateStoreNames)
Internal changes
- Infrastructure for Fixed Key Records:
FixedKeyRecord
:- Not a
Record
sub/superclass to casting toRecord
. - Private constructor to avoid reconstructing record and change key.
- Not a
FixedKeyProcessor(Supplier)
andFixedKeyProcessorContext
interfaces forFixedKeyRecord
.FixedKeyContextualProcessor
abstract class, similar toContextualProcessor
.
Deprecated methods
All transform operations on the KStream will be deprecated in favor of process
and processValues
operations:
KStream#transform
KStream#flatTransform
KStream#transformValues
KStream#flatTransformValues
Proposed Changes
...
KStream#process
and
...
KStream#processValues
...
to replacing most Transformer
s
With the ability to manage forward
calls as part of the Processor
itself, transform
, flatTransform
valueTransform
, flatTransform
, and flatValueTransform
can be replaced by a Processor
process
/processValues
:
Code Block | ||
---|---|---|
| ||
private Topology topology() { final var builder = new StreamsBuilder(); final var input = builder.stream("words", Consumed.with(Serdes.String(), Serdes.String())); input.processValues( .processValues(() -> new FixedKeyContextualProcessor<String, String, String>() { @Override public void process(FixedKeyRecord<String, String> record) { for (final var word : record.value().split(",")) { context().forward(record.withValue("Hello " + word)); } }}, Named.as("process-values-without-repartitioning")) .process(() -> new ContextualProcessor<String, String, String, String>() { @Override public void process(Record<String, String> record) { for (final var sword : record.value().split(",")) { context().forward(record.withKey(word).withValue("Hello " + sword)); } } }}, Named.as("testprocess-with-partitioning")) .to("output", Produced.with(Serdes.String(), Serdes.String())); return builder.build(); } |
New ValueProcessorContext
to enforce only Value changes
Processors that require enforcing the same key to avoid repartitioning at the moment trust internal implementations. As a result the KStream API added *Values
methods to signal when key is not changed, and repartitioning can be avoided.
At the moment record keys can be changed or records recreated between the record entering Processor#process(Record)
and ProcessorContext#forward(Record)
.
As an alternative to enforce the same key, ProcessorContext wrapper could be implemented to define an initial key:
Infrastructure for Fixed Key Records
FixedKeyRecord
Record with immutable key.
Code Block | ||
---|---|---|
public final class FixedKeyRecord<K, V> {
| ||
Code Block | ||
| ||
public class ValueProcessorContext<KForward, VForward> implements ProcessorContext<KForward, VForward> { //... private final KForwardK key; private final V value; public void setRecordKey(KForward initialKey)private { final long timestamp; private final this.key = initialKey;Headers headers; } FixedKeyRecord(final K key, final V value, final long timestamp, publicfinal voidHeaders clearRecordKey(headers) { this.key = nullkey; } @Override this.value = value; public <K extends KForward, V extends VForward> voidif forward(Record<K,timestamp V>< record0) { throw ifnew StreamsException(key != null) { if (!record.key().equals(key)) { "Malformed Record", throw new IllegalArgumentException("KeyTimestamp hasmay changednot whilebe processingnegative. theGot: record" and requires enforcing repartitioning.");+ timestamp) }); } delegate.forward(record)this.timestamp = timestamp; } //... } |
This ValueProcessorContext
could be used internally at the Processor
level, and the method setRecordKey(K)
must be used before calling user-defined Processor
.
Code Block |
---|
public class KStreamValueProcessorSupplier<KIn, VIn, VOut> implements ProcessorSupplier<KIn, VIn, KIn, VOut> { // get ... this.headers = new RecordHeaders(headers); } public K key() { return key; } public static class KStreamValueProcessor<KIn, VIn, VOut> extends ContextualProcessor<KIn, VIn, KIn, VOut> { V value() { return value; } public long timestamp() { return timestamp; } public Headers headers() private final Processor<KIn, VIn, KIn, VOut> processor; private ValueProcessorContext<KIn, VOut> processorContext; public KStreamValueProcessor(Processor<KIn, VIn, KIn, VOut> transformer) {{ return headers; } public <NewV> FixedKeyRecord<K, NewV> withValue(final NewV value) { return new FixedKeyRecord<>(key, value, timestamp, headers); } public FixedKeyRecord<K, V> withTimestamp(final long timestamp) { return new FixedKeyRecord<>(key, value, timestamp, headers); } public FixedKeyRecord<K, V> withHeaders(final Headers headers) { return new this.processor = transformer; } FixedKeyRecord<>(key, value, timestamp, headers); } } |
FixedKeyProcessorSupplier
Code Block |
---|
@FunctionalInterface public interface FixedKeyProcessorSupplier<KIn, VIn, VOut> extends ConnectedStoreProvider, Supplier<FixedKeyProcessor<KIn, VIn, VOut>> { FixedKeyProcessor<KIn, VIn, VOut> @Override get(); } |
FixedKeyProcessor
Code Block |
---|
public interface FixedKeyProcessor<KIn, VIn, VOut> { publicdefault void init(final ProcessorContext<KInFixedKeyProcessorContext<KIn, VOut> context) {} void process(FixedKeyRecord<KIn, super.init(contextVIn> record); default void close() {} } |
FixedKeyContextualProcessor
Helper, same as ContextualProcessor
.
Code Block |
---|
public abstract class FixedKeyContextualProcessor<KIn, VIn, this.processorContextVOut> =implements new ValueProcessorContext<>(context); FixedKeyProcessor<KIn, VIn, VOut> { private FixedKeyProcessorContext<KIn, processor.init(processorContext);VOut> context; protected FixedKeyContextualProcessor() {} @Override public void processinit(final Record<KInFixedKeyProcessorContext<KIn, VIn>VOut> recordcontext) { this.context processorContext.setRecordKey(record.key())= context; } protected final FixedKeyProcessorContext<KIn, VOut> processor.processcontext(record); { processorContext.clearRecordKey(); return context; } } } |
ProcessingContext
To be extended by FixedKeyProcessorContext
and ProcessorContext
Based on the previous example, if key
is changed then processing will fail :
Code Block |
---|
interface ProcessingContext { private TopologyString topologyapplicationId(); { final var builder = new StreamsBuilderTaskId taskId(); Optional<RecordMetadata> recordMetadata(); final var input = builder.stream("words", Consumed.with(Serdes.String(), Serdes.String())); input.processValues(() -> new ContextualProcessor<String, String, String, String>() { @Override Serde<?> keySerde(); Serde<?> valueSerde(); File stateDir(); StreamsMetrics metrics(); <S extends StateStore> S getStateStore(final String name); Cancellable schedule(final Duration interval, public void process(Record<String, String> record) { for (var sfinal : record.value().split(",")) {PunctuationType type, context().forward(record.withKey("FAIL").withValue("Hello " + s)); } final Punctuator }callback); }, Named.as("test"))void commit(); .to("output"Map<String, Produced.with(Serdes.String(), Serdes.String())); return builder.build(); } Object> appConfigsWithPrefix(final String prefix); } |
FixedKeyProcessorContext
Code Block |
---|
public interface FixedKeyProcessorContext<KForward, VForward> extends ProcessingContext {
<K extends KForward, V extends VForward> void forward(FixedKeyRecord<K, V> record);
<K extends KForward, V extends VForward> void forward(FixedKeyRecord<K, V> record, final String childName);
} |
Code Block |
12:08:19.125 [new-process-c2e58cba-3c47-41a9-b221-adea4cd38c87-StreamThread-1] ERROR org.apache.kafka.streams.processor.internals.TaskManager - stream-thread [new-process-c2e58cba-3c47-41a9-b221-adea4cd38c87-StreamThread-1] Failed to process stream task 0_0 due to the following error:
org.apache.kafka.streams.errors.StreamsException: Exception caught in process. taskId=0_0, processor=KSTREAM-SOURCE-0000000000, topic=words, partition=0, offset=6, stacktrace=java.lang.IllegalArgumentException: Key has changed while processing the record and requires enforcing repartitioning. |
Compatibility, Deprecation, and Migration Plan
Modified method KStream#process
should be compatible with previous version, that at the moment is fixed to a Void
return type.
Leaving Transform
API with older ProcessorContext
could be considered a driver to adopt the methods proposed in this KIP.
KStreams#process
Return type change
Changing return type from void
to KStream<KOut, VOut>
is source-compatible, but not binary-compatible. It will require users to recompile the application to use the latest version of the library.
Though users will not require to change their code as the current returned value is void, and the input processor supplier types will include Void
that is the current type for output key and value.
KStream#*transform*
deprecations
This KIP is including the deprecation of the transform
operations on KStream
to propose using the latest Processor API operations.
Transform
API is not marked as deprecated yet, as it requires additional considerationsEven though, the new Processor
API could potentially replace most Transformer
implementations, it is not yet possible to deprecate Transformer
APIs:
Transform
API is broadly adopted for custom processing. Even though most functionality is possible to implement with newProcessor
, this migration is not straightforward.KTable#transformValues
is an interesting use-case forValueTransformerWithKey
as the internal value isChange<V>
— an internal type to handle record changes on KTable — not exposed as public API. An approach would be to deprecate this method in favor of.toStream().processValues().toTable()
.
A new KIP should be proposed to continue the deprecation of Transformer
APIs.
Rejected Alternatives
...
Migrate Transform APIs
...
to the latest ProcessContext
This would involve extending KStream/KTable APIs even more with specific Transform variants. If a new Processor is able to can support most Transform
in the long-term becoming the adopted option to transform values, flat transform, etc.; then we should start getting the new Processor
properly adopted in the DSL and let the usage drive the next steps.
Runtime key validation on KStream#processValues
and changing key output type to Void
On one hand, with Void type for key output, we force the users to cast to Void and change the key to null, though this can be documented on the API, so the users are aware of the peculiarity of forwarding within processValues
.
On the other hand, keeping the key type as output doesn't require to do any change of keys, but this could lead to key-checking runtime exceptions when changing keys.
This KIP proposes adding a new Record type for Fixed Keys, to remove the need for runtime checked and reduce as much as possible the ability to change the key when implementing the processor.
References
- Cleanup old Processor work: https://issues.apache.org/jira/browse/KAFKA-12939
- Reference implementation: https://github.com/jeqo/kafka/tree/kstream-new-process
- Draft implementation of fixed record key: https://github.com/apache/kafka/pull/11854