Class MockProcessorContext<KForward,VForward>
- java.lang.Object
-
- org.apache.kafka.streams.processor.api.MockProcessorContext<KForward,VForward>
-
- All Implemented Interfaces:
ProcessorContext<KForward,VForward>
,org.apache.kafka.streams.processor.internals.RecordCollector.Supplier
public class MockProcessorContext<KForward,VForward> extends Object implements ProcessorContext<KForward,VForward>, org.apache.kafka.streams.processor.internals.RecordCollector.Supplier
MockProcessorContext
is a mock ofProcessorContext
for users to test theirProcessor
,Transformer
, andValueTransformer
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-sinkTopology
and using theTopologyTestDriver
.
-
-
Nested Class Summary
Nested Classes Modifier and Type Class Description static class
MockProcessorContext.CapturedForward<K,V>
static class
MockProcessorContext.CapturedPunctuator
MockProcessorContext.CapturedPunctuator
holds captured punctuators, along with their scheduling information.
-
Constructor Summary
Constructors Constructor Description MockProcessorContext()
MockProcessorContext(Properties config)
MockProcessorContext(Properties config, TaskId taskId, File stateDir)
Create aMockProcessorContext
with a specified taskId and null stateDir.
-
Method Summary
All Methods Instance Methods Concrete Methods Modifier and Type Method Description <S extends StateStore>
voidaddStateStore(S stateStore)
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.boolean
committed()
WhetherProcessorContext.commit()
has been called in this context.<K extends KForward,V extends VForward>
voidforward(Record<K,V> record)
Forwards a record to all child processors.<K extends KForward,V extends VForward>
voidforward(Record<K,V> record, String childName)
Forwards 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>
SgetStateStore(String name)
Get the state store given the store name.StateStoreContext
getStateStoreContext()
Used to get aStateStoreContext
for use withStateStore.init(StateStoreContext, StateStore)
if you need to initialize a store for your tests.Serde<?>
keySerde()
Returns the default key serde.StreamsMetrics
metrics()
Returns Metrics instance.org.apache.kafka.streams.processor.internals.RecordCollector
recordCollector()
Optional<RecordMetadata>
recordMetadata()
The metadata of the source record, if is one.void
resetCommit()
Reset the commit capture tofalse
(whether or not it was previouslytrue
).void
resetForwards()
Clear the captured forwarded data.Cancellable
schedule(Duration interval, PunctuationType type, Punctuator callback)
Schedules a periodic operation for processors.List<MockProcessorContext.CapturedPunctuator>
scheduledPunctuators()
Get the punctuators scheduled so far.void
setRecordMetadata(String topic, int partition, long offset)
The context exposes these metadata for use in the processor.File
stateDir()
Returns the state directory for the partition.TaskId
taskId()
Returns the task id.Serde<?>
valueSerde()
Returns the default value serde.
-
-
-
Constructor Detail
-
MockProcessorContext
public MockProcessorContext()
Create aMockProcessorContext
with dummyconfig
andtaskId
andnull
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 theInMemoryKeyValueStore
, so the stateDir won't matter.
-
MockProcessorContext
public MockProcessorContext(Properties config)
Create aMockProcessorContext
with dummytaskId
andnull
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 theInMemoryKeyValueStore
, so the stateDir won't matter.- Parameters:
config
- a Properties object, used to configure the context and the processor.
-
MockProcessorContext
public MockProcessorContext(Properties config, TaskId taskId, File stateDir)
Create aMockProcessorContext
with a specified taskId and null stateDir.- Parameters:
config
- aProperties
object, used to configure the context and the processor.taskId
- aTaskId
, which the context makes available viataskId()
.stateDir
- aFile
, which the context makes available viwstateDir()
.
-
-
Method Detail
-
applicationId
public String applicationId()
Description copied from interface:ProcessorContext
Returns the application id.- Specified by:
applicationId
in interfaceProcessorContext<KForward,VForward>
- Returns:
- the application id
-
taskId
public TaskId taskId()
Description copied from interface:ProcessorContext
Returns the task id.- Specified by:
taskId
in interfaceProcessorContext<KForward,VForward>
- Returns:
- the task id
-
appConfigs
public Map<String,Object> appConfigs()
Description copied from interface:ProcessorContext
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)
).- Specified by:
appConfigs
in interfaceProcessorContext<KForward,VForward>
- Returns:
- all the key/values from the StreamsConfig properties
-
appConfigsWithPrefix
public Map<String,Object> appConfigsWithPrefix(String prefix)
Description copied from interface:ProcessorContext
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.- Specified by:
appConfigsWithPrefix
in interfaceProcessorContext<KForward,VForward>
- Parameters:
prefix
- the properties prefix- Returns:
- the key/values matching the given prefix from the StreamsConfig properties.
-
keySerde
public Serde<?> keySerde()
Description copied from interface:ProcessorContext
Returns the default key serde.- Specified by:
keySerde
in interfaceProcessorContext<KForward,VForward>
- Returns:
- the key serializer
-
valueSerde
public Serde<?> valueSerde()
Description copied from interface:ProcessorContext
Returns the default value serde.- Specified by:
valueSerde
in interfaceProcessorContext<KForward,VForward>
- Returns:
- the value serializer
-
stateDir
public File stateDir()
Description copied from interface:ProcessorContext
Returns the state directory for the partition.- Specified by:
stateDir
in interfaceProcessorContext<KForward,VForward>
- Returns:
- the state directory
-
metrics
public StreamsMetrics metrics()
Description copied from interface:ProcessorContext
Returns Metrics instance.- Specified by:
metrics
in interfaceProcessorContext<KForward,VForward>
- Returns:
- StreamsMetrics
-
setRecordMetadata
public void setRecordMetadata(String topic, int partition, long offset)
The context exposes these metadata for use in the processor. Normally, they are set by the Kafka Streams framework, but for the purpose of driving unit tests, you can set them directly.- Parameters:
topic
- A topic namepartition
- A partition numberoffset
- A record offset
-
recordMetadata
public Optional<RecordMetadata> recordMetadata()
Description copied from interface:ProcessorContext
The metadata of the source record, if is one. Processors may be invoked to process a source record from an input topic, to run a scheduled punctuation (seeProcessorContext.schedule(Duration, PunctuationType, Punctuator)
), or because a parent processor calledProcessorContext.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
ProcessorContext.forward(Record)
, downstream processors will receive the forwarded record as a regularProcessor.process(Record)
invocation. In other words, it wouldn't be apparent to downstream processors whether or not the record being processed came from an input topic or punctuation and therefore whether or not this metadata is defined. This is why the return type of this method isOptional
.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.- Specified by:
recordMetadata
in interfaceProcessorContext<KForward,VForward>
-
getStateStore
public <S extends StateStore> S getStateStore(String name)
Description copied from interface:ProcessorContext
Get the state store given the store name.- Specified by:
getStateStore
in interfaceProcessorContext<KForward,VForward>
- Type Parameters:
S
- The type or interface of the store to return- Parameters:
name
- The store name- Returns:
- The state store instance
-
addStateStore
public <S extends StateStore> void addStateStore(S stateStore)
-
schedule
public Cancellable schedule(Duration interval, PunctuationType type, Punctuator callback)
Description copied from interface:ProcessorContext
Schedules a periodic operation for processors. A processor may call this method duringinitialization
orProcessor.process(Record)
processing} 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, ...
- Specified by:
schedule
in interfaceProcessorContext<KForward,VForward>
- 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
-
scheduledPunctuators
public List<MockProcessorContext.CapturedPunctuator> scheduledPunctuators()
Get the punctuators scheduled so far. The returned list is not affected by subsequent calls toschedule(...)
.- Returns:
- A list of captured punctuators.
-
forward
public <K extends KForward,V extends VForward> void forward(Record<K,V> record)
Description copied from interface:ProcessorContext
Forwards 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); }
- Specified by:
forward
in interfaceProcessorContext<KForward,VForward>
- Parameters:
record
- The record to forward to all children
-
forward
public <K extends KForward,V extends VForward> void forward(Record<K,V> record, String childName)
Description copied from interface:ProcessorContext
Forwards a record to the specified child processor. SeeProcessorContext.forward(Record)
for considerations.- Specified by:
forward
in interfaceProcessorContext<KForward,VForward>
- Parameters:
record
- The record to forwardchildName
- The name of the child processor to receive the record- See Also:
ProcessorContext.forward(Record)
-
forwarded
public List<MockProcessorContext.CapturedForward<? extends KForward,? extends VForward>> forwarded()
Get all the forwarded data this context has observed. The returned list will not be affected by subsequent interactions with the context. The data in the list is in the same order as the calls toforward(...)
.- Returns:
- A list of records that were previously passed to the context.
-
forwarded
public 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. The returned list will not be affected by subsequent interactions with the context. The data in the list is in the same order as the calls toforward(...)
.- Parameters:
childName
- The child name to retrieve forwards for- Returns:
- A list of records that were previously passed to the context.
-
resetForwards
public void resetForwards()
Clear the captured forwarded data.
-
commit
public void commit()
Description copied from interface:ProcessorContext
Requests a commit.- Specified by:
commit
in interfaceProcessorContext<KForward,VForward>
-
committed
public boolean committed()
WhetherProcessorContext.commit()
has been called in this context.- Returns:
true
iffProcessorContext.commit()
has been called in this context since construction or reset.
-
resetCommit
public void resetCommit()
Reset the commit capture tofalse
(whether or not it was previouslytrue
).
-
recordCollector
public org.apache.kafka.streams.processor.internals.RecordCollector recordCollector()
- Specified by:
recordCollector
in interfaceorg.apache.kafka.streams.processor.internals.RecordCollector.Supplier
-
getStateStoreContext
public StateStoreContext getStateStoreContext()
Used to get aStateStoreContext
for use withStateStore.init(StateStoreContext, StateStore)
if you need to initialize a store for your tests.- Returns:
- a
StateStoreContext
that delegates to this ProcessorContext.
-
-