Child pages
  • KIP-267: Add Processor Unit Test Support to Kafka Streams Test Utils
Skip to end of metadata
Go to start of metadata


Current state: Voting passed

Discussion thread[DISCUSS] KIP-267: Add Processor Unit Test Support to Kafka Streams Test Utils

JIRA KAFKA-6473 - Add MockProcessorContext to public test-utils In Progress

Released: target version 1.2.0

Please keep the discussion on the mailing list rather than commenting on the wiki (wiki discussions get unwieldy fast).


KIP-247 adds support for testing a complete topology, but authors of Processor, Transformer, and ValueTransformer implementations can benefit from writing lighter and faster unit tests.

This isn't impossible today, but it requires writing some fairly complicated mock code for the ProcessorContext. We want to simplify this task by providing a general purpose ProcessorContext for unit testing.

Public Interface


Proposed Changes

We add the above new class to the kafka-streams-test-utils artifact (introduced in 1.1.0).

In the initial release, we mark our new class with annotation @Evolving.

All the methods annotated with @Override are implementing the ProcessorContext interface. 

We add new methods for the purposes of setting some properties (like topic, partition, etc.) that the processor may want to access.

We also capture data that the processor may put into the context, like k/v pairs in calls to forward(), but also state stores that get registered, punctuators that get scheduled, and whether commit() gets called.

We provide a mechanism to reset the commit and forwarded data captures to make it easy to make assertions about the behavior of the processor after various calls to Processor#process().

There is one method we plan not to implement:

  • void schedule(long interval): it's not possible to implement it without providing a handle on the processor to the context. This is a little tricky, since the mocked ProcessorContext may be used to test not just a Processor, but also a Transformer or a ValueTransformer. Since this method is deprecated in the interface, we feel it's reasonable to update client code to use Punctuators instead.

Also note that we won't automatically trigger sheduled punctuators. Instead, we'll just capture them so test code can retrieve and trigger them if desired. This keeps the mock truly a "mock" and not a "driver", which maintains the simplicity of the code and enables users to write linear and sensible tests. If folks want to have their punctuators triggered automatically, they can use the TopologyTestDriver, which does that. This is a potential sharp edge, so I'm planning to make sure it's well documented in both the javadoc and html docs.

This method will throw an UnsupportedOperationException.

Example Usage

This mocked ProcessorContext would enable unit tests like the following:


Compatibility, Deprecation, and Migration Plan

We are only adding a new class. There are no compatiblity issues.

Test Plan

We need to test all added classes with unit tests. Integration or system test are not required.

Rejected Alternatives


  • No labels


  1. The mailing list discussion is over, and the vote passed