Interface ProcessorContext<KForward,VForward>
- Type Parameters:
KForward
- a bound on the types of keys that may be forwardedVForward
- a bound on the types of values that may be forwarded
- All Superinterfaces:
ProcessingContext
- All Known Implementing Classes:
MockProcessorContext
Record
.-
Method Summary
Methods inherited from interface org.apache.kafka.streams.processor.api.ProcessingContext
appConfigs, appConfigsWithPrefix, applicationId, commit, currentStreamTimeMs, currentSystemTimeMs, getStateStore, keySerde, metrics, recordMetadata, schedule, stateDir, taskId, valueSerde
-
Method Details
-
forward
Forward a record to all child processors.Note that the forwarded
Record
is shared between the parent and child processors. And of course, the parent may forward the same object to multiple children, and the child may forward it to grandchildren, etc. Therefore, you should be mindful of mutability.The
Record
class itself is immutable (all the setter-style methods return an independent copy of the instance). However, the key, value, and headers referenced by the Record may themselves be mutable.Some programs may opt to make use of this mutability for high performance, in which case the input record may be mutated and then forwarded by each
Processor
. However, most applications should instead favor safety.Forwarding records safely simply means to make a copy of the record before you mutate it. This is trivial when using the
Record.withKey(Object)
,Record.withValue(Object)
, andRecord.withTimestamp(long)
methods, as each of these methods make a copy of the record as a matter of course. But a little extra care must be taken with headers, since theHeader
class is mutable. The easiest way to safely handle headers is to use theRecord
constructors to make a copy before modifying headers.In other words, this would be considered unsafe:
process(Record inputRecord) { inputRecord.headers().add(...); context.forward(inputRecord); }
This is unsafe because the parent, and potentially siblings, grandparents, etc., all will see this modification to their shared Headers reference. This is a violation of causality and could lead to undefined behavior.A safe usage would look like this:
process(Record inputRecord) { // makes a copy of the headers Record toForward = inputRecord.withHeaders(inputRecord.headers()); // Other options to create a safe copy are: // * use any copy-on-write method, which makes a copy of all fields: // toForward = inputRecord.withValue(); // * explicitly copy all fields: // toForward = new Record(inputRecord.key(), inputRecord.value(), inputRecord.timestamp(), inputRecord.headers()); // * create a fresh, empty Headers: // toForward = new Record(inputRecord.key(), inputRecord.value(), inputRecord.timestamp()); // * etc. // now, we are modifying our own independent copy of the headers. toForward.headers().add(...); context.forward(toForward); }
- Parameters:
record
- The record to forward to all children
-
forward
Forward a record to the specified child processor. Seeforward(Record)
for considerations.- Parameters:
record
- The record to forwardchildName
- The name of the child processor to receive the record- See Also:
-