...
After the improvements, we will replace the easyMock StoreBuilder without mock StoreBuilderwith 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( "store", Stores.inMemoryKeyValueStore("store"), Serdes.Bytes(), Serdes.Bytes(), false, Time.System); ... @Test(expected = TopologyException.class) public void shouldNotAllowToAddStateStoreToNonExistingProcessor() { topology.addStateStore(keyValueStoreBuilder, "no-such-processor"); } } |
...
We add some new classes to the kafka-streams-test-utils
under a new a state package (org.apache.kafka.streams.state packagestate) under streams/test-utils.
We will provide a MockStoreFactory to generate Mock store builders: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 |
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<>(storeName, 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); } } |
I will use the KeyValueStoreBuilder as an example. Window and Session will have a similar structure.
Each Store builder will have a build methodEach Store builder will have a build method:
Code Block |
---|
package org.apache.kafka.streams.internalsstate; import org.apache.kafka.common.serialization.Serde; import org.apache.kafka.streams.MockTime; import org.apache.kafka.streams.state.KeyValueBytesStoreSupplier; import org.apache.kafka.streams.state.KeyValueStorecommon.utils.Time; import org.apache.kafka.streams.state.internals.AbstractStoreBuilderKeyValueStoreBuilder; public class MockKeyValueStoreBuilder<K, V> extends AbstractStoreBuilder<KKeyValueStoreBuilder<K, V, KeyValueStore>V> { private final boolean persistent; private final KeyValueBytesStoreSupplier storeSupplier; final Serde<K> keySerde; final Serde<V> valueSerde; final Time time; public MockKeyValueStoreBuilder(final StringKeyValueBytesStoreSupplier storeNamestoreSupplier, final KeyValueBytesStoreSupplierSerde<K> storeSupplierkeySerde, final Serde<K>Serde<V> keySerdevalueSerde, final Serde<V>boolean valueSerdepersistent, final booleanTime persistenttime) { super(storeNamestoreSupplier, keySerde, valueSerde, new MockTime(0time)); this.persistent = persistent; this.storeSupplier = storeSupplier; } @Override this.keySerde = keySerde; public KeyValueStorethis.valueSerde = valueSerde; this.time = time; } @Override public KeyValueStore<K, V> build() { return new MockKeyValueStore<>(namestoreSupplier, keySerde, storeSupplier.get()valueSerde, persistent, time); } } |
Then in the Storestore, we will build a wrapper around a InMemoryStore, and capture all the get/put calls (excluding Iterators)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.internalsstate; import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.processor.ProcessorContext; import org.apache.kafka.streams.processor.StateRestoreCallback; import org.apache.kafka.streams.processor.StateStore; import org.apache.kafka.streams.state.KeyValueIterator; import org.apache.kafka.streams.state.KeyValueStore; import java.util.LinkedList; import java.util.List; import java.util.concurrent.atomic.AtomicInteger; public class MockKeyValueStore<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); private final String name; private final boolean persistent; public boolean initialized = false; public boolean flushed = false; public boolean closed = true; 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 final KeyValueStore<K, V> innerKeyValueStore; public MockKeyValueStore(final String namepublic 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) { final KeyValueStore<K, V> innerKeyValueStore,context.register(root, stateRestoreCallback); initialized = true; closed = false; } @Override finalpublic booleanvoid persistentflush() { this.name = name instanceLastFlushCount.set(GLOBAL_FLUSH_COUNTER.getAndIncrement()); this.innerKeyValueStore = innerKeyValueStorewrapped().flush(); this.persistentflushed = persistenttrue; } @Override public Stringint namegetLastFlushCount() { return nameinstanceLastFlushCount.get(); } @Override public void init(final ProcessorContext context, close() { wrapped().close(); closed = true; } @Override finalpublic StateStoreboolean rootpersistent() { context.register(root, stateRestoreCallback)return persistent; } initialized = true;@Override public boolean closed = false; isOpen() { } return @Override public void flush() { instanceLastFlushCount.set(GLOBAL_FLUSH_COUNTER.getAndIncrement());!closed; } public final StateRestoreCallback stateRestoreCallback = new StateRestoreCallback() { innerKeyValueStore.flush();@Override flushedpublic = true; } void restore(final byte[] key, public int getLastFlushCount() { return instanceLastFlushCount.get(); } @Override publicfinal void close(byte[] value) { innerKeyValueStore.close(); } }; @Override public closed = true; } void put(final K key, final V value) { @Override public boolean persistent() {capturedPutCalls.add(new KeyValue<>(key, value)); return persistentwrapped().put(keyBytes(key), serdes.rawValue(value)); } @Override public booleanV isOpenputIfAbsent()final { K key, final V return !closed; value) { } public final StateRestoreCallbackV stateRestoreCallbackoriginalValue = new StateRestoreCallbackget(key) { ; if (originalValue == null) @Override{ public void restore(final byte[] put(key, value); capturedPutCalls.add(new KeyValue<>(key, value)); } final byte[] value)return {originalValue; } }@Override }; @Overridepublic V delete(final K key) { public void put(final K key,V finalvalue V value) {= outerValue(wrapped().delete(keyBytes(key))); capturedPutCallscapturedDeleteCalls.add(new KeyValue<>(key, value)); innerKeyValueStore.put(key,return value); } @Override public Vvoid putIfAbsent(final K key, final V value) { final V originalValue = get(key);putAll(final List<KeyValue<K, V>> entries) { iffor (originalValue == nullfinal KeyValue<K, V> entry : entries) { put(entry.key, entry.value); capturedPutCalls.add(new KeyValue<>(key, value))entry); } return originalValue; } @Override public V deleteget(final K key) { V value = innerKeyValueStore.deleteouterValue(wrapped().get(keyBytes(key))); capturedDeleteCallscapturedGetCalls.add(new KeyValue<>(key, value)); return value; } @SuppressWarnings("unchecked") @Override public voidKeyValueIterator<K,V> putAllrange(final List<KeyValue<KK from, V>>final K entriesto) { return fornew MockKeyValueStore.MockKeyValueIterator(final KeyValue<K, V> entry : entries) { put(entry.key, entry.value); capturedPutCalls.add(entry); wrapped().range(Bytes.wrap(serdes.rawKey(from)), Bytes.wrap(serdes.rawKey(to)))); } } @SuppressWarnings("unchecked") @Override public V get(final K keyKeyValueIterator<K,V> all() { Vreturn value = innerKeyValueStore.get(key); new MockKeyValueStore.MockKeyValueIterator(wrapped().all()); } @Override capturedGetCalls.add(new KeyValue<>(key, value));public long approximateNumEntries() { return valuewrapped().approximateNumEntries(); } @Override private public KeyValueIterator<K,V> rangeV outerValue(final K from, final K to) { byte[] value) { return value != null return? innerKeyValueStoreserdes.range(from, to)valueFrom(value) : null; } @Override private Bytes keyBytes(final public KeyValueIterator<K,V> all(K key) { return innerKeyValueStore.all(Bytes.wrap(serdes.rawKey(key)); } private class @Override MockKeyValueIterator implements KeyValueIterator<K, V> public{ long approximateNumEntries() { private final return innerKeyValueStore.approximateNumEntries();KeyValueIterator<Bytes, byte[]> 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.