mirror of https://github.com/apache/kafka.git
KAFKA-13480: Track Position in KeyValue stores (#11514)
Add position tracking to KeyValue stores in support of KIP-796 Reviewers: John Roesler <vvcephei@apache.org>
This commit is contained in:
parent
0f967828e1
commit
23e9818e62
|
|
@ -21,6 +21,9 @@ import org.apache.kafka.common.utils.Bytes;
|
||||||
import org.apache.kafka.streams.KeyValue;
|
import org.apache.kafka.streams.KeyValue;
|
||||||
import org.apache.kafka.streams.processor.ProcessorContext;
|
import org.apache.kafka.streams.processor.ProcessorContext;
|
||||||
import org.apache.kafka.streams.processor.StateStore;
|
import org.apache.kafka.streams.processor.StateStore;
|
||||||
|
import org.apache.kafka.streams.processor.StateStoreContext;
|
||||||
|
import org.apache.kafka.streams.processor.api.RecordMetadata;
|
||||||
|
import org.apache.kafka.streams.processor.internals.StoreToProcessorContextAdapter;
|
||||||
import org.apache.kafka.streams.state.KeyValueIterator;
|
import org.apache.kafka.streams.state.KeyValueIterator;
|
||||||
import org.apache.kafka.streams.state.KeyValueStore;
|
import org.apache.kafka.streams.state.KeyValueStore;
|
||||||
import org.slf4j.Logger;
|
import org.slf4j.Logger;
|
||||||
|
|
@ -40,9 +43,12 @@ public class InMemoryKeyValueStore implements KeyValueStore<Bytes, byte[]> {
|
||||||
private final String name;
|
private final String name;
|
||||||
private final NavigableMap<Bytes, byte[]> map = new TreeMap<>();
|
private final NavigableMap<Bytes, byte[]> map = new TreeMap<>();
|
||||||
private volatile boolean open = false;
|
private volatile boolean open = false;
|
||||||
|
private StateStoreContext context;
|
||||||
|
private Position position;
|
||||||
|
|
||||||
public InMemoryKeyValueStore(final String name) {
|
public InMemoryKeyValueStore(final String name) {
|
||||||
this.name = name;
|
this.name = name;
|
||||||
|
this.position = Position.emptyPosition();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|
@ -62,6 +68,13 @@ public class InMemoryKeyValueStore implements KeyValueStore<Bytes, byte[]> {
|
||||||
open = true;
|
open = true;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void init(final StateStoreContext context,
|
||||||
|
final StateStore root) {
|
||||||
|
init(StoreToProcessorContextAdapter.adapt(context), root);
|
||||||
|
this.context = context;
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public boolean persistent() {
|
public boolean persistent() {
|
||||||
return false;
|
return false;
|
||||||
|
|
@ -72,6 +85,10 @@ public class InMemoryKeyValueStore implements KeyValueStore<Bytes, byte[]> {
|
||||||
return open;
|
return open;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
Position getPosition() {
|
||||||
|
return position;
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public synchronized byte[] get(final Bytes key) {
|
public synchronized byte[] get(final Bytes key) {
|
||||||
return map.get(key);
|
return map.get(key);
|
||||||
|
|
@ -98,6 +115,11 @@ public class InMemoryKeyValueStore implements KeyValueStore<Bytes, byte[]> {
|
||||||
} else {
|
} else {
|
||||||
map.put(key, value);
|
map.put(key, value);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
if (context != null && context.recordMetadata().isPresent()) {
|
||||||
|
final RecordMetadata meta = context.recordMetadata().get();
|
||||||
|
position = position.update(meta.topic(), meta.partition(), meta.offset());
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|
|
||||||
|
|
@ -0,0 +1,96 @@
|
||||||
|
/*
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||||
|
* contributor license agreements. See the NOTICE file distributed with
|
||||||
|
* this work for additional information regarding copyright ownership.
|
||||||
|
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||||
|
* (the "License"); you may not use this file except in compliance with
|
||||||
|
* the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
package org.apache.kafka.streams.state.internals;
|
||||||
|
|
||||||
|
|
||||||
|
import java.util.HashMap;
|
||||||
|
import java.util.Map;
|
||||||
|
import java.util.Map.Entry;
|
||||||
|
import java.util.Objects;
|
||||||
|
import java.util.concurrent.ConcurrentHashMap;
|
||||||
|
import java.util.concurrent.ConcurrentMap;
|
||||||
|
import java.util.concurrent.atomic.AtomicLong;
|
||||||
|
import java.util.function.BiConsumer;
|
||||||
|
|
||||||
|
public class Position {
|
||||||
|
private final ConcurrentMap<String, ConcurrentMap<Integer, AtomicLong>> position;
|
||||||
|
|
||||||
|
public static Position emptyPosition() {
|
||||||
|
final HashMap<String, Map<Integer, Long>> pos = new HashMap<>();
|
||||||
|
return new Position(pos);
|
||||||
|
}
|
||||||
|
|
||||||
|
public static Position fromMap(final Map<String, Map<Integer, Long>> map) {
|
||||||
|
return new Position(map);
|
||||||
|
}
|
||||||
|
|
||||||
|
private Position(final Map<String, Map<Integer, Long>> other) {
|
||||||
|
this.position = new ConcurrentHashMap<>();
|
||||||
|
merge(other, (t, e) -> update(t, e.getKey(), e.getValue().longValue()));
|
||||||
|
}
|
||||||
|
|
||||||
|
public Position update(final String topic, final int partition, final long offset) {
|
||||||
|
position.computeIfAbsent(topic, k -> new ConcurrentHashMap<>());
|
||||||
|
final ConcurrentMap<Integer, AtomicLong> topicMap = position.get(topic);
|
||||||
|
topicMap.computeIfAbsent(partition, k -> new AtomicLong(0));
|
||||||
|
topicMap.get(partition).getAndAccumulate(offset, Math::max);
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void merge(final Position other) {
|
||||||
|
merge(other.position, (a, b) -> update(a, b.getKey(), b.getValue().longValue()));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String toString() {
|
||||||
|
return "Position{" +
|
||||||
|
"position=" + position +
|
||||||
|
'}';
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean equals(final Object o) {
|
||||||
|
if (this == o) {
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
if (o == null || getClass() != o.getClass()) {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
final Position other = (Position) o;
|
||||||
|
final HashMap<String, HashMap<Integer, Long>> position1 = new HashMap<>();
|
||||||
|
merge(position, (t, e) -> position1.computeIfAbsent(t, k -> new HashMap<Integer, Long>()).put(e.getKey(), e.getValue().longValue()));
|
||||||
|
final HashMap<String, HashMap<Integer, Long>> position2 = new HashMap<>();
|
||||||
|
merge(other.position, (t, e) -> position2.computeIfAbsent(t, k -> new HashMap<Integer, Long>()).put(e.getKey(), e.getValue().longValue()));
|
||||||
|
|
||||||
|
return Objects.equals(position1, position2);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int hashCode() {
|
||||||
|
return Objects.hash(position);
|
||||||
|
}
|
||||||
|
|
||||||
|
private void merge(final Map<String, ? extends Map<Integer, ? extends Number>> other, final BiConsumer<String, Entry<Integer, ? extends Number>> func) {
|
||||||
|
for (final Entry<String, ? extends Map<Integer, ? extends Number>> entry : other.entrySet()) {
|
||||||
|
final String topic = entry.getKey();
|
||||||
|
final Map<Integer, ? extends Number> inputMap = entry.getValue();
|
||||||
|
for (final Entry<Integer, ? extends Number> topicEntry : inputMap.entrySet()) {
|
||||||
|
func.accept(topic, topicEntry);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
@ -28,6 +28,7 @@ import org.apache.kafka.streams.processor.BatchingStateRestoreCallback;
|
||||||
import org.apache.kafka.streams.processor.ProcessorContext;
|
import org.apache.kafka.streams.processor.ProcessorContext;
|
||||||
import org.apache.kafka.streams.processor.StateStore;
|
import org.apache.kafka.streams.processor.StateStore;
|
||||||
import org.apache.kafka.streams.processor.StateStoreContext;
|
import org.apache.kafka.streams.processor.StateStoreContext;
|
||||||
|
import org.apache.kafka.streams.processor.api.RecordMetadata;
|
||||||
import org.apache.kafka.streams.state.KeyValueIterator;
|
import org.apache.kafka.streams.state.KeyValueIterator;
|
||||||
import org.apache.kafka.streams.state.KeyValueStore;
|
import org.apache.kafka.streams.state.KeyValueStore;
|
||||||
import org.apache.kafka.streams.state.RocksDBConfigSetter;
|
import org.apache.kafka.streams.state.RocksDBConfigSetter;
|
||||||
|
|
@ -105,6 +106,8 @@ public class RocksDBStore implements KeyValueStore<Bytes, byte[]>, BatchWritingS
|
||||||
private final RocksDBMetricsRecorder metricsRecorder;
|
private final RocksDBMetricsRecorder metricsRecorder;
|
||||||
|
|
||||||
protected volatile boolean open = false;
|
protected volatile boolean open = false;
|
||||||
|
private StateStoreContext context;
|
||||||
|
private Position position;
|
||||||
|
|
||||||
RocksDBStore(final String name,
|
RocksDBStore(final String name,
|
||||||
final String metricsScope) {
|
final String metricsScope) {
|
||||||
|
|
@ -117,6 +120,7 @@ public class RocksDBStore implements KeyValueStore<Bytes, byte[]>, BatchWritingS
|
||||||
this.name = name;
|
this.name = name;
|
||||||
this.parentDir = parentDir;
|
this.parentDir = parentDir;
|
||||||
this.metricsRecorder = metricsRecorder;
|
this.metricsRecorder = metricsRecorder;
|
||||||
|
this.position = Position.emptyPosition();
|
||||||
}
|
}
|
||||||
|
|
||||||
@SuppressWarnings("unchecked")
|
@SuppressWarnings("unchecked")
|
||||||
|
|
@ -186,6 +190,10 @@ public class RocksDBStore implements KeyValueStore<Bytes, byte[]>, BatchWritingS
|
||||||
addValueProvidersToMetricsRecorder();
|
addValueProvidersToMetricsRecorder();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
Position getPosition() {
|
||||||
|
return position;
|
||||||
|
}
|
||||||
|
|
||||||
private void maybeSetUpStatistics(final Map<String, Object> configs) {
|
private void maybeSetUpStatistics(final Map<String, Object> configs) {
|
||||||
if (userSpecifiedOptions.statistics() != null) {
|
if (userSpecifiedOptions.statistics() != null) {
|
||||||
userSpecifiedStatistics = true;
|
userSpecifiedStatistics = true;
|
||||||
|
|
@ -252,6 +260,7 @@ public class RocksDBStore implements KeyValueStore<Bytes, byte[]>, BatchWritingS
|
||||||
// value getter should always read directly from rocksDB
|
// value getter should always read directly from rocksDB
|
||||||
// since it is only for values that are already flushed
|
// since it is only for values that are already flushed
|
||||||
context.register(root, new RocksDBBatchingRestoreCallback(this));
|
context.register(root, new RocksDBBatchingRestoreCallback(this));
|
||||||
|
this.context = context;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|
@ -281,6 +290,11 @@ public class RocksDBStore implements KeyValueStore<Bytes, byte[]>, BatchWritingS
|
||||||
Objects.requireNonNull(key, "key cannot be null");
|
Objects.requireNonNull(key, "key cannot be null");
|
||||||
validateStoreOpen();
|
validateStoreOpen();
|
||||||
dbAccessor.put(key.get(), value);
|
dbAccessor.put(key.get(), value);
|
||||||
|
|
||||||
|
if (context != null && context.recordMetadata().isPresent()) {
|
||||||
|
final RecordMetadata meta = context.recordMetadata().get();
|
||||||
|
position = position.update(meta.topic(), meta.partition(), meta.offset());
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|
|
||||||
|
|
@ -47,6 +47,7 @@ public class InMemoryKeyValueStoreTest extends AbstractKeyValueStoreTest {
|
||||||
private KeyValueStore<Bytes, byte[]> byteStore;
|
private KeyValueStore<Bytes, byte[]> byteStore;
|
||||||
private final Serializer<String> stringSerializer = new StringSerializer();
|
private final Serializer<String> stringSerializer = new StringSerializer();
|
||||||
private final KeyValueStoreTestDriver<Bytes, byte[]> byteStoreDriver = KeyValueStoreTestDriver.create(Bytes.class, byte[].class);
|
private final KeyValueStoreTestDriver<Bytes, byte[]> byteStoreDriver = KeyValueStoreTestDriver.create(Bytes.class, byte[].class);
|
||||||
|
private InMemoryKeyValueStore inMemoryKeyValueStore;
|
||||||
|
|
||||||
@Before
|
@Before
|
||||||
public void createStringKeyValueStore() {
|
public void createStringKeyValueStore() {
|
||||||
|
|
@ -58,6 +59,7 @@ public class InMemoryKeyValueStoreTest extends AbstractKeyValueStoreTest {
|
||||||
new Serdes.ByteArraySerde());
|
new Serdes.ByteArraySerde());
|
||||||
byteStore = storeBuilder.build();
|
byteStore = storeBuilder.build();
|
||||||
byteStore.init(byteStoreContext, byteStore);
|
byteStore.init(byteStoreContext, byteStore);
|
||||||
|
this.inMemoryKeyValueStore = getInMemoryStore();
|
||||||
}
|
}
|
||||||
|
|
||||||
@After
|
@After
|
||||||
|
|
@ -80,6 +82,10 @@ public class InMemoryKeyValueStoreTest extends AbstractKeyValueStoreTest {
|
||||||
return store;
|
return store;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
InMemoryKeyValueStore getInMemoryStore() {
|
||||||
|
return new InMemoryKeyValueStore("in-memory-store-test");
|
||||||
|
}
|
||||||
|
|
||||||
@SuppressWarnings("unchecked")
|
@SuppressWarnings("unchecked")
|
||||||
@Test
|
@Test
|
||||||
public void shouldRemoveKeysWithNullValues() {
|
public void shouldRemoveKeysWithNullValues() {
|
||||||
|
|
@ -237,4 +243,33 @@ public class InMemoryKeyValueStoreTest extends AbstractKeyValueStoreTest {
|
||||||
public void shouldThrowNullPointerIfPrefixKeySerializerIsNull() {
|
public void shouldThrowNullPointerIfPrefixKeySerializerIsNull() {
|
||||||
assertThrows(NullPointerException.class, () -> byteStore.prefixScan("bb", null));
|
assertThrows(NullPointerException.class, () -> byteStore.prefixScan("bb", null));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void shouldMatchPositionAfterPut() {
|
||||||
|
final List<KeyValue<Bytes, byte[]>> entries = new ArrayList<>();
|
||||||
|
entries.add(new KeyValue<>(
|
||||||
|
new Bytes(stringSerializer.serialize(null, "1")),
|
||||||
|
stringSerializer.serialize(null, "a")));
|
||||||
|
entries.add(new KeyValue<>(
|
||||||
|
new Bytes(stringSerializer.serialize(null, "2")),
|
||||||
|
stringSerializer.serialize(null, "b")));
|
||||||
|
entries.add(new KeyValue<>(
|
||||||
|
new Bytes(stringSerializer.serialize(null, "3")),
|
||||||
|
stringSerializer.serialize(null, "c")));
|
||||||
|
|
||||||
|
final MonotonicProcessorRecordContext recordContext = new MonotonicProcessorRecordContext("input", 0);
|
||||||
|
context.setRecordContext(recordContext);
|
||||||
|
inMemoryKeyValueStore.init((StateStoreContext) context, inMemoryKeyValueStore);
|
||||||
|
|
||||||
|
final Position expected = Position.emptyPosition();
|
||||||
|
long offset = 0;
|
||||||
|
for (final KeyValue<Bytes, byte[]> k : entries) {
|
||||||
|
inMemoryKeyValueStore.put(k.key, k.value);
|
||||||
|
expected.update("input", 0, offset);
|
||||||
|
offset++;
|
||||||
|
}
|
||||||
|
|
||||||
|
final Position actual = inMemoryKeyValueStore.getPosition();
|
||||||
|
assertThat(expected, is(actual));
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -0,0 +1,37 @@
|
||||||
|
/*
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||||
|
* contributor license agreements. See the NOTICE file distributed with
|
||||||
|
* this work for additional information regarding copyright ownership.
|
||||||
|
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||||
|
* (the "License"); you may not use this file except in compliance with
|
||||||
|
* the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.apache.kafka.streams.state.internals;
|
||||||
|
|
||||||
|
import org.apache.kafka.common.header.internals.RecordHeaders;
|
||||||
|
import org.apache.kafka.streams.processor.internals.ProcessorRecordContext;
|
||||||
|
|
||||||
|
public class MonotonicProcessorRecordContext extends ProcessorRecordContext {
|
||||||
|
private long counter;
|
||||||
|
|
||||||
|
public MonotonicProcessorRecordContext(final String topic, final int partition) {
|
||||||
|
super(0, 0, partition, topic, new RecordHeaders());
|
||||||
|
this.counter = 0;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public long offset() {
|
||||||
|
final long ret = counter;
|
||||||
|
counter++;
|
||||||
|
return ret;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
@ -0,0 +1,66 @@
|
||||||
|
/*
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||||
|
* contributor license agreements. See the NOTICE file distributed with
|
||||||
|
* this work for additional information regarding copyright ownership.
|
||||||
|
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||||
|
* (the "License"); you may not use this file except in compliance with
|
||||||
|
* the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.apache.kafka.streams.state.internals;
|
||||||
|
|
||||||
|
import static org.junit.Assert.assertEquals;
|
||||||
|
import static org.junit.Assert.assertNotEquals;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import org.junit.Test;
|
||||||
|
|
||||||
|
public class PositionTest {
|
||||||
|
|
||||||
|
private final String topic = "topic";
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void shouldMatchOnEqual() throws IOException {
|
||||||
|
final Position position1 = Position.emptyPosition();
|
||||||
|
final Position position2 = Position.emptyPosition();
|
||||||
|
position1.update("topic1", 0, 1);
|
||||||
|
position2.update("topic1", 0, 1);
|
||||||
|
|
||||||
|
position1.update("topic1", 1, 2);
|
||||||
|
position2.update("topic1", 1, 2);
|
||||||
|
|
||||||
|
position1.update("topic1", 2, 1);
|
||||||
|
position2.update("topic1", 2, 1);
|
||||||
|
|
||||||
|
position1.update("topic2", 0, 0);
|
||||||
|
position2.update("topic2", 0, 0);
|
||||||
|
|
||||||
|
assertEquals(position1, position2);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void shouldNotMatchOnUnEqual() throws IOException {
|
||||||
|
final Position position1 = Position.emptyPosition();
|
||||||
|
final Position position2 = Position.emptyPosition();
|
||||||
|
position1.update("topic1", 0, 1);
|
||||||
|
position2.update("topic1", 0, 1);
|
||||||
|
|
||||||
|
position1.update("topic1", 1, 2);
|
||||||
|
|
||||||
|
position1.update("topic1", 2, 1);
|
||||||
|
position2.update("topic1", 2, 1);
|
||||||
|
|
||||||
|
position1.update("topic2", 0, 0);
|
||||||
|
position2.update("topic2", 0, 0);
|
||||||
|
|
||||||
|
assertNotEquals(position1, position2);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
@ -16,6 +16,7 @@
|
||||||
*/
|
*/
|
||||||
package org.apache.kafka.streams.state.internals;
|
package org.apache.kafka.streams.state.internals;
|
||||||
|
|
||||||
|
import java.util.Optional;
|
||||||
import org.apache.kafka.common.Metric;
|
import org.apache.kafka.common.Metric;
|
||||||
import org.apache.kafka.common.MetricName;
|
import org.apache.kafka.common.MetricName;
|
||||||
import org.apache.kafka.common.metrics.MetricConfig;
|
import org.apache.kafka.common.metrics.MetricConfig;
|
||||||
|
|
@ -110,6 +111,7 @@ public class RocksDBStoreTest extends AbstractKeyValueStoreTest {
|
||||||
|
|
||||||
InternalMockProcessorContext context;
|
InternalMockProcessorContext context;
|
||||||
RocksDBStore rocksDBStore;
|
RocksDBStore rocksDBStore;
|
||||||
|
Position position;
|
||||||
|
|
||||||
@Before
|
@Before
|
||||||
public void setUp() {
|
public void setUp() {
|
||||||
|
|
@ -123,6 +125,7 @@ public class RocksDBStoreTest extends AbstractKeyValueStoreTest {
|
||||||
new StreamsConfig(props)
|
new StreamsConfig(props)
|
||||||
);
|
);
|
||||||
rocksDBStore = getRocksDBStore();
|
rocksDBStore = getRocksDBStore();
|
||||||
|
position = rocksDBStore.getPosition();
|
||||||
}
|
}
|
||||||
|
|
||||||
@After
|
@After
|
||||||
|
|
@ -383,6 +386,35 @@ public class RocksDBStoreTest extends AbstractKeyValueStoreTest {
|
||||||
rocksDBStore.get(new Bytes(stringSerializer.serialize(null, "3")))));
|
rocksDBStore.get(new Bytes(stringSerializer.serialize(null, "3")))));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void shouldMatchPositionAfterPut() {
|
||||||
|
final List<KeyValue<Bytes, byte[]>> entries = new ArrayList<>();
|
||||||
|
entries.add(new KeyValue<>(
|
||||||
|
new Bytes(stringSerializer.serialize(null, "1")),
|
||||||
|
stringSerializer.serialize(null, "a")));
|
||||||
|
entries.add(new KeyValue<>(
|
||||||
|
new Bytes(stringSerializer.serialize(null, "2")),
|
||||||
|
stringSerializer.serialize(null, "b")));
|
||||||
|
entries.add(new KeyValue<>(
|
||||||
|
new Bytes(stringSerializer.serialize(null, "3")),
|
||||||
|
stringSerializer.serialize(null, "c")));
|
||||||
|
|
||||||
|
final MonotonicProcessorRecordContext recordContext = new MonotonicProcessorRecordContext("input", 0);
|
||||||
|
context.setRecordContext(recordContext);
|
||||||
|
rocksDBStore.init((StateStoreContext) context, rocksDBStore);
|
||||||
|
|
||||||
|
final Position expected = Position.emptyPosition();
|
||||||
|
long offset = 0;
|
||||||
|
for (final KeyValue<Bytes, byte[]> k : entries) {
|
||||||
|
rocksDBStore.put(k.key, k.value);
|
||||||
|
expected.update("input", 0, offset);
|
||||||
|
offset++;
|
||||||
|
}
|
||||||
|
|
||||||
|
final Position actual = rocksDBStore.getPosition();
|
||||||
|
assertEquals(expected, actual);
|
||||||
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void shouldReturnKeysWithGivenPrefix() {
|
public void shouldReturnKeysWithGivenPrefix() {
|
||||||
final List<KeyValue<Bytes, byte[]>> entries = new ArrayList<>();
|
final List<KeyValue<Bytes, byte[]>> entries = new ArrayList<>();
|
||||||
|
|
@ -786,6 +818,8 @@ public class RocksDBStoreTest extends AbstractKeyValueStoreTest {
|
||||||
EasyMock.expect(context.appConfigs())
|
EasyMock.expect(context.appConfigs())
|
||||||
.andStubReturn(new StreamsConfig(StreamsTestUtils.getStreamsConfig()).originals());
|
.andStubReturn(new StreamsConfig(StreamsTestUtils.getStreamsConfig()).originals());
|
||||||
EasyMock.expect(context.stateDir()).andStubReturn(dir);
|
EasyMock.expect(context.stateDir()).andStubReturn(dir);
|
||||||
|
final MonotonicProcessorRecordContext processorRecordContext = new MonotonicProcessorRecordContext("test", 0);
|
||||||
|
EasyMock.expect(context.recordMetadata()).andStubReturn(Optional.of(processorRecordContext));
|
||||||
EasyMock.replay(context);
|
EasyMock.replay(context);
|
||||||
|
|
||||||
rocksDBStore.init((StateStoreContext) context, rocksDBStore);
|
rocksDBStore.init((StateStoreContext) context, rocksDBStore);
|
||||||
|
|
@ -818,6 +852,8 @@ public class RocksDBStoreTest extends AbstractKeyValueStoreTest {
|
||||||
EasyMock.expect(context.appConfigs())
|
EasyMock.expect(context.appConfigs())
|
||||||
.andStubReturn(new StreamsConfig(StreamsTestUtils.getStreamsConfig()).originals());
|
.andStubReturn(new StreamsConfig(StreamsTestUtils.getStreamsConfig()).originals());
|
||||||
EasyMock.expect(context.stateDir()).andStubReturn(dir);
|
EasyMock.expect(context.stateDir()).andStubReturn(dir);
|
||||||
|
final MonotonicProcessorRecordContext processorRecordContext = new MonotonicProcessorRecordContext("test", 0);
|
||||||
|
EasyMock.expect(context.recordMetadata()).andStubReturn(Optional.of(processorRecordContext));
|
||||||
EasyMock.replay(context);
|
EasyMock.replay(context);
|
||||||
|
|
||||||
rocksDBStore.init((StateStoreContext) context, rocksDBStore);
|
rocksDBStore.init((StateStoreContext) context, rocksDBStore);
|
||||||
|
|
@ -981,4 +1017,5 @@ public class RocksDBStoreTest extends AbstractKeyValueStoreTest {
|
||||||
return result;
|
return result;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
||||||
Loading…
Reference in New Issue