Status
Current state: "Under Discussion"
Discussion thread: here
JIRA: KAFKA-6460
Motivation
This is for adding mock testing support for StateStore, StoreBuilder, StoreSupplier and other store related components which are used in Streams unit testing. We'd like to use mocks for different types of state stores: KV, window, session - that can be used to record the number of expected put / get calls used in the DSL operator unit testing. These will provide convenience for developers when they are writing unit test for Kafka stream and other related modules.
For example, in the current streams/TopologyTest.java:
... @Test public void shouldFailIfSinkIsParent() { topology.addSource("source", "topic-1"); topology.addSink("sink-1", "topic-2", "source"); try { topology.addSink("sink-2", "topic-3", "sink-1"); fail("Should throw TopologyException for using sink as parent"); } catch (final TopologyException expected) { } } @Test(expected = TopologyException.class) public void shouldNotAllowToAddStateStoreToNonExistingProcessor() { mockStoreBuilder(); EasyMock.replay(storeBuilder); topology.addStateStore(storeBuilder, "no-such-processor"); } @Test public void shouldNotAllowToAddStateStoreToSource() { mockStoreBuilder(); EasyMock.replay(storeBuilder); topology.addSource("source-1", "topic-1"); try { topology.addStateStore(storeBuilder, "source-1"); fail("Should have thrown TopologyException for adding store to source node"); } catch (final TopologyException expected) { } } private void mockStoreBuilder() { EasyMock.expect(storeBuilder.name()).andReturn("store").anyTimes(); EasyMock.expect(storeBuilder.logConfig()).andReturn(Collections.emptyMap()); EasyMock.expect(storeBuilder.loggingEnabled()).andReturn(false); }
One of the goal is to replace the in-test vanilla mockStoreBuilder with a more generic mockStoreBuilder class, which can provide more functionality and can be reuse later.
Public Interfaces
We add some new classes to the kafka-streams-test-utils
under a new org.apache.kafka.streams.state package.
We will provide a MockStoreFactory to generate Mock store builders:
package org.apache.kafka.streams.internals; public class MockStoreFactory<K, V> { final String storeName; final Serde<K> keySerde; final Serde<V> valueSerde; final Time time; final Boolean persistent; public MockStoreFactory (final String storeName, final Serde<K> keySerde, final Serde<V> valueSerde, final Time time) { this.storeName = storeName; this.keySerde = keySerde; this.valueSerde = valueSerde; this.time = time; } public MockKeyValueStoreBuilder createKeyValueStoreBuilder(){ return new MockKeyValueStoreBuilder<K,V>(storeName, keySerde, valueSerde, time); } }
I will use the KeyValueStoreBuilder as an example. Window and Session will have a similar structure.
Each Store builder will have a build method:
package org.apache.kafka.streams.internals; public class MockKeyValueStoreBuilder<K, V> extends AbstractStoreBuilder<K, V, StateStore> { final Boolean persistent; public MockKeyValueStoreBuilder(final String storeName, final Serde<K> keySerde, final Serde<V> valueSerde, final Time time) { super(storeName, keySerde, valueSerde, time); this.persistent = persistent; } @Override public KeyValueStore build() { return new MockKeyValueStore(name); } }
Then in the Store, we will build a wrapper around a InMemoryStore, and capture all the get/put calls (excluding Iterators)
public class MockKeyValueStore extends InMemoryKeyValueStore { private final boolean persistent; public boolean initialized = false; public boolean flushed = false; public final List<KeyValue> capturedPutCalls = new LinkedList<>(); public final List<KeyValue> capturedGetCalls = new LinkedList<>(); public final List<KeyValue> capturedDeleteCalls = new LinkedList<>(); public MockKeyValueStore(final String name, final boolean persistent) { super(name); this.persistent = persistent; } @Override public void flush() { flushed = true; super.flush(); } @Override public boolean persistent() { return persistent; } @Override public void put(final Bytes key, final byte[] value) { super.put(key, value); capturedPutCalls.add(new KeyValue(key, value)); } @Override public byte[] get(final Bytes key) { byte[] value = super.get(key); capturedGetCalls.add(new KeyValue(key, value)); return value; } @Override public byte[] delete(final Bytes key) { byte[] value = super.delete(key); capturedDeleteCalls.add(new KeyValue(key, value)); return value; } @Override public byte[] putIfAbsent(final Bytes key, final byte[] value) { final byte[] originalValue = get(key); if (originalValue == null) { put(key, value); capturedPutCalls.add(new KeyValue(key, value)); } return originalValue; } @Override public void putAll(final List<KeyValue<Bytes, byte[]>> entries) { for (final KeyValue<Bytes, byte[]> entry : entries) { put(entry.key, entry.value); capturedPutCalls.add(entry); } } }
Proposed Changes
Compatibility, Deprecation, and Migration Plan
Until we are refactoring with these new MockStateStores, there shouldn't be any compatibility issues. But the next phase should be refactoring, including:
1) Remove and refactor redundant MockStores (i.e. org.apache.kafka.test.MockKeyValueStore)
2) Examine the current tests (i.e. org.apache.kafka.streams.TopologyTest), remove complicate testing code logics and refactor with the new MockStateStores.
Rejected Alternatives
1) Rebuilding a MockStateStores vs extending an InMemoryStore
- If we are rebuilding the functionality of a store from scratch in memory, it will basically act like an InMemoryStore.
2) Track all calls in a total order.
- Now I haven't got a feature request to track calls within a total order, so now we track the order separately. If there is a need, we can definitely adjust it.
3)Using the MockStateStoreFactory as a main entry point to access stores. (Keeping the state of all StoreBuilder)
- Now the MockStateStoreFactory only provides build() functionality, but not access(i.e. getStore(String storeName)) to StoreBuilders and Stores. Developers have to access these Stores separately. If there is a request to allow access from the factory, we can adjust that, create a mapping to store all the StoreBuilders.