Interface ProcessorContext
-
- All Known Implementing Classes:
MockProcessorContext
public interface ProcessorContext
Processor context interface.
-
-
Method Summary
All Methods Instance Methods Abstract Methods Deprecated Methods Modifier and Type Method Description Map<String,Object>
appConfigs()
Returns all the application config properties as key/value pairs.Map<String,Object>
appConfigsWithPrefix(String prefix)
Returns all the application config properties with the given key prefix, as key/value pairs stripping the prefix.String
applicationId()
Returns the application id.void
commit()
Requests a commit.<K,V>
voidforward(K key, V value)
Forwards a key/value pair to all downstream processors.<K,V>
voidforward(K key, V value, int childIndex)
Deprecated.please useforward(Object, Object, To)
instead<K,V>
voidforward(K key, V value, String childName)
Deprecated.please useforward(Object, Object, To)
instead<K,V>
voidforward(K key, V value, To to)
Forwards a key/value pair to the specified downstream processors.<S extends StateStore>
SgetStateStore(String name)
Get the state store given the store name.Headers
headers()
Returns the headers of the current input record; could be null if it is not available (for example, if this method is invoked from the punctuate call).Serde<?>
keySerde()
Returns the default key serde.StreamsMetrics
metrics()
Returns Metrics instance.long
offset()
Returns the offset of the current input record; could be -1 if it is not available (for example, if this method is invoked from the punctuate call).int
partition()
Returns the partition id of the current input record; could be -1 if it is not available (for example, if this method is invoked from the punctuate call).void
register(StateStore store, StateRestoreCallback stateRestoreCallback)
Registers and possibly restores the specified storage engine.Cancellable
schedule(long intervalMs, PunctuationType type, Punctuator callback)
Deprecated.Useschedule(Duration, PunctuationType, Punctuator)
insteadCancellable
schedule(Duration interval, PunctuationType type, Punctuator callback)
Schedules a periodic operation for processors.File
stateDir()
Returns the state directory for the partition.TaskId
taskId()
Returns the task id.long
timestamp()
Returns the current timestamp.String
topic()
Returns the topic name of the current input record; could be null if it is not available (for example, if this method is invoked from the punctuate call).Serde<?>
valueSerde()
Returns the default value serde.
-
-
-
Method Detail
-
applicationId
String applicationId()
Returns the application id.- Returns:
- the application id
-
taskId
TaskId taskId()
Returns the task id.- Returns:
- the task id
-
keySerde
Serde<?> keySerde()
Returns the default key serde.- Returns:
- the key serializer
-
valueSerde
Serde<?> valueSerde()
Returns the default value serde.- Returns:
- the value serializer
-
stateDir
File stateDir()
Returns the state directory for the partition.- Returns:
- the state directory
-
metrics
StreamsMetrics metrics()
Returns Metrics instance.- Returns:
- StreamsMetrics
-
register
void register(StateStore store, StateRestoreCallback stateRestoreCallback)
Registers and possibly restores the specified storage engine.- Parameters:
store
- the storage enginestateRestoreCallback
- the restoration callback logic for log-backed state stores upon restart- Throws:
IllegalStateException
- If store gets registered after initialized is already finishedStreamsException
- if the store's change log does not contain the partition
-
getStateStore
<S extends StateStore> S getStateStore(String name)
Get the state store given the store name.- Type Parameters:
S
- The type or interface of the store to return- Parameters:
name
- The store name- Returns:
- The state store instance
- Throws:
ClassCastException
- if the return type isn't a type or interface of the actual returned store.
-
schedule
@Deprecated Cancellable schedule(long intervalMs, PunctuationType type, Punctuator callback)
Deprecated.Useschedule(Duration, PunctuationType, Punctuator)
insteadSchedules a periodic operation for processors. A processor may call this method duringinitialization
orprocessing
to schedule a periodic callback — called a punctuation — toPunctuator.punctuate(long)
. The type parameter controls what notion of time is used for punctuation:PunctuationType.STREAM_TIME
— uses "stream time", which is advanced by the processing of messages in accordance with the timestamp as extracted by theTimestampExtractor
in use. The first punctuation will be triggered by the first record that is processed. NOTE: Only advanced if messages arrivePunctuationType.WALL_CLOCK_TIME
— uses system time (the wall-clock time), which is advanced independent of whether new messages arrive. The first punctuation will be triggered after interval has elapsed. NOTE: This is best effort only as its granularity is limited by how long an iteration of the processing loop takes to complete
- with
PunctuationType.STREAM_TIME
, when stream time advances more than interval - with
PunctuationType.WALL_CLOCK_TIME
, on GC pause, too short interval, ...
- Parameters:
intervalMs
- the time interval between punctuations in millisecondstype
- one of:PunctuationType.STREAM_TIME
,PunctuationType.WALL_CLOCK_TIME
callback
- a function consuming timestamps representing the current stream or system time- Returns:
- a handle allowing cancellation of the punctuation schedule established by this method
-
schedule
Cancellable schedule(Duration interval, PunctuationType type, Punctuator callback)
Schedules a periodic operation for processors. A processor may call this method duringinitialization
orprocessing
to schedule a periodic callback — called a punctuation — toPunctuator.punctuate(long)
. The type parameter controls what notion of time is used for punctuation:PunctuationType.STREAM_TIME
— uses "stream time", which is advanced by the processing of messages in accordance with the timestamp as extracted by theTimestampExtractor
in use. The first punctuation will be triggered by the first record that is processed. NOTE: Only advanced if messages arrivePunctuationType.WALL_CLOCK_TIME
— uses system time (the wall-clock time), which is advanced independent of whether new messages arrive. The first punctuation will be triggered after interval has elapsed. NOTE: This is best effort only as its granularity is limited by how long an iteration of the processing loop takes to complete
- with
PunctuationType.STREAM_TIME
, when stream time advances more than interval - with
PunctuationType.WALL_CLOCK_TIME
, on GC pause, too short interval, ...
- Parameters:
interval
- the time interval between punctuations (supported minimum is 1 millisecond)type
- one of:PunctuationType.STREAM_TIME
,PunctuationType.WALL_CLOCK_TIME
callback
- a function consuming timestamps representing the current stream or system time- Returns:
- a handle allowing cancellation of the punctuation schedule established by this method
- Throws:
IllegalArgumentException
- if the interval is not representable in milliseconds
-
forward
<K,V> void forward(K key, V value)
Forwards a key/value pair to all downstream processors. Used the input record's timestamp as timestamp for the output record.- Parameters:
key
- keyvalue
- value
-
forward
<K,V> void forward(K key, V value, To to)
Forwards a key/value pair to the specified downstream processors. Can be used to set the timestamp of the output record.- Parameters:
key
- keyvalue
- valueto
- the options to use when forwarding
-
forward
@Deprecated <K,V> void forward(K key, V value, int childIndex)
Deprecated.please useforward(Object, Object, To)
insteadForwards a key/value pair to one of the downstream processors designated by childIndex.- Parameters:
key
- keyvalue
- valuechildIndex
- index in list of children of this node
-
forward
@Deprecated <K,V> void forward(K key, V value, String childName)
Deprecated.please useforward(Object, Object, To)
insteadForwards a key/value pair to one of the downstream processors designated by the downstream processor name.- Parameters:
key
- keyvalue
- valuechildName
- name of downstream processor
-
commit
void commit()
Requests a commit.
-
topic
String topic()
Returns the topic name of the current input record; could be null if it is not available (for example, if this method is invoked from the punctuate call).- Returns:
- the topic name
-
partition
int partition()
Returns the partition id of the current input record; could be -1 if it is not available (for example, if this method is invoked from the punctuate call).- Returns:
- the partition id
-
offset
long offset()
Returns the offset of the current input record; could be -1 if it is not available (for example, if this method is invoked from the punctuate call).- Returns:
- the offset
-
headers
Headers headers()
Returns the headers of the current input record; could be null if it is not available (for example, if this method is invoked from the punctuate call).- Returns:
- the headers
-
timestamp
long timestamp()
Returns the current timestamp.If it is triggered while processing a record streamed from the source processor, timestamp is defined as the timestamp of the current input record; the timestamp is extracted from
ConsumerRecord
byTimestampExtractor
.If it is triggered while processing a record generated not from the source processor (for example, if this method is invoked from the punctuate call), timestamp is defined as the current task's stream time, which is defined as the largest timestamp of any record processed by the task.
- Returns:
- the timestamp
-
appConfigs
Map<String,Object> appConfigs()
Returns all the application config properties as key/value pairs.The config properties are defined in the
StreamsConfig
object and associated to the ProcessorContext.The type of the values is dependent on the
type
of the property (e.g. the value ofDEFAULT_KEY_SERDE_CLASS_CONFIG
will be of typeClass
, even if it was specified as a String toStreamsConfig(Map)
).- Returns:
- all the key/values from the StreamsConfig properties
-
appConfigsWithPrefix
Map<String,Object> appConfigsWithPrefix(String prefix)
Returns all the application config properties with the given key prefix, as key/value pairs stripping the prefix.The config properties are defined in the
StreamsConfig
object and associated to the ProcessorContext.- Parameters:
prefix
- the properties prefix- Returns:
- the key/values matching the given prefix from the StreamsConfig properties.
-
-