Class 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 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<KForward,​VForward>
        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)
        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
      • 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 (see ProcessorContext.schedule(Duration, PunctuationType, Punctuator) ), or because a parent processor called ProcessorContext.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 regular Processor.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 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.

        Specified by:
        recordMetadata in interface ProcessorContext<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 interface ProcessorContext<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 during initialization or Processor.process(Record) 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<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 to schedule(...).
        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), 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); }

        Specified by:
        forward in interface ProcessorContext<KForward,​VForward>
        Parameters:
        record - The record to forward to all children
      • 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 to forward(...).
        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 to forward(...).
        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.
      • 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