public class MockProcessorContext<KForward,VForward> extends Object implements ProcessorContext<KForward,VForward>, org.apache.kafka.streams.processor.internals.RecordCollector.Supplier
MockProcessorContext
is a mock of ProcessorContext
for users to test their Processor
,
Transformer
, and ValueTransformer
implementations.
The tests for this class (org.apache.kafka.streams.MockProcessorContextTest) include several behavioral tests that serve as example usage.
Note that this class does not take any automated actions (such as firing scheduled punctuators).
It simply captures any data it witnesses.
If you require more automated tests, we recommend wrapping your Processor
in a minimal source-processor-sink
Topology
and using the TopologyTestDriver
.
Modifier and Type | Class and Description |
---|---|
static class |
MockProcessorContext.CapturedForward<K,V> |
static class |
MockProcessorContext.CapturedPunctuator
MockProcessorContext.CapturedPunctuator holds captured punctuators, along with their scheduling information. |
Constructor and Description |
---|
MockProcessorContext()
|
MockProcessorContext(Properties config)
|
MockProcessorContext(Properties config,
TaskId taskId,
File stateDir)
Create a
MockProcessorContext with a specified taskId and null stateDir. |
Modifier and Type | Method and Description |
---|---|
<S extends StateStore> |
addStateStore(S stateStore) |
Map<String,Object> |
appConfigs()
Returns 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.
|
boolean |
committed()
Whether
ProcessingContext.commit() has been called in this context. |
long |
currentStreamTimeMs()
Return the current stream-time in milliseconds.
|
long |
currentSystemTimeMs()
Return the current system timestamp (also called wall-clock time) in milliseconds.
|
<K extends KForward,V extends VForward> |
forward(Record<K,V> record)
Forward a record to all child processors.
|
<K extends KForward,V extends VForward> |
forward(Record<K,V> record,
String childName)
Forward a record to the specified child processor.
|
List<MockProcessorContext.CapturedForward<? extends KForward,? extends VForward>> |
forwarded()
Get all the forwarded data this context has observed.
|
List<MockProcessorContext.CapturedForward<? extends KForward,? extends VForward>> |
forwarded(String childName)
Get all the forwarded data this context has observed for a specific child by name.
|
<S extends StateStore> |
getStateStore(String name)
Get the state store given the store name.
|
StateStoreContext |
getStateStoreContext()
Used to get a
StateStoreContext for use with
StateStore.init(StateStoreContext, StateStore)
if you need to initialize a store for your tests. |
Serde<?> |
keySerde()
Return the default key serde.
|
StreamsMetrics |
metrics()
Return Metrics instance.
|
org.apache.kafka.streams.processor.internals.RecordCollector |
recordCollector() |
Optional<RecordMetadata> |
recordMetadata()
Return the metadata of the current record if available.
|
void |
resetCommit()
Reset the commit capture to
false (whether or not it was previously true ). |
void |
resetForwards()
Clear the captured forwarded data.
|
Cancellable |
schedule(Duration interval,
PunctuationType type,
Punctuator callback)
Schedule a periodic operation for processors.
|
List<MockProcessorContext.CapturedPunctuator> |
scheduledPunctuators()
Get the punctuators scheduled so far.
|
void |
setCurrentStreamTimeMs(long currentStreamTimeMs) |
void |
setCurrentSystemTimeMs(long currentSystemTimeMs) |
void |
setRecordMetadata(String topic,
int partition,
long offset)
The context exposes these metadata for use in the processor.
|
File |
stateDir()
Return the state directory for the partition.
|
TaskId |
taskId()
Return the task id.
|
Serde<?> |
valueSerde()
Return the default value serde.
|
public MockProcessorContext()
MockProcessorContext
with dummy config
and taskId
and null
stateDir
.
Most unit tests using this mock won't need to know the taskId,
and most unit tests should be able to get by with the
InMemoryKeyValueStore
, so the stateDir won't matter.public MockProcessorContext(Properties config)
MockProcessorContext
with dummy taskId
and null
stateDir
.
Most unit tests using this mock won't need to know the taskId,
and most unit tests should be able to get by with the
InMemoryKeyValueStore
, so the stateDir won't matter.config
- a Properties object, used to configure the context and the processor.public MockProcessorContext(Properties config, TaskId taskId, File stateDir)
MockProcessorContext
with a specified taskId and null stateDir.config
- a Properties
object, used to configure the context and the processor.taskId
- a TaskId
, which the context makes available via taskId()
.stateDir
- a File
, which the context makes available viw stateDir()
.public String applicationId()
ProcessingContext
applicationId
in interface ProcessingContext
public TaskId taskId()
ProcessingContext
taskId
in interface ProcessingContext
public Map<String,Object> appConfigs()
ProcessingContext
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)
).
appConfigs
in interface ProcessingContext
public Map<String,Object> appConfigsWithPrefix(String prefix)
ProcessingContext
The config properties are defined in the StreamsConfig
object and associated to the ProcessorContext.
appConfigsWithPrefix
in interface ProcessingContext
prefix
- the properties prefixpublic long currentSystemTimeMs()
ProcessingContext
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()
.
currentSystemTimeMs
in interface ProcessingContext
public long currentStreamTimeMs()
ProcessingContext
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
.
currentStreamTimeMs
in interface ProcessingContext
public Serde<?> keySerde()
ProcessingContext
keySerde
in interface ProcessingContext
public Serde<?> valueSerde()
ProcessingContext
valueSerde
in interface ProcessingContext
public File stateDir()
ProcessingContext
stateDir
in interface ProcessingContext
public StreamsMetrics metrics()
ProcessingContext
metrics
in interface ProcessingContext
public void setRecordMetadata(String topic, int partition, long offset)
topic
- A topic namepartition
- A partition numberoffset
- A record offsetpublic void setCurrentSystemTimeMs(long currentSystemTimeMs)
public void setCurrentStreamTimeMs(long currentStreamTimeMs)
public Optional<RecordMetadata> recordMetadata()
ProcessingContext
ProcessingContext.schedule(Duration, PunctuationType, Punctuator)
),
or because a parent processor called forward(Record)
.
In the case of a punctuation, there is no source record, so this metadata would be
undefined. Note that when a punctuator invokes forward(Record)
,
downstream processors will receive the forwarded record as a regular
Processor.process(Record)
or FixedKeyProcessor.process(FixedKeyRecord)
invocation.
In other words, it wouldn't be apparent to
downstream processors whether the record being processed came from an input topic
or punctuation and therefore whether this metadata is defined. This is why
the return type of this method is Optional
.
If there is any possibility of punctuators upstream, any access
to this field should consider the case of
"recordMetadata().isPresent() == false
".
Of course, it would be safest to always guard this condition.
recordMetadata
in interface ProcessingContext
public <S extends StateStore> S getStateStore(String name)
ProcessingContext
getStateStore
in interface ProcessingContext
S
- The type or interface of the store to returnname
- The store namepublic <S extends StateStore> void addStateStore(S stateStore)
public Cancellable schedule(Duration interval, PunctuationType type, Punctuator callback)
ProcessingContext
initialization
,
processing
,
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, ...schedule
in interface ProcessingContext
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 timepublic List<MockProcessorContext.CapturedPunctuator> scheduledPunctuators()
schedule(...)
.public <K extends KForward,V extends VForward> void forward(Record<K,V> record)
ProcessorContext
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)
,
and Record.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
the Header
class is mutable. The easiest way to
safely handle headers is to use the Record
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);
}
forward
in interface ProcessorContext<KForward,VForward>
record
- The record to forward to all childrenpublic <K extends KForward,V extends VForward> void forward(Record<K,V> record, String childName)
ProcessorContext
ProcessorContext.forward(Record)
for considerations.forward
in interface ProcessorContext<KForward,VForward>
record
- The record to forwardchildName
- The name of the child processor to receive the recordProcessorContext.forward(Record)
public List<MockProcessorContext.CapturedForward<? extends KForward,? extends VForward>> forwarded()
forward(...)
.public List<MockProcessorContext.CapturedForward<? extends KForward,? extends VForward>> forwarded(String childName)
forward(...)
.childName
- The child name to retrieve forwards forpublic void resetForwards()
public void commit()
ProcessingContext
commit()
is only a request for a commit, but it does not execute one.
Hence, when commit()
returns, no commit was executed yet. However, Kafka Streams will commit as soon
as possible, instead of waiting for next commit.interval.ms
to pass.commit
in interface ProcessingContext
public boolean committed()
ProcessingContext.commit()
has been called in this context.true
iff ProcessingContext.commit()
has been called in this context since construction or reset.public void resetCommit()
false
(whether or not it was previously true
).public org.apache.kafka.streams.processor.internals.RecordCollector recordCollector()
recordCollector
in interface org.apache.kafka.streams.processor.internals.RecordCollector.Supplier
public StateStoreContext getStateStoreContext()
StateStoreContext
for use with
StateStore.init(StateStoreContext, StateStore)
if you need to initialize a store for your tests.StateStoreContext
that delegates to this ProcessorContext.