mirror of https://github.com/apache/kafka.git
KAFKA-4654: Improve test coverage for MemoryLRUCacheStore
Author: bbejeck <bbejeck@gmail.com> Reviewers: Damian Guy, Guozhang Wang Closes #2500 from bbejeck/KAFKA-4654_improve_MemroryLRUCache_test_coverage
This commit is contained in:
parent
b865a8b1dc
commit
d24616777a
|
|
@ -16,13 +16,20 @@
|
|||
*/
|
||||
package org.apache.kafka.streams.state.internals;
|
||||
|
||||
import org.apache.kafka.streams.KeyValue;
|
||||
import org.apache.kafka.streams.processor.ProcessorContext;
|
||||
import org.apache.kafka.streams.processor.StateStoreSupplier;
|
||||
import org.apache.kafka.streams.state.KeyValueStore;
|
||||
import org.apache.kafka.streams.state.Stores;
|
||||
import org.junit.Test;
|
||||
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
|
||||
import static org.hamcrest.CoreMatchers.equalTo;
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertThat;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
public class InMemoryLRUCacheStoreTest extends AbstractKeyValueStoreTest {
|
||||
|
|
@ -47,6 +54,43 @@ public class InMemoryLRUCacheStoreTest extends AbstractKeyValueStoreTest {
|
|||
return store;
|
||||
}
|
||||
|
||||
@Test
|
||||
public void shouldPutAllKeyValuePairs() {
|
||||
final List<KeyValue<Integer, String>> kvPairs = Arrays.asList(KeyValue.pair(1, "1"),
|
||||
KeyValue.pair(2, "2"),
|
||||
KeyValue.pair(3, "3"));
|
||||
|
||||
store.putAll(kvPairs);
|
||||
|
||||
assertThat(store.approximateNumEntries(), equalTo(3L));
|
||||
|
||||
for (KeyValue<Integer, String> kvPair : kvPairs) {
|
||||
assertThat(store.get(kvPair.key), equalTo(kvPair.value));
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void shouldUpdateValuesForExistingKeysOnPutAll() {
|
||||
final List<KeyValue<Integer, String>> kvPairs = Arrays.asList(KeyValue.pair(1, "1"),
|
||||
KeyValue.pair(2, "2"),
|
||||
KeyValue.pair(3, "3"));
|
||||
|
||||
store.putAll(kvPairs);
|
||||
|
||||
|
||||
final List<KeyValue<Integer, String>> updatedKvPairs = Arrays.asList(KeyValue.pair(1, "ONE"),
|
||||
KeyValue.pair(2, "TWO"),
|
||||
KeyValue.pair(3, "THREE"));
|
||||
|
||||
store.putAll(updatedKvPairs);
|
||||
|
||||
assertThat(store.approximateNumEntries(), equalTo(3L));
|
||||
|
||||
for (KeyValue<Integer, String> kvPair : updatedKvPairs) {
|
||||
assertThat(store.get(kvPair.key), equalTo(kvPair.value));
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testEvict() {
|
||||
// Create the test driver ...
|
||||
|
|
@ -93,4 +137,5 @@ public class InMemoryLRUCacheStoreTest extends AbstractKeyValueStoreTest {
|
|||
assertTrue(driver.flushedEntryRemoved(3));
|
||||
assertEquals(3, driver.numFlushedEntryRemoved());
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
|||
Loading…
Reference in New Issue