...
Motivation
This is for adding mocks for state stores 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.
Public Interfaces
These will be internal classes, so no public API/interface.
Proposed Changes
We will provide a MockStoreFactory to generate Mock store builders:
For example, in the current streams/TopologyTest.java:
Code Block | ||
---|---|---|
| ||
public class TopologyTest | ||
Code Block | ||
package org.apache.kafka.streams.internals; public class MockStoreFactory<K, V> { private final String storeNameStoreBuilder storeBuilder = EasyMock.createNiceMock(StoreBuilder.class); private final KeyValueStoreBuilder Serde<K> keySerdeglobalStoreBuilder = EasyMock.createNiceMock(KeyValueStoreBuilder.class); private final Serde<V>Topology valueSerde; topology = final Time time; new Topology(); final Boolean persistent; ... @Test public MockStoreFactoryvoid shouldFailIfSinkIsParent(final String storeName,) { topology.addSource("source", "topic-1"); topology.addSink("sink-1", "topic-2", "source"); final Serde<K>try keySerde,{ topology.addSink("sink-2", "topic-3", "sink-1"); fail("Should throw TopologyException finalfor Serde<V> valueSerde,using sink as parent"); } catch (final TopologyException expected) { } } @Test(expected = TopologyException.class) public finalvoid Time time,shouldNotAllowToAddStateStoreToNonExistingProcessor() { mockStoreBuilder(); EasyMock.replay(storeBuilder); topology.addStateStore(storeBuilder, "no-such-processor"); final Boolean persistent) { } @Test public this.storeName = storeName;void shouldNotAllowToAddStateStoreToSource() { this.keySerde = keySerdemockStoreBuilder(); this.valueSerde = valueSerdeEasyMock.replay(storeBuilder); this.time = time; topology.addSource("source-1", "topic-1"); try this.persistent{ = persistent; } public MockKeyValueStoreBuilder createKeyValueStoreBuilder(){ topology.addStateStore(storeBuilder, "source-1"); return new MockKeyValueStoreBuilder<K,V>(storeName, keySerde, valueSerde, time, persistent); } } |
I will use the KeyValueStoreBuilder as an example. Window and Session will have a similar structure.
Each Store builder will have a build method:
Code Block |
---|
package org.apache.kafka.streams.internals; public class MockKeyValueStoreBuilder<K, V> extends AbstractStoreBuilder<K, V, StateStore> { final Boolean persistent; public MockKeyValueStoreBuilder(final String storeName,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); final Serde<K> keySerde, final Serde<V> valueSerde,} } |
One of the goal is to replace the in-test vanilla mockStoreBuilder with a more general purpose mockStoreBuilder class, which simplify writing unit test and can be reuse later.
After the improvements, we will replace the easyMock StoreBuilder with our mockStoreBuilder.
Code Block |
---|
public class TopologyTest { private final StoreBuilder storeBuilder = EasyMock.createNiceMock(StoreBuilder.class); private final Topology topology = new Topology(); private final MockStoreFactory mockStoreFactory = new MockStoreFactory<>(); private final KeyValueStoreBuilder keyValueStoreBuilder = mockStoreFactory.createKeyValueStoreBuilder( final Time timeStores.inMemoryKeyValueStore("store"), Serdes.Bytes(), Serdes.Bytes(), final Boolean persistent) {false, Time.System); ... super(storeName, keySerde, valueSerde, time);@Test(expected = TopologyException.class) public void this.persistent = persistent; } @Override public KeyValueStore build() shouldNotAllowToAddStateStoreToNonExistingProcessor() { return new MockKeyValueStore(name, persistenttopology.addStateStore(keyValueStoreBuilder, "no-such-processor"); } } |
Then in the Store, we will build a wrapper around a InMemoryStore, and capture all the get/put calls (excluding Iterators)
Public Interfaces
We add some new classes to a state package (org.apache.kafka.streams.state) under streams/test-utils.
We will provide a MockStoreFactory to generate mock store builders, I will use the KeyValueStoreBuilder as an example. Window and Session will have a similar structure.
The developers/users can provide their own store as the backend storage, and their own Serde of choice. For example, for simple testing, they can just use an InMemoryKeyValueStore.
Code Block |
---|
package org.apache.kafka.streams.internals;
public class MockStoreFactory<K, V> {
public final Map<String, StoreBuilder> stateStores = new LinkedHashMap<>();
public MockStoreFactory () {
}
public KeyValueStoreBuilder createKeyValueStoreBuilder(KeyValueBytesStoreSupplier keyValueBytesStoreSupplier,
final Serde<K> keySerde,
final Serde<V> valueSerde,
boolean persistent){
String storeName = keyValueBytesStoreSupplier.name();
stateStores.put(storeName, new MockKeyValueStoreBuilder<>(keyValueBytesStoreSupplier, keySerde, valueSerde, persistent));
return (KeyValueStoreBuilder)stateStores.get(storeName);
}
public WindowStoreBuilder createWindowStoreBuilder(KeyValueBytesStoreSupplier keyValueBytesStoreSupplier,
final Serde<K> keySerde,
final Serde<V> valueSerde,
final Time time){
...
}
public SessionStoreBuilder createSessionStoreBuilder(KeyValueBytesStoreSupplier keyValueBytesStoreSupplier,
final Serde<K> keySerde,
final Serde<V> valueSerde,
final Time time){
...
}
public StoreBuilder getStore(String storeName) {
return stateStores.get(storeName);
}
} |
Each Store builder will have a build method:
Code Block |
---|
package org.apache.kafka.streams.state;
import org.apache.kafka.common.serialization.Serde;
import org.apache.kafka.common.utils.Time;
import org.apache.kafka.streams.state.internals.KeyValueStoreBuilder;
public class MockKeyValueStoreBuilder<K, V> extends KeyValueStoreBuilder<K, V> {
private final boolean persistent;
private final KeyValueBytesStoreSupplier storeSupplier;
final Serde<K> keySerde;
final Serde<V> valueSerde;
final Time time;
public MockKeyValueStoreBuilder(final KeyValueBytesStoreSupplier storeSupplier,
final Serde<K> keySerde,
final Serde<V> valueSerde,
final boolean persistent,
final Time time) {
super(storeSupplier, keySerde, valueSerde, time);
this.persistent = persistent;
this.storeSupplier = storeSupplier;
this.keySerde = keySerde;
this.valueSerde = valueSerde;
this.time = time;
}
@Override
public KeyValueStore<K, V> build() {
return new MockKeyValueStore<>(storeSupplier, keySerde, valueSerde, persistent, time);
}
} |
Then in the store, we will build a wrapper around the provided backend store. We will capture each get/put/delete call, the user can write tests accordingly. We will also track if the store has been flushed or closed.
Code Block |
---|
package org.apache.kafka.streams.state;
public class MockKeyValueStore<K, V>
extends WrappedStateStore<KeyValueStore<Bytes, byte[]>, K, V>
implements KeyValueStore<K, V> {
// keep a global counter of flushes and a local reference to which store had which
// flush, so we can reason about the order in which stores get flushed.
private static final AtomicInteger GLOBAL_FLUSH_COUNTER = new AtomicInteger(0);
private final AtomicInteger instanceLastFlushCount = new AtomicInteger(-1);
public boolean initialized = false;
public boolean flushed = false;
public boolean closed = true;
public String name;
public boolean persistent;
protected final Time time;
final Serde<K> keySerde;
final Serde<V> valueSerde;
StateSerdes<K, V> serdes;
public final List<KeyValue<K, V>> capturedPutCalls = new LinkedList<>();
public final List<KeyValue<K, V>> capturedGetCalls = new LinkedList<>();
public final List<KeyValue<K, V>> capturedDeleteCalls = new LinkedList<>();
public MockKeyValueStore(final KeyValueBytesStoreSupplier keyValueBytesStoreSupplier,
final Serde<K> keySerde,
final Serde<V> valueSerde,
final boolean persistent,
final Time time) {
super(keyValueBytesStoreSupplier.get());
this.name = keyValueBytesStoreSupplier.name();
this.time = time != null ? time : Time.SYSTEM;
this.persistent = persistent;
this.keySerde = keySerde;
this.valueSerde = valueSerde;
}
@SuppressWarnings("unchecked")
void initStoreSerde(final ProcessorContext context) {
serdes = new StateSerdes<>(
ProcessorStateManager.storeChangelogTopic(context.applicationId(), name()),
keySerde == null ? (Serde<K>) context.keySerde() : keySerde,
valueSerde == null ? (Serde<V>) context.valueSerde() : valueSerde);
}
@Override
public String name() {
return name;
}
@Override
public void init(final ProcessorContext context,
final StateStore root) {
context.register(root, stateRestoreCallback);
initialized = true;
closed = false;
}
@Override
public void flush() {
instanceLastFlushCount.set(GLOBAL_FLUSH_COUNTER.getAndIncrement());
wrapped().flush();
flushed = true;
}
public int getLastFlushCount() {
return instanceLastFlushCount.get();
}
@Override
public void close() {
wrapped().close();
closed = true;
}
@Override
public boolean persistent() {
return persistent;
}
@Override
public boolean isOpen() {
return !closed;
}
public final StateRestoreCallback stateRestoreCallback = new StateRestoreCallback() {
@Override
public void restore(final byte[] key,
final byte[] value) {
}
};
@Override
public void put(final K key, final V value) {
capturedPutCalls.add(new KeyValue<>(key, value));
wrapped().put(keyBytes(key), serdes.rawValue(value));
}
@Override
public V putIfAbsent(final K key, final V value) {
final V originalValue = get(key);
if (originalValue == null) {
|
Code Block |
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, put(key, value); final boolean persistent) {capturedPutCalls.add(new KeyValue<>(key, value)); super(name);} this.persistent = persistentreturn originalValue; } @Override public voidV flushdelete(final K key) { flushedV value = true; super.flush(outerValue(wrapped().delete(keyBytes(key))); } @Override public boolean persistent() {capturedDeleteCalls.add(new KeyValue<>(key, value)); return persistentvalue; } @Override public void putputAll(final Bytes key, final byte[] value List<KeyValue<K, V>> entries) { for (final KeyValue<K, V> entry : entries) { super.put(entry.key, entry.value); capturedPutCalls.add(new KeyValue(key, value));entry); } } @Override public byte[]V get(final BytesK key) { byte[]V value = super outerValue(wrapped().get(keyBytes(key))); capturedGetCalls.add(new KeyValueKeyValue<>(key, value)); return value; } @SuppressWarnings("unchecked") @Override public KeyValueIterator<K,V> range(final K from, final K to) { } @Override return new MockKeyValueStore.MockKeyValueIterator( public byte[] delete(final Bytes key) { byte[] value = super.delete(key wrapped().range(Bytes.wrap(serdes.rawKey(from)), Bytes.wrap(serdes.rawKey(to)))); } capturedDeleteCalls.add(new KeyValue(key, value));@SuppressWarnings("unchecked") @Override return value; public KeyValueIterator<K,V> } all() { @Override publicreturn byte[] putIfAbsent(final Bytes key, final byte[] value) {new MockKeyValueStore.MockKeyValueIterator(wrapped().all()); } @Override finalpublic byte[] originalValue = get(key);long approximateNumEntries() { ifreturn (originalValue == null) {wrapped().approximateNumEntries(); } private V put(key,outerValue(final byte[] value); { return value != null capturedPutCalls? serdes.addvalueFrom(new KeyValue(key, value)); value) : null; } private Bytes }keyBytes(final K key) { return originalValueBytes.wrap(serdes.rawKey(key)); } @Override private class MockKeyValueIterator public void putAll(final List<KeyValue<Bytes, byte[]>> entries) {implements KeyValueIterator<K, V> { forprivate (final KeyValue<BytesKeyValueIterator<Bytes, byte[]> entry : entries) { put(entry.key, entry.value); capturedPutCalls.add(entry); } } }iter; .... } } |
Proposed Changes
I proposed to add:
- A MockStoreFactory class to produce mock state store builders.
- A mock StateStoreBuilder class for KV, Session and Window.
- A mock StateStore class for KV, Session and Window with tracking.
Compatibility, Deprecation, and Migration Plan
...
2) Examine the current tests (i.e. org.apache.kafka.streams.TopologyTest ), remove complicate and refactor the 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)
...
A discussion that has been brought up is
Jira | ||||||
---|---|---|---|---|---|---|
|
Rejected Alternatives
- Using the current EasyMock implementation. There is no strong argument against the current EasyMock implementation, it is easy to use and lightweight. The main argument for this KIP is to write better tests with an in-house mock state store support.