Versions Compared

Key

  • This line was added.
  • This line was removed.
  • Formatting was changed.

...

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-a state package (org.apache.kafka.streams.stateunder 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.

...

Code Block
package org.apache.kafka.streams.internalsstate;

import org.apache.kafka.common.serialization.Serde;

import org.apache.kafka.streamscommon.utils.MockTimeTime;
import org.apache.kafka.streams.state.KeyValueBytesStoreSupplierinternals.KeyValueStoreBuilder;


import org.apache.kafka.streams.state.KeyValueStore;
import org.apache.kafka.streams.state.internals.AbstractStoreBuilder;



public class MockKeyValueStoreBuilder<K,public class MockKeyValueStoreBuilder<K, V> extends AbstractStoreBuilder<KKeyValueStoreBuilder<K, V, KeyValueStore>V> {

    private final boolean persistent;
    private final KeyValueBytesStoreSupplier storeSupplier;

    public MockKeyValueStoreBuilder(final KeyValueBytesStoreSupplierSerde<K> storeSupplier,keySerde;
    final Serde<V> valueSerde;
    final Time time;

    public MockKeyValueStoreBuilder(final KeyValueBytesStoreSupplier storeSupplier,
                                    final Serde<K> keySerde,
                                    final Serde<V> valueSerde,
                                    final boolean persistent) {,
        super(storeSupplier.name(), keySerde, valueSerde, new MockTime(0));
        this.persistent = persistent;
        this.storeSupplier = storeSupplier;

    }

final Time time) {
 @Override
     public KeyValueStore build() {
   super(storeSupplier, keySerde, valueSerde, time);
        this.persistent = persistent;
        this.storeSupplier = storeSupplier;
     return new MockKeyValueStore<>(name, storeSupplier.get(), persistent)this.keySerde = keySerde;
    }
}

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.

    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
Code Block
package org.apache.kafka.streams.internals;

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 name,
                     final StateStore root) {
     final KeyValueStore<K, V> innerKeyValueStore,
context.register(root, stateRestoreCallback);
        initialized = true;
        closed = false;
    }

    @Override
    finalpublic booleanvoid persistentflush() {
        this.name = nameinstanceLastFlushCount.set(GLOBAL_FLUSH_COUNTER.getAndIncrement());
        this.innerKeyValueStore = innerKeyValueStorewrapped().flush();
        this.persistentflushed = persistenttrue;

    }

    @Override
    public Stringint namegetLastFlushCount() {
        return nameinstanceLastFlushCount.get();
    }

    @Override
    public void initclose(final ProcessorContext context,) {
                     final StateStore root) {
wrapped().close();
        closed = true;
   context.register(root, stateRestoreCallback); }

    @Override
    public initializedboolean = true;persistent() {
        closedreturn = falsepersistent;
    }

    @Override
    public voidboolean flushisOpen() {
        instanceLastFlushCount.set(GLOBAL_FLUSH_COUNTER.getAndIncrement());return !closed;
    }

    public final StateRestoreCallback  innerKeyValueStore.flushstateRestoreCallback = new StateRestoreCallback(); {
        flushed = true;@Override
    }

    public intvoid getLastFlushCount() {restore(final byte[] key,
        return instanceLastFlushCount.get();
    }

    @Override
    public void close() {
        innerKeyValueStore.close();
  final byte[] value) {
      closed = true;}
    };

    @Override
    public booleanvoid persistent() {put(final K key, final V value) {
        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) {
;
        @Override
if (originalValue == null) {
    public void restore(final byte[] key,
        put(key, value);
            capturedPutCalls.add(new KeyValue<>(key, value));
     final byte[] value) {}
        return }originalValue;
    };

    @Override
    public voidV putdelete(final K key, final V value) { K key) {
        V value = outerValue(wrapped().delete(keyBytes(key)));
        capturedPutCallscapturedDeleteCalls.add(new KeyValue<>(key, value));
        innerKeyValueStore.put(key,return value);
    }

    @Override
    public Vvoid putIfAbsentputAll(final K keyList<KeyValue<K, finalV>> V valueentries) {
        final V originalValue = get(key);
        if (originalValue == nullfor (final 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;
    }

    @Override
    public void putAll(final List<KeyValue<K, V>> entries) {
@SuppressWarnings("unchecked")
    @Override
    public forKeyValueIterator<K,V> range(final K KeyValue<Kfrom, V>final entryK : entriesto) {
        return new MockKeyValueStore.MockKeyValueIterator(
  put(entry.key, entry.value);
               capturedPutCalls.add(entrywrapped().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(keynew MockKeyValueStore.MockKeyValueIterator(wrapped().all());
    }

    capturedGetCalls.add(new KeyValue<>(key, value));@Override
    public long approximateNumEntries() {
        return valuewrapped().approximateNumEntries();
    }

    @Override
private    public KeyValueIterator<K,V> rangeV outerValue(final K from, final K tobyte[] value) {
        return innerKeyValueStore.range(from, to) value != null ? serdes.valueFrom(value) : null;
    }

    @Override
    public KeyValueIterator<K,V>  all(private Bytes keyBytes(final K key) {
        return innerKeyValueStore.all()Bytes.wrap(serdes.rawKey(key));
    }

    @Override
private class MockKeyValueIterator implements public long approximateNumEntries()KeyValueIterator<K, V> {

         return innerKeyValueStore.approximateNumEntries();private final KeyValueIterator<Bytes, byte[]> iter;
		....
    }
}


Proposed Changes

I proposed to add:

  1. a MockStoreFactory.java A MockStoreFactory class to produce mock state store builders.
  2. A mock StateStoreBuilder class for KV, Session and Window.
  3. A mock StateStore class for KV, Session and Window with tracking.

...

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
serverASF JIRA
serverId5aa69414-a9e9-3523-82ec-879b028fb15b
keyKAFKA-8630
. In-memory Window/Session store doesn't work well in tests because in the init() method, it has internally cast ProcessorContext into InternalProcessorContext, that way the tester couldn't use MockProcessorContext. Either we twist the init() method in the mock store to accommodate or we can somehow use InternalMockProcessorContext instead?

Rejected Alternatives

  1. 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.