Class MockProcessorContext
- java.lang.Object
-
- org.apache.kafka.streams.processor.MockProcessorContext
-
- All Implemented Interfaces:
org.apache.kafka.streams.processor.internals.RecordCollector.Supplier
,ProcessorContext
public class MockProcessorContext extends Object implements ProcessorContext, 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
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 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.boolean
committed()
WhetherProcessorContext.commit()
has been called in this context.<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.<K,V>
voidforward(K key, V value, String childName)
Deprecated.<K,V>
voidforward(K key, V value, To to)
Forwards a key/value pair to the specified downstream processors.List<MockProcessorContext.CapturedForward>
forwarded()
Get all the forwarded data this context has observed.List<MockProcessorContext.CapturedForward>
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.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).org.apache.kafka.streams.processor.internals.RecordCollector
recordCollector()
void
register(StateStore store, StateRestoreCallback stateRestoreCallbackIsIgnoredInMock)
Registers and possibly restores the specified storage engine.void
resetCommit()
Reset the commit capture tofalse
(whether or not it was previouslytrue
).void
resetForwards()
Clear the captured forwarded data.Cancellable
schedule(long intervalMs, PunctuationType type, Punctuator callback)
Deprecated.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
setHeaders(Headers headers)
The context exposes this metadata for use in the processor.void
setOffset(long offset)
The context exposes this metadata for use in the processor.void
setPartition(int partition)
The context exposes this metadata for use in the processor.void
setRecordMetadata(String topic, int partition, long offset, Headers headers, long timestamp)
The context exposes these metadata for use in the processor.void
setTimestamp(long timestamp)
The context exposes this metadata for use in the processor.void
setTopic(String topic)
The context exposes this metadata for use in the processor.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.
-
-
-
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
- Returns:
- the application id
-
taskId
public TaskId taskId()
Description copied from interface:ProcessorContext
Returns the task id.- Specified by:
taskId
in interfaceProcessorContext
- 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
- 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
- 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
- Returns:
- the key serializer
-
valueSerde
public Serde<?> valueSerde()
Description copied from interface:ProcessorContext
Returns the default value serde.- Specified by:
valueSerde
in interfaceProcessorContext
- 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
- Returns:
- the state directory
-
metrics
public StreamsMetrics metrics()
Description copied from interface:ProcessorContext
Returns Metrics instance.- Specified by:
metrics
in interfaceProcessorContext
- Returns:
- StreamsMetrics
-
setRecordMetadata
public void setRecordMetadata(String topic, int partition, long offset, Headers headers, long timestamp)
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 offsettimestamp
- A record timestamp
-
setTopic
public void setTopic(String topic)
The context exposes this 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 it directly. Setting this attribute doesn't affect the others.- Parameters:
topic
- A topic name
-
setPartition
public void setPartition(int partition)
The context exposes this 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 it directly. Setting this attribute doesn't affect the others.- Parameters:
partition
- A partition number
-
setOffset
public void setOffset(long offset)
The context exposes this 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 it directly. Setting this attribute doesn't affect the others.- Parameters:
offset
- A record offset
-
setHeaders
public void setHeaders(Headers headers)
The context exposes this 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 it directly. Setting this attribute doesn't affect the others.- Parameters:
headers
- Record headers
-
setTimestamp
public void setTimestamp(long timestamp)
The context exposes this 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 it directly. Setting this attribute doesn't affect the others.- Parameters:
timestamp
- A record timestamp
-
topic
public String topic()
Description copied from interface:ProcessorContext
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).- Specified by:
topic
in interfaceProcessorContext
- Returns:
- the topic name
-
partition
public int partition()
Description copied from interface:ProcessorContext
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).- Specified by:
partition
in interfaceProcessorContext
- Returns:
- the partition id
-
offset
public long offset()
Description copied from interface:ProcessorContext
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).- Specified by:
offset
in interfaceProcessorContext
- Returns:
- the offset
-
headers
public Headers headers()
Description copied from interface:ProcessorContext
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).- Specified by:
headers
in interfaceProcessorContext
- Returns:
- the headers
-
timestamp
public long timestamp()
Description copied from interface:ProcessorContext
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.
- Specified by:
timestamp
in interfaceProcessorContext
- Returns:
- the timestamp
-
register
public void register(StateStore store, StateRestoreCallback stateRestoreCallbackIsIgnoredInMock)
Description copied from interface:ProcessorContext
Registers and possibly restores the specified storage engine.- Specified by:
register
in interfaceProcessorContext
- Parameters:
store
- the storage enginestateRestoreCallbackIsIgnoredInMock
- the restoration callback logic for log-backed state stores upon restart
-
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
- Type Parameters:
S
- The type or interface of the store to return- Parameters:
name
- The store name- Returns:
- The state store instance
-
schedule
@Deprecated public Cancellable schedule(long intervalMs, PunctuationType type, Punctuator callback)
Deprecated.Description copied from interface:ProcessorContext
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, ...
- Specified by:
schedule
in interfaceProcessorContext
- 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
public Cancellable schedule(Duration interval, PunctuationType type, Punctuator callback) throws IllegalArgumentException
Description copied from interface:ProcessorContext
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, ...
- Specified by:
schedule
in interfaceProcessorContext
- 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
-
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,V> void forward(K key, V value)
Description copied from interface:ProcessorContext
Forwards a key/value pair to all downstream processors. Used the input record's timestamp as timestamp for the output record.- Specified by:
forward
in interfaceProcessorContext
- Parameters:
key
- keyvalue
- value
-
forward
public <K,V> void forward(K key, V value, To to)
Description copied from interface:ProcessorContext
Forwards a key/value pair to the specified downstream processors. Can be used to set the timestamp of the output record.- Specified by:
forward
in interfaceProcessorContext
- Parameters:
key
- keyvalue
- valueto
- the options to use when forwarding
-
forward
@Deprecated public <K,V> void forward(K key, V value, int childIndex)
Deprecated.Description copied from interface:ProcessorContext
Forwards a key/value pair to one of the downstream processors designated by childIndex.- Specified by:
forward
in interfaceProcessorContext
- Parameters:
key
- keyvalue
- valuechildIndex
- index in list of children of this node
-
forward
@Deprecated public <K,V> void forward(K key, V value, String childName)
Deprecated.Description copied from interface:ProcessorContext
Forwards a key/value pair to one of the downstream processors designated by the downstream processor name.- Specified by:
forward
in interfaceProcessorContext
- Parameters:
key
- keyvalue
- valuechildName
- name of downstream processor
-
forwarded
public List<MockProcessorContext.CapturedForward> 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 key/value pairs that were previously passed to the context.
-
forwarded
public List<MockProcessorContext.CapturedForward> 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 key/value pairs 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
-
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
-
-