Interface ValueTransformerWithKey<K,V,VR>
-
- Type Parameters:
K- key typeV- value typeVR- transformed value type
public interface ValueTransformerWithKey<K,V,VR>TheValueTransformerWithKeyinterface for stateful mapping of a value to a new value (with possible new type). This is a stateful record-by-record operation, i.e,transform(Object, Object)is invoked individually for each record of a stream and can access and modify a state that is available beyond a single call oftransform(Object, Object)(cf.ValueMapperfor stateless value transformation). Additionally, thisValueTransformerWithKeycanschedulea method to becalled periodicallywith the provided context. Note that the key is read-only and should not be modified, as this can lead to corrupt partitioning. IfValueTransformerWithKeyis applied to aKeyValuepair record the record's key is preserved.Use
ValueTransformerWithKeySupplierto provide new instances ofValueTransformerWithKeyto Kafka Stream's runtime.If a record's key and value should be modified
Transformercan be used.
-
-
Method Summary
All Methods Instance Methods Abstract Methods Modifier and Type Method Description voidclose()Close this processor and clean up any resources.voidinit(ProcessorContext context)Initialize this transformer.VRtransform(K readOnlyKey, V value)Transform the given [key and ]value to a new value.
-
-
-
Method Detail
-
init
void init(ProcessorContext context)
Initialize this transformer. This is called once per instance when the topology gets initialized.The provided
contextcan be used to access topology and record meta data, toschedulea method to becalled periodicallyand to access attachedStateStores.Note that
ProcessorContextis updated in the background with the current record's meta data. Thus, it only contains valid record meta data when accessed withintransform(Object, Object).Note that using
ProcessorContext.forward(Object, Object)orProcessorContext.forward(Object, Object, To)is not allowed within any method ofValueTransformerWithKeyand will result in anexception.- Parameters:
context- the context- Throws:
IllegalStateException- If store gets registered after initialization is already finishedStreamsException- if the store's change log does not contain the partition
-
transform
VR transform(K readOnlyKey, V value)
Transform the given [key and ]value to a new value. Additionally, anyStateStorethat isattachedto this operator can be accessed and modified arbitrarily (cf.ProcessorContext.getStateStore(String)).Note, that using
ProcessorContext.forward(Object, Object)orProcessorContext.forward(Object, Object, To)is not allowed withintransformand will result in anexception.- Parameters:
readOnlyKey- the read-only keyvalue- the value to be transformed- Returns:
- the new value
-
close
void close()
Close this processor and clean up any resources.It is not possible to return any new output records within
close(). UsingProcessorContext.forward(Object, Object)orProcessorContext.forward(Object, Object, To), will result in anexception.
-
-