public interface ProcessorContext
Modifier and Type | Method and Description |
---|---|
Map<String,Object> |
appConfigs()
Return all the application config properties as key/value pairs.
|
Map<String,Object> |
appConfigsWithPrefix(String prefix)
Return all the application config properties with the given key prefix, as key/value pairs
stripping the prefix.
|
String |
applicationId()
Return the application id.
|
void |
commit()
Request a commit.
|
long |
currentStreamTimeMs()
Return the current stream-time in milliseconds.
|
long |
currentSystemTimeMs()
Return the current system timestamp (also called wall-clock time) in milliseconds.
|
<K,V> void |
forward(K key,
V value)
Forward a key/value pair to all downstream processors.
|
<K,V> void |
forward(K key,
V value,
To to)
Forward a key/value pair to the specified downstream processors.
|
<S extends StateStore> |
getStateStore(String name)
Get the state store given the store name.
|
org.apache.kafka.common.header.Headers |
headers()
Return the headers of the current input record; could be an empty header if it is not
available.
|
org.apache.kafka.common.serialization.Serde<?> |
keySerde()
Return the default key serde.
|
StreamsMetrics |
metrics()
Return Metrics instance.
|
long |
offset()
Return the offset of the current input record; could be
-1 if it is not
available. |
int |
partition()
Return the partition id of the current input record; could be
-1 if it is not
available. |
void |
register(StateStore store,
StateRestoreCallback stateRestoreCallback)
Register and possibly restores the specified storage engine.
|
Cancellable |
schedule(Duration interval,
PunctuationType type,
Punctuator callback)
Schedule a periodic operation for processors.
|
File |
stateDir()
Return the state directory for the partition.
|
TaskId |
taskId()
Return the task id.
|
long |
timestamp()
Return the current timestamp.
|
String |
topic()
Return the topic name of the current input record; could be
null if it is not
available. |
org.apache.kafka.common.serialization.Serde<?> |
valueSerde()
Return the default value serde.
|
String applicationId()
TaskId taskId()
org.apache.kafka.common.serialization.Serde<?> keySerde()
org.apache.kafka.common.serialization.Serde<?> valueSerde()
File stateDir()
StreamsMetrics metrics()
void register(StateStore store, StateRestoreCallback stateRestoreCallback)
store
- the storage enginestateRestoreCallback
- the restoration callback logic for log-backed state stores upon restartIllegalStateException
- If store gets registered after initialized is already finishedStreamsException
- if the store's change log does not contain the partition<S extends StateStore> S getStateStore(String name)
S
- The type or interface of the store to returnname
- The store nameClassCastException
- if the return type isn't a type or interface of the actual returned store.Cancellable schedule(Duration interval, PunctuationType type, Punctuator callback)
initialization
or
processing
to
schedule a periodic callback — called a punctuation — to Punctuator.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 the TimestampExtractor
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 completePunctuationType.STREAM_TIME
, when stream time advances more than intervalPunctuationType.WALL_CLOCK_TIME
, on GC pause, too short interval, ...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 timeIllegalArgumentException
- if the interval is not representable in milliseconds<K,V> void forward(K key, V value)
If this method is called with Punctuator.punctuate(long)
the record that
is sent downstream won't have any associated record metadata like topic, partition, or offset.
key
- keyvalue
- value<K,V> void forward(K key, V value, To to)
If this method is called with Punctuator.punctuate(long)
the record that
is sent downstream won't have any associated record metadata like topic, partition, or offset.
key
- keyvalue
- valueto
- the options to use when forwardingvoid commit()
String topic()
null
if it is not
available.
For example, if this method is invoked within a punctuation callback
, or while processing a record that was forwarded by a punctuation
callback, the record won't have an associated topic.
Another example is
KTable.transformValues(ValueTransformerWithKeySupplier, String...)
(and siblings), that do not always guarantee to provide a valid topic name, as they might be
executed "out-of-band" due to some internal optimizations applied by the Kafka Streams DSL.
int partition()
-1
if it is not
available.
For example, if this method is invoked within a punctuation callback
, or while processing a record that was forwarded by a punctuation
callback, the record won't have an associated partition id.
Another example is
KTable.transformValues(ValueTransformerWithKeySupplier, String...)
(and siblings), that do not always guarantee to provide a valid partition id, as they might be
executed "out-of-band" due to some internal optimizations applied by the Kafka Streams DSL.
long offset()
-1
if it is not
available.
For example, if this method is invoked within a punctuation callback
, or while processing a record that was forwarded by a punctuation
callback, the record won't have an associated offset.
Another example is
KTable.transformValues(ValueTransformerWithKeySupplier, String...)
(and siblings), that do not always guarantee to provide a valid offset, as they might be
executed "out-of-band" due to some internal optimizations applied by the Kafka Streams DSL.
org.apache.kafka.common.header.Headers headers()
For example, if this method is invoked within a punctuation callback
, or while processing a record that was forwarded by a punctuation
callback, the record might not have any associated headers.
Another example is
KTable.transformValues(ValueTransformerWithKeySupplier, String...)
(and siblings), that do not always guarantee to provide valid headers, as they might be
executed "out-of-band" due to some internal optimizations applied by the Kafka Streams DSL.
long 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
by TimestampExtractor
.
Note, that an upstream Processor
might have set a new timestamp by calling
forward(..., To.all().withTimestamp(...))
.
In particular, some Kafka Streams DSL operators set result record timestamps explicitly,
to guarantee deterministic results.
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.
Map<String,Object> appConfigs()
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 of DEFAULT_KEY_SERDE_CLASS_CONFIG
will be of type Class
, even if it was specified as a String to
StreamsConfig(Map)
).
Map<String,Object> appConfigsWithPrefix(String prefix)
The config properties are defined in the StreamsConfig
object and associated to the ProcessorContext.
prefix
- the properties prefixlong currentSystemTimeMs()
Note: this method returns the internally cached system timestamp from the Kafka Stream runtime.
Thus, it may return a different value compared to System.currentTimeMillis()
.
long currentStreamTimeMs()
Stream-time is the maximum observed record timestamp
so far
(including the currently processed record), i.e., it can be considered a high-watermark.
Stream-time is tracked on a per-task basis and is preserved across restarts and during task migration.
Note: this method is not supported for global processors (cf. Topology.addGlobalStore(org.apache.kafka.streams.state.StoreBuilder<?>, java.lang.String, org.apache.kafka.common.serialization.Deserializer<K>, org.apache.kafka.common.serialization.Deserializer<V>, java.lang.String, java.lang.String, org.apache.kafka.streams.processor.ProcessorSupplier<K, V>)
(...)
and StreamsBuilder.addGlobalStore(org.apache.kafka.streams.state.StoreBuilder<?>, java.lang.String, org.apache.kafka.streams.kstream.Consumed<K, V>, org.apache.kafka.streams.processor.ProcessorSupplier<K, V>)
(...),
because there is no concept of stream-time for this case.
Calling this method in a global processor will result in an UnsupportedOperationException
.