Class 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 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.

    • Constructor Detail

      • MockProcessorContext

        public MockProcessorContext()
        Create a 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.
      • MockProcessorContext

        public MockProcessorContext​(Properties config)
        Create a 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.
        Parameters:
        config - a Properties object, used to configure the context and the processor.
      • MockProcessorContext

        public MockProcessorContext​(Properties config,
                                    TaskId taskId,
                                    File stateDir)
        Create a MockProcessorContext with a specified taskId and null stateDir.
        Parameters:
        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().
    • Method Detail

      • 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 interface ProcessorContext
        Parameters:
        prefix - the properties prefix
        Returns:
        the key/values matching the given prefix from the StreamsConfig properties.
      • 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 name
        partition - A partition number
        offset - A record offset
        timestamp - 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 interface ProcessorContext
        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 interface ProcessorContext
        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 interface ProcessorContext
        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 interface ProcessorContext
        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 by TimestampExtractor.

        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 interface ProcessorContext
        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 interface ProcessorContext
        Parameters:
        store - the storage engine
        stateRestoreCallbackIsIgnoredInMock - 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 interface ProcessorContext
        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 during 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 arrive
        • PunctuationType.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
        Skipping punctuations: Punctuations will not be triggered more than once at any given timestamp. This means that "missed" punctuation will be skipped. It's possible to "miss" a punctuation if:
        Specified by:
        schedule in interface ProcessorContext
        Parameters:
        intervalMs - the time interval between punctuations in milliseconds
        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
      • 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 during 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 arrive
        • PunctuationType.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
        Skipping punctuations: Punctuations will not be triggered more than once at any given timestamp. This means that "missed" punctuation will be skipped. It's possible to "miss" a punctuation if:
        Specified by:
        schedule in interface ProcessorContext
        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 to schedule(...).
        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 interface ProcessorContext
        Parameters:
        key - key
        value - 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 interface ProcessorContext
        Parameters:
        key - key
        value - value
        to - 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 interface ProcessorContext
        Parameters:
        key - key
        value - value
        childIndex - 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 interface ProcessorContext
        Parameters:
        key - key
        value - value
        childName - 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 to forward(...).
        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 to forward(...).
        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.
      • resetCommit

        public void resetCommit()
        Reset the commit capture to false (whether or not it was previously true).
      • recordCollector

        public org.apache.kafka.streams.processor.internals.RecordCollector recordCollector()
        Specified by:
        recordCollector in interface org.apache.kafka.streams.processor.internals.RecordCollector.Supplier