diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/ForwardingCacheFlushListener.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/ForwardingCacheFlushListener.java index 57c5a1a7bb178..6c6b8fd33b8ae 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/ForwardingCacheFlushListener.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/ForwardingCacheFlushListener.java @@ -39,7 +39,7 @@ public void apply(final K key, final ProcessorNode prev = context.currentNode(); context.setCurrentNode(myNode); try { - context.forward(key, new Change<>(newValue, oldValue), To.all().withTimestamp(timestamp)); + context.forward(key, new Change<>(newValue, oldValue), To.all().withTimestamp(timestamp)); } finally { context.setCurrentNode(prev); } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KGroupedStreamImpl.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KGroupedStreamImpl.java index eab1e8f7ca3da..285e77e4bb589 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KGroupedStreamImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KGroupedStreamImpl.java @@ -185,10 +185,11 @@ private KTable doAggregate(final KStreamAggProcessorSupplier> materializedInternal) { return aggregateBuilder.build( functionName, - new KeyValueStoreMaterializer<>(materializedInternal).materialize(), + new KeyValueWithTimestampStoreMaterializer<>(materializedInternal).materialize(), aggregateSupplier, materializedInternal.queryableStoreName(), materializedInternal.keySerde(), - materializedInternal.valueSerde()); + materializedInternal.valueSerde() + ); } } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KGroupedTableImpl.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KGroupedTableImpl.java index 2eca84e9caa79..42f7288d16e50 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KGroupedTableImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KGroupedTableImpl.java @@ -82,7 +82,7 @@ private KTable doAggregate(final ProcessorSupplier> aggre final StatefulProcessorNode statefulProcessorNode = new StatefulProcessorNode<>( funcName, new ProcessorParameters<>(aggregateSupplier, funcName), - new KeyValueStoreMaterializer<>(materialized).materialize(), + new KeyValueWithTimestampStoreMaterializer<>(materialized).materialize(), false ); diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamAggregate.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamAggregate.java index 648c50b7e50cf..8afe64f8959ee 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamAggregate.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamAggregate.java @@ -21,8 +21,12 @@ import org.apache.kafka.streams.processor.AbstractProcessor; import org.apache.kafka.streams.processor.Processor; import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.processor.StateStore; import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; import org.apache.kafka.streams.state.KeyValueStore; +import org.apache.kafka.streams.state.ReadOnlyKeyValueStore; +import org.apache.kafka.streams.state.ValueAndTimestamp; +import org.apache.kafka.streams.state.internals.WrappedStateStore; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -53,7 +57,7 @@ public void enableSendingOldValues() { private class KStreamAggregateProcessor extends AbstractProcessor { - private KeyValueStore store; + private KeyValueStore> store; private StreamsMetricsImpl metrics; private TupleForwarder tupleForwarder; @@ -62,11 +66,14 @@ private class KStreamAggregateProcessor extends AbstractProcessor { public void init(final ProcessorContext context) { super.init(context); metrics = (StreamsMetricsImpl) context.metrics(); - store = (KeyValueStore) context.getStateStore(storeName); + StateStore store = context.getStateStore(storeName); + if (store instanceof WrappedStateStore) { + store = ((WrappedStateStore) store).wrappedStore(); + } + this.store = ((KeyValueWithTimestampStoreMaterializer.KeyValueStoreFacade) store).inner; tupleForwarder = new TupleForwarder<>(store, context, new ForwardingCacheFlushListener(context), sendOldValues); } - @Override public void process(final K key, final V value) { // If the key or value is null we don't need to proceed @@ -79,23 +86,28 @@ key, value, context().topic(), context().partition(), context().offset() return; } - T oldAgg = store.get(key); + final ValueAndTimestamp oldAggWithTimestamp = store.get(key); + final T oldAgg; + final long oldTimestamp; - if (oldAgg == null) { + if (oldAggWithTimestamp == null) { oldAgg = initializer.apply(); + oldTimestamp = -1; + } else { + oldAgg = oldAggWithTimestamp.value(); + oldTimestamp = oldAggWithTimestamp.timestamp(); } - T newAgg = oldAgg; - - // try to add the new value - newAgg = aggregator.apply(key, value, newAgg); + final T newAgg = aggregator.apply(key, value, oldAgg); + final long newTimestamp = Math.max(oldTimestamp, context().timestamp()); // update the store with the new value - store.put(key, newAgg); - tupleForwarder.maybeForward(key, newAgg, sendOldValues ? oldAgg : null); + store.put(key, ValueAndTimestamp.make(newAgg, newTimestamp)); + tupleForwarder.maybeForward(key, newAgg, sendOldValues ? oldAgg : null, newTimestamp); } } + @Override public KTableValueGetterSupplier view() { @@ -114,16 +126,16 @@ public String[] storeNames() { private class KStreamAggregateValueGetter implements KTableValueGetter { - private KeyValueStore store; + private ReadOnlyKeyValueStore> store; @SuppressWarnings("unchecked") @Override public void init(final ProcessorContext context) { - store = (KeyValueStore) context.getStateStore(storeName); + store = (ReadOnlyKeyValueStore>) context.getStateStore(storeName); } @Override - public T get(final K key) { + public ValueAndTimestamp get(final K key) { return store.get(key); } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java index ed5625e3a17f4..e4bca10202631 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java @@ -16,7 +16,9 @@ */ package org.apache.kafka.streams.kstream.internals; +import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.common.serialization.Serde; +import org.apache.kafka.common.utils.Time; import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.kstream.ForeachAction; import org.apache.kafka.streams.kstream.GlobalKTable; @@ -36,6 +38,7 @@ import org.apache.kafka.streams.kstream.ValueMapperWithKey; import org.apache.kafka.streams.kstream.ValueTransformerSupplier; import org.apache.kafka.streams.kstream.ValueTransformerWithKeySupplier; +import org.apache.kafka.streams.kstream.Windowed; import org.apache.kafka.streams.kstream.internals.graph.OptimizableRepartitionNode; import org.apache.kafka.streams.kstream.internals.graph.OptimizableRepartitionNode.OptimizableRepartitionNodeBuilder; import org.apache.kafka.streams.kstream.internals.graph.ProcessorGraphNode; @@ -46,18 +49,31 @@ import org.apache.kafka.streams.kstream.internals.graph.StreamTableJoinNode; import org.apache.kafka.streams.kstream.internals.graph.StreamsGraphNode; import org.apache.kafka.streams.processor.FailOnInvalidTimestamp; +import org.apache.kafka.streams.processor.ProcessorContext; import org.apache.kafka.streams.processor.ProcessorSupplier; +import org.apache.kafka.streams.processor.StateStore; import org.apache.kafka.streams.processor.TopicNameExtractor; +import org.apache.kafka.streams.processor.internals.DefaultRecordConverter; import org.apache.kafka.streams.processor.internals.StaticTopicNameExtractor; +import org.apache.kafka.streams.state.KeyValueIterator; +import org.apache.kafka.streams.state.RecordConverter; import org.apache.kafka.streams.state.StoreBuilder; import org.apache.kafka.streams.state.Stores; +import org.apache.kafka.streams.state.ValueAndTimestamp; +import org.apache.kafka.streams.state.WindowBytesStoreSupplier; import org.apache.kafka.streams.state.WindowStore; +import org.apache.kafka.streams.state.WindowStoreIterator; +import org.apache.kafka.streams.state.internals.KeyValueIteratorFacade; +import org.apache.kafka.streams.state.internals.WindowStoreBuilder; +import org.apache.kafka.streams.state.internals.WrappedStateStore; import java.lang.reflect.Array; import java.time.Duration; +import java.time.Instant; import java.util.Arrays; import java.util.Collections; import java.util.HashSet; +import java.util.Map; import java.util.Objects; import java.util.Set; @@ -854,16 +870,216 @@ private static StoreBuilder> joinWindowStoreBuilder(fin final JoinWindows windows, final Serde keySerde, final Serde valueSerde) { - return Stores.windowStoreBuilder( - Stores.persistentWindowStore( - joinName + "-store", - Duration.ofMillis(windows.size() + windows.gracePeriodMs()), - Duration.ofMillis(windows.size()), - true - ), - keySerde, - valueSerde + final WindowBytesStoreSupplier supplier = Stores.persistentWindowWithTimestampStore( + joinName + "-store", + Duration.ofMillis(windows.size() + windows.gracePeriodMs()), + Duration.ofMillis(windows.size()), + true ); + return new WindowStoreBuilder(supplier, null, null, Time.SYSTEM) { + final StoreBuilder>> inner = Stores.windowWithTimestampStoreBuilder( + supplier, + keySerde, + valueSerde + ); + + @Override + public WindowStoreBuilder withCachingEnabled() { + inner.withCachingEnabled(); + return this; + } + + @Override + public WindowStoreBuilder withCachingDisabled() { + inner.withCachingDisabled(); + return this; + } + + @Override + public WindowStoreBuilder withLoggingEnabled(final Map config) { + inner.withLoggingEnabled(config); + return this; + } + + @Override + public WindowStoreBuilder withLoggingDisabled() { + inner.withLoggingDisabled(); + return this; + } + + @Override + public Map logConfig() { + return inner.logConfig(); + } + + @Override + public boolean loggingEnabled() { + return inner.loggingEnabled(); + } + + @Override + public String name() { + return inner.name(); + } + + @Override + public WindowStore build() { + return new WindowStoreFacade<>(inner.build()); + } + }; + } + + public static class WindowStoreFacade implements WindowStore, RecordConverter { + public final WindowStore> inner; + private final RecordConverter innerRecordConvert; + + public WindowStoreFacade(final WindowStore> store) { + this.inner = store; + final StateStore rootStore = store instanceof WrappedStateStore ? ((WrappedStateStore) store).inner() : store; + innerRecordConvert = rootStore instanceof RecordConverter ? (RecordConverter) rootStore : new DefaultRecordConverter(); + } + + @Override + public void init(final ProcessorContext context, + final StateStore root) { + inner.init(context, root); + } + + @Override + public void put(final A key, + final B value) { + inner.put(key, ValueAndTimestamp.make(value, -1L)); + } + + @Override + public void put(final A key, + final B value, + final long windowStartTimestamp) { + inner.put(key, ValueAndTimestamp.make(value, -1L), windowStartTimestamp); + } + + @Override + public B fetch(final A key, + final long time) { + final ValueAndTimestamp valueAndTimestamp = inner.fetch(key, time); + return valueAndTimestamp == null ? null : valueAndTimestamp.value(); + } + + @Override + @SuppressWarnings("deprecation") + public WindowStoreIterator fetch(final A key, + final long timeFrom, + final long timeTo) { + final KeyValueIterator> innerIterator = inner.fetch(key, timeFrom, timeTo); + return new WindowStoreIteratorFacade(innerIterator); + } + + @Override + public WindowStoreIterator fetch(final A key, + final Instant from, + final Instant to) throws IllegalArgumentException { + final KeyValueIterator> innerIterator = inner.fetch(key, from, to); + return new WindowStoreIteratorFacade(innerIterator); + } + + @Override + @SuppressWarnings("deprecation") + public KeyValueIterator, B> fetch(final A from, + final A to, + final long timeFrom, + final long timeTo) { + final KeyValueIterator, ValueAndTimestamp> innerIterator = inner.fetch(from, to, timeFrom, timeTo); + return new KeyValueIteratorFacade<>(innerIterator); + } + + @Override + public KeyValueIterator, B> fetch(final A from, + final A to, + final Instant fromTime, + final Instant toTime) throws IllegalArgumentException { + final KeyValueIterator, ValueAndTimestamp> innerIterator = inner.fetch(from, to, fromTime, toTime); + return new KeyValueIteratorFacade<>(innerIterator); + } + + @Override + @SuppressWarnings("deprecation") + public KeyValueIterator, B> fetchAll(final long timeFrom, + final long timeTo) { + final KeyValueIterator, ValueAndTimestamp> innerIterator = inner.fetchAll(timeFrom, timeTo); + return new KeyValueIteratorFacade<>(innerIterator); + } + + @Override + public KeyValueIterator, B> fetchAll(final Instant from, + final Instant to) throws IllegalArgumentException { + final KeyValueIterator, ValueAndTimestamp> innerIterator = inner.fetchAll(from, to); + return new KeyValueIteratorFacade<>(innerIterator); + } + + @Override + public KeyValueIterator, B> all() { + final KeyValueIterator, ValueAndTimestamp> innerIterator = inner.all(); + return new KeyValueIteratorFacade<>(innerIterator); + } + + @Override + public void flush() { + inner.flush(); + } + + @Override + public void close() { + inner.close(); + } + + @Override + public boolean isOpen() { + return inner.isOpen(); + } + + @Override + public String name() { + return inner.name(); + } + + @Override + public boolean persistent() { + return inner.persistent(); + } + + @Override + public ConsumerRecord convert(final ConsumerRecord record) { + return innerRecordConvert.convert(record); + } + + private class WindowStoreIteratorFacade implements WindowStoreIterator { + final KeyValueIterator> innerIterator; + + WindowStoreIteratorFacade(final KeyValueIterator> iterator) { + innerIterator = iterator; + } + + @Override + public void close() { + innerIterator.close(); + } + + @Override + public Long peekNextKey() { + return innerIterator.peekNextKey(); + } + + @Override + public boolean hasNext() { + return innerIterator.hasNext(); + } + + @Override + public KeyValue next() { + final KeyValue> innerKeyValue = innerIterator.next(); + return KeyValue.pair(innerKeyValue.key, innerKeyValue.value.value()); + } + } } private class KStreamImplJoin { diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamJoinWindow.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamJoinWindow.java index 34756d47cf2ee..8ec66b5573478 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamJoinWindow.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamJoinWindow.java @@ -20,7 +20,10 @@ import org.apache.kafka.streams.processor.Processor; import org.apache.kafka.streams.processor.ProcessorContext; import org.apache.kafka.streams.processor.ProcessorSupplier; +import org.apache.kafka.streams.processor.StateStore; +import org.apache.kafka.streams.state.ValueAndTimestamp; import org.apache.kafka.streams.state.WindowStore; +import org.apache.kafka.streams.state.internals.WrappedStateStore; class KStreamJoinWindow implements ProcessorSupplier { @@ -37,14 +40,18 @@ public Processor get() { private class KStreamJoinWindowProcessor extends AbstractProcessor { - private WindowStore window; + private WindowStore> window; @SuppressWarnings("unchecked") @Override public void init(final ProcessorContext context) { super.init(context); - window = (WindowStore) context.getStateStore(windowName); + StateStore store = context.getStateStore(windowName); + if (store instanceof WrappedStateStore) { + store = ((WrappedStateStore) store).wrappedStore(); + } + window = ((KStreamImpl.WindowStoreFacade) store).inner; } @Override @@ -54,7 +61,7 @@ public void process(final K key, final V value) { if (key != null) { context().forward(key, value); // Every record basically starts a new window. We're using a window store mostly for the retention. - window.put(key, value, context().timestamp()); + window.put(key, ValueAndTimestamp.make(value, context().timestamp()), context().timestamp()); } } } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoin.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoin.java index 4c6998ad85535..b5473609680a2 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoin.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoin.java @@ -16,14 +16,19 @@ */ package org.apache.kafka.streams.kstream.internals; +import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.kstream.ValueJoiner; import org.apache.kafka.streams.processor.AbstractProcessor; import org.apache.kafka.streams.processor.Processor; import org.apache.kafka.streams.processor.ProcessorContext; import org.apache.kafka.streams.processor.ProcessorSupplier; +import org.apache.kafka.streams.processor.StateStore; +import org.apache.kafka.streams.processor.To; import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; +import org.apache.kafka.streams.state.ValueAndTimestamp; import org.apache.kafka.streams.state.WindowStore; import org.apache.kafka.streams.state.WindowStoreIterator; +import org.apache.kafka.streams.state.internals.WrappedStateStore; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -53,7 +58,7 @@ public Processor get() { private class KStreamKStreamJoinProcessor extends AbstractProcessor { - private WindowStore otherWindow; + private WindowStore> otherWindow; private StreamsMetricsImpl metrics; @SuppressWarnings("unchecked") @@ -62,7 +67,11 @@ public void init(final ProcessorContext context) { super.init(context); metrics = (StreamsMetricsImpl) context.metrics(); - otherWindow = (WindowStore) context.getStateStore(otherWindowName); + StateStore store = context.getStateStore(otherWindowName); + if (store instanceof WrappedStateStore) { + store = ((WrappedStateStore) store).wrappedStore(); + } + otherWindow = ((KStreamImpl.WindowStoreFacade) store).inner; } @@ -88,10 +97,12 @@ key, value, context().topic(), context().partition(), context().offset() final long timeFrom = Math.max(0L, context().timestamp() - joinBeforeMs); final long timeTo = Math.max(0L, context().timestamp() + joinAfterMs); - try (final WindowStoreIterator iter = otherWindow.fetch(key, timeFrom, timeTo)) { + try (final WindowStoreIterator> iter = otherWindow.fetch(key, timeFrom, timeTo)) { while (iter.hasNext()) { needOuterJoin = false; - context().forward(key, joiner.apply(value, iter.next().value)); + final KeyValue> other = iter.next(); + final long resultTimestamp = Math.max(context().timestamp(), other.value.timestamp()); + context().forward(key, joiner.apply(value, other.value.value()), To.all().withTimestamp(resultTimestamp)); } if (needOuterJoin) { diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKTableJoinProcessor.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKTableJoinProcessor.java index 70b9bade2dc8f..0cb135346d5b7 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKTableJoinProcessor.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKTableJoinProcessor.java @@ -21,6 +21,7 @@ import org.apache.kafka.streams.processor.AbstractProcessor; import org.apache.kafka.streams.processor.ProcessorContext; import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; +import org.apache.kafka.streams.state.ValueAndTimestamp; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -68,7 +69,13 @@ key, value, context().topic(), context().partition(), context().offset() metrics.skippedRecordsSensor().record(); } else { final K2 mappedKey = keyMapper.apply(key, value); - final V2 value2 = mappedKey == null ? null : valueGetter.get(mappedKey); + final V2 value2; + if (mappedKey != null) { + final ValueAndTimestamp tableValueAndTimestamp = valueGetter.get(mappedKey); + value2 = tableValueAndTimestamp == null ? null : tableValueAndTimestamp.value(); + } else { + value2 = null; + } if (leftJoin || value2 != null) { context().forward(key, joiner.apply(value, value2)); } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamReduce.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamReduce.java index 09e4fab48e557..da110336ca827 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamReduce.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamReduce.java @@ -20,8 +20,12 @@ import org.apache.kafka.streams.processor.AbstractProcessor; import org.apache.kafka.streams.processor.Processor; import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.processor.StateStore; import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; import org.apache.kafka.streams.state.KeyValueStore; +import org.apache.kafka.streams.state.ReadOnlyKeyValueStore; +import org.apache.kafka.streams.state.ValueAndTimestamp; +import org.apache.kafka.streams.state.internals.WrappedStateStore; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -50,7 +54,7 @@ public void enableSendingOldValues() { private class KStreamReduceProcessor extends AbstractProcessor { - private KeyValueStore store; + private KeyValueStore> store; private TupleForwarder tupleForwarder; private StreamsMetricsImpl metrics; @@ -60,7 +64,11 @@ public void init(final ProcessorContext context) { super.init(context); metrics = (StreamsMetricsImpl) context.metrics(); - store = (KeyValueStore) context.getStateStore(storeName); + StateStore store = context.getStateStore(storeName); + if (store instanceof WrappedStateStore) { + store = ((WrappedStateStore) store).wrappedStore(); + } + this.store = ((KeyValueWithTimestampStoreMaterializer.KeyValueStoreFacade) store).inner; tupleForwarder = new TupleForwarder<>(store, context, new ForwardingCacheFlushListener(context), sendOldValues); } @@ -77,19 +85,29 @@ key, value, context().topic(), context().partition(), context().offset() return; } - final V oldAgg = store.get(key); - V newAgg = oldAgg; + final ValueAndTimestamp oldAggWithTimestamp = store.get(key); + + final V oldAgg; + final V newAgg; + long resultTimestamp = context().timestamp(); + + if (oldAggWithTimestamp != null) { + oldAgg = oldAggWithTimestamp.value(); + resultTimestamp = Math.max(resultTimestamp, oldAggWithTimestamp.timestamp()); + } else { + oldAgg = null; + } // try to add the new value - if (newAgg == null) { + if (oldAgg == null) { newAgg = value; } else { - newAgg = reducer.apply(newAgg, value); + newAgg = reducer.apply(oldAgg, value); } // update the store with the new value - store.put(key, newAgg); - tupleForwarder.maybeForward(key, newAgg, sendOldValues ? oldAgg : null); + store.put(key, ValueAndTimestamp.make(newAgg, resultTimestamp)); + tupleForwarder.maybeForward(key, newAgg, sendOldValues ? oldAgg : null, resultTimestamp); } } @@ -111,16 +129,16 @@ public String[] storeNames() { private class KStreamReduceValueGetter implements KTableValueGetter { - private KeyValueStore store; + private ReadOnlyKeyValueStore> store; @SuppressWarnings("unchecked") @Override public void init(final ProcessorContext context) { - store = (KeyValueStore) context.getStateStore(storeName); + store = (ReadOnlyKeyValueStore>) context.getStateStore(storeName); } @Override - public V get(final K key) { + public ValueAndTimestamp get(final K key) { return store.get(key); } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSessionWindowAggregate.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSessionWindowAggregate.java index 13f4a6eb4092a..1937f54f322a0 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSessionWindowAggregate.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSessionWindowAggregate.java @@ -28,10 +28,13 @@ import org.apache.kafka.streams.processor.AbstractProcessor; import org.apache.kafka.streams.processor.Processor; import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.processor.StateStore; import org.apache.kafka.streams.processor.internals.InternalProcessorContext; import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; import org.apache.kafka.streams.state.KeyValueIterator; import org.apache.kafka.streams.state.SessionStore; +import org.apache.kafka.streams.state.ValueAndTimestamp; +import org.apache.kafka.streams.state.internals.WrappedStateStore; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -77,7 +80,7 @@ public void enableSendingOldValues() { private class KStreamSessionWindowAggregateProcessor extends AbstractProcessor { - private SessionStore store; + private SessionStore> store; private TupleForwarder, Agg> tupleForwarder; private StreamsMetricsImpl metrics; private InternalProcessorContext internalProcessorContext; @@ -91,7 +94,11 @@ public void init(final ProcessorContext context) { metrics = (StreamsMetricsImpl) context.metrics(); lateRecordDropSensor = Sensors.lateRecordDropSensor(internalProcessorContext); - store = (SessionStore) context.getStateStore(storeName); + StateStore store = context.getStateStore(storeName); + if (store instanceof WrappedStateStore) { + store = ((WrappedStateStore) store).wrappedStore(); + } + this.store = ((SessionWindowedKStreamImpl.SessionStoreFacade) store).inner; tupleForwarder = new TupleForwarder<>(store, context, new ForwardingCacheFlushListener(context), sendOldValues); } @@ -111,38 +118,44 @@ value, context().topic(), context().partition(), context().offset() final long closeTime = internalProcessorContext.streamTime() - windows.gracePeriodMs(); final long timestamp = context().timestamp(); - final List, Agg>> merged = new ArrayList<>(); + final List, ValueAndTimestamp>> merged = new ArrayList<>(); final SessionWindow newSessionWindow = new SessionWindow(timestamp, timestamp); SessionWindow mergedWindow = newSessionWindow; Agg agg = initializer.apply(); + long resultTimestamp = timestamp; try ( - final KeyValueIterator, Agg> iterator = store.findSessions( + final KeyValueIterator, ValueAndTimestamp> iterator = store.findSessions( key, timestamp - windows.inactivityGap(), timestamp + windows.inactivityGap() ) ) { while (iterator.hasNext()) { - final KeyValue, Agg> next = iterator.next(); + final KeyValue, ValueAndTimestamp> next = iterator.next(); merged.add(next); - agg = sessionMerger.apply(key, agg, next.value); + agg = sessionMerger.apply(key, agg, next.value.value()); + resultTimestamp = Math.max(resultTimestamp, next.value.timestamp()); mergedWindow = mergeSessionWindow(mergedWindow, (SessionWindow) next.key.window()); } } if (mergedWindow.end() > closeTime) { if (!mergedWindow.equals(newSessionWindow)) { - for (final KeyValue, Agg> session : merged) { + for (final KeyValue, ValueAndTimestamp> session : merged) { store.remove(session.key); - tupleForwarder.maybeForward(session.key, null, sendOldValues ? session.value : null); + tupleForwarder.maybeForward( + session.key, + null, + sendOldValues ? session.value.value() : null, + resultTimestamp); } } agg = aggregator.apply(key, value, agg); final Windowed sessionKey = new Windowed<>(key, mergedWindow); - store.put(sessionKey, agg); - tupleForwarder.maybeForward(sessionKey, agg, null); + store.put(sessionKey, ValueAndTimestamp.make(agg, resultTimestamp)); + tupleForwarder.maybeForward(sessionKey, agg, null, resultTimestamp); } else { LOG.debug( "Skipping record for expired window. key=[{}] topic=[{}] partition=[{}] offset=[{}] timestamp=[{}] window=[{},{}) expiration=[{}]", @@ -175,25 +188,30 @@ public String[] storeNames() { } private class KTableSessionWindowValueGetter implements KTableValueGetter, Agg> { - private SessionStore store; + private SessionStore> store; @SuppressWarnings("unchecked") @Override public void init(final ProcessorContext context) { - store = (SessionStore) context.getStateStore(storeName); + StateStore store = context.getStateStore(storeName); + if (store instanceof WrappedStateStore) { + store = ((WrappedStateStore) store).wrappedStore(); + } + this.store = ((SessionWindowedKStreamImpl.SessionStoreFacade) store).inner; } @Override - public Agg get(final Windowed key) { - try (final KeyValueIterator, Agg> iter = store.findSessions(key.key(), key.window().end(), key.window().end())) { + public ValueAndTimestamp get(final Windowed key) { + try (final KeyValueIterator, ValueAndTimestamp> iter + = store.findSessions(key.key(), key.window().end(), key.window().end())) { if (!iter.hasNext()) { return null; } - final Agg value = iter.next().value; + final ValueAndTimestamp valueAndTimestamp = iter.next().value; if (iter.hasNext()) { throw new ProcessorStateException(String.format("Iterator for key [%s] on session store has more than one value", key)); } - return value; + return valueAndTimestamp; } } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregate.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregate.java index 0edbe4e456745..0cc4b9cabd895 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregate.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregate.java @@ -26,9 +26,13 @@ import org.apache.kafka.streams.processor.AbstractProcessor; import org.apache.kafka.streams.processor.Processor; import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.processor.StateStore; import org.apache.kafka.streams.processor.internals.InternalProcessorContext; import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; +import org.apache.kafka.streams.state.ReadOnlyWindowStore; +import org.apache.kafka.streams.state.ValueAndTimestamp; import org.apache.kafka.streams.state.WindowStore; +import org.apache.kafka.streams.state.internals.WrappedStateStore; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -70,7 +74,7 @@ public void enableSendingOldValues() { private class KStreamWindowAggregateProcessor extends AbstractProcessor { - private WindowStore windowStore; + private WindowStore> windowStore; private TupleForwarder, Agg> tupleForwarder; private StreamsMetricsImpl metrics; private InternalProcessorContext internalProcessorContext; @@ -86,7 +90,11 @@ public void init(final ProcessorContext context) { lateRecordDropSensor = Sensors.lateRecordDropSensor(internalProcessorContext); - windowStore = (WindowStore) context.getStateStore(storeName); + StateStore store = context.getStateStore(storeName); + if (store instanceof WrappedStateStore) { + store = ((WrappedStateStore) store).wrappedStore(); + } + windowStore = ((KStreamImpl.WindowStoreFacade) store).inner; tupleForwarder = new TupleForwarder<>(windowStore, context, new ForwardingCacheFlushListener, V>(context), sendOldValues); } @@ -112,17 +120,23 @@ value, context().topic(), context().partition(), context().offset() final Long windowStart = entry.getKey(); final long windowEnd = entry.getValue().end(); if (windowEnd > closeTime) { - Agg oldAgg = windowStore.fetch(key, windowStart); + final ValueAndTimestamp oldAggWithTimestamp = windowStore.fetch(key, windowStart); - if (oldAgg == null) { + final Agg oldAgg; + final long resultTimestamp; + if (oldAggWithTimestamp == null) { oldAgg = initializer.apply(); + resultTimestamp = context().timestamp(); + } else { + oldAgg = oldAggWithTimestamp.value(); + resultTimestamp = Math.max(context().timestamp(), oldAggWithTimestamp.timestamp()); } final Agg newAgg = aggregator.apply(key, value, oldAgg); // update the store with the new value - windowStore.put(key, newAgg, windowStart); - tupleForwarder.maybeForward(new Windowed<>(key, entry.getValue()), newAgg, sendOldValues ? oldAgg : null); + windowStore.put(key, ValueAndTimestamp.make(newAgg, resultTimestamp), windowStart); + tupleForwarder.maybeForward(new Windowed<>(key, entry.getValue()), newAgg, sendOldValues ? oldAgg : null, resultTimestamp); } else { log.debug( "Skipping record for expired window. key=[{}] topic=[{}] partition=[{}] offset=[{}] timestamp=[{}] window=[{},{}) expiration=[{}]", @@ -152,17 +166,21 @@ public String[] storeNames() { private class KStreamWindowAggregateValueGetter implements KTableValueGetter, Agg> { - private WindowStore windowStore; + private ReadOnlyWindowStore> windowStore; @SuppressWarnings("unchecked") @Override public void init(final ProcessorContext context) { - windowStore = (WindowStore) context.getStateStore(storeName); + StateStore store = context.getStateStore(storeName); + if (store instanceof WrappedStateStore) { + store = ((WrappedStateStore) store).wrappedStore(); + } + windowStore = ((KStreamImpl.WindowStoreFacade) store).inner; } @SuppressWarnings("unchecked") @Override - public Agg get(final Windowed windowedKey) { + public ValueAndTimestamp get(final Windowed windowedKey) { final K key = windowedKey.key(); final W window = (W) windowedKey.window(); diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableAggregate.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableAggregate.java index b04a729a44702..e2b11fec0d850 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableAggregate.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableAggregate.java @@ -22,7 +22,10 @@ import org.apache.kafka.streams.processor.AbstractProcessor; import org.apache.kafka.streams.processor.Processor; import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.processor.StateStore; import org.apache.kafka.streams.state.KeyValueStore; +import org.apache.kafka.streams.state.ValueAndTimestamp; +import org.apache.kafka.streams.state.internals.WrappedStateStore; public class KTableAggregate implements KTableProcessorSupplier { @@ -54,14 +57,18 @@ public Processor> get() { } private class KTableAggregateProcessor extends AbstractProcessor> { - private KeyValueStore store; + private KeyValueStore> store; private TupleForwarder tupleForwarder; @SuppressWarnings("unchecked") @Override public void init(final ProcessorContext context) { super.init(context); - store = (KeyValueStore) context.getStateStore(storeName); + StateStore store = context.getStateStore(storeName); + if (store instanceof WrappedStateStore) { + store = ((WrappedStateStore) store).wrappedStore(); + } + this.store = ((KeyValueWithTimestampStoreMaterializer.KeyValueStoreFacade) store).inner; tupleForwarder = new TupleForwarder<>(store, context, new ForwardingCacheFlushListener(context), sendOldValues); } @@ -75,7 +82,16 @@ public void process(final K key, final Change value) { throw new StreamsException("Record key for KTable aggregate operator with state " + storeName + " should not be null."); } - T oldAgg = store.get(key); + final ValueAndTimestamp oldAggWithTimestamp = store.get(key); + T oldAgg; + final long resultTimestamp; + if (oldAggWithTimestamp != null) { + oldAgg = oldAggWithTimestamp.value(); + resultTimestamp = Math.max(oldAggWithTimestamp.timestamp(), context().timestamp()); + } else { + oldAgg = null; + resultTimestamp = context().timestamp(); + } if (oldAgg == null) { oldAgg = initializer.apply(); @@ -94,8 +110,8 @@ public void process(final K key, final Change value) { } // update the store with the new value - store.put(key, newAgg); - tupleForwarder.maybeForward(key, newAgg, sendOldValues ? oldAgg : null); + store.put(key, ValueAndTimestamp.make(newAgg, resultTimestamp)); + tupleForwarder.maybeForward(key, newAgg, sendOldValues ? oldAgg : null, resultTimestamp); } } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableFilter.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableFilter.java index 1312e2fad17ea..3e5a26d0311fd 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableFilter.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableFilter.java @@ -20,7 +20,10 @@ import org.apache.kafka.streams.processor.AbstractProcessor; import org.apache.kafka.streams.processor.Processor; import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.processor.StateStore; import org.apache.kafka.streams.state.KeyValueStore; +import org.apache.kafka.streams.state.ValueAndTimestamp; +import org.apache.kafka.streams.state.internals.WrappedStateStore; class KTableFilter implements KTableProcessorSupplier { @@ -62,7 +65,7 @@ private V computeValue(final K key, final V value) { } private class KTableFilterProcessor extends AbstractProcessor> { - private KeyValueStore store; + private KeyValueStore> store; private TupleForwarder tupleForwarder; @SuppressWarnings("unchecked") @@ -70,7 +73,11 @@ private class KTableFilterProcessor extends AbstractProcessor> { public void init(final ProcessorContext context) { super.init(context); if (queryableName != null) { - store = (KeyValueStore) context.getStateStore(queryableName); + StateStore store = context.getStateStore(queryableName); + if (store instanceof WrappedStateStore) { + store = ((WrappedStateStore) store).wrappedStore(); + } + this.store = ((KeyValueWithTimestampStoreMaterializer.KeyValueStoreFacade) store).inner; tupleForwarder = new TupleForwarder<>(store, context, new ForwardingCacheFlushListener(context), sendOldValues); } } @@ -85,7 +92,7 @@ public void process(final K key, final Change change) { } if (queryableName != null) { - store.put(key, newValue); + store.put(key, ValueAndTimestamp.make(newValue, context().timestamp())); tupleForwarder.maybeForward(key, newValue, oldValue); } else { context().forward(key, new Change<>(newValue, oldValue)); @@ -130,8 +137,19 @@ public void init(final ProcessorContext context) { } @Override - public V get(final K key) { - return computeValue(key, parentGetter.get(key)); + public ValueAndTimestamp get(final K key) { + final ValueAndTimestamp parentValueAndTimestamp = parentGetter.get(key); + final V parentValue; + final long parentTimestamp; + if (parentValueAndTimestamp != null) { + parentValue = parentValueAndTimestamp.value(); + parentTimestamp = parentValueAndTimestamp.timestamp(); + } else { + parentValue = null; + parentTimestamp = -1L; + } + final V resultValue = computeValue(key, parentValue); + return ValueAndTimestamp.make(resultValue, parentTimestamp); } @Override diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableInnerJoin.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableInnerJoin.java index 58110b4845e46..1d41c314f2b47 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableInnerJoin.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableInnerJoin.java @@ -21,7 +21,9 @@ import org.apache.kafka.streams.processor.AbstractProcessor; import org.apache.kafka.streams.processor.Processor; import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.processor.To; import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; +import org.apache.kafka.streams.state.ValueAndTimestamp; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -88,21 +90,23 @@ change, context().topic(), context().partition(), context().offset() R newValue = null; R oldValue = null; + long resultTimestamp = context().timestamp(); - final V2 value2 = valueGetter.get(key); - if (value2 == null) { + final ValueAndTimestamp valueAndTimestamp2 = valueGetter.get(key); + if (valueAndTimestamp2 == null || valueAndTimestamp2.value() == null) { return; } if (change.newValue != null) { - newValue = joiner.apply(change.newValue, value2); + newValue = joiner.apply(change.newValue, valueAndTimestamp2.value()); + resultTimestamp = Math.max(resultTimestamp, valueAndTimestamp2.timestamp()); } if (sendOldValues && change.oldValue != null) { - oldValue = joiner.apply(change.oldValue, value2); + oldValue = joiner.apply(change.oldValue, valueAndTimestamp2.value()); } - context().forward(key, new Change<>(newValue, oldValue)); + context().forward(key, new Change<>(newValue, oldValue), To.all().withTimestamp(resultTimestamp)); } @Override @@ -129,14 +133,17 @@ public void init(final ProcessorContext context) { } @Override - public R get(final K key) { - final V1 value1 = valueGetter1.get(key); + public ValueAndTimestamp get(final K key) { + final ValueAndTimestamp valueAndTimestamp1 = valueGetter1.get(key); - if (value1 != null) { - final V2 value2 = valueGetter2.get(keyValueMapper.apply(key, value1)); + if (valueAndTimestamp1 != null) { + final ValueAndTimestamp valueAndTimestamp2 = valueGetter2.get(keyValueMapper.apply(key, valueAndTimestamp1.value())); - if (value2 != null) { - return joiner.apply(value1, value2); + if (valueAndTimestamp2 != null) { + final R joinResult = joiner.apply(valueAndTimestamp1.value(), valueAndTimestamp2.value()); + return ValueAndTimestamp.make( + joinResult, + Math.max(valueAndTimestamp1.timestamp(), valueAndTimestamp2.timestamp())); } else { return null; } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableJoinMerger.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableJoinMerger.java index 78c1dc6f48001..f4ab10df71731 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableJoinMerger.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableJoinMerger.java @@ -19,7 +19,10 @@ import org.apache.kafka.streams.processor.AbstractProcessor; import org.apache.kafka.streams.processor.Processor; import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.processor.StateStore; import org.apache.kafka.streams.state.KeyValueStore; +import org.apache.kafka.streams.state.ValueAndTimestamp; +import org.apache.kafka.streams.state.internals.WrappedStateStore; import java.util.Collections; import java.util.HashSet; @@ -79,7 +82,7 @@ public void enableSendingOldValues() { } private class KTableKTableJoinMergeProcessor extends AbstractProcessor> { - private KeyValueStore store; + private KeyValueStore> store; private TupleForwarder tupleForwarder; @SuppressWarnings("unchecked") @@ -87,7 +90,11 @@ private class KTableKTableJoinMergeProcessor extends AbstractProcessor) context.getStateStore(queryableName); + StateStore store = context.getStateStore(queryableName); + if (store instanceof WrappedStateStore) { + store = ((WrappedStateStore) store).wrappedStore(); + } + this.store = ((KeyValueWithTimestampStoreMaterializer.KeyValueStoreFacade) store).inner; tupleForwarder = new TupleForwarder<>(store, context, new ForwardingCacheFlushListener(context), sendOldValues); @@ -97,7 +104,7 @@ public void init(final ProcessorContext context) { @Override public void process(final K key, final Change value) { if (queryableName != null) { - store.put(key, value.newValue); + store.put(key, ValueAndTimestamp.make(value.newValue, context().timestamp())); tupleForwarder.maybeForward(key, value.newValue, sendOldValues ? value.oldValue : null); } else { if (sendOldValues) { diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableLeftJoin.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableLeftJoin.java index cbb63c6ba46a4..696f96b261216 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableLeftJoin.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableLeftJoin.java @@ -20,7 +20,9 @@ import org.apache.kafka.streams.processor.AbstractProcessor; import org.apache.kafka.streams.processor.Processor; import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.processor.To; import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; +import org.apache.kafka.streams.state.ValueAndTimestamp; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -86,10 +88,18 @@ change, context().topic(), context().partition(), context().offset() R newValue = null; R oldValue = null; - - final V2 value2 = valueGetter.get(key); - if (value2 == null && change.newValue == null && change.oldValue == null) { - return; + long resultTimestamp = context().timestamp(); + + final ValueAndTimestamp valueAndTimestamp2 = valueGetter.get(key); + final V2 value2; + if (valueAndTimestamp2 == null) { + if (change.newValue == null && change.oldValue == null) { + return; + } + value2 = null; + } else { + value2 = valueAndTimestamp2.value(); + resultTimestamp = Math.max(resultTimestamp, valueAndTimestamp2.timestamp()); } if (change.newValue != null) { @@ -100,7 +110,7 @@ change, context().topic(), context().partition(), context().offset() oldValue = joiner.apply(change.oldValue, value2); } - context().forward(key, new Change<>(newValue, oldValue)); + context().forward(key, new Change<>(newValue, oldValue), To.all().withTimestamp(resultTimestamp)); } @Override @@ -127,12 +137,24 @@ public void init(final ProcessorContext context) { } @Override - public R get(final K key) { - final V1 value1 = valueGetter1.get(key); - - if (value1 != null) { - final V2 value2 = valueGetter2.get(key); - return joiner.apply(value1, value2); + public ValueAndTimestamp get(final K key) { + final ValueAndTimestamp valueAndTimestamp1 = valueGetter1.get(key); + + if (valueAndTimestamp1 != null) { + final ValueAndTimestamp valueAndTimestamp2 = valueGetter2.get(key); + final V2 value2; + final long timestamp2; + if (valueAndTimestamp2 != null) { + value2 = valueAndTimestamp2.value(); + timestamp2 = valueAndTimestamp2.timestamp(); + } else { + value2 = null; + timestamp2 = -1; + } + final R joinResult = joiner.apply(valueAndTimestamp1.value(), value2); + return ValueAndTimestamp.make( + joinResult, + Math.max(valueAndTimestamp1.timestamp(), timestamp2)); } else { return null; } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableOuterJoin.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableOuterJoin.java index 27eb698778248..352d1196fe889 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableOuterJoin.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableOuterJoin.java @@ -20,7 +20,9 @@ import org.apache.kafka.streams.processor.AbstractProcessor; import org.apache.kafka.streams.processor.Processor; import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.processor.To; import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; +import org.apache.kafka.streams.state.ValueAndTimestamp; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -85,10 +87,18 @@ change, context().topic(), context().partition(), context().offset() R newValue = null; R oldValue = null; - - final V2 value2 = valueGetter.get(key); - if (value2 == null && change.newValue == null && change.oldValue == null) { - return; + long resultTimestamp = context().timestamp(); + + final ValueAndTimestamp valueAndTimestamp2 = valueGetter.get(key); + final V2 value2; + if (valueAndTimestamp2 == null) { + if (change.newValue == null && change.oldValue == null) { + return; + } + value2 = null; + } else { + value2 = valueAndTimestamp2.value(); + resultTimestamp = Math.max(resultTimestamp, valueAndTimestamp2.timestamp()); } if (value2 != null || change.newValue != null) { @@ -99,7 +109,7 @@ change, context().topic(), context().partition(), context().offset() oldValue = joiner.apply(change.oldValue, value2); } - context().forward(key, new Change<>(newValue, oldValue)); + context().forward(key, new Change<>(newValue, oldValue), To.all().withTimestamp(resultTimestamp)); } @Override @@ -126,16 +136,33 @@ public void init(final ProcessorContext context) { } @Override - public R get(final K key) { + public ValueAndTimestamp get(final K key) { R newValue = null; - final V1 value1 = valueGetter1.get(key); - final V2 value2 = valueGetter2.get(key); + long resultTimestamp = -1; + + final ValueAndTimestamp valueAndTimestamp1 = valueGetter1.get(key); + final ValueAndTimestamp valueAndTimestamp2 = valueGetter2.get(key); + + final V1 value1; + final V2 value2; + if (valueAndTimestamp1 != null) { + value1 = valueAndTimestamp1.value(); + resultTimestamp = valueAndTimestamp1.timestamp(); + } else { + value1 = null; + } + if (valueAndTimestamp2 != null) { + value2 = valueAndTimestamp2.value(); + resultTimestamp = valueAndTimestamp1 != null ? Math.max(resultTimestamp, valueAndTimestamp2.timestamp()) : valueAndTimestamp2.timestamp(); + } else { + value2 = null; + } if (value1 != null || value2 != null) { newValue = joiner.apply(value1, value2); } - return newValue; + return ValueAndTimestamp.make(newValue, resultTimestamp); } @Override diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableRightJoin.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableRightJoin.java index 1e634d00448f9..88cf4d322e186 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableRightJoin.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableRightJoin.java @@ -20,7 +20,9 @@ import org.apache.kafka.streams.processor.AbstractProcessor; import org.apache.kafka.streams.processor.Processor; import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.processor.To; import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; +import org.apache.kafka.streams.state.ValueAndTimestamp; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -85,19 +87,22 @@ change, context().topic(), context().partition(), context().offset() final R newValue; R oldValue = null; + long resultTimestamp = context().timestamp(); - final V2 value2 = valueGetter.get(key); - if (value2 == null) { + final ValueAndTimestamp valueAndTimestamp2 = valueGetter.get(key); + if (valueAndTimestamp2 == null) { return; } + final V2 value2 = valueAndTimestamp2.value(); newValue = joiner.apply(change.newValue, value2); + resultTimestamp = Math.max(resultTimestamp, valueAndTimestamp2.timestamp()); if (sendOldValues) { oldValue = joiner.apply(change.oldValue, value2); } - context().forward(key, new Change<>(newValue, oldValue)); + context().forward(key, new Change<>(newValue, oldValue), To.all().withTimestamp(resultTimestamp)); } @Override @@ -124,12 +129,22 @@ public void init(final ProcessorContext context) { } @Override - public R get(final K key) { - final V2 value2 = valueGetter2.get(key); - - if (value2 != null) { - final V1 value1 = valueGetter1.get(key); - return joiner.apply(value1, value2); + public ValueAndTimestamp get(final K key) { + final ValueAndTimestamp valueAndTimestamp2 = valueGetter2.get(key); + + if (valueAndTimestamp2 != null) { + long resultTimestamp = valueAndTimestamp2.timestamp(); + final ValueAndTimestamp valueAndTimestamp1 = valueGetter1.get(key); + final V1 value1; + if (valueAndTimestamp1 != null) { + value1 = valueAndTimestamp1.value(); + resultTimestamp = Math.max(resultTimestamp, valueAndTimestamp1.timestamp()); + } else { + value1 = null; + } + + final R joinResult = joiner.apply(value1, valueAndTimestamp2.value()); + return ValueAndTimestamp.make(joinResult, resultTimestamp); } else { return null; } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableMapValues.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableMapValues.java index aae1437ea7656..e858edda00d5f 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableMapValues.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableMapValues.java @@ -20,7 +20,10 @@ import org.apache.kafka.streams.processor.AbstractProcessor; import org.apache.kafka.streams.processor.Processor; import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.processor.StateStore; import org.apache.kafka.streams.state.KeyValueStore; +import org.apache.kafka.streams.state.ValueAndTimestamp; +import org.apache.kafka.streams.state.internals.WrappedStateStore; class KTableMapValues implements KTableProcessorSupplier { @@ -82,7 +85,7 @@ private V1 computeValue(final K key, final V value) { } private class KTableMapValuesProcessor extends AbstractProcessor> { - private KeyValueStore store; + private KeyValueStore> store; private TupleForwarder tupleForwarder; @SuppressWarnings("unchecked") @@ -90,7 +93,11 @@ private class KTableMapValuesProcessor extends AbstractProcessor> { public void init(final ProcessorContext context) { super.init(context); if (queryableName != null) { - store = (KeyValueStore) context.getStateStore(queryableName); + StateStore store = context.getStateStore(queryableName); + if (store instanceof WrappedStateStore) { + store = ((WrappedStateStore) store).wrappedStore(); + } + this.store = ((KeyValueWithTimestampStoreMaterializer.KeyValueStoreFacade) store).inner; tupleForwarder = new TupleForwarder<>(store, context, new ForwardingCacheFlushListener(context), sendOldValues); } } @@ -101,7 +108,7 @@ public void process(final K key, final Change change) { final V1 oldValue = sendOldValues ? computeValue(key, change.oldValue) : null; if (queryableName != null) { - store.put(key, newValue); + store.put(key, ValueAndTimestamp.make(newValue, context().timestamp())); tupleForwarder.maybeForward(key, newValue, oldValue); } else { context().forward(key, new Change<>(newValue, oldValue)); @@ -123,8 +130,11 @@ public void init(final ProcessorContext context) { } @Override - public V1 get(final K key) { - return computeValue(key, parentGetter.get(key)); + public ValueAndTimestamp get(final K key) { + final ValueAndTimestamp parentValueAndTimestamp = parentGetter.get(key); + final V parentValue = parentValueAndTimestamp == null ? null : parentValueAndTimestamp.value(); + final V1 resultValue = computeValue(key, parentValue); + return resultValue == null ? null : ValueAndTimestamp.make(resultValue, parentValueAndTimestamp.timestamp()); // this is a potential NPE -- if `parentValueAndTimestamp == null` we could also directly return `null`, but this would be a semantic change } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableMaterializedValueGetterSupplier.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableMaterializedValueGetterSupplier.java index 0c17d5908f831..c1807d33da566 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableMaterializedValueGetterSupplier.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableMaterializedValueGetterSupplier.java @@ -18,7 +18,10 @@ package org.apache.kafka.streams.kstream.internals; import org.apache.kafka.streams.processor.ProcessorContext; -import org.apache.kafka.streams.state.KeyValueStore; +import org.apache.kafka.streams.processor.StateStore; +import org.apache.kafka.streams.state.ReadOnlyKeyValueStore; +import org.apache.kafka.streams.state.ValueAndTimestamp; +import org.apache.kafka.streams.state.internals.WrappedStateStore; public class KTableMaterializedValueGetterSupplier implements KTableValueGetterSupplier { @@ -38,16 +41,20 @@ public String[] storeNames() { } private class KTableMaterializedValueGetter implements KTableValueGetter { - private KeyValueStore store; + private ReadOnlyKeyValueStore> store; @SuppressWarnings("unchecked") @Override public void init(final ProcessorContext context) { - store = (KeyValueStore) context.getStateStore(storeName); + StateStore store = context.getStateStore(storeName); + if (store instanceof WrappedStateStore) { + store = ((WrappedStateStore) store).wrappedStore(); + } + this.store = ((KeyValueWithTimestampStoreMaterializer.KeyValueStoreFacade) store).inner; } @Override - public V get(final K key) { + public ValueAndTimestamp get(final K key) { return store.get(key); } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableReduce.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableReduce.java index 38c5a11f34db1..da25de6daaf00 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableReduce.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableReduce.java @@ -21,7 +21,10 @@ import org.apache.kafka.streams.processor.AbstractProcessor; import org.apache.kafka.streams.processor.Processor; import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.processor.StateStore; import org.apache.kafka.streams.state.KeyValueStore; +import org.apache.kafka.streams.state.ValueAndTimestamp; +import org.apache.kafka.streams.state.internals.WrappedStateStore; public class KTableReduce implements KTableProcessorSupplier { @@ -49,14 +52,18 @@ public Processor> get() { private class KTableReduceProcessor extends AbstractProcessor> { - private KeyValueStore store; + private KeyValueStore> store; private TupleForwarder tupleForwarder; @SuppressWarnings("unchecked") @Override public void init(final ProcessorContext context) { super.init(context); - store = (KeyValueStore) context.getStateStore(storeName); + StateStore store = context.getStateStore(storeName); + if (store instanceof WrappedStateStore) { + store = ((WrappedStateStore) store).wrappedStore(); + } + this.store = ((KeyValueWithTimestampStoreMaterializer.KeyValueStoreFacade) store).inner; tupleForwarder = new TupleForwarder<>(store, context, new ForwardingCacheFlushListener(context), sendOldValues); } @@ -70,15 +77,25 @@ public void process(final K key, final Change value) { throw new StreamsException("Record key for KTable reduce operator with state " + storeName + " should not be null."); } - final V oldAgg = store.get(key); + final ValueAndTimestamp oldAggWithTimestamp = store.get(key); + final V oldAgg; + long resultTimestamp; + if (oldAggWithTimestamp != null) { + oldAgg = oldAggWithTimestamp.value(); + resultTimestamp = oldAggWithTimestamp.timestamp(); + } else { + oldAgg = null; + resultTimestamp = context().timestamp(); + } V newAgg = oldAgg; // first try to add the new value if (value.newValue != null) { - if (newAgg == null) { + if (oldAgg == null) { newAgg = value.newValue; } else { - newAgg = addReducer.apply(newAgg, value.newValue); + newAgg = addReducer.apply(oldAgg, value.newValue); + resultTimestamp = Math.max(resultTimestamp, context().timestamp()); } } @@ -88,8 +105,8 @@ public void process(final K key, final Change value) { } // update the store with the new value - store.put(key, newAgg); - tupleForwarder.maybeForward(key, newAgg, sendOldValues ? oldAgg : null); + store.put(key, ValueAndTimestamp.make(newAgg, resultTimestamp)); + tupleForwarder.maybeForward(key, newAgg, sendOldValues ? oldAgg : null, resultTimestamp); } } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableRepartitionMap.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableRepartitionMap.java index e86445a311384..a3b718002e0b8 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableRepartitionMap.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableRepartitionMap.java @@ -22,6 +22,7 @@ import org.apache.kafka.streams.processor.AbstractProcessor; import org.apache.kafka.streams.processor.Processor; import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.state.ValueAndTimestamp; /** * KTable repartition map functions are not exposed to public APIs, but only used for keyed aggregations. @@ -112,8 +113,11 @@ public void init(final ProcessorContext context) { } @Override - public KeyValue get(final K key) { - return mapper.apply(key, parentGetter.get(key)); + public ValueAndTimestamp> get(final K key) { + final ValueAndTimestamp parentValueAndTimestamp = parentGetter.get(key); + final V parentValue = parentValueAndTimestamp == null ? null : parentValueAndTimestamp.value(); + final KeyValue result = mapper.apply(key, parentValue); + return ValueAndTimestamp.make(result, parentValueAndTimestamp.timestamp()); // this is a potential NPE -- if `parentValueAndTimestamp == null` we could also directly return `null`, but this would be a semantic change } @Override diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableSource.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableSource.java index 6fc57bcf582cd..6145da80fb991 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableSource.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableSource.java @@ -20,8 +20,11 @@ import org.apache.kafka.streams.processor.Processor; import org.apache.kafka.streams.processor.ProcessorContext; import org.apache.kafka.streams.processor.ProcessorSupplier; +import org.apache.kafka.streams.processor.StateStore; import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; import org.apache.kafka.streams.state.KeyValueStore; +import org.apache.kafka.streams.state.ValueAndTimestamp; +import org.apache.kafka.streams.state.internals.WrappedStateStore; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -34,7 +37,8 @@ public class KTableSource implements ProcessorSupplier { private String queryableName; private boolean sendOldValues; - public KTableSource(final String storeName, final String queryableName) { + public KTableSource(final String storeName, + final String queryableName) { Objects.requireNonNull(storeName, "storeName can't be null"); this.storeName = storeName; @@ -66,7 +70,7 @@ public void materialize() { private class KTableSourceProcessor extends AbstractProcessor { - private KeyValueStore store; + private KeyValueStore> store; private TupleForwarder tupleForwarder; private StreamsMetricsImpl metrics; @@ -76,13 +80,18 @@ public void init(final ProcessorContext context) { super.init(context); metrics = (StreamsMetricsImpl) context.metrics(); if (queryableName != null) { - store = (KeyValueStore) context.getStateStore(queryableName); + StateStore store = context.getStateStore(storeName); + if (store instanceof WrappedStateStore) { + store = ((WrappedStateStore) store).wrappedStore(); + } + this.store = ((KeyValueWithTimestampStoreMaterializer.KeyValueStoreFacade) store).inner; tupleForwarder = new TupleForwarder<>(store, context, new ForwardingCacheFlushListener(context), sendOldValues); } } @Override - public void process(final K key, final V value) { + public void process(final K key, + final V value) { // if the key is null, then ignore the record if (key == null) { LOG.warn( @@ -94,9 +103,9 @@ public void process(final K key, final V value) { } if (queryableName != null) { - final V oldValue = sendOldValues ? store.get(key) : null; - store.put(key, value); - tupleForwarder.maybeForward(key, value, oldValue); + final ValueAndTimestamp oldValue = sendOldValues ? store.get(key) : null; + store.put(key, ValueAndTimestamp.make(value, context().timestamp())); + tupleForwarder.maybeForward(key, value, oldValue == null ? null : oldValue.value()); } else { context().forward(key, new Change<>(value, null)); } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableSourceValueGetterSupplier.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableSourceValueGetterSupplier.java index 6882dacce934e..bd6c514fd4502 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableSourceValueGetterSupplier.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableSourceValueGetterSupplier.java @@ -17,7 +17,10 @@ package org.apache.kafka.streams.kstream.internals; import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.processor.StateStore; import org.apache.kafka.streams.state.ReadOnlyKeyValueStore; +import org.apache.kafka.streams.state.ValueAndTimestamp; +import org.apache.kafka.streams.state.internals.WrappedStateStore; public class KTableSourceValueGetterSupplier implements KTableValueGetterSupplier { @@ -38,20 +41,23 @@ public String[] storeNames() { private class KTableSourceValueGetter implements KTableValueGetter { - ReadOnlyKeyValueStore store = null; + ReadOnlyKeyValueStore> store = null; @SuppressWarnings("unchecked") public void init(final ProcessorContext context) { - store = (ReadOnlyKeyValueStore) context.getStateStore(storeName); + StateStore store = context.getStateStore(storeName); + if (store instanceof WrappedStateStore) { + store = ((WrappedStateStore) store).wrappedStore(); + } + this.store = store == null ? null : ((KeyValueWithTimestampStoreMaterializer.KeyValueStoreFacade) store).inner; } - public V get(final K key) { + public ValueAndTimestamp get(final K key) { return store.get(key); } @Override - public void close() { - } + public void close() {} } } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableTransformValues.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableTransformValues.java index 88cea4fc7c75d..e2a23acf2786b 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableTransformValues.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableTransformValues.java @@ -21,8 +21,11 @@ import org.apache.kafka.streams.processor.AbstractProcessor; import org.apache.kafka.streams.processor.Processor; import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.processor.StateStore; import org.apache.kafka.streams.processor.internals.ForwardingDisabledProcessorContext; import org.apache.kafka.streams.state.KeyValueStore; +import org.apache.kafka.streams.state.ValueAndTimestamp; +import org.apache.kafka.streams.state.internals.WrappedStateStore; import java.util.Objects; @@ -76,7 +79,7 @@ public void enableSendingOldValues() { private class KTableTransformValuesProcessor extends AbstractProcessor> { private final ValueTransformerWithKey valueTransformer; - private KeyValueStore store; + private KeyValueStore> store; private TupleForwarder tupleForwarder; private KTableTransformValuesProcessor(final ValueTransformerWithKey valueTransformer) { @@ -92,7 +95,11 @@ public void init(final ProcessorContext context) { if (queryableName != null) { final ForwardingCacheFlushListener flushListener = new ForwardingCacheFlushListener<>(context); - store = (KeyValueStore) context.getStateStore(queryableName); + StateStore store = context.getStateStore(queryableName); + if (store instanceof WrappedStateStore) { + store = ((WrappedStateStore) store).wrappedStore(); + } + this.store = ((KeyValueWithTimestampStoreMaterializer.KeyValueStoreFacade) store).inner; tupleForwarder = new TupleForwarder<>(store, context, flushListener, sendOldValues); } } @@ -105,8 +112,14 @@ public void process(final K key, final Change change) { final V1 oldValue = sendOldValues ? valueTransformer.transform(key, change.oldValue) : null; context().forward(key, new Change<>(newValue, oldValue)); } else { - final V1 oldValue = sendOldValues ? store.get(key) : null; - store.put(key, newValue); + final V1 oldValue; + if (sendOldValues) { + final ValueAndTimestamp oldValueAndTimestamp = store.get(key); + oldValue = oldValueAndTimestamp == null ? null : oldValueAndTimestamp.value(); + } else { + oldValue = null; + } + store.put(key, ValueAndTimestamp.make(newValue, context().timestamp())); tupleForwarder.maybeForward(key, newValue, oldValue); } } @@ -131,13 +144,15 @@ private class KTableTransformValuesGetter implements KTableValueGetter { @Override public void init(final ProcessorContext context) { parentGetter.init(context); - valueTransformer.init(new ForwardingDisabledProcessorContext(context)); } @Override - public V1 get(final K key) { - return valueTransformer.transform(key, parentGetter.get(key)); + public ValueAndTimestamp get(final K key) { + final ValueAndTimestamp valueAndTimestamp = parentGetter.get(key); + final V value = valueAndTimestamp == null ? null : valueAndTimestamp.value(); + final V1 result = valueTransformer.transform(key, value); + return ValueAndTimestamp.make(result, valueAndTimestamp.timestamp()); // this is a potential NPE -- if `parentValueAndTimestamp == null` we could also directly return `null`, but this would be a semantic change } @Override diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableValueGetter.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableValueGetter.java index edd964474e232..a2695d069f14b 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableValueGetter.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableValueGetter.java @@ -17,12 +17,13 @@ package org.apache.kafka.streams.kstream.internals; import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.state.ValueAndTimestamp; public interface KTableValueGetter { void init(ProcessorContext context); - V get(K key); + ValueAndTimestamp get(K key); void close(); } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KeyValueStoreMaterializer.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KeyValueStoreMaterializer.java deleted file mode 100644 index 67872be0a3b8e..0000000000000 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KeyValueStoreMaterializer.java +++ /dev/null @@ -1,57 +0,0 @@ -/* - * 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.kstream.internals; - -import org.apache.kafka.common.utils.Bytes; -import org.apache.kafka.streams.state.KeyValueBytesStoreSupplier; -import org.apache.kafka.streams.state.KeyValueStore; -import org.apache.kafka.streams.state.StoreBuilder; -import org.apache.kafka.streams.state.Stores; - -public class KeyValueStoreMaterializer { - private final MaterializedInternal> materialized; - - public KeyValueStoreMaterializer(final MaterializedInternal> materialized) { - this.materialized = materialized; - } - - /** - * @return StoreBuilder - */ - public StoreBuilder> materialize() { - KeyValueBytesStoreSupplier supplier = (KeyValueBytesStoreSupplier) materialized.storeSupplier(); - if (supplier == null) { - final String name = materialized.storeName(); - supplier = Stores.persistentKeyValueStore(name); - } - final StoreBuilder> builder = Stores.keyValueStoreBuilder( - supplier, - materialized.keySerde(), - materialized.valueSerde()); - - if (materialized.loggingEnabled()) { - builder.withLoggingEnabled(materialized.logConfig()); - } else { - builder.withLoggingDisabled(); - } - - if (materialized.cachingEnabled()) { - builder.withCachingEnabled(); - } - return builder; - } -} diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KeyValueWithTimestampStoreMaterializer.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KeyValueWithTimestampStoreMaterializer.java new file mode 100644 index 0000000000000..b36c7072433dd --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KeyValueWithTimestampStoreMaterializer.java @@ -0,0 +1,217 @@ +/* + * 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.kstream.internals; + +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.common.utils.Bytes; +import org.apache.kafka.common.utils.Time; +import org.apache.kafka.streams.KeyValue; +import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.processor.StateStore; +import org.apache.kafka.streams.processor.internals.DefaultRecordConverter; +import org.apache.kafka.streams.state.KeyValueBytesStoreSupplier; +import org.apache.kafka.streams.state.KeyValueIterator; +import org.apache.kafka.streams.state.KeyValueStore; +import org.apache.kafka.streams.state.RecordConverter; +import org.apache.kafka.streams.state.StoreBuilder; +import org.apache.kafka.streams.state.Stores; +import org.apache.kafka.streams.state.ValueAndTimestamp; +import org.apache.kafka.streams.state.internals.KeyValueIteratorFacade; +import org.apache.kafka.streams.state.internals.KeyValueStoreBuilder; +import org.apache.kafka.streams.state.internals.WrappedStateStore; + +import java.util.List; +import java.util.Map; + +public class KeyValueWithTimestampStoreMaterializer { + private final MaterializedInternal> materialized; + + public KeyValueWithTimestampStoreMaterializer(final MaterializedInternal> materialized) { + this.materialized = materialized; + } + + /** + * @return StoreBuilder + */ + public StoreBuilder> materialize() { + KeyValueBytesStoreSupplier supplier = (KeyValueBytesStoreSupplier) materialized.storeSupplier(); + if (supplier == null) { + supplier = Stores.persistentKeyValueWithTimestampStore(materialized.storeName()); + } + + final KeyValueBytesStoreSupplier innerSupplier = supplier; + final KeyValueStoreBuilder builder = new KeyValueStoreBuilder(supplier, null, null, Time.SYSTEM) { + final StoreBuilder>> inner = + Stores.keyValueWithTimestampStoreBuilder( + innerSupplier, + materialized.keySerde(), + materialized.valueSerde()); + + @Override + public KeyValueStoreBuilder withCachingEnabled() { + inner.withCachingEnabled(); + return this; + } + + @Override + public KeyValueStoreBuilder withCachingDisabled() { + inner.withCachingDisabled(); + return this; + } + + @Override + public KeyValueStoreBuilder withLoggingEnabled(final Map config) { + inner.withLoggingEnabled(config); + return this; + } + + @Override + public KeyValueStoreBuilder withLoggingDisabled() { + inner.withLoggingDisabled(); + return this; + } + + @Override + public Map logConfig() { + return inner.logConfig(); + } + + @Override + public boolean loggingEnabled() { + return inner.loggingEnabled(); + } + + @Override + public String name() { + return inner.name(); + } + + @Override + public KeyValueStore build() { + return new KeyValueStoreFacade<>(inner.build()); + } + }; + + if (materialized.loggingEnabled()) { + builder.withLoggingEnabled(materialized.logConfig()); + } else { + builder.withLoggingDisabled(); + } + + if (materialized.cachingEnabled()) { + builder.withCachingEnabled(); + } + return builder; + } + + public static class KeyValueStoreFacade implements KeyValueStore, RecordConverter { + public final KeyValueStore> inner; + private final RecordConverter innerRecordConvert; + + public KeyValueStoreFacade(final KeyValueStore> store) { + inner = store; + final StateStore rootStore = store instanceof WrappedStateStore ? ((WrappedStateStore) store).inner() : store; + innerRecordConvert = rootStore instanceof RecordConverter ? (RecordConverter) rootStore : new DefaultRecordConverter(); + } + + @Override + public void init(final ProcessorContext context, + final StateStore root) { + inner.init(context, root); + } + + @Override + public void put(final A key, + final B value) { + inner.put(key, ValueAndTimestamp.make(value, -1L)); + } + + @Override + public B putIfAbsent(final A key, + final B value) { + final ValueAndTimestamp old = inner.putIfAbsent(key, ValueAndTimestamp.make(value, -1L)); + return old == null ? null : old.value(); + } + + @Override + public void putAll(final List> entries) { + for (final KeyValue entry : entries) { + inner.put(entry.key, ValueAndTimestamp.make(entry.value, -1L)); + } + } + + @Override + public B delete(final A key) { + final ValueAndTimestamp old = inner.delete(key); + return old == null ? null : old.value(); + } + + @Override + public B get(final A key) { + final ValueAndTimestamp valueAndTimestamp = inner.get(key); + return valueAndTimestamp == null ? null : valueAndTimestamp.value(); + } + + @Override + public KeyValueIterator range(final A from, + final A to) { + final KeyValueIterator> innerIterator = inner.range(from, to); + return new KeyValueIteratorFacade<>(innerIterator); + } + + @Override + public KeyValueIterator all() { + final KeyValueIterator> innerIterator = inner.all(); + return new KeyValueIteratorFacade<>(innerIterator); + } + + @Override + public void flush() { + inner.flush(); + } + + @Override + public void close() { + inner.close(); + } + + @Override + public boolean isOpen() { + return inner.isOpen(); + } + + @Override + public long approximateNumEntries() { + return inner.approximateNumEntries(); + } + + @Override + public String name() { + return inner.name(); + } + + @Override + public boolean persistent() { + return inner.persistent(); + } + + @Override + public ConsumerRecord convert(final ConsumerRecord record) { + return innerRecordConvert.convert(record); + } + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/SessionWindowedKStreamImpl.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/SessionWindowedKStreamImpl.java index 8c731be287d9d..8ab026ce895e0 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/SessionWindowedKStreamImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/SessionWindowedKStreamImpl.java @@ -16,9 +16,11 @@ */ package org.apache.kafka.streams.kstream.internals; +import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.common.serialization.Serde; import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.utils.Bytes; +import org.apache.kafka.common.utils.Time; import org.apache.kafka.streams.kstream.Aggregator; import org.apache.kafka.streams.kstream.Initializer; import org.apache.kafka.streams.kstream.KTable; @@ -30,11 +32,22 @@ import org.apache.kafka.streams.kstream.Windowed; import org.apache.kafka.streams.kstream.WindowedSerdes; import org.apache.kafka.streams.kstream.internals.graph.StreamsGraphNode; +import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.processor.StateStore; +import org.apache.kafka.streams.processor.internals.DefaultRecordConverter; +import org.apache.kafka.streams.state.KeyValueIterator; +import org.apache.kafka.streams.state.RecordConverter; import org.apache.kafka.streams.state.SessionBytesStoreSupplier; import org.apache.kafka.streams.state.SessionStore; import org.apache.kafka.streams.state.StoreBuilder; import org.apache.kafka.streams.state.Stores; +import org.apache.kafka.streams.state.ValueAndTimestamp; +import org.apache.kafka.streams.state.internals.KeyValueIteratorFacade; +import org.apache.kafka.streams.state.internals.SessionStoreBuilder; +import org.apache.kafka.streams.state.internals.WrappedStateStore; +import java.time.Duration; +import java.util.Map; import java.util.Objects; import java.util.Set; @@ -191,16 +204,64 @@ private StoreBuilder> materialize(final MaterializedInt + " grace=[" + windows.gracePeriodMs() + "]," + " retention=[" + retentionPeriod + "]"); } - supplier = Stores.persistentSessionStore( + supplier = Stores.persistentSessionWithTimestampStore( materialized.storeName(), - retentionPeriod + Duration.ofMillis(retentionPeriod) ); } - final StoreBuilder> builder = Stores.sessionStoreBuilder( - supplier, - materialized.keySerde(), - materialized.valueSerde() - ); + + final SessionBytesStoreSupplier innerSupplier = supplier; + final SessionStoreBuilder builder = new SessionStoreBuilder(supplier, null, null, Time.SYSTEM) { + final StoreBuilder>> inner = Stores.sessionWithTimestampStoreBuilder( + innerSupplier, + materialized.keySerde(), + materialized.valueSerde() + ); + + @Override + public SessionStoreBuilder withCachingEnabled() { + inner.withCachingEnabled(); + return this; + } + + @Override + public SessionStoreBuilder withCachingDisabled() { + inner.withCachingDisabled(); + return this; + } + + @Override + public SessionStoreBuilder withLoggingEnabled(final Map config) { + inner.withLoggingEnabled(config); + return this; + } + + @Override + public SessionStoreBuilder withLoggingDisabled() { + inner.withLoggingDisabled(); + return this; + } + + @Override + public Map logConfig() { + return inner.logConfig(); + } + + @Override + public boolean loggingEnabled() { + return inner.loggingEnabled(); + } + + @Override + public String name() { + return inner.name(); + } + + @Override + public SessionStore build() { + return new SessionStoreFacade<>(inner.build()); + } + }; if (materialized.loggingEnabled()) { builder.withLoggingEnabled(materialized.logConfig()); @@ -211,6 +272,7 @@ private StoreBuilder> materialize(final MaterializedInt if (materialized.cachingEnabled()) { builder.withCachingEnabled(); } + return builder; } @@ -221,4 +283,94 @@ private Merger mergerForAggregator(final Aggregator aggregator) { private Aggregator aggregatorForReducer(final Reducer reducer) { return (aggKey, value, aggregate) -> aggregate == null ? value : reducer.apply(aggregate, value); } + + public static class SessionStoreFacade implements SessionStore, RecordConverter { + public final SessionStore> inner; + private final RecordConverter innerRecordConvert; + + public SessionStoreFacade(final SessionStore> store) { + this.inner = store; + final StateStore rootStore = store instanceof WrappedStateStore ? ((WrappedStateStore) store).inner() : store; + innerRecordConvert = rootStore instanceof RecordConverter ? (RecordConverter) rootStore : new DefaultRecordConverter(); + } + + @Override + public void init(final ProcessorContext context, + final StateStore root) { + inner.init(context, root); + } + + @Override + public void put(final Windowed sessionKey, + final B aggregate) { + inner.put(sessionKey, ValueAndTimestamp.make(aggregate, -1L)); + } + + @Override + public void remove(final Windowed sessionKey) { + inner.remove(sessionKey); + } + + @Override + public KeyValueIterator, B> fetch(final A key) { + final KeyValueIterator, ValueAndTimestamp> innerIterator = inner.fetch(key); + return new KeyValueIteratorFacade<>(innerIterator); + } + + @Override + public KeyValueIterator, B> fetch(final A from, + final A to) { + final KeyValueIterator, ValueAndTimestamp> innerIterator = inner.fetch(from, to); + return new KeyValueIteratorFacade<>(innerIterator); + } + + @Override + public KeyValueIterator, B> findSessions(final A key, + final long earliestSessionEndTime, + final long latestSessionStartTime) { + final KeyValueIterator, ValueAndTimestamp> innerIterator + = inner.findSessions(key, earliestSessionEndTime, latestSessionStartTime); + return new KeyValueIteratorFacade<>(innerIterator); + } + + @Override + public KeyValueIterator, B> findSessions(final A keyFrom, + final A keyTo, + final long earliestSessionEndTime, + final long latestSessionStartTime) { + final KeyValueIterator, ValueAndTimestamp> innerIterator + = inner.findSessions(keyFrom, keyTo, earliestSessionEndTime, latestSessionStartTime); + return new KeyValueIteratorFacade<>(innerIterator); + } + + @Override + public void flush() { + inner.flush(); + } + + @Override + public void close() { + inner.close(); + } + + @Override + public boolean isOpen() { + return inner.isOpen(); + } + + @Override + public String name() { + return inner.name(); + } + + @Override + public boolean persistent() { + return inner.persistent(); + } + + @Override + public ConsumerRecord convert(final ConsumerRecord record) { + return innerRecordConvert.convert(record); + } + } } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/TimeWindowedKStreamImpl.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/TimeWindowedKStreamImpl.java index f549ce91b0c42..f4676e4bb5d52 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/TimeWindowedKStreamImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/TimeWindowedKStreamImpl.java @@ -19,6 +19,7 @@ import org.apache.kafka.common.serialization.Serde; import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.utils.Bytes; +import org.apache.kafka.common.utils.Time; import org.apache.kafka.streams.kstream.Aggregator; import org.apache.kafka.streams.kstream.Initializer; import org.apache.kafka.streams.kstream.KTable; @@ -31,10 +32,13 @@ import org.apache.kafka.streams.kstream.internals.graph.StreamsGraphNode; import org.apache.kafka.streams.state.StoreBuilder; import org.apache.kafka.streams.state.Stores; +import org.apache.kafka.streams.state.ValueAndTimestamp; import org.apache.kafka.streams.state.WindowBytesStoreSupplier; import org.apache.kafka.streams.state.WindowStore; +import org.apache.kafka.streams.state.internals.WindowStoreBuilder; import java.time.Duration; +import java.util.Map; import java.util.Objects; import java.util.Set; @@ -163,13 +167,13 @@ private StoreBuilder> materialize(final MaterializedInte if ((windows.size() + windows.gracePeriodMs()) > retentionPeriod) { throw new IllegalArgumentException("The retention period of the window store " - + name + " must be no smaller than its window size plus the grace period." - + " Got size=[" + windows.size() + "]," - + " grace=[" + windows.gracePeriodMs() + "]," - + " retention=[" + retentionPeriod + "]"); + + name + " must be no smaller than its window size plus the grace period." + + " Got size=[" + windows.size() + "]," + + " grace=[" + windows.gracePeriodMs() + "]," + + " retention=[" + retentionPeriod + "]"); } - supplier = Stores.persistentWindowStore( + supplier = Stores.persistentWindowWithTimestampStore( materialized.storeName(), Duration.ofMillis(retentionPeriod), Duration.ofMillis(windows.size()), @@ -184,10 +188,10 @@ private StoreBuilder> materialize(final MaterializedInte if ((windows.size() + windows.gracePeriodMs()) > windows.maintainMs()) { throw new IllegalArgumentException("The retention period of the window store " - + name + " must be no smaller than its window size plus the grace period." - + " Got size=[" + windows.size() + "]," - + " grace=[" + windows.gracePeriodMs() + "]," - + " retention=[" + windows.maintainMs() + "]"); + + name + " must be no smaller than its window size plus the grace period." + + " Got size=[" + windows.size() + "]," + + " grace=[" + windows.gracePeriodMs() + "]," + + " retention=[" + windows.maintainMs() + "]"); } supplier = Stores.persistentWindowStore( @@ -199,11 +203,59 @@ private StoreBuilder> materialize(final MaterializedInte ); } } - final StoreBuilder> builder = Stores.windowStoreBuilder( - supplier, - materialized.keySerde(), - materialized.valueSerde() - ); + + final WindowBytesStoreSupplier innerSupplier = supplier; + final WindowStoreBuilder builder = new WindowStoreBuilder(supplier, null, null, Time.SYSTEM) { + StoreBuilder>> inner = Stores.windowWithTimestampStoreBuilder( + innerSupplier, + materialized.keySerde(), + materialized.valueSerde() + ); + + @Override + public WindowStoreBuilder withCachingEnabled() { + inner.withCachingEnabled(); + return this; + } + + @Override + public WindowStoreBuilder withCachingDisabled() { + inner.withCachingDisabled(); + return this; + } + + @Override + public WindowStoreBuilder withLoggingEnabled(final Map config) { + inner.withLoggingEnabled(config); + return this; + } + + @Override + public WindowStoreBuilder withLoggingDisabled() { + inner.withLoggingDisabled(); + return this; + } + + @Override + public Map logConfig() { + return inner.logConfig(); + } + + @Override + public boolean loggingEnabled() { + return inner.loggingEnabled(); + } + + @Override + public String name() { + return inner.name(); + } + + @Override + public WindowStore build() { + return new KStreamImpl.WindowStoreFacade<>(inner.build()); + } + }; if (materialized.loggingEnabled()) { builder.withLoggingEnabled(materialized.logConfig()); diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/TupleForwarder.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/TupleForwarder.java index 127057f20d769..f9d79a74dd12d 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/TupleForwarder.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/TupleForwarder.java @@ -18,6 +18,7 @@ import org.apache.kafka.streams.processor.ProcessorContext; import org.apache.kafka.streams.processor.StateStore; +import org.apache.kafka.streams.processor.To; import org.apache.kafka.streams.state.internals.CachedStateStore; import org.apache.kafka.streams.state.internals.WrappedStateStore; @@ -38,13 +39,26 @@ class TupleForwarder { final ProcessorContext context, final ForwardingCacheFlushListener flushListener, final boolean sendOldValues) { - this.cachedStateStore = cachedStateStore(store); + this.cachedStateStore = cachedStateStore(maybeRemoveStoreFacade(store)); this.context = context; if (this.cachedStateStore != null) { cachedStateStore.setFlushListener(flushListener, sendOldValues); } } + private StateStore maybeRemoveStoreFacade(final StateStore store) { + if (store instanceof KeyValueWithTimestampStoreMaterializer.KeyValueStoreFacade) { + return ((KeyValueWithTimestampStoreMaterializer.KeyValueStoreFacade) store).inner; + } + if (store instanceof KStreamImpl.WindowStoreFacade) { + return ((KStreamImpl.WindowStoreFacade) store).inner; + } + if (store instanceof SessionWindowedKStreamImpl.SessionStoreFacade) { + return ((SessionWindowedKStreamImpl.SessionStoreFacade) store).inner; + } + return store; + } + private CachedStateStore cachedStateStore(final StateStore store) { if (store instanceof CachedStateStore) { return (CachedStateStore) store; @@ -71,4 +85,14 @@ public void maybeForward(final K key, context.forward(key, new Change<>(newValue, oldValue)); } } + + public void maybeForward(final K key, + final V newValue, + final V oldValue, + final long timestamp) { + if (cachedStateStore == null) { + context.forward(key, new Change<>(newValue, oldValue), To.all().withTimestamp(timestamp)); + } + } + } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/KTableKTableJoinNode.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/KTableKTableJoinNode.java index aeda0d9e19d13..911d279e3fd6d 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/KTableKTableJoinNode.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/KTableKTableJoinNode.java @@ -20,7 +20,7 @@ import org.apache.kafka.common.utils.Bytes; import org.apache.kafka.streams.kstream.ValueJoiner; import org.apache.kafka.streams.kstream.internals.Change; -import org.apache.kafka.streams.kstream.internals.KeyValueStoreMaterializer; +import org.apache.kafka.streams.kstream.internals.KeyValueWithTimestampStoreMaterializer; import org.apache.kafka.streams.kstream.internals.MaterializedInternal; import org.apache.kafka.streams.processor.internals.InternalTopologyBuilder; import org.apache.kafka.streams.state.KeyValueStore; @@ -87,7 +87,7 @@ public void writeToTopology(final InternalTopologyBuilder topologyBuilder) { if (materializedInternal != null) { final StoreBuilder> storeBuilder = - new KeyValueStoreMaterializer<>(materializedInternal).materialize(); + new KeyValueWithTimestampStoreMaterializer<>(materializedInternal).materialize(); topologyBuilder.addStateStore(storeBuilder, mergeProcessorName); } } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/StreamStreamJoinNode.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/StreamStreamJoinNode.java index d080c18ad0a5e..b696999a56804 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/StreamStreamJoinNode.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/StreamStreamJoinNode.java @@ -34,7 +34,6 @@ public class StreamStreamJoinNode extends BaseJoinProcessorNode> otherWindowStoreBuilder; private final Joined joined; - private StreamStreamJoinNode(final String nodeName, final ValueJoiner valueJoiner, final ProcessorParameters joinThisProcessorParameters, diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/TableProcessorNode.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/TableProcessorNode.java index 0409c62818e55..5743ff837a43e 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/TableProcessorNode.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/TableProcessorNode.java @@ -18,7 +18,7 @@ package org.apache.kafka.streams.kstream.internals.graph; import org.apache.kafka.common.utils.Bytes; -import org.apache.kafka.streams.kstream.internals.KeyValueStoreMaterializer; +import org.apache.kafka.streams.kstream.internals.KeyValueWithTimestampStoreMaterializer; import org.apache.kafka.streams.kstream.internals.MaterializedInternal; import org.apache.kafka.streams.processor.StateStore; import org.apache.kafka.streams.processor.internals.InternalTopologyBuilder; @@ -67,7 +67,7 @@ public void writeToTopology(final InternalTopologyBuilder topologyBuilder) { if (shouldMaterialize) { // TODO: we are enforcing this as a keyvalue store, but it should go beyond any type of stores topologyBuilder.addStateStore( - new KeyValueStoreMaterializer<>( + new KeyValueWithTimestampStoreMaterializer<>( (MaterializedInternal>) materializedInternal ).materialize(), processorName); diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/TableSourceNode.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/TableSourceNode.java index 53061dc88ea58..b244309aa5ccd 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/TableSourceNode.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/TableSourceNode.java @@ -20,7 +20,7 @@ import org.apache.kafka.common.utils.Bytes; import org.apache.kafka.streams.kstream.internals.ConsumedInternal; import org.apache.kafka.streams.kstream.internals.KTableSource; -import org.apache.kafka.streams.kstream.internals.KeyValueStoreMaterializer; +import org.apache.kafka.streams.kstream.internals.KeyValueWithTimestampStoreMaterializer; import org.apache.kafka.streams.kstream.internals.MaterializedInternal; import org.apache.kafka.streams.processor.internals.InternalTopologyBuilder; import org.apache.kafka.streams.state.KeyValueStore; @@ -79,7 +79,7 @@ public void writeToTopology(final InternalTopologyBuilder topologyBuilder) { // TODO: we assume source KTables can only be key-value stores for now. // should be expanded for other types of stores as well. final StoreBuilder> storeBuilder = - new KeyValueStoreMaterializer<>((MaterializedInternal>) materializedInternal).materialize(); + new KeyValueWithTimestampStoreMaterializer<>((MaterializedInternal>) materializedInternal).materialize(); if (isGlobalKTable) { topologyBuilder.addGlobalStore(storeBuilder, diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalProcessorContextImpl.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalProcessorContextImpl.java index ccf033799b167..56a39e1213c8a 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalProcessorContextImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalProcessorContextImpl.java @@ -115,4 +115,4 @@ public Cancellable schedule(final Duration interval, final PunctuationType type, public long streamTime() { throw new UnsupportedOperationException("Stream-time is not defined for global tasks."); } -} \ No newline at end of file +} diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilder.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilder.java index 0648fecc5c5e1..f28e22f6d03f0 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilder.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilder.java @@ -29,7 +29,9 @@ import org.apache.kafka.streams.processor.TopicNameExtractor; import org.apache.kafka.streams.state.StoreBuilder; import org.apache.kafka.streams.state.internals.SessionStoreBuilder; +import org.apache.kafka.streams.state.internals.SessionWithTimestampStoreBuilder; import org.apache.kafka.streams.state.internals.WindowStoreBuilder; +import org.apache.kafka.streams.state.internals.WindowWithTimestampStoreBuilder; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -140,8 +142,12 @@ public StateStore build() { long retentionPeriod() { if (builder instanceof WindowStoreBuilder) { return ((WindowStoreBuilder) builder).retentionPeriod(); + } else if (builder instanceof WindowWithTimestampStoreBuilder) { + return ((WindowWithTimestampStoreBuilder) builder).retentionPeriod(); } else if (builder instanceof SessionStoreBuilder) { return ((SessionStoreBuilder) builder).retentionPeriod(); + } else if (builder instanceof SessionWithTimestampStoreBuilder) { + return ((SessionWithTimestampStoreBuilder) builder).retentionPeriod(); } else { throw new IllegalStateException("retentionPeriod is not supported when not a window store"); } @@ -160,7 +166,10 @@ private String name() { } private boolean isWindowStore() { - return builder instanceof WindowStoreBuilder || builder instanceof SessionStoreBuilder; + return builder instanceof WindowStoreBuilder + || builder instanceof WindowWithTimestampStoreBuilder + || builder instanceof SessionStoreBuilder + || builder instanceof SessionWithTimestampStoreBuilder; } // Apparently Java strips the generics from this method because we're using the raw type for builder, diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java index 9ecc73c14e4d1..cbfd510f5c96a 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java @@ -83,7 +83,9 @@ public StateStore getStateStore(final String name) { final StateStore global = stateManager.getGlobalStore(name); if (global != null) { - if (global instanceof KeyValueStore) { + if (global instanceof StateStoreReadOnlyDecorator) { + return global; + } else if (global instanceof KeyValueStore) { return new KeyValueStoreReadOnlyDecorator((KeyValueStore) global); } else if (global instanceof WindowStore) { return new WindowStoreReadOnlyDecorator((WindowStore) global); @@ -105,7 +107,9 @@ public StateStore getStateStore(final String name) { } final StateStore store = stateManager.getStore(name); - if (store instanceof KeyValueStore) { + if (store instanceof StateStoreReadWriteDecorator) { + return global; + } else if (store instanceof KeyValueStore) { return new KeyValueStoreReadWriteDecorator((KeyValueStore) store); } else if (store instanceof WindowStore) { return new WindowStoreReadWriteDecorator((WindowStore) store); @@ -273,24 +277,24 @@ public long approximateNumEntries() { } @Override - public void put(final K key, - final V value) { + public final void put(final K key, + final V value) { throw new UnsupportedOperationException(ERROR_MESSAGE); } @Override - public V putIfAbsent(final K key, - final V value) { + public final V putIfAbsent(final K key, + final V value) { throw new UnsupportedOperationException(ERROR_MESSAGE); } @Override - public void putAll(final List entries) { + public final void putAll(final List entries) { throw new UnsupportedOperationException(ERROR_MESSAGE); } @Override - public V delete(final K key) { + public final V delete(final K key) { throw new UnsupportedOperationException(ERROR_MESSAGE); } } @@ -304,15 +308,15 @@ private WindowStoreReadOnlyDecorator(final WindowStore inner) { } @Override - public void put(final K key, - final V value) { + public final void put(final K key, + final V value) { throw new UnsupportedOperationException(ERROR_MESSAGE); } @Override - public void put(final K key, - final V value, - final long windowStartTimestamp) { + public final void put(final K key, + final V value, + final long windowStartTimestamp) { throw new UnsupportedOperationException(ERROR_MESSAGE); } @@ -376,13 +380,13 @@ public KeyValueIterator, AGG> findSessions(final K keyFrom, } @Override - public void remove(final Windowed sessionKey) { + public final void remove(final Windowed sessionKey) { throw new UnsupportedOperationException(ERROR_MESSAGE); } @Override - public void put(final Windowed sessionKey, - final AGG aggregate) { + public final void put(final Windowed sessionKey, + final AGG aggregate) { throw new UnsupportedOperationException(ERROR_MESSAGE); } @@ -398,35 +402,35 @@ public KeyValueIterator, AGG> fetch(final K from, } } - private abstract static class StateStoreReadWriteDecorator extends AbstractStateStore { + public abstract static class StateStoreReadWriteDecorator extends AbstractStateStore { static final String ERROR_MESSAGE = "This method may only be called by Kafka Streams"; - private StateStoreReadWriteDecorator(final T inner) { + protected StateStoreReadWriteDecorator(final T inner) { super(inner); } @SuppressWarnings("unchecked") - T wrapped() { + protected T wrapped() { return (T) super.wrappedStore(); } @Override - public void init(final ProcessorContext context, - final StateStore root) { + public final void init(final ProcessorContext context, + final StateStore root) { throw new UnsupportedOperationException(ERROR_MESSAGE); } @Override - public void close() { + public final void close() { throw new UnsupportedOperationException(ERROR_MESSAGE); } } - private static class KeyValueStoreReadWriteDecorator + public static class KeyValueStoreReadWriteDecorator extends StateStoreReadWriteDecorator> implements KeyValueStore { - private KeyValueStoreReadWriteDecorator(final KeyValueStore inner) { + public KeyValueStoreReadWriteDecorator(final KeyValueStore inner) { super(inner); } @@ -474,11 +478,11 @@ public V delete(final K key) { } } - private static class WindowStoreReadWriteDecorator + public static class WindowStoreReadWriteDecorator extends StateStoreReadWriteDecorator> implements WindowStore { - private WindowStoreReadWriteDecorator(final WindowStore inner) { + public WindowStoreReadWriteDecorator(final WindowStore inner) { super(inner); } @@ -531,11 +535,11 @@ public KeyValueIterator, V> fetchAll(final long timeFrom, } } - private static class SessionStoreReadWriteDecorator + public static class SessionStoreReadWriteDecorator extends StateStoreReadWriteDecorator> implements SessionStore { - private SessionStoreReadWriteDecorator(final SessionStore inner) { + public SessionStoreReadWriteDecorator(final SessionStore inner) { super(inner); } @@ -560,7 +564,8 @@ public void remove(final Windowed sessionKey) { } @Override - public void put(final Windowed sessionKey, final AGG aggregate) { + public void put(final Windowed sessionKey, + final AGG aggregate) { wrapped().put(sessionKey, aggregate); } @@ -575,4 +580,5 @@ public KeyValueIterator, AGG> fetch(final K from, return wrapped().fetch(from, to); } } + } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorNode.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorNode.java index 01e3e56bbb5b6..ebaac0595c904 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorNode.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorNode.java @@ -104,6 +104,9 @@ public void close() { if (processor != null) { processor.close(); } + if (nodeMetrics == null) { + throw new IllegalStateException(); + } nodeMetrics.nodeDestructionSensor.record(time.nanoseconds() - startNs); nodeMetrics.removeAllSensors(); } catch (final Exception e) { diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorStateManager.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorStateManager.java index 669eb1527122f..a5e353d0451fa 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorStateManager.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorStateManager.java @@ -23,8 +23,8 @@ import org.apache.kafka.streams.processor.StateRestoreCallback; import org.apache.kafka.streams.processor.StateStore; import org.apache.kafka.streams.processor.TaskId; -import org.apache.kafka.streams.state.internals.OffsetCheckpoint; import org.apache.kafka.streams.state.RecordConverter; +import org.apache.kafka.streams.state.internals.OffsetCheckpoint; import org.apache.kafka.streams.state.internals.WrappedStateStore; import org.slf4j.Logger; diff --git a/streams/src/main/java/org/apache/kafka/streams/state/QueryableStoreTypes.java b/streams/src/main/java/org/apache/kafka/streams/state/QueryableStoreTypes.java index 297e18157edfc..aa50776431534 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/QueryableStoreTypes.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/QueryableStoreTypes.java @@ -42,6 +42,17 @@ public static QueryableStoreType> keyValueSto return new KeyValueStoreType<>(); } + /** + * A {@link QueryableStoreType} that accepts {@link ReadOnlyKeyValueStore ReadOnlyKeyValueStore>}. + * + * @param key type of the store + * @param value type of the store + * @return {@link QueryableStoreTypes.KeyValueWithTimestampStoreType} + */ + public static QueryableStoreType>> keyValueWithTimestampStore() { + return new KeyValueWithTimestampStoreType<>(); + } + /** * A {@link QueryableStoreType} that accepts {@link ReadOnlyWindowStore}. * @@ -53,6 +64,17 @@ public static QueryableStoreType> windowStore() return new WindowStoreType<>(); } + /** + * A {@link QueryableStoreType} that accepts {@link ReadOnlyWindowStore ReadOnlyWindowStore>}. + * + * @param key type of the store + * @param value type of the store + * @return {@link QueryableStoreTypes.WindowWithTimestampStoreType} + */ + public static QueryableStoreType>> windowWithTimestampStore() { + return new WindowWithTimestampStoreType<>(); + } + /** * A {@link QueryableStoreType} that accepts {@link ReadOnlySessionStore}. * @@ -64,6 +86,17 @@ public static QueryableStoreType> sessionStore return new SessionStoreType<>(); } + /** + * A {@link QueryableStoreType} that accepts {@link ReadOnlySessionStore ReadOnlySessionStore>}. + * + * @param key type of the store + * @param value type of the store + * @return {@link QueryableStoreTypes.SessionWithTimestampStoreType} + */ + public static QueryableStoreType>> sessionWithTimestampStore() { + return new SessionStoreType<>(); + } + private static abstract class QueryableStoreTypeMatcher implements QueryableStoreType { private final Class matchTo; @@ -79,7 +112,8 @@ public boolean accepts(final StateStore stateStore) { } } - static class KeyValueStoreType extends QueryableStoreTypeMatcher> { + public static class KeyValueStoreType extends QueryableStoreTypeMatcher> { + KeyValueStoreType() { super(ReadOnlyKeyValueStore.class); } @@ -92,7 +126,21 @@ public ReadOnlyKeyValueStore create(final StateStoreProvider storeProvider } - static class WindowStoreType extends QueryableStoreTypeMatcher> { + private static class KeyValueWithTimestampStoreType extends QueryableStoreTypeMatcher>> { + + KeyValueWithTimestampStoreType() { + super(ReadOnlyKeyValueStore.class); + } + + @Override + public ReadOnlyKeyValueStore> create(final StateStoreProvider storeProvider, + final String storeName) { + return new CompositeReadOnlyKeyValueStore<>(storeProvider, this, storeName); + } + } + + public static class WindowStoreType extends QueryableStoreTypeMatcher> { + WindowStoreType() { super(ReadOnlyWindowStore.class); } @@ -104,14 +152,41 @@ public ReadOnlyWindowStore create(final StateStoreProvider storeProvider, } } - static class SessionStoreType extends QueryableStoreTypeMatcher> { + private static class WindowWithTimestampStoreType extends QueryableStoreTypeMatcher>> { + + WindowWithTimestampStoreType() { + super(ReadOnlyWindowStore.class); + } + + @Override + public ReadOnlyWindowStore> create(final StateStoreProvider storeProvider, + final String storeName) { + return new CompositeReadOnlyWindowStore<>(storeProvider, this, storeName); + } + } + + public static class SessionStoreType extends QueryableStoreTypeMatcher> { + SessionStoreType() { super(ReadOnlySessionStore.class); } + @Override public ReadOnlySessionStore create(final StateStoreProvider storeProvider, final String storeName) { return new CompositeReadOnlySessionStore<>(storeProvider, this, storeName); } } + + private static class SessionWithTimestampStoreType extends QueryableStoreTypeMatcher>> { + + SessionWithTimestampStoreType() { + super(ReadOnlySessionStore.class); + } + + @Override + public ReadOnlySessionStore> create(final StateStoreProvider storeProvider, final String storeName) { + return new CompositeReadOnlySessionStore<>(storeProvider, this, storeName); + } + } } diff --git a/streams/src/main/java/org/apache/kafka/streams/state/Stores.java b/streams/src/main/java/org/apache/kafka/streams/state/Stores.java index 8236c1ad010cb..dd8b91e6ccf1c 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/Stores.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/Stores.java @@ -24,12 +24,18 @@ import org.apache.kafka.streams.internals.ApiUtils; import org.apache.kafka.streams.state.internals.InMemoryKeyValueStore; import org.apache.kafka.streams.state.internals.KeyValueStoreBuilder; +import org.apache.kafka.streams.state.internals.KeyValueWithTimestampStoreBuilder; import org.apache.kafka.streams.state.internals.MemoryNavigableLRUCache; import org.apache.kafka.streams.state.internals.RocksDbKeyValueBytesStoreSupplier; +import org.apache.kafka.streams.state.internals.RocksDbKeyValueWithTimestampBytesStoreSupplier; import org.apache.kafka.streams.state.internals.RocksDbSessionBytesStoreSupplier; +import org.apache.kafka.streams.state.internals.RocksDbSessionWithTimestampBytesStoreSupplier; import org.apache.kafka.streams.state.internals.RocksDbWindowBytesStoreSupplier; +import org.apache.kafka.streams.state.internals.RocksDbWindowWithTimestampBytesStoreSupplier; import org.apache.kafka.streams.state.internals.SessionStoreBuilder; +import org.apache.kafka.streams.state.internals.SessionWithTimestampStoreBuilder; import org.apache.kafka.streams.state.internals.WindowStoreBuilder; +import org.apache.kafka.streams.state.internals.WindowWithTimestampStoreBuilder; import java.time.Duration; import java.util.Objects; @@ -74,21 +80,45 @@ * } */ @InterfaceStability.Evolving -public class Stores { +public final class Stores { /** * Create a persistent {@link KeyValueBytesStoreSupplier}. + *

+ * This store supplier can be passed into a {@link KeyValueStoreBuilder}. + * If you want to create a {@link KeyValueWithTimestampStoreBuilder KeyValueWithTimestampStore} you should use + * {@link #persistentKeyValueWithTimestampStore(String)} to create a store supplier instead. + * * @param name name of the store (cannot be {@code null}) * @return an instance of a {@link KeyValueBytesStoreSupplier} that can be used - * to build a persistent store + * to build a persistent key-value store */ public static KeyValueBytesStoreSupplier persistentKeyValueStore(final String name) { Objects.requireNonNull(name, "name cannot be null"); return new RocksDbKeyValueBytesStoreSupplier(name); } + /** + * Create a persistent {@link KeyValueBytesStoreSupplier}. + *

+ * This store supplier can be passed into a {@link KeyValueWithTimestampStoreBuilder}. + * If you want to create a {@link KeyValueStoreBuilder KeyValueStore} you should use + * {@link #persistentKeyValueStore(String)} to create a store supplier instead. + * + * @param name name of the store (cannot be {@code null}) + * @return an instance of a {@link KeyValueBytesStoreSupplier} that can be used + * to build a persistent store + */ + public static KeyValueBytesStoreSupplier persistentKeyValueWithTimestampStore(final String name) { + Objects.requireNonNull(name, "name cannot be null"); + return new RocksDbKeyValueWithTimestampBytesStoreSupplier(name); + } + /** * Create an in-memory {@link KeyValueBytesStoreSupplier}. + *

+ * This store supplier can be passed into a {@link KeyValueStoreBuilder} or {@link KeyValueWithTimestampStoreBuilder}. + * * @param name name of the store (cannot be {@code null}) * @return an instance of a {@link KeyValueBytesStoreSupplier} than can be used to * build an in-memory store @@ -115,6 +145,9 @@ public String metricsScope() { /** * Create a LRU Map {@link KeyValueBytesStoreSupplier}. + *

+ * This store supplier can be passed into a {@link KeyValueStoreBuilder} or {@link KeyValueWithTimestampStoreBuilder}. + * * @param name name of the store (cannot be {@code null}) * @param maxCacheSize maximum number of items in the LRU (cannot be negative) * @return an instance of a {@link KeyValueBytesStoreSupplier} that can be used to build @@ -146,10 +179,10 @@ public String metricsScope() { /** * Create a persistent {@link WindowBytesStoreSupplier}. * @param name name of the store (cannot be {@code null}) - * @param retentionPeriod length of time to retain data in the store (cannot be negative). - * Note that the retention period must be at least long enough to contain the + * @param retentionPeriod length of time to retain data in the store (cannot be negative) + * (note that the retention period must be at least long enough to contain the * windowed data's entire life cycle, from window-start through window-end, - * and for the entire grace period. + * and for the entire grace period) * @param numSegments number of db segments (cannot be zero or negative) * @param windowSize size of the windows that are stored (cannot be negative). Note: the window size * is not stored with the records, so this value is used to compute the keys that @@ -176,17 +209,23 @@ public static WindowBytesStoreSupplier persistentWindowStore(final String name, retentionPeriod, windowSize, retainDuplicates, - legacySegmentInterval + legacySegmentInterval, + false ); } /** * Create a persistent {@link WindowBytesStoreSupplier}. + *

+ * This store supplier can be passed into a {@link WindowStoreBuilder}. + * If you want to create a {@link WindowWithTimestampStoreBuilder WindowWithTimestampStore} you should use + * {@link #persistentWindowWithTimestampStore(String, Duration, Duration, boolean)} to create a store supplier instead. + * * @param name name of the store (cannot be {@code null}) * @param retentionPeriod length of time to retain data in the store (cannot be negative) - * Note that the retention period must be at least long enough to contain the + * (note that the retention period must be at least long enough to contain the * windowed data's entire life cycle, from window-start through window-end, - * and for the entire grace period. + * and for the entire grace period) * @param windowSize size of the windows (cannot be negative) * @param retainDuplicates whether or not to retain duplicates. * @return an instance of {@link WindowBytesStoreSupplier} @@ -196,6 +235,38 @@ public static WindowBytesStoreSupplier persistentWindowStore(final String name, final Duration retentionPeriod, final Duration windowSize, final boolean retainDuplicates) throws IllegalArgumentException { + return persistentWindowStore(name, retentionPeriod, windowSize, retainDuplicates, false); + } + + /** + * Create a persistent {@link WindowBytesStoreSupplier}. + *

+ * This store supplier can be passed into a {@link WindowWithTimestampStoreBuilder}. + * If you want to create a {@link WindowStoreBuilder WindowWithTimestampStore} you should use + * {@link #persistentWindowStore(String, Duration, Duration, boolean)} to create a store supplier instead. + * + * @param name name of the store (cannot be {@code null}) + * @param retentionPeriod length of time to retain data in the store (cannot be negative) + * (note that the retention period must be at least long enough to contain the + * windowed data's entire life cycle, from window-start through window-end, + * and for the entire grace period) + * @param windowSize size of the windows (cannot be negative) + * @param retainDuplicates whether or not to retain duplicates. + * @return an instance of {@link WindowBytesStoreSupplier} + * @throws IllegalArgumentException if {@code retentionPeriod} or {@code windowSize} can't be represented as {@code long milliseconds} + */ + public static WindowBytesStoreSupplier persistentWindowWithTimestampStore(final String name, + final Duration retentionPeriod, + final Duration windowSize, + final boolean retainDuplicates) throws IllegalArgumentException { + return persistentWindowStore(name, retentionPeriod, windowSize, retainDuplicates, true); + } + + private static WindowBytesStoreSupplier persistentWindowStore(final String name, + final Duration retentionPeriod, + final Duration windowSize, + final boolean retainDuplicates, + final boolean withTimestampStore) { Objects.requireNonNull(name, "name cannot be null"); final String rpMsgPrefix = prepareMillisCheckFailMsgPrefix(retentionPeriod, "retentionPeriod"); final long retentionMs = ApiUtils.validateMillisecondDuration(retentionPeriod, rpMsgPrefix); @@ -204,14 +275,15 @@ public static WindowBytesStoreSupplier persistentWindowStore(final String name, final long defaultSegmentInterval = Math.max(retentionMs / 2, 60_000L); - return persistentWindowStore(name, retentionMs, windowSizeMs, retainDuplicates, defaultSegmentInterval); + return persistentWindowStore(name, retentionMs, windowSizeMs, retainDuplicates, defaultSegmentInterval, withTimestampStore); } private static WindowBytesStoreSupplier persistentWindowStore(final String name, final long retentionPeriod, final long windowSize, final boolean retainDuplicates, - final long segmentInterval) { + final long segmentInterval, + final boolean withTimestampStore) { Objects.requireNonNull(name, "name cannot be null"); if (retentionPeriod < 0L) { throw new IllegalArgumentException("retentionPeriod cannot be negative"); @@ -224,35 +296,40 @@ private static WindowBytesStoreSupplier persistentWindowStore(final String name, } if (windowSize > retentionPeriod) { throw new IllegalArgumentException("The retention period of the window store " - + name + " must be no smaller than its window size. Got size=[" - + windowSize + "], retention=[" + retentionPeriod + "]"); + + name + " must be no smaller than its window size. Got size=[" + + windowSize + "], retention=[" + retentionPeriod + "]"); } - return new RocksDbWindowBytesStoreSupplier(name, retentionPeriod, segmentInterval, windowSize, retainDuplicates); + if (withTimestampStore) { + return new RocksDbWindowWithTimestampBytesStoreSupplier(name, retentionPeriod, segmentInterval, windowSize, retainDuplicates); + } else { + return new RocksDbWindowBytesStoreSupplier(name, retentionPeriod, segmentInterval, windowSize, retainDuplicates); + } } /** * Create a persistent {@link SessionBytesStoreSupplier}. * @param name name of the store (cannot be {@code null}) * @param retentionPeriod length ot time to retain data in the store (cannot be negative) - * Note that the retention period must be at least long enough to contain the + * (note that the retention period must be at least long enough to contain the * windowed data's entire life cycle, from window-start through window-end, - * and for the entire grace period. + * and for the entire grace period) * @return an instance of a {@link SessionBytesStoreSupplier} * @deprecated since 2.1 Use {@link Stores#persistentSessionStore(String, Duration)} instead */ @Deprecated public static SessionBytesStoreSupplier persistentSessionStore(final String name, final long retentionPeriod) { - Objects.requireNonNull(name, "name cannot be null"); - if (retentionPeriod < 0) { - throw new IllegalArgumentException("retentionPeriod cannot be negative"); - } - return new RocksDbSessionBytesStoreSupplier(name, retentionPeriod); + return persistentSessionStore(name, retentionPeriod, false); } /** * Create a persistent {@link SessionBytesStoreSupplier}. + *

+ * This store supplier can be passed into a {@link SessionStoreBuilder}. + * If you want to create a {@link SessionWithTimestampStoreBuilder SessionWithTimestampStore} you should use + * {@link #persistentSessionWithTimestampStore(String, Duration)} to create a store supplier instead. + * * @param name name of the store (cannot be {@code null}) * @param retentionPeriod length ot time to retain data in the store (cannot be negative) * Note that the retention period must be at least long enough to contain the @@ -260,36 +337,51 @@ public static SessionBytesStoreSupplier persistentSessionStore(final String name * and for the entire grace period. * @return an instance of a {@link SessionBytesStoreSupplier} */ - @SuppressWarnings("deprecation") public static SessionBytesStoreSupplier persistentSessionStore(final String name, final Duration retentionPeriod) { final String msgPrefix = prepareMillisCheckFailMsgPrefix(retentionPeriod, "retentionPeriod"); - return persistentSessionStore(name, ApiUtils.validateMillisecondDuration(retentionPeriod, msgPrefix)); + return persistentSessionStore(name, ApiUtils.validateMillisecondDuration(retentionPeriod, msgPrefix), false); } - /** - * Creates a {@link StoreBuilder} that can be used to build a {@link WindowStore}. - * @param supplier a {@link WindowBytesStoreSupplier} (cannot be {@code null}) - * @param keySerde the key serde to use - * @param valueSerde the value serde to use; if the serialized bytes is null for put operations, - * it is treated as delete - * @param key type - * @param value type - * @return an instance of {@link StoreBuilder} than can build a {@link WindowStore} + * Create a persistent {@link SessionBytesStoreSupplier}. + *

+ * This store supplier can be passed into a {@link SessionWithTimestampStoreBuilder}. + * If you want to create a {@link SessionStoreBuilder SessionStore} you should use + * {@link #persistentSessionStore(String, Duration)} to create a store supplier instead. + * + * @param name name of the store (cannot be {@code null}) + * @param retentionPeriod length ot time to retain data in the store (cannot be negative) + * Note that the retention period must be at least long enough to contain the + * windowed data's entire life cycle, from window-start through window-end, + * and for the entire grace period. + * @return an instance of a {@link SessionBytesStoreSupplier} */ - public static StoreBuilder> windowStoreBuilder(final WindowBytesStoreSupplier supplier, - final Serde keySerde, - final Serde valueSerde) { - Objects.requireNonNull(supplier, "supplier cannot be null"); - return new WindowStoreBuilder<>(supplier, keySerde, valueSerde, Time.SYSTEM); + public static SessionBytesStoreSupplier persistentSessionWithTimestampStore(final String name, + final Duration retentionPeriod) { + final String msgPrefix = prepareMillisCheckFailMsgPrefix(retentionPeriod, "retentionPeriod"); + return persistentSessionStore(name, ApiUtils.validateMillisecondDuration(retentionPeriod, msgPrefix), true); + } + + private static SessionBytesStoreSupplier persistentSessionStore(final String name, + final long retentionPeriod, + final boolean withTimestampStore) { + Objects.requireNonNull(name, "name cannot be null"); + if (retentionPeriod < 0) { + throw new IllegalArgumentException("retentionPeriod cannot be negative"); + } + if (withTimestampStore) { + return new RocksDbSessionWithTimestampBytesStoreSupplier(name, retentionPeriod); + } else { + return new RocksDbSessionBytesStoreSupplier(name, retentionPeriod); + } } /** * Creates a {@link StoreBuilder} than can be used to build a {@link KeyValueStore}. * @param supplier a {@link KeyValueBytesStoreSupplier} (cannot be {@code null}) * @param keySerde the key serde to use - * @param valueSerde the value serde to use; if the serialized bytes is null for put operations, + * @param valueSerde the value serde to use; if the serialized bytes is {@code null} for put operations, * it is treated as delete * @param key type * @param value type @@ -302,21 +394,59 @@ public static StoreBuilder> keyValueStoreBuilder(fina return new KeyValueStoreBuilder<>(supplier, keySerde, valueSerde, Time.SYSTEM); } + public static StoreBuilder>> keyValueWithTimestampStoreBuilder(final KeyValueBytesStoreSupplier supplier, + final Serde keySerde, + final Serde valueSerde) { + Objects.requireNonNull(supplier, "supplier cannot be null"); + return new KeyValueWithTimestampStoreBuilder<>(supplier, keySerde, valueSerde, Time.SYSTEM); + } + + /** + * Creates a {@link StoreBuilder} that can be used to build a {@link WindowStore}. + * @param supplier a {@link WindowBytesStoreSupplier} (cannot be {@code null}) + * @param keySerde the key serde to use + * @param valueSerde the value serde to use; if the serialized bytes is {@code null} for put operations, + * it is treated as delete + * @param key type + * @param value type + * @return an instance of {@link StoreBuilder} than can build a {@link WindowStore} + */ + public static StoreBuilder> windowStoreBuilder(final WindowBytesStoreSupplier supplier, + final Serde keySerde, + final Serde valueSerde) { + Objects.requireNonNull(supplier, "supplier cannot be null"); + return new WindowStoreBuilder<>(supplier, keySerde, valueSerde, Time.SYSTEM); + } + + public static StoreBuilder>> windowWithTimestampStoreBuilder(final WindowBytesStoreSupplier supplier, + final Serde keySerde, + final Serde valueSerde) { + Objects.requireNonNull(supplier, "supplier cannot be null"); + return new WindowWithTimestampStoreBuilder<>(supplier, keySerde, valueSerde, Time.SYSTEM); + } + /** * Creates a {@link StoreBuilder} that can be used to build a {@link SessionStore}. * @param supplier a {@link SessionBytesStoreSupplier} (cannot be {@code null}) * @param keySerde the key serde to use - * @param valueSerde the value serde to use; if the serialized bytes is null for put operations, + * @param valueSerde the value serde to use; if the serialized bytes is {@code null} for put operations, * it is treated as delete * @param key type * @param value type * @return an instance of {@link StoreBuilder} than can build a {@link SessionStore} - * */ + */ public static StoreBuilder> sessionStoreBuilder(final SessionBytesStoreSupplier supplier, final Serde keySerde, final Serde valueSerde) { Objects.requireNonNull(supplier, "supplier cannot be null"); return new SessionStoreBuilder<>(supplier, keySerde, valueSerde, Time.SYSTEM); } -} + public static StoreBuilder>> sessionWithTimestampStoreBuilder(final SessionBytesStoreSupplier supplier, + final Serde keySerde, + final Serde valueSerde) { + Objects.requireNonNull(supplier, "supplier cannot be null"); + return new SessionWithTimestampStoreBuilder<>(supplier, keySerde, valueSerde, Time.SYSTEM); + } + +} diff --git a/streams/src/main/java/org/apache/kafka/streams/state/ValueAndTimestamp.java b/streams/src/main/java/org/apache/kafka/streams/state/ValueAndTimestamp.java new file mode 100644 index 0000000000000..6991cff5e46dc --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/state/ValueAndTimestamp.java @@ -0,0 +1,73 @@ +/* + * 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; + +import org.apache.kafka.streams.KeyValue; + +import java.util.Objects; + +/** + * Combines a value from a {@link KeyValue} with a timestamp. + * + * @param + */ +public class ValueAndTimestamp { + private final V value; + private final long timestamp; + + private ValueAndTimestamp(final V value, + final long timestamp) { + this.value = value; + this.timestamp = timestamp; + } + + public static ValueAndTimestamp make(final V value, + final long timestamp) { + return value == null ? null : new ValueAndTimestamp<>(value, timestamp); + } + + public V value() { + return value; + } + + public long timestamp() { + return timestamp; + } + + @Override + public String toString() { + return "<" + value + "," + timestamp + ">"; + } + + @Override + public boolean equals(final Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + final ValueAndTimestamp that = (ValueAndTimestamp) o; + return timestamp == that.timestamp && + Objects.equals(value, that.value); + } + + @Override + public int hashCode() { + return Objects.hash(value, timestamp); + } +} \ No newline at end of file diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/Segments.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/AbstractSegments.java similarity index 79% rename from streams/src/main/java/org/apache/kafka/streams/state/internals/Segments.java rename to streams/src/main/java/org/apache/kafka/streams/state/internals/AbstractSegments.java index d4aedce353192..acc566349210a 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/Segments.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/AbstractSegments.java @@ -35,18 +35,19 @@ import java.util.TreeMap; /** - * Manages the {@link Segment}s that are used by the {@link RocksDBSegmentedBytesStore} + * Manages the {@link PlainSegment}s or {@link SegmentWithTimestamp} that are used by the {@link RocksDBSegmentedBytesStore} + * and {@link RocksDBSegmentedWithTimestampBytesStore}. */ -class Segments { - private static final Logger log = LoggerFactory.getLogger(Segments.class); +abstract class AbstractSegments { + private static final Logger log = LoggerFactory.getLogger(AbstractSegments.class); - private final TreeMap segments = new TreeMap<>(); - private final String name; + final TreeMap segments = new TreeMap<>(); + final String name; private final long retentionPeriod; private final long segmentInterval; private final SimpleDateFormat formatter; - Segments(final String name, final long retentionPeriod, final long segmentInterval) { + AbstractSegments(final String name, final long retentionPeriod, final long segmentInterval) { this.name = name; this.segmentInterval = segmentInterval; this.retentionPeriod = retentionPeriod; @@ -67,15 +68,15 @@ String segmentName(final long segmentId) { return name + "." + segmentId * segmentInterval; } - Segment getSegmentForTimestamp(final long timestamp) { + S getSegmentForTimestamp(final long timestamp) { return segments.get(segmentId(timestamp)); } - Segment getOrCreateSegmentIfLive(final long segmentId, final InternalProcessorContext context) { + S getOrCreateSegmentIfLive(final long segmentId, final InternalProcessorContext context) { final long minLiveTimestamp = context.streamTime() - retentionPeriod; final long minLiveSegment = segmentId(minLiveTimestamp); - final Segment toReturn; + final S toReturn; if (segmentId >= minLiveSegment) { // The segment is live. get it, ensure it's open, and return it. toReturn = getOrCreateSegment(segmentId, context); @@ -87,21 +88,7 @@ Segment getOrCreateSegmentIfLive(final long segmentId, final InternalProcessorCo return toReturn; } - private Segment getOrCreateSegment(final long segmentId, final InternalProcessorContext context) { - if (segments.containsKey(segmentId)) { - return segments.get(segmentId); - } else { - final Segment newSegment = new Segment(segmentName(segmentId), name, segmentId); - final Segment shouldBeNull = segments.put(segmentId, newSegment); - - if (shouldBeNull != null) { - throw new IllegalStateException("Segment already exists. Possible concurrent access."); - } - - newSegment.openDB(context); - return newSegment; - } - } + abstract S getOrCreateSegment(final long segmentId, final InternalProcessorContext context); void openExisting(final InternalProcessorContext context) { try { @@ -135,13 +122,13 @@ void openExisting(final InternalProcessorContext context) { cleanupEarlierThan(minLiveSegment); } - List segments(final long timeFrom, final long timeTo) { - final List result = new ArrayList<>(); - final NavigableMap segmentsInRange = segments.subMap( + List segments(final long timeFrom, final long timeTo) { + final List result = new ArrayList<>(); + final NavigableMap segmentsInRange = segments.subMap( segmentId(timeFrom), true, segmentId(timeTo), true ); - for (final Segment segment : segmentsInRange.values()) { + for (final S segment : segmentsInRange.values()) { if (segment.isOpen()) { result.add(segment); } @@ -149,9 +136,9 @@ List segments(final long timeFrom, final long timeTo) { return result; } - List allSegments() { - final List result = new ArrayList<>(); - for (final Segment segment : segments.values()) { + List allSegments() { + final List result = new ArrayList<>(); + for (final S segment : segments.values()) { if (segment.isOpen()) { result.add(segment); } @@ -160,26 +147,26 @@ List allSegments() { } void flush() { - for (final Segment segment : segments.values()) { + for (final S segment : segments.values()) { segment.flush(); } } public void close() { - for (final Segment segment : segments.values()) { + for (final S segment : segments.values()) { segment.close(); } segments.clear(); } private void cleanupEarlierThan(final long minLiveSegment) { - final Iterator> toRemove = + final Iterator> toRemove = segments.headMap(minLiveSegment, false).entrySet().iterator(); while (toRemove.hasNext()) { - final Map.Entry next = toRemove.next(); + final Map.Entry next = toRemove.next(); toRemove.remove(); - final Segment segment = next.getValue(); + final S segment = next.getValue(); segment.close(); try { segment.destroy(); diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingKeyValueStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingKeyValueStore.java index 9ecae5a8c95e8..7ca0f8ba98976 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingKeyValueStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingKeyValueStore.java @@ -36,15 +36,15 @@ class CachingKeyValueStore extends WrappedStateStore.AbstractStateStore implements KeyValueStore, CachedStateStore { - private final KeyValueStore underlying; + final KeyValueStore underlying; private final Serde keySerde; private final Serde valueSerde; - private CacheFlushListener flushListener; - private boolean sendOldValues; + CacheFlushListener flushListener; + boolean sendOldValues; private String cacheName; private ThreadCache cache; private InternalProcessorContext context; - private StateSerdes serdes; + StateSerdes serdes; private Thread streamThread; private final ReadWriteLock lock = new ReentrantReadWriteLock(); @@ -83,8 +83,8 @@ private void initInternal(final ProcessorContext context) { }); } - private void putAndMaybeForward(final ThreadCache.DirtyEntry entry, - final InternalProcessorContext context) { + void putAndMaybeForward(final ThreadCache.DirtyEntry entry, + final InternalProcessorContext context) { final ProcessorRecordContext current = context.recordContext(); try { context.setRecordContext(entry.entry().context()); diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingKeyValueWithTimestampStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingKeyValueWithTimestampStore.java new file mode 100644 index 0000000000000..a24ded947bd8e --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingKeyValueWithTimestampStore.java @@ -0,0 +1,71 @@ +/* + * 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.serialization.Serde; +import org.apache.kafka.common.utils.Bytes; +import org.apache.kafka.streams.processor.internals.InternalProcessorContext; +import org.apache.kafka.streams.processor.internals.ProcessorRecordContext; +import org.apache.kafka.streams.state.KeyValueStore; + +class CachingKeyValueWithTimestampStore extends CachingKeyValueStore { + + CachingKeyValueWithTimestampStore(final KeyValueStore underlying, final Serde keySerde, final Serde valueSerde) { + super(underlying, keySerde, valueSerde); + } + + @SuppressWarnings("unchecked") + void putAndMaybeForward(final ThreadCache.DirtyEntry entry, final InternalProcessorContext context) { + final ProcessorRecordContext current = context.recordContext(); + try { + context.setRecordContext(entry.entry().context()); + if (flushListener != null) { + V oldValue = null; + if (sendOldValues) { + final byte[] oldRawValueAndTimestamp = underlying.get(entry.key()); + if (oldRawValueAndTimestamp != null) { + final byte[] oldRawValue = new byte[oldRawValueAndTimestamp.length - 8]; + System.arraycopy(oldRawValueAndTimestamp, 8, oldRawValue, 0, oldRawValue.length); + oldValue = ((KeyValueWithTimestampStoreBuilder.ValueAndTimestampDeserializer) serdes.valueDeserializer()) + .valueDeserializer.deserialize(serdes.topic(), oldRawValue); + } + } + // we rely on underlying store to handle null new value bytes as deletes + final byte[] rawValueAndTimestamp = entry.newValue(); + final byte[] rawValue; + if (rawValueAndTimestamp != null) { + rawValue = new byte[rawValueAndTimestamp.length - 8]; + System.arraycopy(rawValueAndTimestamp, 8, rawValue, 0, rawValue.length); + } else { + rawValue = null; + } + + underlying.put(entry.key(), rawValueAndTimestamp); + flushListener.apply( + serdes.keyFrom(entry.key().get()), + ((KeyValueWithTimestampStoreBuilder.ValueAndTimestampDeserializer) serdes.valueDeserializer()) + .valueDeserializer.deserialize(serdes.topic(), rawValue), + oldValue, + entry.entry().context().timestamp()); + } else { + underlying.put(entry.key(), entry.newValue()); + } + } finally { + context.setRecordContext(current); + } + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingSessionStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingSessionStore.java index ccf824c9cd1de..c4d2e5ce202e8 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingSessionStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingSessionStore.java @@ -35,18 +35,19 @@ class CachingSessionStore extends WrappedStateStore.AbstractStateStore implements SessionStore, CachedStateStore, AGG> { - private final SessionStore bytesStore; private final SessionKeySchema keySchema; private final Serde keySerde; private final Serde aggSerde; - private final SegmentedCacheFunction cacheFunction; private String cacheName; private ThreadCache cache; - private StateSerdes serdes; private InternalProcessorContext context; - private CacheFlushListener, AGG> flushListener; - private boolean sendOldValues; - private String topic; + + final SessionStore bytesStore; + final SegmentedCacheFunction cacheFunction; + CacheFlushListener, AGG> flushListener; + boolean sendOldValues; + StateSerdes serdes; + String topic; CachingSessionStore(final SessionStore bytesStore, final Serde keySerde, @@ -164,7 +165,7 @@ public KeyValueIterator, byte[]> fetch(final Bytes from, final B return findSessions(from, to, 0, Long.MAX_VALUE); } - private void putAndMaybeForward(final ThreadCache.DirtyEntry entry, final InternalProcessorContext context) { + void putAndMaybeForward(final ThreadCache.DirtyEntry entry, final InternalProcessorContext context) { final Bytes binaryKey = cacheFunction.key(entry.key()); final ProcessorRecordContext current = context.recordContext(); context.setRecordContext(entry.entry().context()); @@ -188,7 +189,7 @@ private void putAndMaybeForward(final ThreadCache.DirtyEntry entry, final Intern } } - private AGG fetchPrevious(final Bytes rawKey, final Window window) { + AGG fetchPrevious(final Bytes rawKey, final Window window) { try (final KeyValueIterator, byte[]> iterator = bytesStore.findSessions(rawKey, window.start(), window.end())) { if (!iterator.hasNext()) { return null; diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingSessionWithTimestampStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingSessionWithTimestampStore.java new file mode 100644 index 0000000000000..0f05da8e08584 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingSessionWithTimestampStore.java @@ -0,0 +1,82 @@ +/* + * 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.serialization.Serde; +import org.apache.kafka.common.utils.Bytes; +import org.apache.kafka.streams.kstream.Window; +import org.apache.kafka.streams.kstream.Windowed; +import org.apache.kafka.streams.processor.internals.InternalProcessorContext; +import org.apache.kafka.streams.processor.internals.ProcessorRecordContext; +import org.apache.kafka.streams.state.KeyValueIterator; +import org.apache.kafka.streams.state.SessionStore; + +class CachingSessionWithTimestampStore extends CachingSessionStore { + + CachingSessionWithTimestampStore(final SessionStore bytesStore, + final Serde keySerde, + final Serde aggSerde, + final long segmentInterval) { + super(bytesStore, keySerde, aggSerde, segmentInterval); + } + + @SuppressWarnings("unchecked") + void putAndMaybeForward(final ThreadCache.DirtyEntry entry, final InternalProcessorContext context) { + final Bytes binaryKey = cacheFunction.key(entry.key()); + final ProcessorRecordContext current = context.recordContext(); + context.setRecordContext(entry.entry().context()); + try { + final Windowed key = SessionKeySchema.from(binaryKey.get(), serdes.keyDeserializer(), topic); + final Bytes rawKey = Bytes.wrap(serdes.rawKey(key.key())); + if (flushListener != null) { + final byte[] rawValueAndTimestamp = entry.newValue(); + final byte[] rawValue; + if (rawValueAndTimestamp != null) { + rawValue = new byte[rawValueAndTimestamp.length - 8]; + System.arraycopy(rawValueAndTimestamp, 8, rawValue, 0, rawValue.length); + } else { + rawValue = null; + } + final AGG newValue = ((KeyValueWithTimestampStoreBuilder.ValueAndTimestampDeserializer) serdes.valueDeserializer()) + .valueDeserializer.deserialize(serdes.topic(), rawValue); + final AGG oldValue = newValue == null || sendOldValues ? fetchPrevious(rawKey, key.window()) : null; + if (!(newValue == null && oldValue == null)) { + flushListener.apply(key, newValue, oldValue, entry.entry().context().timestamp()); + } + } + bytesStore.put(new Windowed<>(rawKey, key.window()), entry.newValue()); + } finally { + context.setRecordContext(current); + } + } + + @SuppressWarnings("unchecked") + @Override + AGG fetchPrevious(final Bytes rawKey, final Window window) { + try (final KeyValueIterator, byte[]> iterator = bytesStore.findSessions(rawKey, window.start(), window.end())) { + if (!iterator.hasNext()) { + return null; + } + final byte[] rawValueAndTimestamp = iterator.next().value; + final byte[] rawValue = new byte[rawValueAndTimestamp.length - 8]; + System.arraycopy(rawValueAndTimestamp, 8, rawValue, 0, rawValue.length); + return ((KeyValueWithTimestampStoreBuilder.ValueAndTimestampDeserializer) serdes.valueDeserializer()) + .valueDeserializer.deserialize(serdes.topic(), rawValue); + } + } + +} diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingWindowStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingWindowStore.java index afe9b34323081..430016aeba3af 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingWindowStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingWindowStore.java @@ -39,17 +39,16 @@ class CachingWindowStore extends WrappedStateStore.AbstractStateStore impl private final Serde valueSerde; private final long windowSize; private final SegmentedBytesStore.KeySchema keySchema = new WindowKeySchema(); - + private final SegmentedCacheFunction cacheFunction; private String name; private ThreadCache cache; - private boolean sendOldValues; - private StateSerdes serdes; private InternalProcessorContext context; private StateSerdes bytesSerdes; - private CacheFlushListener, V> flushListener; - private final SegmentedCacheFunction cacheFunction; + boolean sendOldValues; + CacheFlushListener, V> flushListener; + StateSerdes serdes; CachingWindowStore(final WindowStore underlying, final Serde keySerde, @@ -100,10 +99,10 @@ public void apply(final List entries) { }); } - private void maybeForward(final ThreadCache.DirtyEntry entry, - final Bytes key, - final Windowed windowedKey, - final InternalProcessorContext context) { + void maybeForward(final ThreadCache.DirtyEntry entry, + final Bytes key, + final Windowed windowedKey, + final InternalProcessorContext context) { if (flushListener != null) { final ProcessorRecordContext current = context.recordContext(); context.setRecordContext(entry.entry().context()); @@ -236,7 +235,7 @@ public KeyValueIterator, byte[]> fetch(final Bytes from, final B ); } - private V fetchPrevious(final Bytes key, final long timestamp) { + V fetchPrevious(final Bytes key, final long timestamp) { final byte[] value = underlying.fetch(key, timestamp); if (value != null) { return serdes.valueFrom(value); diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingWindowWithTimestampStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingWindowWithTimestampStore.java new file mode 100644 index 0000000000000..b4107085e306a --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingWindowWithTimestampStore.java @@ -0,0 +1,67 @@ +/* + * 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.serialization.Serde; +import org.apache.kafka.common.utils.Bytes; +import org.apache.kafka.streams.kstream.Windowed; +import org.apache.kafka.streams.processor.internals.InternalProcessorContext; +import org.apache.kafka.streams.processor.internals.ProcessorRecordContext; +import org.apache.kafka.streams.state.WindowStore; + +class CachingWindowWithTimestampStore extends CachingWindowStore { + + CachingWindowWithTimestampStore(final WindowStore underlying, + final Serde keySerde, + final Serde valueSerde, + final long windowSize, + final long segmentInterval) { + super(underlying, keySerde, valueSerde, windowSize, segmentInterval); + } + + @SuppressWarnings("unchecked") + @Override + void maybeForward(final ThreadCache.DirtyEntry entry, + final Bytes key, + final Windowed windowedKey, + final InternalProcessorContext context) { + if (flushListener != null) { + final ProcessorRecordContext current = context.recordContext(); + context.setRecordContext(entry.entry().context()); + try { + final V oldValue = sendOldValues ? fetchPrevious(key, windowedKey.window().start()) : null; + final byte[] rawValueAndTimestamp = entry.newValue(); + final byte[] rawValue; + if (rawValueAndTimestamp != null) { + rawValue = new byte[rawValueAndTimestamp.length - 8]; + System.arraycopy(rawValueAndTimestamp, 8, rawValue, 0, rawValue.length); + } else { + rawValue = null; + } + flushListener.apply( + windowedKey, + ((KeyValueWithTimestampStoreBuilder.ValueAndTimestampDeserializer) serdes.valueDeserializer()) + .valueDeserializer.deserialize(serdes.topic(), rawValue), + oldValue, + entry.entry().context().timestamp()); + } finally { + context.setRecordContext(current); + } + } + } + +} diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingKeyValueBytesStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingKeyValueBytesStore.java index 94c250c63fb0d..93702ac65a46e 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingKeyValueBytesStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingKeyValueBytesStore.java @@ -29,8 +29,8 @@ import java.util.List; public class ChangeLoggingKeyValueBytesStore extends WrappedStateStore.AbstractStateStore implements KeyValueStore { - private final KeyValueStore inner; - private StoreChangeLogger changeLogger; + final KeyValueStore inner; + StoreChangeLogger changeLogger; ChangeLoggingKeyValueBytesStore(final KeyValueStore inner) { super(inner); diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingKeyValueWithTimestampBytesStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingKeyValueWithTimestampBytesStore.java new file mode 100644 index 0000000000000..dc2cf9d8a771c --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingKeyValueWithTimestampBytesStore.java @@ -0,0 +1,63 @@ +/* + * 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.serialization.LongDeserializer; +import org.apache.kafka.common.utils.Bytes; +import org.apache.kafka.streams.KeyValue; +import org.apache.kafka.streams.state.KeyValueStore; + +import java.util.List; + +public class ChangeLoggingKeyValueWithTimestampBytesStore extends ChangeLoggingKeyValueBytesStore { + private final LongDeserializer longDeserializer = new LongDeserializer(); + + ChangeLoggingKeyValueWithTimestampBytesStore(final KeyValueStore inner) { + super(inner); + } + + @Override + public void put(final Bytes key, + final byte[] valueAndTimestamp) { + if (valueAndTimestamp != null) { + inner.put(key, valueAndTimestamp); + log(key, valueAndTimestamp); + } else { + inner.put(key, null); + changeLogger.logChange(key, null); + } + } + + @Override + public void putAll(final List> entries) { + inner.putAll(entries); + for (final KeyValue entry : entries) { + log(entry.key, entry.value); + } + } + + private void log(final Bytes key, + final byte[] valueAndTimestamp) { + final byte[] rawTimestamp = new byte[8]; + final byte[] rawValue = new byte[valueAndTimestamp.length - 8]; + + System.arraycopy(valueAndTimestamp, 0, rawTimestamp, 0, 8); + System.arraycopy(valueAndTimestamp, 8, rawValue, 0, valueAndTimestamp.length - 8); + + changeLogger.logChange(key, rawValue, longDeserializer.deserialize(null, rawTimestamp)); + } +} \ No newline at end of file diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingSessionWithTimestampBytesStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingSessionWithTimestampBytesStore.java new file mode 100644 index 0000000000000..5645ccf01bd27 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingSessionWithTimestampBytesStore.java @@ -0,0 +1,32 @@ +/* + * 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.utils.Bytes; +import org.apache.kafka.streams.state.SessionStore; + +/** + * Simple wrapper around a {@link SegmentedBytesStore} to support writing + * updates to a changelog + */ +class ChangeLoggingSessionWithTimestampBytesStore extends ChangeLoggingSessionBytesStore { + + ChangeLoggingSessionWithTimestampBytesStore(final SessionStore bytesStore) { + super(bytesStore); + } + +} diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingWindowWithTimestampBytesStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingWindowWithTimestampBytesStore.java new file mode 100644 index 0000000000000..ab5ea8424dd75 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingWindowWithTimestampBytesStore.java @@ -0,0 +1,33 @@ +/* + * 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.utils.Bytes; +import org.apache.kafka.streams.state.WindowStore; + +/** + * Simple wrapper around a {@link SegmentedBytesStore} to support writing + * updates to a changelog + */ +class ChangeLoggingWindowWithTimestampBytesStore extends ChangeLoggingWindowBytesStore { + + ChangeLoggingWindowWithTimestampBytesStore(final WindowStore bytesStore, + final boolean retainDuplicates) { + super(bytesStore, retainDuplicates); + } + +} diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/GlobalStateStoreProvider.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/GlobalStateStoreProvider.java index 0db69d0d27f72..c7ee0c50c0a42 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/GlobalStateStoreProvider.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/GlobalStateStoreProvider.java @@ -17,8 +17,15 @@ package org.apache.kafka.streams.state.internals; import org.apache.kafka.streams.errors.InvalidStateStoreException; +import org.apache.kafka.streams.kstream.internals.KStreamImpl; +import org.apache.kafka.streams.kstream.internals.KeyValueWithTimestampStoreMaterializer; +import org.apache.kafka.streams.kstream.internals.SessionWindowedKStreamImpl; import org.apache.kafka.streams.processor.StateStore; +import org.apache.kafka.streams.state.KeyValueStore; import org.apache.kafka.streams.state.QueryableStoreType; +import org.apache.kafka.streams.state.QueryableStoreTypes; +import org.apache.kafka.streams.state.SessionStore; +import org.apache.kafka.streams.state.WindowStore; import java.util.Collections; import java.util.List; @@ -41,6 +48,28 @@ public List stores(final String storeName, final QueryableStoreType qu if (!store.isOpen()) { throw new InvalidStateStoreException("the state store, " + storeName + ", is not open."); } + + if (store instanceof KeyValueWithTimestampStoreMaterializer.KeyValueStoreFacade) { + final KeyValueStore innerStore = ((KeyValueWithTimestampStoreMaterializer.KeyValueStoreFacade) store).inner; + if (queryableStoreType instanceof QueryableStoreTypes.KeyValueStoreType) { + return (List) Collections.singletonList(new ReadOnlyKeyValueStoreFacade(innerStore)); + } + return (List) Collections.singletonList(innerStore); + } + if (store instanceof KStreamImpl.WindowStoreFacade) { + final WindowStore innerStore = ((KStreamImpl.WindowStoreFacade) store).inner; + if (queryableStoreType instanceof QueryableStoreTypes.WindowStoreType) { + return (List) Collections.singletonList(new ReadOnlyWindowStoreFacade(innerStore)); + } + return (List) Collections.singletonList(innerStore); + } + if (store instanceof SessionWindowedKStreamImpl.SessionStoreFacade) { + final SessionStore innerStore = ((SessionWindowedKStreamImpl.SessionStoreFacade) store).inner; + if (queryableStoreType instanceof QueryableStoreTypes.SessionStoreType) { + return (List) Collections.singletonList(new ReadOnlySessionStoreFacade(innerStore)); + } + return (List) Collections.singletonList(innerStore); + } return (List) Collections.singletonList(store); } } diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryKeyValueStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryKeyValueStore.java index d6dd42ac264ff..b37ae476bc6a6 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryKeyValueStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryKeyValueStore.java @@ -32,13 +32,14 @@ import java.util.TreeMap; public class InMemoryKeyValueStore implements KeyValueStore { - private final String name; - private final Serde keySerde; - private final Serde valueSerde; private final NavigableMap map; private volatile boolean open = false; - private StateSerdes serdes; + final String name; + final Serde keySerde; + final Serde valueSerde; + StateSerdes serdes; + public InMemoryKeyValueStore(final String name, final Serde keySerde, @@ -47,8 +48,6 @@ public InMemoryKeyValueStore(final String name, this.keySerde = keySerde; this.valueSerde = valueSerde; - // TODO: when we have serde associated with class types, we can - // improve this situation by passing the comparator here. this.map = new TreeMap<>(); } @@ -58,15 +57,9 @@ public String name() { } @Override - @SuppressWarnings("unchecked") public void init(final ProcessorContext context, final StateStore root) { - // construct the serde - this.serdes = new StateSerdes<>( - ProcessorStateManager.storeChangelogTopic(context.applicationId(), name), - keySerde == null ? (Serde) context.keySerde() : keySerde, - valueSerde == null ? (Serde) context.valueSerde() : valueSerde); - + initStateSerdes(context); if (root != null) { // register the store context.register(root, (key, value) -> { @@ -82,6 +75,14 @@ public void init(final ProcessorContext context, this.open = true; } + @SuppressWarnings("unchecked") + void initStateSerdes(final ProcessorContext context) { + this.serdes = new StateSerdes<>( + ProcessorStateManager.storeChangelogTopic(context.applicationId(), name), + keySerde == null ? (Serde) context.keySerde() : keySerde, + valueSerde == null ? (Serde) context.valueSerde() : valueSerde); + } + @Override public boolean persistent() { return false; @@ -159,10 +160,10 @@ public void close() { this.open = false; } - private static class InMemoryKeyValueIterator implements KeyValueIterator { + static class InMemoryKeyValueIterator implements KeyValueIterator { private final Iterator> iter; - private InMemoryKeyValueIterator(final Iterator> iter) { + InMemoryKeyValueIterator(final Iterator> iter) { this.iter = iter; } diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryKeyValueWithTimestampStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryKeyValueWithTimestampStore.java new file mode 100644 index 0000000000000..7e6e90cea064e --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryKeyValueWithTimestampStore.java @@ -0,0 +1,45 @@ +/* + * 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.serialization.Serde; +import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.processor.internals.ProcessorStateManager; +import org.apache.kafka.streams.state.StateSerdes; +import org.apache.kafka.streams.state.ValueAndTimestamp; + +import static org.apache.kafka.streams.state.internals.KeyValueWithTimestampStoreBuilder.ValueAndTimestampSerde; + +public class InMemoryKeyValueWithTimestampStore extends InMemoryKeyValueStore> { + + public InMemoryKeyValueWithTimestampStore(final String name, + final Serde keySerde, + final ValueAndTimestampSerde valueSerde) { + super(name, keySerde, valueSerde); + } + + @Override + @SuppressWarnings("unchecked") + void initStateSerdes(final ProcessorContext context) { + this.serdes = new StateSerdes<>( + ProcessorStateManager.storeChangelogTopic(context.applicationId(), name), + keySerde == null ? (Serde) context.keySerde() : keySerde, + ((ValueAndTimestampSerde) valueSerde).initialized() ? + valueSerde : + ((ValueAndTimestampSerde) valueSerde).init((Serde) context.valueSerde())); + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/KeyValueIteratorFacade.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/KeyValueIteratorFacade.java new file mode 100644 index 0000000000000..2be3fcdb9c440 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/KeyValueIteratorFacade.java @@ -0,0 +1,50 @@ +/* + * 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.streams.KeyValue; +import org.apache.kafka.streams.state.KeyValueIterator; +import org.apache.kafka.streams.state.ValueAndTimestamp; + +public class KeyValueIteratorFacade implements KeyValueIterator { + private final KeyValueIterator> innerIterator; + + public KeyValueIteratorFacade(final KeyValueIterator> iterator) { + innerIterator = iterator; + } + + @Override + public void close() { + innerIterator.close(); + } + + @Override + public K peekNextKey() { + return innerIterator.peekNextKey(); + } + + @Override + public boolean hasNext() { + return innerIterator.hasNext(); + } + + @Override + public KeyValue next() { + final KeyValue> innerKeyValue = innerIterator.next(); + return KeyValue.pair(innerKeyValue.key, innerKeyValue.value.value()); + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/KeyValueToKeyValueWithTimestampByteProxyStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/KeyValueToKeyValueWithTimestampByteProxyStore.java new file mode 100644 index 0000000000000..871804f2b8039 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/KeyValueToKeyValueWithTimestampByteProxyStore.java @@ -0,0 +1,121 @@ +/* + * 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.clients.consumer.ConsumerRecord; +import org.apache.kafka.common.utils.Bytes; +import org.apache.kafka.streams.KeyValue; +import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.processor.StateStore; +import org.apache.kafka.streams.state.KeyValueIterator; +import org.apache.kafka.streams.state.KeyValueStore; +import org.apache.kafka.streams.state.RecordConverter; + +import java.util.List; + +import static org.apache.kafka.streams.state.internals.StoreProxyUtils.getValue; +import static org.apache.kafka.streams.state.internals.StoreProxyUtils.getValueWithUnknownTimestamp; + +public class KeyValueToKeyValueWithTimestampByteProxyStore implements KeyValueStore, RecordConverter { + final KeyValueStore store; + + KeyValueToKeyValueWithTimestampByteProxyStore(final KeyValueStore store) { + this.store = store; + } + + @Override + public void put(final Bytes key, + final byte[] valueWithTimestamp) { + store.put(key, getValue(valueWithTimestamp)); + } + + @Override + public byte[] putIfAbsent(final Bytes key, + final byte[] valueWithTimestamp) { + return getValueWithUnknownTimestamp(store.putIfAbsent(key, getValue(valueWithTimestamp))); + } + + @Override + public void putAll(final List> entries) { + for (final KeyValue entry : entries) { + final byte[] valueWithTimestamp = entry.value; + store.put(entry.key, getValue(valueWithTimestamp)); + } + } + + @Override + public byte[] delete(final Bytes key) { + return getValueWithUnknownTimestamp(store.delete(key)); + } + + @Override + public String name() { + return store.name(); + } + + @Override + public void init(final ProcessorContext context, + final StateStore root) { + store.init(context, root); + } + + @Override + public void flush() { + store.flush(); + } + + @Override + public void close() { + store.close(); + } + + @Override + public boolean persistent() { + return true; + } + + @Override + public boolean isOpen() { + return store.isOpen(); + } + + @Override + public byte[] get(final Bytes key) { + return getValueWithUnknownTimestamp(store.get(key)); + } + + @Override + public KeyValueIterator range(final Bytes from, + final Bytes to) { + return new StoreProxyUtils.KeyValueIteratorProxy<>(store.range(from, to)); + } + + @Override + public KeyValueIterator all() { + return new StoreProxyUtils.KeyValueIteratorProxy<>(store.all()); + } + + @Override + public long approximateNumEntries() { + return store.approximateNumEntries(); + } + + @Override + public ConsumerRecord convert(final ConsumerRecord record) { + return null; + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/KeyValueWithTimestampStoreBuilder.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/KeyValueWithTimestampStoreBuilder.java new file mode 100644 index 0000000000000..47e9b8e09feaf --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/KeyValueWithTimestampStoreBuilder.java @@ -0,0 +1,217 @@ +/* + * 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.serialization.Deserializer; +import org.apache.kafka.common.serialization.LongDeserializer; +import org.apache.kafka.common.serialization.LongSerializer; +import org.apache.kafka.common.serialization.Serde; +import org.apache.kafka.common.serialization.Serializer; +import org.apache.kafka.common.utils.Bytes; +import org.apache.kafka.common.utils.Time; +import org.apache.kafka.streams.state.KeyValueBytesStoreSupplier; +import org.apache.kafka.streams.state.KeyValueStore; +import org.apache.kafka.streams.state.RecordConverter; +import org.apache.kafka.streams.state.ValueAndTimestamp; + +import java.util.Arrays; +import java.util.Map; +import java.util.Objects; + +public class KeyValueWithTimestampStoreBuilder extends AbstractStoreBuilder, KeyValueStore>> { + + private final KeyValueBytesStoreSupplier storeSupplier; + + public KeyValueWithTimestampStoreBuilder(final KeyValueBytesStoreSupplier storeSupplier, + final Serde keySerde, + final Serde valueSerde, + final Time time) { + super(storeSupplier.name(), keySerde, new ValueAndTimestampSerde<>(valueSerde), time); + Objects.requireNonNull(storeSupplier, "bytesStoreSupplier can't be null"); + this.storeSupplier = storeSupplier; + } + + @Override + public KeyValueStore> build() { + KeyValueStore store = storeSupplier.get(); + if (!(store instanceof RecordConverter) && store.persistent()) { + store = new KeyValueToKeyValueWithTimestampByteProxyStore(store); + } + return new MeteredKeyValueWithTimestampStore<>( + maybeWrapCaching(maybeWrapLogging(store)), + storeSupplier.metricsScope(), + time, + keySerde, + valueSerde); + } + + private KeyValueStore maybeWrapCaching(final KeyValueStore inner) { + if (!enableCaching) { + return inner; + } + return new CachingKeyValueWithTimestampStore<>(inner, keySerde, valueSerde); + } + + private KeyValueStore maybeWrapLogging(final KeyValueStore inner) { + if (!enableLogging) { + return inner; + } + return new ChangeLoggingKeyValueWithTimestampBytesStore(inner); + } + + + + // TODO: where should we move those classes (of can we keep them here)? + + public static class ValueAndTimestampSerde implements Serde> { + private ValueAndTimestampSerializer valueAndTimestampSerializer; + private ValueAndTimestampDeserializer valueAndTimestampDeserializer; + private boolean initialized = false; + + public ValueAndTimestampSerde(final Serde valueSerde) { + if (valueSerde != null) { + valueAndTimestampSerializer = new ValueAndTimestampSerializer<>(valueSerde.serializer()); + valueAndTimestampDeserializer = new ValueAndTimestampDeserializer<>(valueSerde.deserializer()); + initialized = true; + } + } + + public ValueAndTimestampSerde init(final Serde valueSerde) { + valueAndTimestampSerializer = new ValueAndTimestampSerializer<>(valueSerde.serializer()); + valueAndTimestampDeserializer = new ValueAndTimestampDeserializer<>(valueSerde.deserializer()); + initialized = true; + return this; + } + + public boolean initialized() { + return initialized; + } + + @Override + public void configure(final Map configs, final boolean isKey) { + if (valueAndTimestampSerializer != null) { + valueAndTimestampSerializer.configure(configs, isKey); + } + if (valueAndTimestampDeserializer != null) { + valueAndTimestampDeserializer.configure(configs, isKey); + } + } + + @Override + public void close() { + if (valueAndTimestampSerializer != null) { + valueAndTimestampSerializer.close(); + } + if (valueAndTimestampDeserializer != null) { + valueAndTimestampDeserializer.close(); + } + } + + @Override + public Serializer> serializer() { + return valueAndTimestampSerializer; + } + + @Override + public Deserializer> deserializer() { + return valueAndTimestampDeserializer; + } + } + + static class ValueAndTimestampSerializer implements Serializer> { + public final Serializer valueSerializer; + private final Serializer timestampSerializer; + + ValueAndTimestampSerializer(final Serializer valueSerializer) { + this.valueSerializer = valueSerializer; + timestampSerializer = new LongSerializer(); + } + + @Override + public void configure(final Map configs, + final boolean isKey) { + valueSerializer.configure(configs, isKey); + timestampSerializer.configure(configs, isKey); + } + + @Override + public byte[] serialize(final String topic, + final ValueAndTimestamp data) { + if (data == null) { + return null; + } + return serialize(topic, data.value(), data.timestamp()); + } + + public byte[] serialize(final String topic, + final V data, + final long timestamp) { + if (data == null) { + return null; + } + final byte[] rawValue = valueSerializer.serialize(topic, data); + final byte[] rawTimestamp = timestampSerializer.serialize(topic, timestamp); + final byte[] rawValueAndTimestamp = new byte[rawTimestamp.length + rawValue.length]; + System.arraycopy(rawTimestamp, 0, rawValueAndTimestamp, 0, rawTimestamp.length); + System.arraycopy(rawValue, 0, rawValueAndTimestamp, rawTimestamp.length, rawValue.length); + return rawValueAndTimestamp; + } + + @Override + public void close() { + valueSerializer.close(); + timestampSerializer.close(); + } + } + + static class ValueAndTimestampDeserializer implements Deserializer> { + public final Deserializer valueDeserializer; + private final Deserializer timestampDeserializer; + + ValueAndTimestampDeserializer(final Deserializer valueDeserializer) { + this.valueDeserializer = valueDeserializer; + timestampDeserializer = new LongDeserializer(); + } + + @Override + public void configure(final Map configs, + final boolean isKey) { + valueDeserializer.configure(configs, isKey); + timestampDeserializer.configure(configs, isKey); + } + + @Override + public ValueAndTimestamp deserialize(final String topic, + final byte[] data) { + if (data == null) { + return null; + } + final long timestamp = timestampDeserializer.deserialize(topic, Arrays.copyOfRange(data, 0, 8)); + final V value = valueDeserializer.deserialize(topic, Arrays.copyOfRange(data, 8, data.length)); + return ValueAndTimestamp.make(value, timestamp); + } + + @Override + public void close() { + valueDeserializer.close(); + timestampDeserializer.close(); + + } + } + + +} \ No newline at end of file diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredKeyValueStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredKeyValueStore.java index f3d1cae65d6f3..792e4b2ebb3d9 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredKeyValueStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredKeyValueStore.java @@ -48,9 +48,9 @@ public class MeteredKeyValueStore extends WrappedStateStore.AbstractStateStore implements KeyValueStore { private final KeyValueStore inner; - private final Serde keySerde; - private final Serde valueSerde; - private StateSerdes serdes; + final Serde keySerde; + final Serde valueSerde; + StateSerdes serdes; private final String metricScope; protected final Time time; @@ -78,7 +78,6 @@ public class MeteredKeyValueStore extends WrappedStateStore.AbstractStateS this.valueSerde = valueSerde; } - @SuppressWarnings("unchecked") @Override public void init(final ProcessorContext context, final StateStore root) { @@ -89,10 +88,7 @@ public void init(final ProcessorContext context, final Map taskTags = metrics.tagMap("task-id", taskName, metricScope + "-id", "all"); final Map storeTags = metrics.tagMap("task-id", taskName, metricScope + "-id", name()); - serdes = new StateSerdes<>( - ProcessorStateManager.storeChangelogTopic(context.applicationId(), name()), - keySerde == null ? (Serde) context.keySerde() : keySerde, - valueSerde == null ? (Serde) context.valueSerde() : valueSerde); + initStateSerdes(context); putTime = createTaskAndStoreLatencyAndThroughputSensors(DEBUG, "put", metrics, metricsGroup, taskName, name(), taskTags, storeTags); putIfAbsentTime = createTaskAndStoreLatencyAndThroughputSensors(DEBUG, "put-if-absent", metrics, metricsGroup, taskName, name(), taskTags, storeTags); @@ -117,6 +113,14 @@ public void init(final ProcessorContext context, } } + @SuppressWarnings("unchecked") + void initStateSerdes(final ProcessorContext context) { + serdes = new StateSerdes<>( + ProcessorStateManager.storeChangelogTopic(context.applicationId(), name()), + keySerde == null ? (Serde) context.keySerde() : keySerde, + valueSerde == null ? (Serde) context.valueSerde() : valueSerde); + } + @Override public void close() { super.close(); diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredKeyValueWithTimestampStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredKeyValueWithTimestampStore.java new file mode 100644 index 0000000000000..2b017bf6763a5 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredKeyValueWithTimestampStore.java @@ -0,0 +1,57 @@ +/* + * 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.serialization.Serde; +import org.apache.kafka.common.utils.Bytes; +import org.apache.kafka.common.utils.Time; +import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.processor.internals.ProcessorStateManager; +import org.apache.kafka.streams.state.KeyValueStore; +import org.apache.kafka.streams.state.StateSerdes; +import org.apache.kafka.streams.state.ValueAndTimestamp; + +/** + * A Metered {@link KeyValueStore} wrapper that is used for recording operation metrics, and hence its + * inner KeyValueStore implementation do not need to provide its own metrics collecting functionality. + * The inner {@link KeyValueStore} of this class is of type <Bytes,byte[]>, hence we use {@link Serde}s + * to convert from <K,V> to <Bytes,byte[]> + * @param + * @param + */ +public class MeteredKeyValueWithTimestampStore extends MeteredKeyValueStore> { + + MeteredKeyValueWithTimestampStore(final KeyValueStore inner, + final String metricScope, + final Time time, + final Serde keySerde, + final Serde> valueSerde) { + super(inner, metricScope, time, keySerde, valueSerde); + } + + @SuppressWarnings("unchecked") + @Override + void initStateSerdes(final ProcessorContext context) { + serdes = new StateSerdes<>( + ProcessorStateManager.storeChangelogTopic(context.applicationId(), name()), + keySerde == null ? (Serde) context.keySerde() : keySerde, + ((KeyValueWithTimestampStoreBuilder.ValueAndTimestampSerde) valueSerde).initialized() ? + valueSerde : + ((KeyValueWithTimestampStoreBuilder.ValueAndTimestampSerde) valueSerde).init((Serde) context.valueSerde())); + } + +} \ No newline at end of file diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredSessionStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredSessionStore.java index 31a039b644f61..f55ee463247fc 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredSessionStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredSessionStore.java @@ -39,10 +39,7 @@ public class MeteredSessionStore extends WrappedStateStore.AbstractStateStore implements SessionStore { private final SessionStore inner; private final String metricScope; - private final Serde keySerde; - private final Serde valueSerde; private final Time time; - private StateSerdes serdes; private StreamsMetricsImpl metrics; private Sensor putTime; private Sensor fetchTime; @@ -50,6 +47,10 @@ public class MeteredSessionStore extends WrappedStateStore.AbstractStateSt private Sensor removeTime; private String taskName; + final Serde keySerde; + final Serde valueSerde; + StateSerdes serdes; + MeteredSessionStore(final SessionStore inner, final String metricScope, final Serde keySerde, @@ -63,15 +64,10 @@ public class MeteredSessionStore extends WrappedStateStore.AbstractStateSt this.time = time; } - @SuppressWarnings("unchecked") @Override public void init(final ProcessorContext context, final StateStore root) { - //noinspection unchecked - serdes = new StateSerdes<>( - ProcessorStateManager.storeChangelogTopic(context.applicationId(), name()), - keySerde == null ? (Serde) context.keySerde() : keySerde, - valueSerde == null ? (Serde) context.valueSerde() : valueSerde); + initStateSerdes(context); metrics = (StreamsMetricsImpl) context.metrics(); taskName = context.taskId().toString(); @@ -98,6 +94,14 @@ public void init(final ProcessorContext context, } } + @SuppressWarnings("unchecked") + void initStateSerdes(final ProcessorContext context) { + serdes = new StateSerdes<>( + ProcessorStateManager.storeChangelogTopic(context.applicationId(), name()), + keySerde == null ? (Serde) context.keySerde() : keySerde, + valueSerde == null ? (Serde) context.valueSerde() : valueSerde); + } + @Override public void close() { super.close(); diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredSessionWithTimestampStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredSessionWithTimestampStore.java new file mode 100644 index 0000000000000..772778fde13c4 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredSessionWithTimestampStore.java @@ -0,0 +1,49 @@ +/* + * 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.serialization.Serde; +import org.apache.kafka.common.utils.Bytes; +import org.apache.kafka.common.utils.Time; +import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.processor.internals.ProcessorStateManager; +import org.apache.kafka.streams.state.SessionStore; +import org.apache.kafka.streams.state.StateSerdes; +import org.apache.kafka.streams.state.ValueAndTimestamp; + +public class MeteredSessionWithTimestampStore extends MeteredSessionStore> { + + MeteredSessionWithTimestampStore(final SessionStore inner, + final String metricScope, + final Serde keySerde, + final Serde> valueSerde, + final Time time) { + super(inner, metricScope, keySerde, valueSerde, time); + } + + @SuppressWarnings("unchecked") + @Override + void initStateSerdes(final ProcessorContext context) { + serdes = new StateSerdes<>( + ProcessorStateManager.storeChangelogTopic(context.applicationId(), name()), + keySerde == null ? (Serde) context.keySerde() : keySerde, + ((KeyValueWithTimestampStoreBuilder.ValueAndTimestampSerde) valueSerde).initialized() ? + valueSerde : + ((KeyValueWithTimestampStoreBuilder.ValueAndTimestampSerde) valueSerde).init((Serde) context.valueSerde())); + } + +} diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredWindowStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredWindowStore.java index 166c300b81789..587daab259824 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredWindowStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredWindowStore.java @@ -41,16 +41,17 @@ public class MeteredWindowStore extends WrappedStateStore.AbstractStateSto private final WindowStore inner; private final String metricScope; private final Time time; - private final Serde keySerde; - private final Serde valueSerde; private StreamsMetricsImpl metrics; private Sensor putTime; private Sensor fetchTime; private Sensor flushTime; - private StateSerdes serdes; private ProcessorContext context; private String taskName; + final Serde keySerde; + final Serde valueSerde; + StateSerdes serdes; + MeteredWindowStore(final WindowStore inner, final String metricScope, final Time time, @@ -64,15 +65,11 @@ public class MeteredWindowStore extends WrappedStateStore.AbstractStateSto this.valueSerde = valueSerde; } - @SuppressWarnings("unchecked") @Override public void init(final ProcessorContext context, final StateStore root) { this.context = context; - serdes = new StateSerdes<>( - ProcessorStateManager.storeChangelogTopic(context.applicationId(), name()), - keySerde == null ? (Serde) context.keySerde() : keySerde, - valueSerde == null ? (Serde) context.valueSerde() : valueSerde); + initStateStoreSerdes(context); metrics = (StreamsMetricsImpl) context.metrics(); taskName = context.taskId().toString(); @@ -98,6 +95,14 @@ public void init(final ProcessorContext context, } } + @SuppressWarnings("unchecked") + void initStateStoreSerdes(final ProcessorContext context) { + serdes = new StateSerdes<>( + ProcessorStateManager.storeChangelogTopic(context.applicationId(), name()), + keySerde == null ? (Serde) context.keySerde() : keySerde, + valueSerde == null ? (Serde) context.valueSerde() : valueSerde); + } + @Override public void close() { super.close(); diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredWindowWithTimestampStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredWindowWithTimestampStore.java new file mode 100644 index 0000000000000..cea6aa9ce0249 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredWindowWithTimestampStore.java @@ -0,0 +1,49 @@ +/* + * 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.serialization.Serde; +import org.apache.kafka.common.utils.Bytes; +import org.apache.kafka.common.utils.Time; +import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.processor.internals.ProcessorStateManager; +import org.apache.kafka.streams.state.StateSerdes; +import org.apache.kafka.streams.state.ValueAndTimestamp; +import org.apache.kafka.streams.state.WindowStore; + +public class MeteredWindowWithTimestampStore extends MeteredWindowStore> { + + MeteredWindowWithTimestampStore(final WindowStore inner, + final String metricScope, + final Time time, + final Serde keySerde, + final Serde> valueSerde) { + super(inner, metricScope, time, keySerde, valueSerde); + } + + @SuppressWarnings("unchecked") + @Override + void initStateStoreSerdes(final ProcessorContext context) { + serdes = new StateSerdes<>( + ProcessorStateManager.storeChangelogTopic(context.applicationId(), name()), + keySerde == null ? (Serde) context.keySerde() : keySerde, + ((KeyValueWithTimestampStoreBuilder.ValueAndTimestampSerde) valueSerde).initialized() ? + valueSerde : + ((KeyValueWithTimestampStoreBuilder.ValueAndTimestampSerde) valueSerde).init((Serde) context.valueSerde())); + } + +} diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/PlainSegment.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/PlainSegment.java new file mode 100644 index 0000000000000..de0943574103b --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/PlainSegment.java @@ -0,0 +1,68 @@ +/* + * 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.utils.Utils; +import org.apache.kafka.streams.processor.ProcessorContext; + +import java.io.IOException; +import java.util.Objects; + +class PlainSegment extends RocksDBStore implements Comparable, Segment { + public final long id; + + PlainSegment(final String segmentName, final String windowName, final long id) { + super(segmentName, windowName); + this.id = id; + } + + @Override + public void destroy() throws IOException { + Utils.delete(dbDir); + } + + @Override + public int compareTo(final PlainSegment segment) { + return Long.compare(id, segment.id); + } + + @Override + public void openDB(final ProcessorContext context) { + super.openDB(context); + // skip the registering step + internalProcessorContext = context; + } + + @Override + public String toString() { + return "PlainSegment(id=" + id + ", name=" + name() + ")"; + } + + @Override + public boolean equals(final Object obj) { + if (obj == null || getClass() != obj.getClass()) { + return false; + } + final PlainSegment segment = (PlainSegment) obj; + return id == segment.id; + } + + @Override + public int hashCode() { + return Objects.hash(id); + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/PlainSegments.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/PlainSegments.java new file mode 100644 index 0000000000000..bcfd2d46a786e --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/PlainSegments.java @@ -0,0 +1,46 @@ +/* + * 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.streams.processor.internals.InternalProcessorContext; + +/** + * Manages the {@link PlainSegment}s that are used by the {@link RocksDBSegmentedBytesStore} + */ +class PlainSegments extends AbstractSegments { + + PlainSegments(final String name, final long retentionPeriod, final long segmentInterval) { + super(name, retentionPeriod, segmentInterval); + } + + @Override + PlainSegment getOrCreateSegment(final long segmentId, final InternalProcessorContext context) { + if (segments.containsKey(segmentId)) { + return segments.get(segmentId); + } else { + final PlainSegment newSegment = new PlainSegment(segmentName(segmentId), name, segmentId); + final PlainSegment shouldBeNull = segments.put(segmentId, newSegment); + + if (shouldBeNull != null) { + throw new IllegalStateException("PlainSegment already exists. Possible concurrent access."); + } + + newSegment.openDB(context); + return newSegment; + } + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/ReadOnlyKeyValueStoreFacade.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/ReadOnlyKeyValueStoreFacade.java new file mode 100644 index 0000000000000..3d62714275922 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/ReadOnlyKeyValueStoreFacade.java @@ -0,0 +1,54 @@ +/* + * 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.streams.state.KeyValueIterator; +import org.apache.kafka.streams.state.KeyValueStore; +import org.apache.kafka.streams.state.ReadOnlyKeyValueStore; +import org.apache.kafka.streams.state.ValueAndTimestamp; + +public class ReadOnlyKeyValueStoreFacade implements ReadOnlyKeyValueStore { + protected final KeyValueStore> inner; + + protected ReadOnlyKeyValueStoreFacade(final KeyValueStore> store) { + inner = store; + } + + @Override + public V get(final K key) { + final ValueAndTimestamp valueAndTimestamp = inner.get(key); + return valueAndTimestamp.value(); + } + + @Override + public KeyValueIterator range(final K from, + final K to) { + final KeyValueIterator> innerIterator = inner.range(from, to); + return new KeyValueIteratorFacade<>(innerIterator); + } + + @Override + public KeyValueIterator all() { + final KeyValueIterator> innerIterator = inner.all(); + return new KeyValueIteratorFacade<>(innerIterator); + } + + @Override + public long approximateNumEntries() { + return inner.approximateNumEntries(); + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/ReadOnlySessionStoreFacade.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/ReadOnlySessionStoreFacade.java new file mode 100644 index 0000000000000..5c3c17af32af9 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/ReadOnlySessionStoreFacade.java @@ -0,0 +1,44 @@ +/* + * 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.streams.kstream.Windowed; +import org.apache.kafka.streams.state.KeyValueIterator; +import org.apache.kafka.streams.state.ReadOnlySessionStore; +import org.apache.kafka.streams.state.SessionStore; +import org.apache.kafka.streams.state.ValueAndTimestamp; + +public class ReadOnlySessionStoreFacade implements ReadOnlySessionStore { + protected final SessionStore> inner; + + protected ReadOnlySessionStoreFacade(final SessionStore> store) { + inner = store; + } + + @Override + public KeyValueIterator, V> fetch(final K key) { + final KeyValueIterator, ValueAndTimestamp> innerIterator = inner.fetch(key); + return new KeyValueIteratorFacade<>(innerIterator); + } + + @Override + public KeyValueIterator, V> fetch(final K from, + final K to) { + final KeyValueIterator, ValueAndTimestamp> innerIterator = inner.fetch(from, to); + return new KeyValueIteratorFacade<>(innerIterator); + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/ReadOnlyWindowStoreFacade.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/ReadOnlyWindowStoreFacade.java new file mode 100644 index 0000000000000..27e7907e487b5 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/ReadOnlyWindowStoreFacade.java @@ -0,0 +1,128 @@ +/* + * 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.streams.KeyValue; +import org.apache.kafka.streams.kstream.Windowed; +import org.apache.kafka.streams.state.KeyValueIterator; +import org.apache.kafka.streams.state.ReadOnlyWindowStore; +import org.apache.kafka.streams.state.ValueAndTimestamp; +import org.apache.kafka.streams.state.WindowStore; +import org.apache.kafka.streams.state.WindowStoreIterator; + +import java.time.Instant; + +public class ReadOnlyWindowStoreFacade implements ReadOnlyWindowStore { + protected final WindowStore> inner; + + protected ReadOnlyWindowStoreFacade(final WindowStore> store) { + inner = store; + } + + @Override + public V fetch(final K key, + final long time) { + final ValueAndTimestamp valueAndTimestamp = inner.fetch(key, time); + return valueAndTimestamp == null ? null : valueAndTimestamp.value(); + } + + @Override + @SuppressWarnings("deprecation") + public WindowStoreIterator fetch(final K key, + final long timeFrom, + final long timeTo) { + final KeyValueIterator> innerIterator = inner.fetch(key, timeFrom, timeTo); + return new WindowStoreIteratorFacade<>(innerIterator); + } + + @Override + public WindowStoreIterator fetch(final K key, + final Instant from, + final Instant to) throws IllegalArgumentException { + final KeyValueIterator> innerIterator = inner.fetch(key, from, to); + return new WindowStoreIteratorFacade<>(innerIterator); + } + + @Override + @SuppressWarnings("deprecation") + public KeyValueIterator, V> fetch(final K from, + final K to, + final long timeFrom, + final long timeTo) { + final KeyValueIterator, ValueAndTimestamp> innerIterator = inner.fetch(from, to, timeFrom, timeTo); + return new KeyValueIteratorFacade<>(innerIterator); + } + + @Override + public KeyValueIterator, V> fetch(final K from, + final K to, + final Instant fromTime, + final Instant toTime) throws IllegalArgumentException { + final KeyValueIterator, ValueAndTimestamp> innerIterator = inner.fetch(from, to, fromTime, toTime); + return new KeyValueIteratorFacade<>(innerIterator); + } + + @Override + public KeyValueIterator, V> all() { + final KeyValueIterator, ValueAndTimestamp> innerIterator = inner.all(); + return new KeyValueIteratorFacade<>(innerIterator); + } + + @Override + @SuppressWarnings("deprecation") + public KeyValueIterator, V> fetchAll(final long timeFrom, + final long timeTo) { + final KeyValueIterator, ValueAndTimestamp> innerIterator = inner.fetchAll(timeFrom, timeTo); + return new KeyValueIteratorFacade<>(innerIterator); + } + + @Override + public KeyValueIterator, V> fetchAll(final Instant from, + final Instant to) throws IllegalArgumentException { + final KeyValueIterator, ValueAndTimestamp> innerIterator = inner.fetchAll(from, to); + return new KeyValueIteratorFacade<>(innerIterator); + } + + private static class WindowStoreIteratorFacade implements WindowStoreIterator { + final KeyValueIterator> innerIterator; + + WindowStoreIteratorFacade(final KeyValueIterator> iterator) { + innerIterator = iterator; + } + + @Override + public void close() { + innerIterator.close(); + } + + @Override + public Long peekNextKey() { + return innerIterator.peekNextKey(); + } + + @Override + public boolean hasNext() { + return innerIterator.hasNext(); + } + + @Override + public KeyValue next() { + final KeyValue> innerKeyValue = innerIterator.next(); + return KeyValue.pair(innerKeyValue.key, innerKeyValue.value.value()); + } + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBGenericOptionsToDbOptionsColumnFamilyOptionsFacade.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBGenericOptionsToDbOptionsColumnFamilyOptionsFacade.java new file mode 100644 index 0000000000000..92bd7b97fe603 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBGenericOptionsToDbOptionsColumnFamilyOptionsFacade.java @@ -0,0 +1,1363 @@ +/* + * 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.rocksdb.AbstractComparator; +import org.rocksdb.AbstractSlice; +import org.rocksdb.AccessHint; +import org.rocksdb.BuiltinComparator; +import org.rocksdb.Cache; +import org.rocksdb.ColumnFamilyOptions; +import org.rocksdb.CompactionOptionsFIFO; +import org.rocksdb.CompactionOptionsUniversal; +import org.rocksdb.CompactionPriority; +import org.rocksdb.CompactionStyle; +import org.rocksdb.CompressionOptions; +import org.rocksdb.CompressionType; +import org.rocksdb.DBOptions; +import org.rocksdb.DbPath; +import org.rocksdb.Env; +import org.rocksdb.InfoLogLevel; +import org.rocksdb.Logger; +import org.rocksdb.MemTableConfig; +import org.rocksdb.MergeOperator; +import org.rocksdb.Options; +import org.rocksdb.RateLimiter; +import org.rocksdb.SstFileManager; +import org.rocksdb.Statistics; +import org.rocksdb.TableFormatConfig; +import org.rocksdb.WALRecoveryMode; + +import java.util.Collection; +import java.util.List; + +/** + * The generic {@link Options} class allows users to set all configs on one object if only default column family + * is used. Because we use multiple column families, we need to use {@link DBOptions} and {@link ColumnFamilyOptions} + * that cover a part of all options each. + * + * This class do the translation between generic {@link Options} into {@link DBOptions} and {@link ColumnFamilyOptions}. + */ +class RocksDBGenericOptionsToDbOptionsColumnFamilyOptionsFacade extends Options { + private final DBOptions dbOptions; + private final ColumnFamilyOptions columnFamilyOptions; + + RocksDBGenericOptionsToDbOptionsColumnFamilyOptionsFacade(final DBOptions dbOptions, + final ColumnFamilyOptions columnFamilyOptions) { + this.dbOptions = dbOptions; + this.columnFamilyOptions = columnFamilyOptions; + } + + @Override + public Options setIncreaseParallelism(final int totalThreads) { + dbOptions.setIncreaseParallelism(totalThreads); + return this; + } + + @Override + public Options setCreateIfMissing(final boolean flag) { + dbOptions.setCreateIfMissing(flag); + return this; + } + + @Override + public Options setCreateMissingColumnFamilies(final boolean flag) { + dbOptions.setCreateMissingColumnFamilies(flag); + return this; + } + + @Override + public Options setEnv(final Env env) { + dbOptions.setEnv(env); + return this; + } + + @Override + public Env getEnv() { + return dbOptions.getEnv(); + } + + @Override + public Options prepareForBulkLoad() { + /* From https://github.com/facebook/rocksdb/wiki/RocksDB-FAQ + * + * Q: What's the fastest way to load data into RocksDB? + * + * A: A fast way to direct insert data to the DB: + * + * 1. using single writer thread and insert in sorted order + * 2. batch hundreds of keys into one write batch + * 3. use vector memtable + * 4. make sure options.max_background_flushes is at least 4 + * 5. before inserting the data, + * disable automatic compaction, + * set options.level0_file_num_compaction_trigger, + * options.level0_slowdown_writes_trigger + * and options.level0_stop_writes_trigger to very large. + * After inserting all the data, issue a manual compaction. + * + * 3-5 will be automatically done if you call Options::PrepareForBulkLoad() to your option + */ + // don't think we can do (1) + // (2) is done via bulk-loading API + // doing (3) to (5) below + // TODO enabling this breaks StreamThreadTest#shouldRecorverFromInvalidOffsetExceptionOnRestoreAndFinishRestore + // -> unclear why + //columnFamilyOptions.setMemTableConfig(new VectorMemTableConfig()); + dbOptions.setMaxBackgroundFlushes(4); + columnFamilyOptions.setDisableAutoCompactions(true); + columnFamilyOptions.setLevel0FileNumCompactionTrigger(1 << 30); + columnFamilyOptions.setLevel0SlowdownWritesTrigger(1 << 30); + columnFamilyOptions.setLevel0StopWritesTrigger(1 << 30); + return this; + } + + @Override + public boolean createIfMissing() { + return dbOptions.createIfMissing(); + } + + @Override + public boolean createMissingColumnFamilies() { + return dbOptions.createMissingColumnFamilies(); + } + + @Override + public Options optimizeForSmallDb() { + dbOptions.optimizeForSmallDb(); + columnFamilyOptions.optimizeForSmallDb(); + return this; + } + + @Override + public Options optimizeForPointLookup(final long blockCacheSizeMb) { + columnFamilyOptions.optimizeForPointLookup(blockCacheSizeMb); + return this; + } + + @Override + public Options optimizeLevelStyleCompaction() { + columnFamilyOptions.optimizeLevelStyleCompaction(); + return this; + } + + @Override + public Options optimizeLevelStyleCompaction(final long memtableMemoryBudget) { + columnFamilyOptions.optimizeLevelStyleCompaction(memtableMemoryBudget); + return this; + } + + @Override + public Options optimizeUniversalStyleCompaction() { + columnFamilyOptions.optimizeUniversalStyleCompaction(); + return this; + } + + @Override + public Options optimizeUniversalStyleCompaction(final long memtableMemoryBudget) { + columnFamilyOptions.optimizeUniversalStyleCompaction(memtableMemoryBudget); + return this; + } + + @Override + public Options setComparator(final BuiltinComparator builtinComparator) { + columnFamilyOptions.setComparator(builtinComparator); + return this; + } + + @Override + public Options setComparator(final AbstractComparator> comparator) { + columnFamilyOptions.setComparator(comparator); + return this; + } + + @Override + public Options setMergeOperatorName(final String name) { + columnFamilyOptions.setMergeOperatorName(name); + return this; + } + + @Override + public Options setMergeOperator(final MergeOperator mergeOperator) { + columnFamilyOptions.setMergeOperator(mergeOperator); + return this; + } + + @Override + public Options setWriteBufferSize(final long writeBufferSize) { + columnFamilyOptions.setWriteBufferSize(writeBufferSize); + return this; + } + + @Override + public long writeBufferSize() { + return columnFamilyOptions.writeBufferSize(); + } + + @Override + public Options setMaxWriteBufferNumber(final int maxWriteBufferNumber) { + columnFamilyOptions.setMaxWriteBufferNumber(maxWriteBufferNumber); + return this; + } + + @Override + public int maxWriteBufferNumber() { + return columnFamilyOptions.maxWriteBufferNumber(); + } + + @Override + public boolean errorIfExists() { + return dbOptions.errorIfExists(); + } + + @Override + public Options setErrorIfExists(final boolean errorIfExists) { + dbOptions.setErrorIfExists(errorIfExists); + return this; + } + + @Override + public boolean paranoidChecks() { + final boolean columnFamilyParanoidFileChecks = columnFamilyOptions.paranoidFileChecks(); + final boolean dbOptionsParanoidChecks = dbOptions.paranoidChecks(); + + if (columnFamilyParanoidFileChecks != dbOptionsParanoidChecks) { + throw new IllegalStateException("Config for paranoid checks for RockDB and ColumnFamilies should be the same."); + } + + return dbOptionsParanoidChecks; + } + + @Override + public Options setParanoidChecks(final boolean paranoidChecks) { + columnFamilyOptions.paranoidFileChecks(); + dbOptions.setParanoidChecks(paranoidChecks); + return this; + } + + @Override + public int maxOpenFiles() { + return dbOptions.maxOpenFiles(); + } + + @Override + public Options setMaxFileOpeningThreads(final int maxFileOpeningThreads) { + dbOptions.setMaxFileOpeningThreads(maxFileOpeningThreads); + return this; + } + + @Override + public int maxFileOpeningThreads() { + return dbOptions.maxFileOpeningThreads(); + } + + @Override + public Options setMaxTotalWalSize(final long maxTotalWalSize) { + dbOptions.setMaxTotalWalSize(maxTotalWalSize); + return this; + } + + @Override + public long maxTotalWalSize() { + return dbOptions.maxTotalWalSize(); + } + + @Override + public Options setMaxOpenFiles(final int maxOpenFiles) { + dbOptions.setMaxOpenFiles(maxOpenFiles); + return this; + } + + @Override + public boolean useFsync() { + return dbOptions.useFsync(); + } + + @Override + public Options setUseFsync(final boolean useFsync) { + dbOptions.setUseFsync(useFsync); + return this; + } + + @Override + public Options setDbPaths(final Collection dbPaths) { + dbOptions.setDbPaths(dbPaths); + return this; + } + + @Override + public List dbPaths() { + return dbOptions.dbPaths(); + } + + @Override + public String dbLogDir() { + return dbOptions.dbLogDir(); + } + + @Override + public Options setDbLogDir(final String dbLogDir) { + dbOptions.setDbLogDir(dbLogDir); + return this; + } + + @Override + public String walDir() { + return dbOptions.walDir(); + } + + @Override + public Options setWalDir(final String walDir) { + dbOptions.setWalDir(walDir); + return this; + } + + @Override + public long deleteObsoleteFilesPeriodMicros() { + return dbOptions.deleteObsoleteFilesPeriodMicros(); + } + + @Override + public Options setDeleteObsoleteFilesPeriodMicros(final long micros) { + dbOptions.setDeleteObsoleteFilesPeriodMicros(micros); + return this; + } + + @Override + public int maxBackgroundCompactions() { + return dbOptions.maxBackgroundCompactions(); + } + + @Override + public Options setStatistics(final Statistics statistics) { + dbOptions.setStatistics(statistics); + return this; + } + + @Override + public Statistics statistics() { + return dbOptions.statistics(); + } + + @Override + public void setBaseBackgroundCompactions(final int baseBackgroundCompactions) { + dbOptions.setBaseBackgroundCompactions(baseBackgroundCompactions); + } + + @Override + public int baseBackgroundCompactions() { + return dbOptions.baseBackgroundCompactions(); + } + + @Override + public Options setMaxBackgroundCompactions(final int maxBackgroundCompactions) { + dbOptions.setMaxBackgroundCompactions(maxBackgroundCompactions); + return this; + } + + @Override + public void setMaxSubcompactions(final int maxSubcompactions) { + dbOptions.setMaxSubcompactions(maxSubcompactions); + } + + @Override + public int maxSubcompactions() { + return dbOptions.maxSubcompactions(); + } + + @Override + public int maxBackgroundFlushes() { + return dbOptions.maxBackgroundFlushes(); + } + + @Override + public Options setMaxBackgroundFlushes(final int maxBackgroundFlushes) { + dbOptions.setMaxBackgroundFlushes(maxBackgroundFlushes); + return this; + } + + @Override + public int maxBackgroundJobs() { + return dbOptions.maxBackgroundJobs(); + } + + @Override + public Options setMaxBackgroundJobs(final int maxBackgroundJobs) { + dbOptions.setMaxBackgroundJobs(maxBackgroundJobs); + return this; + } + + @Override + public long maxLogFileSize() { + return dbOptions.maxLogFileSize(); + } + + @Override + public Options setMaxLogFileSize(final long maxLogFileSize) { + dbOptions.setMaxLogFileSize(maxLogFileSize); + return this; + } + + @Override + public long logFileTimeToRoll() { + return dbOptions.logFileTimeToRoll(); + } + + @Override + public Options setLogFileTimeToRoll(final long logFileTimeToRoll) { + dbOptions.setLogFileTimeToRoll(logFileTimeToRoll); + return this; + } + + @Override + public long keepLogFileNum() { + return dbOptions.keepLogFileNum(); + } + + @Override + public Options setKeepLogFileNum(final long keepLogFileNum) { + dbOptions.setKeepLogFileNum(keepLogFileNum); + return this; + } + + @Override + public Options setRecycleLogFileNum(final long recycleLogFileNum) { + dbOptions.setRecycleLogFileNum(recycleLogFileNum); + return this; + } + + @Override + public long recycleLogFileNum() { + return dbOptions.recycleLogFileNum(); + } + + @Override + public long maxManifestFileSize() { + return dbOptions.maxManifestFileSize(); + } + + @Override + public Options setMaxManifestFileSize(final long maxManifestFileSize) { + dbOptions.setMaxManifestFileSize(maxManifestFileSize); + return this; + } + + @Override + public Options setMaxTableFilesSizeFIFO(final long maxTableFilesSize) { + columnFamilyOptions.setMaxTableFilesSizeFIFO(maxTableFilesSize); + return this; + } + + @Override + public long maxTableFilesSizeFIFO() { + return columnFamilyOptions.maxTableFilesSizeFIFO(); + } + + @Override + public int tableCacheNumshardbits() { + return dbOptions.tableCacheNumshardbits(); + } + + @Override + public Options setTableCacheNumshardbits(final int tableCacheNumshardbits) { + dbOptions.setTableCacheNumshardbits(tableCacheNumshardbits); + return this; + } + + @Override + public long walTtlSeconds() { + return dbOptions.walTtlSeconds(); + } + + @Override + public Options setWalTtlSeconds(final long walTtlSeconds) { + dbOptions.setWalTtlSeconds(walTtlSeconds); + return this; + } + + @Override + public long walSizeLimitMB() { + return dbOptions.walSizeLimitMB(); + } + + @Override + public Options setWalSizeLimitMB(final long sizeLimitMB) { + dbOptions.setWalSizeLimitMB(sizeLimitMB); + return this; + } + + @Override + public long manifestPreallocationSize() { + return dbOptions.manifestPreallocationSize(); + } + + @Override + public Options setManifestPreallocationSize(final long size) { + dbOptions.setManifestPreallocationSize(size); + return this; + } + + @Override + public Options setUseDirectReads(final boolean useDirectReads) { + dbOptions.setUseDirectReads(useDirectReads); + return this; + } + + @Override + public boolean useDirectReads() { + return dbOptions.useDirectReads(); + } + + @Override + public Options setUseDirectIoForFlushAndCompaction(final boolean useDirectIoForFlushAndCompaction) { + dbOptions.setUseDirectIoForFlushAndCompaction(useDirectIoForFlushAndCompaction); + return this; + } + + @Override + public boolean useDirectIoForFlushAndCompaction() { + return dbOptions.useDirectIoForFlushAndCompaction(); + } + + @Override + public Options setAllowFAllocate(final boolean allowFAllocate) { + dbOptions.setAllowFAllocate(allowFAllocate); + return this; + } + + @Override + public boolean allowFAllocate() { + return dbOptions.allowFAllocate(); + } + + @Override + public boolean allowMmapReads() { + return dbOptions.allowMmapReads(); + } + + @Override + public Options setAllowMmapReads(final boolean allowMmapReads) { + dbOptions.setAllowMmapReads(allowMmapReads); + return this; + } + + @Override + public boolean allowMmapWrites() { + return dbOptions.allowMmapWrites(); + } + + @Override + public Options setAllowMmapWrites(final boolean allowMmapWrites) { + dbOptions.setAllowMmapWrites(allowMmapWrites); + return this; + } + + @Override + public boolean isFdCloseOnExec() { + return dbOptions.isFdCloseOnExec(); + } + + @Override + public Options setIsFdCloseOnExec(final boolean isFdCloseOnExec) { + dbOptions.setIsFdCloseOnExec(isFdCloseOnExec); + return this; + } + + @Override + public int statsDumpPeriodSec() { + return dbOptions.statsDumpPeriodSec(); + } + + @Override + public Options setStatsDumpPeriodSec(final int statsDumpPeriodSec) { + dbOptions.setStatsDumpPeriodSec(statsDumpPeriodSec); + return this; + } + + @Override + public boolean adviseRandomOnOpen() { + return dbOptions.adviseRandomOnOpen(); + } + + @Override + public Options setAdviseRandomOnOpen(final boolean adviseRandomOnOpen) { + dbOptions.setAdviseRandomOnOpen(adviseRandomOnOpen); + return this; + } + + @Override + public Options setDbWriteBufferSize(final long dbWriteBufferSize) { + dbOptions.setDbWriteBufferSize(dbWriteBufferSize); + return this; + } + + @Override + public long dbWriteBufferSize() { + return dbOptions.dbWriteBufferSize(); + } + + @Override + public Options setAccessHintOnCompactionStart(final AccessHint accessHint) { + dbOptions.setAccessHintOnCompactionStart(accessHint); + return this; + } + + @Override + public AccessHint accessHintOnCompactionStart() { + return dbOptions.accessHintOnCompactionStart(); + } + + @Override + public Options setNewTableReaderForCompactionInputs(final boolean newTableReaderForCompactionInputs) { + dbOptions.setNewTableReaderForCompactionInputs(newTableReaderForCompactionInputs); + return this; + } + + @Override + public boolean newTableReaderForCompactionInputs() { + return dbOptions.newTableReaderForCompactionInputs(); + } + + @Override + public Options setCompactionReadaheadSize(final long compactionReadaheadSize) { + dbOptions.setCompactionReadaheadSize(compactionReadaheadSize); + return this; + } + + @Override + public long compactionReadaheadSize() { + return dbOptions.compactionReadaheadSize(); + } + + @Override + public Options setRandomAccessMaxBufferSize(final long randomAccessMaxBufferSize) { + dbOptions.setRandomAccessMaxBufferSize(randomAccessMaxBufferSize); + return this; + } + + @Override + public long randomAccessMaxBufferSize() { + return dbOptions.randomAccessMaxBufferSize(); + } + + @Override + public Options setWritableFileMaxBufferSize(final long writableFileMaxBufferSize) { + dbOptions.setWritableFileMaxBufferSize(writableFileMaxBufferSize); + return this; + } + + @Override + public long writableFileMaxBufferSize() { + return dbOptions.writableFileMaxBufferSize(); + } + + @Override + public boolean useAdaptiveMutex() { + return dbOptions.useAdaptiveMutex(); + } + + @Override + public Options setUseAdaptiveMutex(final boolean useAdaptiveMutex) { + dbOptions.setUseAdaptiveMutex(useAdaptiveMutex); + return this; + } + + @Override + public long bytesPerSync() { + return dbOptions.bytesPerSync(); + } + + @Override + public Options setBytesPerSync(final long bytesPerSync) { + dbOptions.setBytesPerSync(bytesPerSync); + return this; + } + + @Override + public Options setWalBytesPerSync(final long walBytesPerSync) { + dbOptions.setWalBytesPerSync(walBytesPerSync); + return this; + } + + @Override + public long walBytesPerSync() { + return dbOptions.walBytesPerSync(); + } + + @Override + public Options setEnableThreadTracking(final boolean enableThreadTracking) { + dbOptions.setEnableThreadTracking(enableThreadTracking); + return this; + } + + @Override + public boolean enableThreadTracking() { + return dbOptions.enableThreadTracking(); + } + + @Override + public Options setDelayedWriteRate(final long delayedWriteRate) { + dbOptions.setDelayedWriteRate(delayedWriteRate); + return this; + } + + @Override + public long delayedWriteRate() { + return dbOptions.delayedWriteRate(); + } + + @Override + public Options setAllowConcurrentMemtableWrite(final boolean allowConcurrentMemtableWrite) { + dbOptions.setAllowConcurrentMemtableWrite(allowConcurrentMemtableWrite); + return this; + } + + @Override + public boolean allowConcurrentMemtableWrite() { + return dbOptions.allowConcurrentMemtableWrite(); + } + + @Override + public Options setEnableWriteThreadAdaptiveYield(final boolean enableWriteThreadAdaptiveYield) { + dbOptions.setEnableWriteThreadAdaptiveYield(enableWriteThreadAdaptiveYield); + return this; + } + + @Override + public boolean enableWriteThreadAdaptiveYield() { + return dbOptions.enableWriteThreadAdaptiveYield(); + } + + @Override + public Options setWriteThreadMaxYieldUsec(final long writeThreadMaxYieldUsec) { + dbOptions.setWriteThreadMaxYieldUsec(writeThreadMaxYieldUsec); + return this; + } + + @Override + public long writeThreadMaxYieldUsec() { + return dbOptions.writeThreadMaxYieldUsec(); + } + + @Override + public Options setWriteThreadSlowYieldUsec(final long writeThreadSlowYieldUsec) { + dbOptions.setWriteThreadSlowYieldUsec(writeThreadSlowYieldUsec); + return this; + } + + @Override + public long writeThreadSlowYieldUsec() { + return dbOptions.writeThreadSlowYieldUsec(); + } + + @Override + public Options setSkipStatsUpdateOnDbOpen(final boolean skipStatsUpdateOnDbOpen) { + dbOptions.setSkipStatsUpdateOnDbOpen(skipStatsUpdateOnDbOpen); + return this; + } + + @Override + public boolean skipStatsUpdateOnDbOpen() { + return dbOptions.skipStatsUpdateOnDbOpen(); + } + + @Override + public Options setWalRecoveryMode(final WALRecoveryMode walRecoveryMode) { + dbOptions.setWalRecoveryMode(walRecoveryMode); + return this; + } + + @Override + public WALRecoveryMode walRecoveryMode() { + return dbOptions.walRecoveryMode(); + } + + @Override + public Options setAllow2pc(final boolean allow2pc) { + dbOptions.setAllow2pc(allow2pc); + return this; + } + + @Override + public boolean allow2pc() { + return dbOptions.allow2pc(); + } + + @Override + public Options setRowCache(final Cache rowCache) { + dbOptions.setRowCache(rowCache); + return this; + } + + @Override + public Cache rowCache() { + return dbOptions.rowCache(); + } + + @Override + public Options setFailIfOptionsFileError(final boolean failIfOptionsFileError) { + dbOptions.setFailIfOptionsFileError(failIfOptionsFileError); + return this; + } + + @Override + public boolean failIfOptionsFileError() { + return dbOptions.failIfOptionsFileError(); + } + + @Override + public Options setDumpMallocStats(final boolean dumpMallocStats) { + dbOptions.setDumpMallocStats(dumpMallocStats); + return this; + } + + @Override + public boolean dumpMallocStats() { + return dbOptions.dumpMallocStats(); + } + + @Override + public Options setAvoidFlushDuringRecovery(final boolean avoidFlushDuringRecovery) { + dbOptions.setAvoidFlushDuringRecovery(avoidFlushDuringRecovery); + return this; + } + + @Override + public boolean avoidFlushDuringRecovery() { + return dbOptions.avoidFlushDuringRecovery(); + } + + @Override + public Options setAvoidFlushDuringShutdown(final boolean avoidFlushDuringShutdown) { + dbOptions.setAvoidFlushDuringShutdown(avoidFlushDuringShutdown); + return this; + } + + @Override + public boolean avoidFlushDuringShutdown() { + return dbOptions.avoidFlushDuringShutdown(); + } + + @Override + public MemTableConfig memTableConfig() { + return columnFamilyOptions.memTableConfig(); + } + + @Override + public Options setMemTableConfig(final MemTableConfig config) { + columnFamilyOptions.setMemTableConfig(config); + return this; + } + + @Override + public Options setRateLimiter(final RateLimiter rateLimiter) { + dbOptions.setRateLimiter(rateLimiter); + return this; + } + + @Override + public Options setSstFileManager(final SstFileManager sstFileManager) { + dbOptions.setSstFileManager(sstFileManager); + return this; + } + + @Override + public Options setLogger(final Logger logger) { + dbOptions.setLogger(logger); + return this; + } + + @Override + public Options setInfoLogLevel(final InfoLogLevel infoLogLevel) { + dbOptions.setInfoLogLevel(infoLogLevel); + return this; + } + + @Override + public InfoLogLevel infoLogLevel() { + return dbOptions.infoLogLevel(); + } + + @Override + public String memTableFactoryName() { + return columnFamilyOptions.memTableFactoryName(); + } + + @Override + public TableFormatConfig tableFormatConfig() { + return columnFamilyOptions.tableFormatConfig(); + } + + @Override + public Options setTableFormatConfig(final TableFormatConfig config) { + columnFamilyOptions.setTableFormatConfig(config); + return this; + } + + @Override + public String tableFactoryName() { + return columnFamilyOptions.tableFactoryName(); + } + + @Override + public Options useFixedLengthPrefixExtractor(final int n) { + columnFamilyOptions.useFixedLengthPrefixExtractor(n); + return this; + } + + @Override + public Options useCappedPrefixExtractor(final int n) { + columnFamilyOptions.useCappedPrefixExtractor(n); + return this; + } + + @Override + public CompressionType compressionType() { + return columnFamilyOptions.compressionType(); + } + + @Override + public Options setCompressionPerLevel(final List compressionLevels) { + columnFamilyOptions.setCompressionPerLevel(compressionLevels); + return this; + } + + @Override + public List compressionPerLevel() { + return columnFamilyOptions.compressionPerLevel(); + } + + @Override + public Options setCompressionType(final CompressionType compressionType) { + columnFamilyOptions.setCompressionType(compressionType); + return this; + } + + + @Override + public Options setBottommostCompressionType(final CompressionType bottommostCompressionType) { + columnFamilyOptions.setBottommostCompressionType(bottommostCompressionType); + return this; + } + + @Override + public CompressionType bottommostCompressionType() { + return columnFamilyOptions.bottommostCompressionType(); + } + + @Override + public Options setCompressionOptions(final CompressionOptions compressionOptions) { + columnFamilyOptions.setCompressionOptions(compressionOptions); + return this; + } + + @Override + public CompressionOptions compressionOptions() { + return columnFamilyOptions.compressionOptions(); + } + + @Override + public CompactionStyle compactionStyle() { + return columnFamilyOptions.compactionStyle(); + } + + @Override + public Options setCompactionStyle(final CompactionStyle compactionStyle) { + columnFamilyOptions.setCompactionStyle(compactionStyle); + return this; + } + + @Override + public int numLevels() { + return columnFamilyOptions.numLevels(); + } + + @Override + public Options setNumLevels(final int numLevels) { + columnFamilyOptions.setNumLevels(numLevels); + return this; + } + + @Override + public int levelZeroFileNumCompactionTrigger() { + return columnFamilyOptions.levelZeroFileNumCompactionTrigger(); + } + + @Override + public Options setLevelZeroFileNumCompactionTrigger(final int numFiles) { + columnFamilyOptions.setLevelZeroFileNumCompactionTrigger(numFiles); + return this; + } + + @Override + public int levelZeroSlowdownWritesTrigger() { + return columnFamilyOptions.levelZeroSlowdownWritesTrigger(); + } + + @Override + public Options setLevelZeroSlowdownWritesTrigger(final int numFiles) { + columnFamilyOptions.setLevelZeroSlowdownWritesTrigger(numFiles); + return this; + } + + @Override + public int levelZeroStopWritesTrigger() { + return columnFamilyOptions.levelZeroStopWritesTrigger(); + } + + @Override + public Options setLevelZeroStopWritesTrigger(final int numFiles) { + columnFamilyOptions.setLevelZeroStopWritesTrigger(numFiles); + return this; + } + + @Override + public long targetFileSizeBase() { + return columnFamilyOptions.targetFileSizeBase(); + } + + @Override + public Options setTargetFileSizeBase(final long targetFileSizeBase) { + columnFamilyOptions.setTargetFileSizeBase(targetFileSizeBase); + return this; + } + + @Override + public int targetFileSizeMultiplier() { + return columnFamilyOptions.targetFileSizeMultiplier(); + } + + @Override + public Options setTargetFileSizeMultiplier(final int multiplier) { + columnFamilyOptions.setTargetFileSizeMultiplier(multiplier); + return this; + } + + @Override + public Options setMaxBytesForLevelBase(final long maxBytesForLevelBase) { + columnFamilyOptions.setMaxBytesForLevelBase(maxBytesForLevelBase); + return this; + } + + @Override + public long maxBytesForLevelBase() { + return columnFamilyOptions.maxBytesForLevelBase(); + } + + @Override + public Options setLevelCompactionDynamicLevelBytes(final boolean enableLevelCompactionDynamicLevelBytes) { + columnFamilyOptions.setLevelCompactionDynamicLevelBytes(enableLevelCompactionDynamicLevelBytes); + return this; + } + + @Override + public boolean levelCompactionDynamicLevelBytes() { + return columnFamilyOptions.levelCompactionDynamicLevelBytes(); + } + + @Override + public double maxBytesForLevelMultiplier() { + return columnFamilyOptions.maxBytesForLevelMultiplier(); + } + + @Override + public Options setMaxBytesForLevelMultiplier(final double multiplier) { + columnFamilyOptions.setMaxBytesForLevelMultiplier(multiplier); + return this; + } + + @Override + public long maxCompactionBytes() { + return columnFamilyOptions.maxCompactionBytes(); + } + + @Override + public Options setMaxCompactionBytes(final long maxCompactionBytes) { + columnFamilyOptions.setMaxCompactionBytes(maxCompactionBytes); + return this; + } + + @Override + public long arenaBlockSize() { + return columnFamilyOptions.arenaBlockSize(); + } + + @Override + public Options setArenaBlockSize(final long arenaBlockSize) { + columnFamilyOptions.setArenaBlockSize(arenaBlockSize); + return this; + } + + @Override + public boolean disableAutoCompactions() { + return columnFamilyOptions.disableAutoCompactions(); + } + + @Override + public Options setDisableAutoCompactions(final boolean disableAutoCompactions) { + columnFamilyOptions.setDisableAutoCompactions(disableAutoCompactions); + return this; + } + + @Override + public long maxSequentialSkipInIterations() { + return columnFamilyOptions.maxSequentialSkipInIterations(); + } + + @Override + public Options setMaxSequentialSkipInIterations(final long maxSequentialSkipInIterations) { + columnFamilyOptions.setMaxSequentialSkipInIterations(maxSequentialSkipInIterations); + return this; + } + + @Override + public boolean inplaceUpdateSupport() { + return columnFamilyOptions.inplaceUpdateSupport(); + } + + @Override + public Options setInplaceUpdateSupport(final boolean inplaceUpdateSupport) { + columnFamilyOptions.setInplaceUpdateSupport(inplaceUpdateSupport); + return this; + } + + @Override + public long inplaceUpdateNumLocks() { + return columnFamilyOptions.inplaceUpdateNumLocks(); + } + + @Override + public Options setInplaceUpdateNumLocks(final long inplaceUpdateNumLocks) { + columnFamilyOptions.setInplaceUpdateNumLocks(inplaceUpdateNumLocks); + return this; + } + + @Override + public double memtablePrefixBloomSizeRatio() { + return columnFamilyOptions.memtablePrefixBloomSizeRatio(); + } + + @Override + public Options setMemtablePrefixBloomSizeRatio(final double memtablePrefixBloomSizeRatio) { + columnFamilyOptions.setMemtablePrefixBloomSizeRatio(memtablePrefixBloomSizeRatio); + return this; + } + + @Override + public int bloomLocality() { + return columnFamilyOptions.bloomLocality(); + } + + @Override + public Options setBloomLocality(final int bloomLocality) { + columnFamilyOptions.setBloomLocality(bloomLocality); + return this; + } + + @Override + public long maxSuccessiveMerges() { + return columnFamilyOptions.maxSuccessiveMerges(); + } + + @Override + public Options setMaxSuccessiveMerges(final long maxSuccessiveMerges) { + columnFamilyOptions.setMaxSuccessiveMerges(maxSuccessiveMerges); + return this; + } + + @Override + public int minWriteBufferNumberToMerge() { + return columnFamilyOptions.minWriteBufferNumberToMerge(); + } + + @Override + public Options setMinWriteBufferNumberToMerge(final int minWriteBufferNumberToMerge) { + columnFamilyOptions.setMinWriteBufferNumberToMerge(minWriteBufferNumberToMerge); + return this; + } + + @Override + public Options setOptimizeFiltersForHits(final boolean optimizeFiltersForHits) { + columnFamilyOptions.setOptimizeFiltersForHits(optimizeFiltersForHits); + return this; + } + + @Override + public boolean optimizeFiltersForHits() { + return columnFamilyOptions.optimizeFiltersForHits(); + } + + @Override + public Options setMemtableHugePageSize(final long memtableHugePageSize) { + columnFamilyOptions.setMemtableHugePageSize(memtableHugePageSize); + return this; + } + + @Override + public long memtableHugePageSize() { + return columnFamilyOptions.memtableHugePageSize(); + } + + @Override + public Options setSoftPendingCompactionBytesLimit(final long softPendingCompactionBytesLimit) { + columnFamilyOptions.setSoftPendingCompactionBytesLimit(softPendingCompactionBytesLimit); + return this; + } + + @Override + public long softPendingCompactionBytesLimit() { + return columnFamilyOptions.softPendingCompactionBytesLimit(); + } + + @Override + public Options setHardPendingCompactionBytesLimit(final long hardPendingCompactionBytesLimit) { + columnFamilyOptions.setHardPendingCompactionBytesLimit(hardPendingCompactionBytesLimit); + return this; + } + + @Override + public long hardPendingCompactionBytesLimit() { + return columnFamilyOptions.hardPendingCompactionBytesLimit(); + } + + @Override + public Options setLevel0FileNumCompactionTrigger(final int level0FileNumCompactionTrigger) { + columnFamilyOptions.setLevel0FileNumCompactionTrigger(level0FileNumCompactionTrigger); + return this; + } + + @Override + public int level0FileNumCompactionTrigger() { + return columnFamilyOptions.level0FileNumCompactionTrigger(); + } + + @Override + public Options setLevel0SlowdownWritesTrigger(final int level0SlowdownWritesTrigger) { + columnFamilyOptions.setLevel0SlowdownWritesTrigger(level0SlowdownWritesTrigger); + return this; + } + + @Override + public int level0SlowdownWritesTrigger() { + return columnFamilyOptions.level0SlowdownWritesTrigger(); + } + + @Override + public Options setLevel0StopWritesTrigger(final int level0StopWritesTrigger) { + columnFamilyOptions.setLevel0StopWritesTrigger(level0StopWritesTrigger); + return this; + } + + @Override + public int level0StopWritesTrigger() { + return columnFamilyOptions.level0StopWritesTrigger(); + } + + @Override + public Options setMaxBytesForLevelMultiplierAdditional(final int[] maxBytesForLevelMultiplierAdditional) { + columnFamilyOptions.setMaxBytesForLevelMultiplierAdditional(maxBytesForLevelMultiplierAdditional); + return this; + } + + @Override + public int[] maxBytesForLevelMultiplierAdditional() { + return columnFamilyOptions.maxBytesForLevelMultiplierAdditional(); + } + + @Override + public Options setParanoidFileChecks(final boolean paranoidFileChecks) { + columnFamilyOptions.setParanoidFileChecks(paranoidFileChecks); + return this; + } + + @Override + public boolean paranoidFileChecks() { + return columnFamilyOptions.paranoidFileChecks(); + } + + @Override + public Options setMaxWriteBufferNumberToMaintain(final int maxWriteBufferNumberToMaintain) { + columnFamilyOptions.setMaxWriteBufferNumberToMaintain(maxWriteBufferNumberToMaintain); + return this; + } + + @Override + public int maxWriteBufferNumberToMaintain() { + return columnFamilyOptions.maxWriteBufferNumberToMaintain(); + } + + @Override + public Options setCompactionPriority(final CompactionPriority compactionPriority) { + columnFamilyOptions.setCompactionPriority(compactionPriority); + return this; + } + + @Override + public CompactionPriority compactionPriority() { + return columnFamilyOptions.compactionPriority(); + } + + @Override + public Options setReportBgIoStats(final boolean reportBgIoStats) { + columnFamilyOptions.setReportBgIoStats(reportBgIoStats); + return this; + } + + @Override + public boolean reportBgIoStats() { + return columnFamilyOptions.reportBgIoStats(); + } + + @Override + public Options setCompactionOptionsUniversal(final CompactionOptionsUniversal compactionOptionsUniversal) { + columnFamilyOptions.setCompactionOptionsUniversal(compactionOptionsUniversal); + return this; + } + + @Override + public CompactionOptionsUniversal compactionOptionsUniversal() { + return columnFamilyOptions.compactionOptionsUniversal(); + } + + @Override + public Options setCompactionOptionsFIFO(final CompactionOptionsFIFO compactionOptionsFIFO) { + columnFamilyOptions.setCompactionOptionsFIFO(compactionOptionsFIFO); + return this; + } + + @Override + public CompactionOptionsFIFO compactionOptionsFIFO() { + return columnFamilyOptions.compactionOptionsFIFO(); + } + + @Override + public Options setForceConsistencyChecks(final boolean forceConsistencyChecks) { + columnFamilyOptions.setForceConsistencyChecks(forceConsistencyChecks); + return this; + } + + @Override + public boolean forceConsistencyChecks() { + return columnFamilyOptions.forceConsistencyChecks(); + } + + @Override + public void close() { + columnFamilyOptions.close(); + dbOptions.close(); + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBSegmentedBytesStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBSegmentedBytesStore.java index 17079b964b505..649d1e327d658 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBSegmentedBytesStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBSegmentedBytesStore.java @@ -41,74 +41,73 @@ import static org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl.addInvocationRateAndCount; -class RocksDBSegmentedBytesStore implements SegmentedBytesStore { +public class RocksDBSegmentedBytesStore implements SegmentedBytesStore { private static final Logger LOG = LoggerFactory.getLogger(RocksDBSegmentedBytesStore.class); private final String name; - private final Segments segments; + private final PlainSegments segments; private final String metricScope; - private final KeySchema keySchema; + private final KeySchema keySchema; private InternalProcessorContext context; private volatile boolean open; - private Set bulkLoadSegments; + private Set bulkLoadSegments; private Sensor expiredRecordSensor; RocksDBSegmentedBytesStore(final String name, final String metricScope, final long retention, final long segmentInterval, - final KeySchema keySchema) { + final KeySchema keySchema) { this.name = name; this.metricScope = metricScope; this.keySchema = keySchema; - this.segments = new Segments(name, retention, segmentInterval); + this.segments = new PlainSegments(name, retention, segmentInterval); } @Override public KeyValueIterator fetch(final Bytes key, final long from, final long to) { - final List searchSpace = keySchema.segmentsToSearch(segments, from, to); + final List searchSpace = keySchema.segmentsToSearch(segments, from, to); final Bytes binaryFrom = keySchema.lowerRangeFixedSize(key, from); final Bytes binaryTo = keySchema.upperRangeFixedSize(key, to); - return new SegmentIterator(searchSpace.iterator(), - keySchema.hasNextCondition(key, key, from, to), - binaryFrom, binaryTo); + return new SegmentIterator<>(searchSpace.iterator(), + keySchema.hasNextCondition(key, key, from, to), + binaryFrom, binaryTo); } @Override public KeyValueIterator fetch(final Bytes keyFrom, final Bytes keyTo, final long from, final long to) { - final List searchSpace = keySchema.segmentsToSearch(segments, from, to); + final List searchSpace = keySchema.segmentsToSearch(segments, from, to); final Bytes binaryFrom = keySchema.lowerRange(keyFrom, from); final Bytes binaryTo = keySchema.upperRange(keyTo, to); - return new SegmentIterator(searchSpace.iterator(), - keySchema.hasNextCondition(keyFrom, keyTo, from, to), - binaryFrom, binaryTo); + return new SegmentIterator<>(searchSpace.iterator(), + keySchema.hasNextCondition(keyFrom, keyTo, from, to), + binaryFrom, binaryTo); } @Override public KeyValueIterator all() { + final List searchSpace = segments.allSegments(); - final List searchSpace = segments.allSegments(); - - return new SegmentIterator(searchSpace.iterator(), - keySchema.hasNextCondition(null, null, 0, Long.MAX_VALUE), - null, null); + return new SegmentIterator<>(searchSpace.iterator(), + keySchema.hasNextCondition(null, null, 0, Long.MAX_VALUE), + null, null); } @Override public KeyValueIterator fetchAll(final long timeFrom, final long timeTo) { - final List searchSpace = segments.segments(timeFrom, timeTo); + final List searchSpace = segments.segments(timeFrom, timeTo); - return new SegmentIterator(searchSpace.iterator(), - keySchema.hasNextCondition(null, null, timeFrom, timeTo), - null, null); + return new SegmentIterator<>(searchSpace.iterator(), + keySchema.hasNextCondition(null, null, timeFrom, timeTo), + null, null); } @Override public void remove(final Bytes key) { - final Segment segment = segments.getSegmentForTimestamp(keySchema.segmentTimestamp(key)); + final PlainSegment segment = segments.getSegmentForTimestamp(keySchema.segmentTimestamp(key)); if (segment == null) { return; } @@ -119,7 +118,7 @@ public void remove(final Bytes key) { public void put(final Bytes key, final byte[] value) { final long timestamp = keySchema.segmentTimestamp(key); final long segmentId = segments.segmentId(timestamp); - final Segment segment = segments.getOrCreateSegmentIfLive(segmentId, context); + final PlainSegment segment = segments.getOrCreateSegmentIfLive(segmentId, context); if (segment == null) { expiredRecordSensor.record(); LOG.debug("Skipping record for expired segment."); @@ -130,7 +129,7 @@ public void put(final Bytes key, final byte[] value) { @Override public byte[] get(final Bytes key) { - final Segment segment = segments.getSegmentForTimestamp(keySchema.segmentTimestamp(key)); + final PlainSegment segment = segments.getSegmentForTimestamp(keySchema.segmentTimestamp(key)); if (segment == null) { return null; } @@ -195,16 +194,16 @@ public boolean isOpen() { } // Visible for testing - List getSegments() { + List getSegments() { return segments.allSegments(); } // Visible for testing void restoreAllInternal(final Collection> records) { try { - final Map writeBatchMap = getWriteBatches(records); - for (final Map.Entry entry : writeBatchMap.entrySet()) { - final Segment segment = entry.getKey(); + final Map writeBatchMap = getWriteBatches(records); + for (final Map.Entry entry : writeBatchMap.entrySet()) { + final PlainSegment segment = entry.getKey(); final WriteBatch batch = entry.getValue(); segment.write(batch); } @@ -214,11 +213,11 @@ void restoreAllInternal(final Collection> records) { } // Visible for testing - Map getWriteBatches(final Collection> records) { - final Map writeBatchMap = new HashMap<>(); + Map getWriteBatches(final Collection> records) { + final Map writeBatchMap = new HashMap<>(); for (final KeyValue record : records) { final long segmentId = segments.segmentId(keySchema.segmentTimestamp(Bytes.wrap(record.key))); - final Segment segment = segments.getOrCreateSegmentIfLive(segmentId, context); + final PlainSegment segment = segments.getOrCreateSegmentIfLive(segmentId, context); if (segment != null) { // This handles the case that state store is moved to a new client and does not // have the local RocksDB instance for the segment. In this case, toggleDBForBulkLoading @@ -247,7 +246,7 @@ Map getWriteBatches(final Collection keySchema; + private InternalProcessorContext context; + private volatile boolean open; + private Set bulkLoadSegments; + private Sensor expiredRecordSensor; + + RocksDBSegmentedWithTimestampBytesStore(final String name, + final String metricScope, + final long retention, + final long segmentInterval, + final KeySchema keySchema) { + this.name = name; + this.metricScope = metricScope; + this.keySchema = keySchema; + this.segments = new WithTimestampSegments(name, retention, segmentInterval); + } + + @Override + public KeyValueIterator fetch(final Bytes key, final long from, final long to) { + final List searchSpace = keySchema.segmentsToSearch(segments, from, to); + + final Bytes binaryFrom = keySchema.lowerRangeFixedSize(key, from); + final Bytes binaryTo = keySchema.upperRangeFixedSize(key, to); + + return new SegmentIterator<>(searchSpace.iterator(), + keySchema.hasNextCondition(key, key, from, to), + binaryFrom, binaryTo); + } + + @Override + public KeyValueIterator fetch(final Bytes keyFrom, final Bytes keyTo, final long from, final long to) { + final List searchSpace = keySchema.segmentsToSearch(segments, from, to); + + final Bytes binaryFrom = keySchema.lowerRange(keyFrom, from); + final Bytes binaryTo = keySchema.upperRange(keyTo, to); + + return new SegmentIterator<>(searchSpace.iterator(), + keySchema.hasNextCondition(keyFrom, keyTo, from, to), + binaryFrom, binaryTo); + } + + @Override + public KeyValueIterator all() { + final List searchSpace = segments.allSegments(); + + return new SegmentIterator<>(searchSpace.iterator(), + keySchema.hasNextCondition(null, null, 0, Long.MAX_VALUE), + null, null); + } + + @Override + public KeyValueIterator fetchAll(final long timeFrom, final long timeTo) { + final List searchSpace = segments.segments(timeFrom, timeTo); + + return new SegmentIterator<>(searchSpace.iterator(), + keySchema.hasNextCondition(null, null, timeFrom, timeTo), + null, null); + } + + @Override + public void remove(final Bytes key) { + final WithTimestampSegment segment = segments.getSegmentForTimestamp(keySchema.segmentTimestamp(key)); + if (segment == null) { + return; + } + segment.delete(key); + } + + @Override + public void put(final Bytes key, final byte[] value) { + final long timestamp = keySchema.segmentTimestamp(key); + final long segmentId = segments.segmentId(timestamp); + final WithTimestampSegment segment = segments.getOrCreateSegmentIfLive(segmentId, context); + if (segment == null) { + expiredRecordSensor.record(); + LOG.debug("Skipping record for expired segment."); + } else { + segment.put(key, value); + } + } + + @Override + public byte[] get(final Bytes key) { + final WithTimestampSegment segment = segments.getSegmentForTimestamp(keySchema.segmentTimestamp(key)); + if (segment == null) { + return null; + } + return segment.get(key); + } + + @Override + public String name() { + return name; + } + + @Override + public void init(final ProcessorContext context, final StateStore root) { + this.context = (InternalProcessorContext) context; + + final StreamsMetricsImpl metrics = this.context.metrics(); + + final String taskName = context.taskId().toString(); + + expiredRecordSensor = metrics.storeLevelSensor( + taskName, + name(), + "expired-window-record-drop", + Sensor.RecordingLevel.INFO + ); + addInvocationRateAndCount( + expiredRecordSensor, + "stream-" + metricScope + "-metrics", + metrics.tagMap("task-id", taskName, metricScope + "-id", name()), + "expired-window-record-drop" + ); + + segments.openExisting(this.context); + + bulkLoadSegments = new HashSet<>(segments.allSegments()); + + // register and possibly restore the state from the logs + context.register(root, new RocksDBSegmentsBatchingRestoreCallback()); + + open = true; + } + + @Override + public void flush() { + segments.flush(); + } + + @Override + public void close() { + open = false; + segments.close(); + } + + @Override + public boolean persistent() { + return true; + } + + @Override + public boolean isOpen() { + return open; + } + + // Visible for testing + List getSegments() { + return segments.allSegments(); + } + + // Visible for testing + void restoreAllInternal(final Collection> records) { + try { + final Map writeBatchMap = getWriteBatches(records); + for (final Map.Entry entry : writeBatchMap.entrySet()) { + final WithTimestampSegment segment = entry.getKey(); + final WriteBatch batch = entry.getValue(); + segment.write(batch); + } + } catch (final RocksDBException e) { + throw new ProcessorStateException("Error restoring batch to store " + this.name, e); + } + } + + // Visible for testing + Map getWriteBatches(final Collection> records) { + final Map writeBatchMap = new HashMap<>(); + for (final KeyValue record : records) { + final long segmentId = segments.segmentId(keySchema.segmentTimestamp(Bytes.wrap(record.key))); + final WithTimestampSegment segment = segments.getOrCreateSegmentIfLive(segmentId, context); + if (segment != null) { + // This handles the case that state store is moved to a new client and does not + // have the local RocksDB instance for the segment. In this case, toggleDBForBulkLoading + // will only close the database and open it again with bulk loading enabled. + if (!bulkLoadSegments.contains(segment)) { + segment.toggleDbForBulkLoading(true); + // If the store does not exist yet, the getOrCreateSegmentIfLive will call openDB that + // makes the open flag for the newly created store. + // if the store does exist already, then toggleDbForBulkLoading will make sure that + // the store is already open here. + bulkLoadSegments = new HashSet<>(segments.allSegments()); + } + try { + final WriteBatch batch = writeBatchMap.computeIfAbsent(segment, s -> new WriteBatch()); + if (record.value == null) { + batch.delete(record.key); + } else { + batch.put(record.key, record.value); + } + } catch (final RocksDBException e) { + throw new ProcessorStateException("Error restoring batch to store " + this.name, e); + } + } + } + return writeBatchMap; + } + + private void toggleForBulkLoading(final boolean prepareForBulkload) { + for (final WithTimestampSegment segment: segments.allSegments()) { + segment.toggleDbForBulkLoading(prepareForBulkload); + } + } + + private class RocksDBSegmentsBatchingRestoreCallback extends AbstractNotifyingBatchingRestoreCallback { + + @Override + public void restoreAll(final Collection> records) { + restoreAllInternal(records); + } + + @Override + public void onRestoreStart(final TopicPartition topicPartition, + final String storeName, + final long startingOffset, + final long endingOffset) { + toggleForBulkLoading(true); + } + + @Override + public void onRestoreEnd(final TopicPartition topicPartition, + final String storeName, + final long totalRestored) { + toggleForBulkLoading(false); + } + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBSessionWithTimestampStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBSessionWithTimestampStore.java new file mode 100644 index 0000000000000..3a9699641749c --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBSessionWithTimestampStore.java @@ -0,0 +1,95 @@ +/* + * 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.serialization.Serde; +import org.apache.kafka.common.utils.Bytes; +import org.apache.kafka.streams.kstream.Windowed; +import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.processor.StateStore; +import org.apache.kafka.streams.processor.internals.ProcessorStateManager; +import org.apache.kafka.streams.state.KeyValueIterator; +import org.apache.kafka.streams.state.SessionStore; +import org.apache.kafka.streams.state.StateSerdes; + + +public class RocksDBSessionWithTimestampStore extends WrappedStateStore.AbstractStateStore implements SessionStore { + + private final Serde keySerde; + private final Serde aggSerde; + private final SegmentedBytesStore bytesStore; + + private StateSerdes serdes; + private String topic; + + RocksDBSessionWithTimestampStore(final SegmentedBytesStore bytesStore, + final Serde keySerde, + final Serde aggSerde) { + super(bytesStore); + this.keySerde = keySerde; + this.bytesStore = bytesStore; + this.aggSerde = aggSerde; + } + + @Override + @SuppressWarnings("unchecked") + public void init(final ProcessorContext context, final StateStore root) { + final String storeName = bytesStore.name(); + topic = ProcessorStateManager.storeChangelogTopic(context.applicationId(), storeName); + + serdes = new StateSerdes<>( + topic, + keySerde == null ? (Serde) context.keySerde() : keySerde, + aggSerde == null ? (Serde) context.valueSerde() : aggSerde); + + bytesStore.init(context, root); + } + + @Override + public KeyValueIterator, AGG> findSessions(final K key, final long earliestSessionEndTime, final long latestSessionStartTime) { + return findSessions(key, key, earliestSessionEndTime, latestSessionStartTime); + } + + @Override + public KeyValueIterator, AGG> findSessions(final K keyFrom, final K keyTo, final long earliestSessionEndTime, final long latestSessionStartTime) { + final KeyValueIterator bytesIterator = bytesStore.fetch( + Bytes.wrap(serdes.rawKey(keyFrom)), Bytes.wrap(serdes.rawKey(keyTo)), + earliestSessionEndTime, latestSessionStartTime + ); + return new WrappedSessionStoreIterator<>(bytesIterator, serdes); + } + + @Override + public KeyValueIterator, AGG> fetch(final K key) { + return findSessions(key, 0, Long.MAX_VALUE); + } + + @Override + public KeyValueIterator, AGG> fetch(final K from, final K to) { + return findSessions(from, to, 0, Long.MAX_VALUE); + } + + @Override + public void remove(final Windowed key) { + bytesStore.remove(Bytes.wrap(SessionKeySchema.toBinary(key, serdes.keySerializer(), topic))); + } + + @Override + public void put(final Windowed sessionKey, final AGG aggregate) { + bytesStore.put(Bytes.wrap(SessionKeySchema.toBinary(sessionKey, serdes.keySerializer(), topic)), serdes.rawValue(aggregate)); + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBWindowWithTimestampStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBWindowWithTimestampStore.java new file mode 100644 index 0000000000000..95648b08d93ff --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBWindowWithTimestampStore.java @@ -0,0 +1,120 @@ +/* + * 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.serialization.Serde; +import org.apache.kafka.common.utils.Bytes; +import org.apache.kafka.streams.kstream.Windowed; +import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.processor.StateStore; +import org.apache.kafka.streams.processor.internals.ProcessorStateManager; +import org.apache.kafka.streams.state.KeyValueIterator; +import org.apache.kafka.streams.state.StateSerdes; +import org.apache.kafka.streams.state.WindowStore; +import org.apache.kafka.streams.state.WindowStoreIterator; + +public class RocksDBWindowWithTimestampStore extends WrappedStateStore.AbstractStateStore implements WindowStore { + + private final Serde keySerde; + private final Serde valueSerde; + private final boolean retainDuplicates; + private final long windowSize; + private final SegmentedBytesStore bytesStore; + + private ProcessorContext context; + private StateSerdes serdes; + private int seqnum = 0; + + RocksDBWindowWithTimestampStore(final SegmentedBytesStore bytesStore, + final Serde keySerde, + final Serde valueSerde, + final boolean retainDuplicates, + final long windowSize) { + super(bytesStore); + this.keySerde = keySerde; + this.valueSerde = valueSerde; + this.bytesStore = bytesStore; + this.retainDuplicates = retainDuplicates; + this.windowSize = windowSize; + } + + @Override + @SuppressWarnings("unchecked") + public void init(final ProcessorContext context, final StateStore root) { + this.context = context; + // construct the serde + serdes = new StateSerdes<>(ProcessorStateManager.storeChangelogTopic(context.applicationId(), bytesStore.name()), + keySerde == null ? (Serde) context.keySerde() : keySerde, + valueSerde == null ? (Serde) context.valueSerde() : valueSerde); + + bytesStore.init(context, root); + } + + @Override + public void put(final K key, final V value) { + put(key, value, context.timestamp()); + } + + @Override + public void put(final K key, final V value, final long windowStartTimestamp) { + maybeUpdateSeqnumForDups(); + + bytesStore.put(WindowKeySchema.toStoreKeyBinary(key, windowStartTimestamp, seqnum, serdes), serdes.rawValue(value)); + } + + @Override + public V fetch(final K key, final long timestamp) { + final byte[] bytesValue = bytesStore.get(WindowKeySchema.toStoreKeyBinary(key, timestamp, seqnum, serdes)); + if (bytesValue == null) { + return null; + } + return serdes.valueFrom(bytesValue); + } + + @SuppressWarnings("deprecation") + @Override + public WindowStoreIterator fetch(final K key, final long timeFrom, final long timeTo) { + final KeyValueIterator bytesIterator = bytesStore.fetch(Bytes.wrap(serdes.rawKey(key)), timeFrom, timeTo); + return new WindowStoreIteratorWrapper<>(bytesIterator, serdes, windowSize).valuesIterator(); + } + + @SuppressWarnings("deprecation") + @Override + public KeyValueIterator, V> fetch(final K from, final K to, final long timeFrom, final long timeTo) { + final KeyValueIterator bytesIterator = bytesStore.fetch(Bytes.wrap(serdes.rawKey(from)), Bytes.wrap(serdes.rawKey(to)), timeFrom, timeTo); + return new WindowStoreIteratorWrapper<>(bytesIterator, serdes, windowSize).keyValueIterator(); + } + + @Override + public KeyValueIterator, V> all() { + final KeyValueIterator bytesIterator = bytesStore.all(); + return new WindowStoreIteratorWrapper<>(bytesIterator, serdes, windowSize).keyValueIterator(); + } + + @SuppressWarnings("deprecation") + @Override + public KeyValueIterator, V> fetchAll(final long timeFrom, final long timeTo) { + final KeyValueIterator bytesIterator = bytesStore.fetchAll(timeFrom, timeTo); + return new WindowStoreIteratorWrapper<>(bytesIterator, serdes, windowSize).keyValueIterator(); + } + + private void maybeUpdateSeqnumForDups() { + if (retainDuplicates) { + seqnum = (seqnum + 1) & 0x7FFFFFFF; + } + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBWithTimestampStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBWithTimestampStore.java new file mode 100644 index 0000000000000..face29d126b0a --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBWithTimestampStore.java @@ -0,0 +1,704 @@ +/* + * 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.clients.consumer.ConsumerRecord; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.utils.AbstractIterator; +import org.apache.kafka.common.utils.Bytes; +import org.apache.kafka.common.utils.Utils; +import org.apache.kafka.streams.KeyValue; +import org.apache.kafka.streams.StreamsConfig; +import org.apache.kafka.streams.errors.InvalidStateStoreException; +import org.apache.kafka.streams.errors.ProcessorStateException; +import org.apache.kafka.streams.processor.AbstractNotifyingBatchingRestoreCallback; +import org.apache.kafka.streams.processor.BatchingStateRestoreCallback; +import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.processor.StateStore; +import org.apache.kafka.streams.state.KeyValueIterator; +import org.apache.kafka.streams.state.KeyValueStore; +import org.apache.kafka.streams.state.RecordConverter; +import org.apache.kafka.streams.state.RocksDBConfigSetter; +import org.rocksdb.BlockBasedTableConfig; +import org.rocksdb.ColumnFamilyDescriptor; +import org.rocksdb.ColumnFamilyHandle; +import org.rocksdb.ColumnFamilyOptions; +import org.rocksdb.CompactionStyle; +import org.rocksdb.CompressionType; +import org.rocksdb.DBOptions; +import org.rocksdb.FlushOptions; +import org.rocksdb.InfoLogLevel; +import org.rocksdb.Options; +import org.rocksdb.RocksDB; +import org.rocksdb.RocksDBException; +import org.rocksdb.RocksIterator; +import org.rocksdb.WriteBatch; +import org.rocksdb.WriteOptions; + +import java.io.File; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.nio.file.Files; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.Comparator; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.NoSuchElementException; +import java.util.Objects; +import java.util.Set; +import java.util.regex.Pattern; + +import static org.apache.kafka.streams.state.internals.StoreProxyUtils.getValueWithUnknownTimestamp; + +/** + * A persistent key-value store based on RocksDB. + * + * Note that the use of array-typed keys is discouraged because they result in incorrect caching behavior. + * If you intend to work on byte arrays as key, for example, you may want to wrap them with the {@code Bytes} class, + * i.e. use {@code RocksDBStore} rather than {@code RocksDBStore}. + */ +public class RocksDBWithTimestampStore implements KeyValueStore, RecordConverter { + + private static final Pattern SST_FILE_EXTENSION = Pattern.compile(".*\\.sst"); + + private static final CompressionType COMPRESSION_TYPE = CompressionType.NO_COMPRESSION; + private static final CompactionStyle COMPACTION_STYLE = CompactionStyle.UNIVERSAL; + private static final long WRITE_BUFFER_SIZE = 16 * 1024 * 1024L; + private static final long BLOCK_CACHE_SIZE = 50 * 1024 * 1024L; + private static final long BLOCK_SIZE = 4096L; + private static final int MAX_WRITE_BUFFERS = 3; + private static final String DB_FILE_DIR = "rocksdb"; + + private final String name; + private final String parentDir; + private final Set openIterators = Collections.synchronizedSet(new HashSet<>()); + + File dbDir; + private RocksDB db; + private ColumnFamilyHandle noTimestampColumnFamily; + private ColumnFamilyHandle withTimestampColumnFamily; + + // the following option objects will be created in the constructor and closed in the close() method + private RocksDBGenericOptionsToDbOptionsColumnFamilyOptionsFacade userSpecifiedOptions; + private WriteOptions wOptions; + private FlushOptions fOptions; + + private volatile boolean prepareForBulkload = false; + ProcessorContext internalProcessorContext; + // visible for testing + volatile BatchingStateRestoreCallback batchingStateRestoreCallback = null; + + protected volatile boolean open = false; + + public RocksDBWithTimestampStore(final String name) { + this(name, DB_FILE_DIR); + } + + public RocksDBWithTimestampStore(final String name, final String parentDir) { + this.name = name; + this.parentDir = parentDir; + } + + @SuppressWarnings("unchecked") + void openDB(final ProcessorContext context) { + // initialize the default rocksdb options + final BlockBasedTableConfig tableConfig = new BlockBasedTableConfig(); + tableConfig.setBlockCacheSize(BLOCK_CACHE_SIZE); + tableConfig.setBlockSize(BLOCK_SIZE); + + final DBOptions dbOptions = new DBOptions(); + final ColumnFamilyOptions columnFamilyOptions = new ColumnFamilyOptions(); + userSpecifiedOptions = new RocksDBGenericOptionsToDbOptionsColumnFamilyOptionsFacade(dbOptions, columnFamilyOptions); + userSpecifiedOptions.setTableFormatConfig(tableConfig); + userSpecifiedOptions.setWriteBufferSize(WRITE_BUFFER_SIZE); + userSpecifiedOptions.setCompressionType(COMPRESSION_TYPE); + userSpecifiedOptions.setCompactionStyle(COMPACTION_STYLE); + userSpecifiedOptions.setMaxWriteBufferNumber(MAX_WRITE_BUFFERS); + userSpecifiedOptions.setCreateIfMissing(true); + userSpecifiedOptions.setErrorIfExists(false); + userSpecifiedOptions.setInfoLogLevel(InfoLogLevel.ERROR_LEVEL); + // this is the recommended way to increase parallelism in RocksDb + // note that the current implementation of setIncreaseParallelism affects the number + // of compaction threads but not flush threads (the latter remains one). Also + // the parallelism value needs to be at least two because of the code in + // https://github.com/facebook/rocksdb/blob/62ad0a9b19f0be4cefa70b6b32876e764b7f3c11/util/options.cc#L580 + // subtracts one from the value passed to determine the number of compaction threads + // (this could be a bug in the RocksDB code and their devs have been contacted). + userSpecifiedOptions.setIncreaseParallelism(Math.max(Runtime.getRuntime().availableProcessors(), 2)); + + wOptions = new WriteOptions(); + wOptions.setDisableWAL(true); + + fOptions = new FlushOptions(); + fOptions.setWaitForFlush(true); + + final Map configs = context.appConfigs(); + final Class configSetterClass = + (Class) configs.get(StreamsConfig.ROCKSDB_CONFIG_SETTER_CLASS_CONFIG); + + if (configSetterClass != null) { + final RocksDBConfigSetter configSetter = Utils.newInstance(configSetterClass); + configSetter.setConfig(name, userSpecifiedOptions, configs); + } + + if (prepareForBulkload) { + userSpecifiedOptions.prepareForBulkLoad(); + } + + dbDir = new File(new File(context.stateDir(), parentDir), name); + + try { + final List columnFamilyDescriptors = new ArrayList(2) { + { + add(new ColumnFamilyDescriptor(RocksDB.DEFAULT_COLUMN_FAMILY, columnFamilyOptions)); + add(new ColumnFamilyDescriptor("keyValueWithTimestamp".getBytes(StandardCharsets.UTF_8), columnFamilyOptions)); + } + }; + final List columnFamilies = new ArrayList<>(columnFamilyDescriptors.size()); + + Files.createDirectories(dbDir.getParentFile().toPath()); + + try { + db = RocksDB.open(dbOptions, dbDir.getAbsolutePath(), columnFamilyDescriptors, columnFamilies); + } catch (final RocksDBException e) { + if ("Column family not found: : keyValueWithTimestamp".equals(e.getMessage())) { + try { + db = RocksDB.open(dbOptions, dbDir.getAbsolutePath(), columnFamilyDescriptors.subList(0, 1), columnFamilies); + columnFamilies.add(db.createColumnFamily(columnFamilyDescriptors.get(1))); + } catch (final RocksDBException fatal) { + throw new ProcessorStateException("Error opening store " + name + " at location " + dbDir.toString(), fatal); + } + columnFamilies.get(0).close(); + columnFamilies.get(1).close(); + columnFamilies.clear(); + db.close(); + try { + db = RocksDB.open(dbOptions, dbDir.getAbsolutePath(), columnFamilyDescriptors, columnFamilies); + } catch (final RocksDBException fatal) { + throw new ProcessorStateException("Error opening store " + name + " at location " + dbDir.toString(), fatal); + } + } else { + throw new ProcessorStateException("Error opening store " + name + " at location " + dbDir.toString(), e); + } + } + noTimestampColumnFamily = columnFamilies.get(0); + withTimestampColumnFamily = columnFamilies.get(1); + } catch (final IOException fatal) { + throw new ProcessorStateException(fatal); + } + + open = true; + } + + public void init(final ProcessorContext context, + final StateStore root) { + // open the DB dir + internalProcessorContext = context; + openDB(context); + batchingStateRestoreCallback = new RocksDBBatchingRestoreCallback(this); + + // value getter should always read directly from rocksDB + // since it is only for values that are already flushed + context.register(root, batchingStateRestoreCallback); + } + + // visible for testing + boolean isPrepareForBulkload() { + return prepareForBulkload; + } + + @Override + public String name() { + return name; + } + + @Override + public boolean persistent() { + return true; + } + + @Override + public boolean isOpen() { + return open; + } + + @Override + public synchronized byte[] get(final Bytes key) { + validateStoreOpen(); + return getInternal(key.get()); + } + + private void validateStoreOpen() { + if (!open) { + throw new InvalidStateStoreException("Store " + name + " is currently closed"); + } + } + + private byte[] getInternal(final byte[] rawKey) { + try { + final byte[] rawValueWithTimestamp = db.get(withTimestampColumnFamily, rawKey); + if (rawValueWithTimestamp != null) { + return rawValueWithTimestamp; + } + + final byte[] rawValue = db.get(noTimestampColumnFamily, rawKey); + if (rawValue != null) { + final byte[] rawValueWithUnknownTimestamp = getValueWithUnknownTimestamp(rawValue); + // this does only work, because the changelog topic contains correct data already + // for other format changes, we cannot take this short cut and can only migrate data + // from old to new store on put() + putInternal(rawKey, rawValueWithUnknownTimestamp); + return rawValueWithUnknownTimestamp; + } + + return null; + } catch (final RocksDBException e) { + // String format is happening in wrapping stores. So formatted message is thrown from wrapping stores. + throw new ProcessorStateException("Error while getting value for key %s from store " + name, e); + } + } + + void toggleDbForBulkLoading(final boolean prepareForBulkload) { + if (prepareForBulkload) { + // if the store is not empty, we need to compact to get around the num.levels check for bulk loading + final String[] sstFileNames = dbDir.list((dir, name) -> SST_FILE_EXTENSION.matcher(name).matches()); + + if (sstFileNames != null && sstFileNames.length > 0) { + try { + db.compactRange(true, 1, 0); + db.compactRange(true, 1, 0); + } catch (final RocksDBException e) { + throw new ProcessorStateException("Error while range compacting during restoring store " + name, e); + } + } + } + + close(); + this.prepareForBulkload = prepareForBulkload; + openDB(internalProcessorContext); + } + + @SuppressWarnings("unchecked") + @Override + public synchronized void put(final Bytes key, + final byte[] rawValueWithTimestamp) { + Objects.requireNonNull(key, "key cannot be null"); + validateStoreOpen(); + putInternal(key.get(), rawValueWithTimestamp); + } + + @Override + public synchronized byte[] putIfAbsent(final Bytes key, + final byte[] rawValueWithTimestamp) { + Objects.requireNonNull(key, "key cannot be null"); + final byte[] originalValueWithTimestamp = get(key); + if (originalValueWithTimestamp == null) { + put(key, rawValueWithTimestamp); + } + return originalValueWithTimestamp; + } + + private void restoreAllInternal(final Collection> records) { + try (final WriteBatch batch = new WriteBatch()) { + for (final KeyValue record : records) { + if (record.value == null) { + batch.delete(noTimestampColumnFamily, record.key); + batch.delete(withTimestampColumnFamily, record.key); + } else { + batch.delete(noTimestampColumnFamily, record.key); + batch.put(withTimestampColumnFamily, record.key, record.value); + } + } + write(batch); + } catch (final RocksDBException e) { + throw new ProcessorStateException("Error restoring batch to store " + this.name, e); + } + } + + private void putInternal(final byte[] rawKey, + final byte[] rawValueWithTimestamp) { + if (rawValueWithTimestamp == null) { + try { + db.delete(noTimestampColumnFamily, wOptions, rawKey); + db.delete(withTimestampColumnFamily, wOptions, rawKey); + } catch (final RocksDBException e) { + // String format is happening in wrapping stores. So formatted message is thrown from wrapping stores. + throw new ProcessorStateException("Error while removing key %s from store " + name, e); + } + } else { + try { + db.delete(noTimestampColumnFamily, wOptions, rawKey); + db.put(withTimestampColumnFamily, wOptions, rawKey, rawValueWithTimestamp); + } catch (final RocksDBException e) { + // String format is happening in wrapping stores. So formatted message is thrown from wrapping stores. + throw new ProcessorStateException("Error while putting key %s value %s into store " + name, e); + } + } + } + + void write(final WriteBatch batch) throws RocksDBException { + db.write(wOptions, batch); + } + + @Override + public void putAll(final List> entries) { + try (final WriteBatch batch = new WriteBatch()) { + for (final KeyValue entry : entries) { + Objects.requireNonNull(entry.key, "key cannot be null"); + if (entry.value == null) { + batch.delete(noTimestampColumnFamily, entry.key.get()); + batch.delete(withTimestampColumnFamily, entry.key.get()); + } else { + batch.delete(noTimestampColumnFamily, entry.key.get()); + batch.put(withTimestampColumnFamily, entry.key.get(), entry.value); + } + } + write(batch); + } catch (final RocksDBException e) { + throw new ProcessorStateException("Error while batch writing to store " + name, e); + } + + } + + @Override + public synchronized byte[] delete(final Bytes key) { + Objects.requireNonNull(key, "key cannot be null"); + final byte[] valueWithTimestamp = get(key); + put(key, null); + return valueWithTimestamp; + } + + @Override + public synchronized KeyValueIterator range(final Bytes from, + final Bytes to) { + Objects.requireNonNull(from, "from cannot be null"); + Objects.requireNonNull(to, "to cannot be null"); + validateStoreOpen(); + + // query rocksdb + final RocksDBRangeIterator rocksDBRangeIterator = new RocksDBRangeIterator( + name, + db.newIterator(withTimestampColumnFamily), + db.newIterator(noTimestampColumnFamily), + from, + to); + openIterators.add(rocksDBRangeIterator); + + return rocksDBRangeIterator; + } + + @Override + public synchronized KeyValueIterator all() { + validateStoreOpen(); + // query rocksdb + final RocksIterator innerIterWithTimestamp = db.newIterator(withTimestampColumnFamily); + innerIterWithTimestamp.seekToFirst(); + final RocksIterator innerIterNoTimestamp = db.newIterator(noTimestampColumnFamily); + innerIterNoTimestamp.seekToFirst(); + final RocksDbIterator rocksDbIterator = new RocksDbIterator(name, innerIterWithTimestamp, innerIterNoTimestamp); + openIterators.add(rocksDbIterator); + return rocksDbIterator; + } + + /** + * Return an approximate count of key-value mappings in this store. + * + * RocksDB cannot return an exact entry count without doing a + * full scan, so this method relies on the rocksdb.estimate-num-keys + * property to get an approximate count. The returned size also includes + * a count of dirty keys in the store's in-memory cache, which may lead to some + * double-counting of entries and inflate the estimate. + * + * @return an approximate count of key-value mappings in the store. + */ + @Override + public long approximateNumEntries() { + validateStoreOpen(); + final long value; + try { + value = db.getLongProperty(noTimestampColumnFamily, "rocksdb.estimate-num-keys") + + db.getLongProperty(withTimestampColumnFamily, "rocksdb.estimate-num-keys"); + } catch (final RocksDBException e) { + throw new ProcessorStateException("Error fetching property from store " + name, e); + } + if (isOverflowing(value)) { + return Long.MAX_VALUE; + } + return value; + } + + private boolean isOverflowing(final long value) { + // RocksDB returns an unsigned 8-byte integer, which could overflow long + // and manifest as a negative value. + return value < 0; + } + + @Override + public synchronized void flush() { + if (db == null) { + return; + } + // flush RocksDB + flushInternal(); + } + /** + * @throws ProcessorStateException if flushing failed because of any internal store exceptions + */ + private void flushInternal() { + try { + db.flush(fOptions, noTimestampColumnFamily); + db.flush(fOptions, withTimestampColumnFamily); + } catch (final RocksDBException e) { + throw new ProcessorStateException("Error while executing flush from store " + name, e); + } + } + + @Override + public synchronized void close() { + if (!open) { + return; + } + + open = false; + closeOpenIterators(); + noTimestampColumnFamily.close(); + withTimestampColumnFamily.close(); + userSpecifiedOptions.close(); + wOptions.close(); + fOptions.close(); + db.close(); + + userSpecifiedOptions = null; + wOptions = null; + fOptions = null; + db = null; + } + + private void closeOpenIterators() { + final HashSet iterators; + synchronized (openIterators) { + iterators = new HashSet<>(openIterators); + } + for (final KeyValueIterator iterator : iterators) { + iterator.close(); + } + } + + @Override + public ConsumerRecord convert(final ConsumerRecord record) { + final byte[] rawValue = record.value(); + return new ConsumerRecord<>( + record.topic(), + record.partition(), + record.offset(), + record.timestamp(), + record.timestampType(), + (long) ConsumerRecord.NULL_CHECKSUM, + record.serializedKeySize(), + 8 + record.serializedValueSize(), + record.key(), + ByteBuffer.allocate(8 + rawValue.length) + .putLong(record.timestamp()) + .put(rawValue).array(), + record.headers(), + record.leaderEpoch() + ); + } + + private class RocksDbIterator extends AbstractIterator> implements KeyValueIterator { + // RocksDB's JNI interface does not expose getters/setters that allow the + // comparator to be pluggable, and the default is lexicographic, so it's + // safe to just force lexicographic comparator here for now. + private final Comparator comparator = Bytes.BYTES_LEXICO_COMPARATOR; + + private final String storeName; + private final RocksIterator iterWithTimestamp; + private final RocksIterator iterNoTimestamp; + + private volatile boolean open = true; + + private KeyValue nextWithTimestamp; + private KeyValue nextNoTimestamp; + private KeyValue next; + + RocksDbIterator(final String storeName, + final RocksIterator iterWithTimestamp, + final RocksIterator iterNoTimestamp) { + this.iterWithTimestamp = iterWithTimestamp; + this.iterNoTimestamp = iterNoTimestamp; + this.storeName = storeName; + } + + @Override + public synchronized boolean hasNext() { + if (!open) { + throw new InvalidStateStoreException(String.format("RocksDB store %s has closed", storeName)); + } + return super.hasNext(); + } + + @Override + public synchronized KeyValue next() { + return super.next(); + } + + @Override + public KeyValue makeNext() { + if (nextNoTimestamp == null && iterNoTimestamp.isValid()) { + nextNoTimestamp = getKeyValueNoTimestamp(); + } + + if (nextWithTimestamp == null && iterWithTimestamp.isValid()) { + nextWithTimestamp = getKeyValueWithTimestamp(); + } + + if (nextNoTimestamp == null) { + if (nextWithTimestamp == null) { + return allDone(); + } else { + next = nextWithTimestamp; + nextWithTimestamp = null; + iterWithTimestamp.next(); + } + } else { + if (nextWithTimestamp == null) { + next = nextNoTimestamp; + nextNoTimestamp = null; + iterNoTimestamp.next(); + } else { + if (comparator.compare(nextNoTimestamp.key.get(), nextWithTimestamp.key.get()) <= 0) { + next = nextNoTimestamp; + nextNoTimestamp = null; + iterNoTimestamp.next(); + } else { + next = nextWithTimestamp; + nextWithTimestamp = null; + iterWithTimestamp.next(); + } + } + + } + + return next; + } + + + private KeyValue getKeyValueWithTimestamp() { + return new KeyValue<>(new Bytes(iterWithTimestamp.key()), iterWithTimestamp.value()); + } + + private KeyValue getKeyValueNoTimestamp() { + return new KeyValue<>(new Bytes(iterNoTimestamp.key()), getValueWithUnknownTimestamp(iterNoTimestamp.value())); + } + + @Override + public void remove() { + throw new UnsupportedOperationException("RocksDB iterator does not support remove()"); + } + + @Override + public synchronized void close() { + openIterators.remove(this); + iterNoTimestamp.close(); + iterWithTimestamp.close(); + open = false; + } + + @Override + public Bytes peekNextKey() { + if (!hasNext()) { + throw new NoSuchElementException(); + } + return next.key; + } + } + + private class RocksDBRangeIterator extends RocksDbIterator { + // RocksDB's JNI interface does not expose getters/setters that allow the + // comparator to be pluggable, and the default is lexicographic, so it's + // safe to just force lexicographic comparator here for now. + private final Comparator comparator = Bytes.BYTES_LEXICO_COMPARATOR; + private final byte[] rawToKey; + + RocksDBRangeIterator(final String storeName, + final RocksIterator iterWithTimestamp, + final RocksIterator iterNoTimestamp, + final Bytes from, + final Bytes to) { + super(storeName, iterWithTimestamp, iterNoTimestamp); + iterWithTimestamp.seek(from.get()); + iterNoTimestamp.seek(from.get()); + rawToKey = to.get(); + if (rawToKey == null) { + throw new NullPointerException("RocksDBRangeIterator: RawToKey is null for key " + to); + } + } + + @Override + public KeyValue makeNext() { + final KeyValue next = super.makeNext(); + + if (next == null) { + return allDone(); + } else { + if (comparator.compare(next.key.get(), rawToKey) <= 0) + return next; + else + return allDone(); + } + } + } + + // not private for testing + static class RocksDBBatchingRestoreCallback extends AbstractNotifyingBatchingRestoreCallback { + + private final RocksDBWithTimestampStore rocksDBStore; + + RocksDBBatchingRestoreCallback(final RocksDBWithTimestampStore rocksDBStore) { + this.rocksDBStore = rocksDBStore; + } + + @Override + public void restoreAll(final Collection> records) { + rocksDBStore.restoreAllInternal(records); + } + + @Override + public void onRestoreStart(final TopicPartition topicPartition, + final String storeName, + final long startingOffset, + final long endingOffset) { + rocksDBStore.toggleDbForBulkLoading(true); + } + + @Override + public void onRestoreEnd(final TopicPartition topicPartition, + final String storeName, + final long totalRestored) { + rocksDBStore.toggleDbForBulkLoading(false); + } + } + + // for testing + public Options getOptions() { + return userSpecifiedOptions; + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDbKeyValueWithTimestampBytesStoreSupplier.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDbKeyValueWithTimestampBytesStoreSupplier.java new file mode 100644 index 0000000000000..9c29f2737ea1f --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDbKeyValueWithTimestampBytesStoreSupplier.java @@ -0,0 +1,45 @@ +/* + * 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.utils.Bytes; +import org.apache.kafka.streams.state.KeyValueBytesStoreSupplier; +import org.apache.kafka.streams.state.KeyValueStore; + +public class RocksDbKeyValueWithTimestampBytesStoreSupplier implements KeyValueBytesStoreSupplier { + + private final String name; + + public RocksDbKeyValueWithTimestampBytesStoreSupplier(final String name) { + this.name = name; + } + + @Override + public String name() { + return name; + } + + @Override + public KeyValueStore get() { + return new RocksDBWithTimestampStore(name); + } + + @Override + public String metricsScope() { + return "rocksdb-state"; + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDbSessionBytesStoreSupplier.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDbSessionBytesStoreSupplier.java index e88755b2fa10c..1bdafcec93d69 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDbSessionBytesStoreSupplier.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDbSessionBytesStoreSupplier.java @@ -43,7 +43,7 @@ public SessionStore get() { metricsScope(), retentionPeriod, segmentIntervalMs(), - new SessionKeySchema()); + new SessionKeySchema<>()); return new RocksDBSessionStore<>(segmented, Serdes.Bytes(), Serdes.ByteArray()); } diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDbSessionWithTimestampBytesStoreSupplier.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDbSessionWithTimestampBytesStoreSupplier.java new file mode 100644 index 0000000000000..45c6b6f48f051 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDbSessionWithTimestampBytesStoreSupplier.java @@ -0,0 +1,65 @@ +/* + * 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.serialization.Serdes; +import org.apache.kafka.common.utils.Bytes; +import org.apache.kafka.streams.state.SessionBytesStoreSupplier; +import org.apache.kafka.streams.state.SessionStore; + +public class RocksDbSessionWithTimestampBytesStoreSupplier implements SessionBytesStoreSupplier { + private final String name; + private final long retentionPeriod; + + public RocksDbSessionWithTimestampBytesStoreSupplier(final String name, + final long retentionPeriod) { + this.name = name; + this.retentionPeriod = retentionPeriod; + } + + @Override + public String name() { + return name; + } + + @Override + public SessionStore get() { + final RocksDBSegmentedWithTimestampBytesStore segmented = new RocksDBSegmentedWithTimestampBytesStore( + name, + metricsScope(), + retentionPeriod, + segmentIntervalMs(), + new SessionKeySchema<>()); + return new RocksDBSessionWithTimestampStore<>(segmented, Serdes.Bytes(), Serdes.ByteArray()); + } + + @Override + public String metricsScope() { + return "rocksdb-session-state"; + } + + @Override + public long segmentIntervalMs() { + // Selected somewhat arbitrarily. Profiling may reveal a different value is preferable. + return Math.max(retentionPeriod / 2, 60_000L); + } + + @Override + public long retentionPeriod() { + return retentionPeriod; + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDbWindowBytesStoreSupplier.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDbWindowBytesStoreSupplier.java index b9b72792d8be0..bed0b86fc2da8 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDbWindowBytesStoreSupplier.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDbWindowBytesStoreSupplier.java @@ -52,7 +52,7 @@ public WindowStore get() { metricsScope(), retentionPeriod, segmentInterval, - new WindowKeySchema() + new WindowKeySchema<>() ); return new RocksDBWindowStore<>(segmentedBytesStore, Serdes.Bytes(), diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDbWindowWithTimestampBytesStoreSupplier.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDbWindowWithTimestampBytesStoreSupplier.java new file mode 100644 index 0000000000000..bae12dfec1d1f --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDbWindowWithTimestampBytesStoreSupplier.java @@ -0,0 +1,95 @@ +/* + * 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.serialization.Serdes; +import org.apache.kafka.common.utils.Bytes; +import org.apache.kafka.streams.state.WindowBytesStoreSupplier; +import org.apache.kafka.streams.state.WindowStore; + +public class RocksDbWindowWithTimestampBytesStoreSupplier implements WindowBytesStoreSupplier { + private final String name; + private final long retentionPeriod; + private final long segmentInterval; + private final long windowSize; + private final boolean retainDuplicates; + + public RocksDbWindowWithTimestampBytesStoreSupplier(final String name, + final long retentionPeriod, + final long segmentInterval, + final long windowSize, + final boolean retainDuplicates) { + this.name = name; + this.retentionPeriod = retentionPeriod; + this.segmentInterval = segmentInterval; + this.windowSize = windowSize; + this.retainDuplicates = retainDuplicates; + } + + @Override + public String name() { + return name; + } + + @Override + public WindowStore get() { + final RocksDBSegmentedWithTimestampBytesStore segmentedBytesStore = new RocksDBSegmentedWithTimestampBytesStore( + name, + metricsScope(), + retentionPeriod, + segmentInterval, + new WindowKeySchema<>() + ); + return new RocksDBWindowWithTimestampStore<>(segmentedBytesStore, + Serdes.Bytes(), + Serdes.ByteArray(), + retainDuplicates, + windowSize); + + } + + @Override + public String metricsScope() { + return "rocksdb-window-state"; + } + + @Deprecated + @Override + public int segments() { + return (int) (retentionPeriod / segmentInterval) + 1; + } + + @Override + public long segmentIntervalMs() { + return segmentInterval; + } + + @Override + public long windowSize() { + return windowSize; + } + + @Override + public boolean retainDuplicates() { + return retainDuplicates; + } + + @Override + public long retentionPeriod() { + return retentionPeriod; + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/Segment.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/Segment.java index f95e3955c108d..8687ffc1a45a0 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/Segment.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/Segment.java @@ -16,52 +16,17 @@ */ package org.apache.kafka.streams.state.internals; -import org.apache.kafka.common.utils.Utils; -import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.common.utils.Bytes; +import org.apache.kafka.streams.processor.StateStore; +import org.apache.kafka.streams.state.KeyValueIterator; import java.io.IOException; -import java.util.Objects; -class Segment extends RocksDBStore implements Comparable { - public final long id; +public interface Segment extends StateStore { - Segment(final String segmentName, final String windowName, final long id) { - super(segmentName, windowName); - this.id = id; - } + void destroy() throws IOException; - void destroy() throws IOException { - Utils.delete(dbDir); - } + KeyValueIterator all(); - @Override - public int compareTo(final Segment segment) { - return Long.compare(id, segment.id); - } - - @Override - public void openDB(final ProcessorContext context) { - super.openDB(context); - // skip the registering step - internalProcessorContext = context; - } - - @Override - public String toString() { - return "Segment(id=" + id + ", name=" + name() + ")"; - } - - @Override - public boolean equals(final Object obj) { - if (obj == null || getClass() != obj.getClass()) { - return false; - } - final Segment segment = (Segment) obj; - return id == segment.id; - } - - @Override - public int hashCode() { - return Objects.hash(id); - } + KeyValueIterator range(final Bytes from, final Bytes to); } diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/SegmentIterator.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/SegmentIterator.java index 5b0781211e9ef..f0b13ab256432 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/SegmentIterator.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/SegmentIterator.java @@ -20,25 +20,24 @@ import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.errors.InvalidStateStoreException; import org.apache.kafka.streams.state.KeyValueIterator; -import org.apache.kafka.streams.state.KeyValueStore; import java.util.Iterator; import java.util.NoSuchElementException; /** - * Iterate over multiple Segments + * Iterate over multiple PlainSegments */ -class SegmentIterator implements KeyValueIterator { +class SegmentIterator implements KeyValueIterator { private final Bytes from; private final Bytes to; - protected final Iterator segments; + protected final Iterator segments; protected final HasNextCondition hasNextCondition; - protected KeyValueStore currentSegment; - protected KeyValueIterator currentIterator; + private S currentSegment; + KeyValueIterator currentIterator; - SegmentIterator(final Iterator segments, + SegmentIterator(final Iterator segments, final HasNextCondition hasNextCondition, final Bytes from, final Bytes to) { diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/SegmentedBytesStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/SegmentedBytesStore.java index ce528ed2f120f..879f8e9e38e22 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/SegmentedBytesStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/SegmentedBytesStore.java @@ -96,7 +96,7 @@ public interface SegmentedBytesStore extends StateStore { */ byte[] get(Bytes key); - interface KeySchema { + interface KeySchema { /** * Given a range of record keys and a time, construct a Segmented key that represents @@ -154,7 +154,7 @@ interface KeySchema { /** * Create an implementation of {@link HasNextCondition} that knows when - * to stop iterating over the Segments. Used during {@link SegmentedBytesStore#fetch(Bytes, Bytes, long, long)} operations + * to stop iterating over the PlainSegments. Used during {@link SegmentedBytesStore#fetch(Bytes, Bytes, long, long)} operations * @param binaryKeyFrom the first key in the range * @param binaryKeyTo the last key in the range * @param from starting time range @@ -171,6 +171,6 @@ interface KeySchema { * @param to * @return List of segments to search */ - List segmentsToSearch(Segments segments, long from, long to); + List segmentsToSearch(AbstractSegments segments, long from, long to); } } diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/SessionKeySchema.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/SessionKeySchema.java index 8ba78cc57d8fe..6c0d67ae485e5 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/SessionKeySchema.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/SessionKeySchema.java @@ -22,13 +22,12 @@ import org.apache.kafka.streams.kstream.Window; import org.apache.kafka.streams.kstream.Windowed; import org.apache.kafka.streams.kstream.internals.SessionWindow; -import org.apache.kafka.streams.state.KeyValueIterator; import java.nio.ByteBuffer; import java.util.List; -public class SessionKeySchema implements SegmentedBytesStore.KeySchema { +public class SessionKeySchema implements SegmentedBytesStore.KeySchema { private static final int TIMESTAMP_SIZE = 8; private static final int SUFFIX_SIZE = 2 * TIMESTAMP_SIZE; @@ -69,29 +68,26 @@ public long segmentTimestamp(final Bytes key) { @Override public HasNextCondition hasNextCondition(final Bytes binaryKeyFrom, final Bytes binaryKeyTo, final long from, final long to) { - return new HasNextCondition() { - @Override - public boolean hasNext(final KeyValueIterator iterator) { - while (iterator.hasNext()) { - final Bytes bytes = iterator.peekNextKey(); - final Windowed windowedKey = SessionKeySchema.from(bytes); - if ((binaryKeyFrom == null || windowedKey.key().compareTo(binaryKeyFrom) >= 0) - && (binaryKeyTo == null || windowedKey.key().compareTo(binaryKeyTo) <= 0) - && windowedKey.window().end() >= from - && windowedKey.window().start() <= to) { - return true; - } - iterator.next(); + return iterator -> { + while (iterator.hasNext()) { + final Bytes bytes = iterator.peekNextKey(); + final Windowed windowedKey = SessionKeySchema.from(bytes); + if ((binaryKeyFrom == null || windowedKey.key().compareTo(binaryKeyFrom) >= 0) + && (binaryKeyTo == null || windowedKey.key().compareTo(binaryKeyTo) <= 0) + && windowedKey.window().end() >= from + && windowedKey.window().start() <= to) { + return true; } - return false; + iterator.next(); } + return false; }; } @Override - public List segmentsToSearch(final Segments segments, - final long from, - final long to) { + public List segmentsToSearch(final AbstractSegments segments, + final long from, + final long to) { return segments.segments(from, Long.MAX_VALUE); } diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/SessionStoreBuilder.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/SessionStoreBuilder.java index b4338952a667a..b861e5ec327a2 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/SessionStoreBuilder.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/SessionStoreBuilder.java @@ -22,7 +22,6 @@ import org.apache.kafka.streams.state.SessionBytesStoreSupplier; import org.apache.kafka.streams.state.SessionStore; - public class SessionStoreBuilder extends AbstractStoreBuilder> { private final SessionBytesStoreSupplier storeSupplier; @@ -37,21 +36,23 @@ public SessionStoreBuilder(final SessionBytesStoreSupplier storeSupplier, @Override public SessionStore build() { - return new MeteredSessionStore<>(maybeWrapCaching(maybeWrapLogging(storeSupplier.get())), - storeSupplier.metricsScope(), - keySerde, - valueSerde, - time); + return new MeteredSessionStore<>( + maybeWrapCaching(maybeWrapLogging(storeSupplier.get())), + storeSupplier.metricsScope(), + keySerde, + valueSerde, + time); } private SessionStore maybeWrapCaching(final SessionStore inner) { if (!enableCaching) { return inner; } - return new CachingSessionStore<>(inner, - keySerde, - valueSerde, - storeSupplier.segmentIntervalMs()); + return new CachingSessionStore<>( + inner, + keySerde, + valueSerde, + storeSupplier.segmentIntervalMs()); } private SessionStore maybeWrapLogging(final SessionStore inner) { diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/SessionToSessionWithTimestampByteProxyStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/SessionToSessionWithTimestampByteProxyStore.java new file mode 100644 index 0000000000000..c29da05c09cb5 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/SessionToSessionWithTimestampByteProxyStore.java @@ -0,0 +1,103 @@ +/* + * 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.utils.Bytes; +import org.apache.kafka.streams.kstream.Windowed; +import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.processor.StateStore; +import org.apache.kafka.streams.state.KeyValueIterator; +import org.apache.kafka.streams.state.SessionStore; + +import static org.apache.kafka.streams.state.internals.StoreProxyUtils.getValue; + +class SessionToSessionWithTimestampByteProxyStore implements SessionStore { + final SessionStore store; + + SessionToSessionWithTimestampByteProxyStore(final SessionStore store) { + this.store = store; + } + + @Override + public KeyValueIterator, byte[]> findSessions(final Bytes key, + final long earliestSessionEndTime, + final long latestSessionStartTime) { + return new StoreProxyUtils.KeyValueIteratorProxy<>( + store.findSessions(key, earliestSessionEndTime, latestSessionStartTime)); + } + + @Override + public KeyValueIterator, byte[]> findSessions(final Bytes keyFrom, + final Bytes keyTo, + final long earliestSessionEndTime, + final long latestSessionStartTime) { + return new StoreProxyUtils.KeyValueIteratorProxy<>( + store.findSessions(keyFrom, keyTo, earliestSessionEndTime, latestSessionStartTime)); + } + + @Override + public void remove(final Windowed sessionKey) { + store.remove(sessionKey); + } + + @Override + public void put(final Windowed sessionKey, + final byte[] aggregateWithTimestamp) { + store.put(sessionKey, getValue(aggregateWithTimestamp)); + } + + @Override + public String name() { + return store.name(); + } + + @Override + public void init(final ProcessorContext context, + final StateStore root) { + store.init(context, root); + } + + @Override + public void flush() { + store.flush(); + } + + @Override + public void close() { + store.close(); + } + + @Override + public boolean persistent() { + return true; + } + + @Override + public boolean isOpen() { + return store.isOpen(); + } + + @Override + public KeyValueIterator, byte[]> fetch(final Bytes key) { + return new StoreProxyUtils.KeyValueIteratorProxy<>(store.fetch(key)); + } + + @Override + public KeyValueIterator, byte[]> fetch(final Bytes from, final Bytes to) { + return new StoreProxyUtils.KeyValueIteratorProxy<>(store.fetch(from, to)); + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/SessionWithTimestampStoreBuilder.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/SessionWithTimestampStoreBuilder.java new file mode 100644 index 0000000000000..400d42649bc42 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/SessionWithTimestampStoreBuilder.java @@ -0,0 +1,74 @@ +/* + * 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.serialization.Serde; +import org.apache.kafka.common.utils.Bytes; +import org.apache.kafka.common.utils.Time; +import org.apache.kafka.streams.state.RecordConverter; +import org.apache.kafka.streams.state.SessionBytesStoreSupplier; +import org.apache.kafka.streams.state.SessionStore; +import org.apache.kafka.streams.state.ValueAndTimestamp; + +public class SessionWithTimestampStoreBuilder extends AbstractStoreBuilder, SessionStore>> { + + private final SessionBytesStoreSupplier storeSupplier; + + public SessionWithTimestampStoreBuilder(final SessionBytesStoreSupplier storeSupplier, + final Serde keySerde, + final Serde valueSerde, + final Time time) { + super(storeSupplier.name(), keySerde, new KeyValueWithTimestampStoreBuilder.ValueAndTimestampSerde<>(valueSerde), time); + this.storeSupplier = storeSupplier; + } + + @Override + public MeteredSessionWithTimestampStore build() { + SessionStore store = storeSupplier.get(); + if (!(store instanceof RecordConverter) && store.persistent()) { + store = new SessionToSessionWithTimestampByteProxyStore(store); + } + return new MeteredSessionWithTimestampStore<>( + maybeWrapCaching(maybeWrapLogging(store)), + storeSupplier.metricsScope(), + keySerde, + valueSerde, + time); + } + + private SessionStore maybeWrapCaching(final SessionStore inner) { + if (!enableCaching) { + return inner; + } + return new CachingSessionWithTimestampStore<>( + inner, + keySerde, + valueSerde, + storeSupplier.segmentIntervalMs()); + } + + private SessionStore maybeWrapLogging(final SessionStore inner) { + if (!enableLogging) { + return inner; + } + return new ChangeLoggingSessionWithTimestampBytesStore(inner); + } + + public long retentionPeriod() { + return storeSupplier.retentionPeriod(); + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/StoreChangeLogger.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/StoreChangeLogger.java index 74134d63bb2fb..bf4a3014a16d7 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/StoreChangeLogger.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/StoreChangeLogger.java @@ -32,12 +32,12 @@ */ class StoreChangeLogger { - protected final StateSerdes serialization; - private final String topic; private final int partition; private final ProcessorContext context; private final RecordCollector collector; + private final Serializer keySerializer; + private final Serializer valueSerializer; StoreChangeLogger(final String storeName, final ProcessorContext context, @@ -49,20 +49,22 @@ private StoreChangeLogger(final String storeName, final ProcessorContext context, final int partition, final StateSerdes serialization) { - this.topic = ProcessorStateManager.storeChangelogTopic(context.applicationId(), storeName); + topic = ProcessorStateManager.storeChangelogTopic(context.applicationId(), storeName); this.context = context; this.partition = partition; - this.serialization = serialization; this.collector = ((RecordCollector.Supplier) context).recordCollector(); + keySerializer = serialization.keySerializer(); + valueSerializer = serialization.valueSerializer(); } void logChange(final K key, final V value) { - if (collector != null) { - final Serializer keySerializer = serialization.keySerializer(); - final Serializer valueSerializer = serialization.valueSerializer(); - // Sending null headers to changelog topics (KIP-244) - collector.send(this.topic, key, value, null, this.partition, context.timestamp(), keySerializer, valueSerializer); - } + logChange(key, value, context.timestamp()); + } + + void logChange(final K key, + final V value, + final long timestamp) { + collector.send(topic, key, value, null, partition, timestamp, keySerializer, valueSerializer); } } diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/StoreProxyUtils.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/StoreProxyUtils.java new file mode 100644 index 0000000000000..e445a779c014f --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/StoreProxyUtils.java @@ -0,0 +1,83 @@ +/* + * 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.serialization.LongSerializer; +import org.apache.kafka.streams.KeyValue; +import org.apache.kafka.streams.state.KeyValueIterator; +import org.apache.kafka.streams.state.WindowStoreIterator; + +class StoreProxyUtils { + private static final byte[] UNKNOWN_TIMESTAMP_BYTE_ARRAY = new LongSerializer().serialize(null, -1L); + + static byte[] getValue(final byte[] rawValueWithTimestamp) { + if (rawValueWithTimestamp == null) { + return null; + } + final byte[] rawValue = new byte[rawValueWithTimestamp.length - 8]; + // TODO: should we use `ByteBuffer` instead of `System.arraycopy` ? + System.arraycopy(rawValueWithTimestamp, 8, rawValue, 0, rawValue.length); + return rawValue; + } + + static byte[] getValueWithUnknownTimestamp(final byte[] rawValue) { + if (rawValue == null) { + return null; + } + final byte[] rawValueWithUnknownTimestamp = new byte[8 + rawValue.length]; + // TODO: should we use `ByteBuffer` instead of `System.arraycopy` ? + System.arraycopy(UNKNOWN_TIMESTAMP_BYTE_ARRAY, 0, rawValueWithUnknownTimestamp, 0, 8); + System.arraycopy(rawValue, 0, rawValueWithUnknownTimestamp, 8, rawValue.length); + return rawValueWithUnknownTimestamp; + } + + static class KeyValueIteratorProxy implements KeyValueIterator { + private final KeyValueIterator innerIterator; + + KeyValueIteratorProxy(final KeyValueIterator innerIterator) { + this.innerIterator = innerIterator; + } + + @Override + public void close() { + innerIterator.close(); + } + + @Override + public K peekNextKey() { + return innerIterator.peekNextKey(); + } + + @Override + public boolean hasNext() { + return innerIterator.hasNext(); + } + + @Override + public KeyValue next() { + final KeyValue plainKeyValue = innerIterator.next(); + return KeyValue.pair(plainKeyValue.key, getValueWithUnknownTimestamp(plainKeyValue.value)); + } + } + + static class WindowIteratorProxy extends KeyValueIteratorProxy implements WindowStoreIterator { + + WindowIteratorProxy(final KeyValueIterator innerIterator) { + super(innerIterator); + } + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/StreamThreadStateStoreProvider.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/StreamThreadStateStoreProvider.java index e097963b3f218..34711ce0b828d 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/StreamThreadStateStoreProvider.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/StreamThreadStateStoreProvider.java @@ -17,10 +17,17 @@ package org.apache.kafka.streams.state.internals; import org.apache.kafka.streams.errors.InvalidStateStoreException; +import org.apache.kafka.streams.kstream.internals.KStreamImpl; +import org.apache.kafka.streams.kstream.internals.KeyValueWithTimestampStoreMaterializer; +import org.apache.kafka.streams.kstream.internals.SessionWindowedKStreamImpl; import org.apache.kafka.streams.processor.StateStore; import org.apache.kafka.streams.processor.internals.StreamThread; import org.apache.kafka.streams.processor.internals.Task; +import org.apache.kafka.streams.state.KeyValueStore; import org.apache.kafka.streams.state.QueryableStoreType; +import org.apache.kafka.streams.state.QueryableStoreTypes; +import org.apache.kafka.streams.state.SessionStore; +import org.apache.kafka.streams.state.WindowStore; import java.util.ArrayList; import java.util.Collections; @@ -56,7 +63,30 @@ public List stores(final String storeName, final QueryableStoreType qu throw new InvalidStateStoreException("Cannot get state store " + storeName + " for task " + streamTask + " because the store is not open. The state store may have migrated to another instances."); } - stores.add((T) store); + if (store instanceof KeyValueWithTimestampStoreMaterializer.KeyValueStoreFacade) { + final KeyValueStore innerStore = ((KeyValueWithTimestampStoreMaterializer.KeyValueStoreFacade) store).inner; + if (queryableStoreType instanceof QueryableStoreTypes.KeyValueStoreType) { + stores.add((T) new ReadOnlyKeyValueStoreFacade(innerStore)); + } else { + stores.add((T) innerStore); + } + } else if (store instanceof KStreamImpl.WindowStoreFacade) { + final WindowStore innerStore = ((KStreamImpl.WindowStoreFacade) store).inner; + if (queryableStoreType instanceof QueryableStoreTypes.WindowStoreType) { + stores.add((T) new ReadOnlyWindowStoreFacade(innerStore)); + } else { + stores.add((T) innerStore); + } + } else if (store instanceof SessionWindowedKStreamImpl.SessionStoreFacade) { + final SessionStore innerStore = ((SessionWindowedKStreamImpl.SessionStoreFacade) store).inner; + if (queryableStoreType instanceof QueryableStoreTypes.SessionStoreType) { + stores.add((T) new ReadOnlySessionStoreFacade(innerStore)); + } else { + stores.add((T) innerStore); + } + } else { + stores.add((T) store); + } } } return stores; diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/WindowKeySchema.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/WindowKeySchema.java index 0b3ba9e7d1ded..fd023bf647e44 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/WindowKeySchema.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/WindowKeySchema.java @@ -28,7 +28,7 @@ import java.nio.ByteBuffer; import java.util.List; -public class WindowKeySchema implements RocksDBSegmentedBytesStore.KeySchema { +public class WindowKeySchema implements RocksDBSegmentedBytesStore.KeySchema { private static final int SEQNUM_SIZE = 4; private static final int TIMESTAMP_SIZE = 8; @@ -88,7 +88,7 @@ public boolean hasNext(final KeyValueIterator iterator) { } @Override - public List segmentsToSearch(final Segments segments, final long from, final long to) { + public List segmentsToSearch(final AbstractSegments segments, final long from, final long to) { return segments.segments(from, to); } diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/WindowStoreBuilder.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/WindowStoreBuilder.java index 058a2498dc5c1..42ea75304d175 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/WindowStoreBuilder.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/WindowStoreBuilder.java @@ -24,7 +24,7 @@ public class WindowStoreBuilder extends AbstractStoreBuilder> { - private final WindowBytesStoreSupplier storeSupplier; + final WindowBytesStoreSupplier storeSupplier; public WindowStoreBuilder(final WindowBytesStoreSupplier storeSupplier, final Serde keySerde, diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/WindowToWindowWithTimestampByteProxyStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/WindowToWindowWithTimestampByteProxyStore.java new file mode 100644 index 0000000000000..cca0f4d9e86dd --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/WindowToWindowWithTimestampByteProxyStore.java @@ -0,0 +1,138 @@ +/* + * 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.utils.Bytes; +import org.apache.kafka.streams.kstream.Windowed; +import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.processor.StateStore; +import org.apache.kafka.streams.state.KeyValueIterator; +import org.apache.kafka.streams.state.WindowStore; +import org.apache.kafka.streams.state.WindowStoreIterator; + +import java.time.Instant; + +import static org.apache.kafka.streams.state.internals.StoreProxyUtils.getValue; +import static org.apache.kafka.streams.state.internals.StoreProxyUtils.getValueWithUnknownTimestamp; + +class WindowToWindowWithTimestampByteProxyStore implements WindowStore { + final WindowStore store; + + WindowToWindowWithTimestampByteProxyStore(final WindowStore store) { + this.store = store; + } + + @Override + public void put(final Bytes key, + final byte[] valueWithTimestamp) { + store.put(key, getValue(valueWithTimestamp)); + } + + @Override + public void put(final Bytes key, final byte[] value, + final long windowStartTimestamp) { + store.put(key, value); + } + + @Override + public byte[] fetch(final Bytes key, + final long time) { + return getValueWithUnknownTimestamp(store.fetch(key, time)); + } + + @Override + @SuppressWarnings("deprecation") + public WindowStoreIterator fetch(final Bytes key, + final long timeFrom, + final long timeTo) { + return new StoreProxyUtils.WindowIteratorProxy(store.fetch(key, timeFrom, timeTo)); + } + + @Override + public WindowStoreIterator fetch(final Bytes key, + final Instant from, + final Instant to) { + return new StoreProxyUtils.WindowIteratorProxy(store.fetch(key, from, to)); + } + + @Override + @SuppressWarnings("deprecation") + public KeyValueIterator, byte[]> fetch(final Bytes from, + final Bytes to, + final long timeFrom, + final long timeTo) { + return new StoreProxyUtils.KeyValueIteratorProxy<>(store.fetch(from, to, timeFrom, timeTo)); + } + + @Override + public KeyValueIterator, byte[]> fetch(final Bytes from, + final Bytes to, + final Instant fromTime, + final Instant toTime) { + return new StoreProxyUtils.KeyValueIteratorProxy<>(store.fetch(from, to, fromTime, toTime)); + } + + @Override + public KeyValueIterator, byte[]> all() { + return new StoreProxyUtils.KeyValueIteratorProxy<>(store.all()); + } + + @Override + @SuppressWarnings("deprecation") + public KeyValueIterator, byte[]> fetchAll(final long timeFrom, + final long timeTo) { + return new StoreProxyUtils.KeyValueIteratorProxy<>(store.fetchAll(timeFrom, timeTo)); + } + + @Override + public KeyValueIterator, byte[]> fetchAll(final Instant from, + final Instant to) { + return new StoreProxyUtils.KeyValueIteratorProxy<>(store.fetchAll(from, to)); + } + + @Override + public String name() { + return store.name(); + } + + @Override + public void init(final ProcessorContext context, + final StateStore root) { + store.init(context, root); + } + + @Override + public void flush() { + store.flush(); + } + + @Override + public void close() { + store.close(); + } + + @Override + public boolean persistent() { + return true; + } + + @Override + public boolean isOpen() { + return store.isOpen(); + } + +} diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/WindowWithTimestampStoreBuilder.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/WindowWithTimestampStoreBuilder.java new file mode 100644 index 0000000000000..3d1ffda249a22 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/WindowWithTimestampStoreBuilder.java @@ -0,0 +1,75 @@ +/* + * 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.serialization.Serde; +import org.apache.kafka.common.utils.Bytes; +import org.apache.kafka.common.utils.Time; +import org.apache.kafka.streams.state.RecordConverter; +import org.apache.kafka.streams.state.ValueAndTimestamp; +import org.apache.kafka.streams.state.WindowBytesStoreSupplier; +import org.apache.kafka.streams.state.WindowStore; + +public class WindowWithTimestampStoreBuilder extends AbstractStoreBuilder, WindowStore>> { + + private final WindowBytesStoreSupplier storeSupplier; + + public WindowWithTimestampStoreBuilder(final WindowBytesStoreSupplier storeSupplier, + final Serde keySerde, + final Serde valueSerde, + final Time time) { + super(storeSupplier.name(), keySerde, new KeyValueWithTimestampStoreBuilder.ValueAndTimestampSerde<>(valueSerde), time); + this.storeSupplier = storeSupplier; + } + + @Override + public WindowStore> build() { + WindowStore store = storeSupplier.get(); + if (!(store instanceof RecordConverter) && store.persistent()) { + store = new WindowToWindowWithTimestampByteProxyStore(store); + } + return new MeteredWindowWithTimestampStore<>( + maybeWrapCaching(maybeWrapLogging(store)), + storeSupplier.metricsScope(), + time, + keySerde, + valueSerde); + } + + private WindowStore maybeWrapCaching(final WindowStore inner) { + if (!enableCaching) { + return inner; + } + return new CachingWindowWithTimestampStore<>( + inner, + keySerde, + valueSerde, + storeSupplier.windowSize(), + storeSupplier.segmentIntervalMs()); + } + + private WindowStore maybeWrapLogging(final WindowStore inner) { + if (!enableLogging) { + return inner; + } + return new ChangeLoggingWindowWithTimestampBytesStore(inner, storeSupplier.retainDuplicates()); + } + + public long retentionPeriod() { + return storeSupplier.retentionPeriod(); + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/WithTimestampSegment.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/WithTimestampSegment.java new file mode 100644 index 0000000000000..d30b78d03e488 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/WithTimestampSegment.java @@ -0,0 +1,68 @@ +/* + * 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.utils.Utils; +import org.apache.kafka.streams.processor.ProcessorContext; + +import java.io.IOException; +import java.util.Objects; + +class WithTimestampSegment extends RocksDBWithTimestampStore implements Comparable, Segment { + public final long id; + + WithTimestampSegment(final String segmentName, final String windowName, final long id) { + super(segmentName, windowName); + this.id = id; + } + + @Override + public void destroy() throws IOException { + Utils.delete(dbDir); + } + + @Override + public int compareTo(final WithTimestampSegment segment) { + return Long.compare(id, segment.id); + } + + @Override + public void openDB(final ProcessorContext context) { + super.openDB(context); + // skip the registering step + internalProcessorContext = context; + } + + @Override + public String toString() { + return "WithTimestampSegment(id=" + id + ", name=" + name() + ")"; + } + + @Override + public boolean equals(final Object obj) { + if (obj == null || getClass() != obj.getClass()) { + return false; + } + final WithTimestampSegment segment = (WithTimestampSegment) obj; + return id == segment.id; + } + + @Override + public int hashCode() { + return Objects.hash(id); + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/WithTimestampSegments.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/WithTimestampSegments.java new file mode 100644 index 0000000000000..b229b1044534a --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/WithTimestampSegments.java @@ -0,0 +1,46 @@ +/* + * 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.streams.processor.internals.InternalProcessorContext; + +/** + * Manages the {@link PlainSegment}s that are used by the {@link RocksDBSegmentedBytesStore} + */ +class WithTimestampSegments extends AbstractSegments { + + WithTimestampSegments(final String name, final long retentionPeriod, final long segmentInterval) { + super(name, retentionPeriod, segmentInterval); + } + + @Override + WithTimestampSegment getOrCreateSegment(final long segmentId, final InternalProcessorContext context) { + if (segments.containsKey(segmentId)) { + return segments.get(segmentId); + } else { + final WithTimestampSegment newSegment = new WithTimestampSegment(segmentName(segmentId), name, segmentId); + final WithTimestampSegment shouldBeNull = segments.put(segmentId, newSegment); + + if (shouldBeNull != null) { + throw new IllegalStateException("PlainSegment already exists. Possible concurrent access."); + } + + newSegment.openDB(context); + return newSegment; + } + } +} diff --git a/streams/src/test/java/org/apache/kafka/streams/StreamsBuilderTest.java b/streams/src/test/java/org/apache/kafka/streams/StreamsBuilderTest.java index 9e88a87b40201..9d0a7208ea399 100644 --- a/streams/src/test/java/org/apache/kafka/streams/StreamsBuilderTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/StreamsBuilderTest.java @@ -29,6 +29,7 @@ import org.apache.kafka.streams.processor.internals.InternalTopologyBuilder; import org.apache.kafka.streams.processor.internals.ProcessorTopology; import org.apache.kafka.streams.state.KeyValueStore; +import org.apache.kafka.streams.state.ValueAndTimestamp; import org.apache.kafka.streams.test.ConsumerRecordFactory; import org.apache.kafka.test.MockMapper; import org.apache.kafka.test.MockPredicate; @@ -306,9 +307,9 @@ public void shouldUseSerdesDefinedInMaterializedToConsumeTable() { driver.pipeInput(recordFactory.create(topic, 1L, "value1")); driver.pipeInput(recordFactory.create(topic, 2L, "value2")); - final KeyValueStore store = driver.getKeyValueStore("store"); - assertThat(store.get(1L), equalTo("value1")); - assertThat(store.get(2L), equalTo("value2")); + final KeyValueStore> store = driver.getKeyValueWithTimestampStore("store"); + assertThat(store.get(1L).value(), equalTo("value1")); + assertThat(store.get(2L).value(), equalTo("value2")); assertThat(results.get(1L), equalTo("value1")); assertThat(results.get(2L), equalTo("value2")); } @@ -327,10 +328,10 @@ public void shouldUseSerdesDefinedInMaterializedToConsumeGlobalTable() { try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { driver.pipeInput(recordFactory.create(topic, 1L, "value1")); driver.pipeInput(recordFactory.create(topic, 2L, "value2")); - final KeyValueStore store = driver.getKeyValueStore("store"); + final KeyValueStore> store = driver.getKeyValueWithTimestampStore("store"); - assertThat(store.get(1L), equalTo("value1")); - assertThat(store.get(2L), equalTo("value2")); + assertThat(store.get(1L).value(), equalTo("value1")); + assertThat(store.get(2L).value(), equalTo("value2")); } } diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/AbstractJoinIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/AbstractJoinIntegrationTest.java index badaa36cd5521..6a2c0bb11a07a 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/AbstractJoinIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/AbstractJoinIntegrationTest.java @@ -35,6 +35,7 @@ import org.apache.kafka.streams.state.KeyValueIterator; import org.apache.kafka.streams.state.QueryableStoreTypes; import org.apache.kafka.streams.state.ReadOnlyKeyValueStore; +import org.apache.kafka.streams.state.ValueAndTimestamp; import org.apache.kafka.test.IntegrationTest; import org.apache.kafka.test.TestUtils; import org.junit.After; @@ -259,14 +260,14 @@ void runTest(final String expectedFinalResult, final String storeName) throws Ex * Checks the embedded queryable state store snapshot */ private void checkQueryableStore(final String queryableName, final String expectedFinalResult) { - final ReadOnlyKeyValueStore store = streams.store(queryableName, QueryableStoreTypes.keyValueStore()); + final ReadOnlyKeyValueStore> store = streams.store(queryableName, QueryableStoreTypes.keyValueWithTimestampStore()); - final KeyValueIterator all = store.all(); - final KeyValue onlyEntry = all.next(); + final KeyValueIterator> all = store.all(); + final KeyValue> onlyEntry = all.next(); try { assertThat(onlyEntry.key, is(anyUniqueKey)); - assertThat(onlyEntry.value, is(expectedFinalResult)); + assertThat(onlyEntry.value.value(), is(expectedFinalResult)); assertThat(all.hasNext(), is(false)); } finally { all.close(); diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/EosIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/EosIntegrationTest.java index f43b3969210f5..b412c000c9988 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/EosIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/EosIntegrationTest.java @@ -39,6 +39,7 @@ import org.apache.kafka.streams.state.ReadOnlyKeyValueStore; import org.apache.kafka.streams.state.StoreBuilder; import org.apache.kafka.streams.state.Stores; +import org.apache.kafka.streams.state.ValueAndTimestamp; import org.apache.kafka.test.IntegrationTest; import org.apache.kafka.test.StreamsTestUtils; import org.apache.kafka.test.TestUtils; @@ -744,12 +745,12 @@ private Set> getMaxPerKey(final List> private void verifyStateStore(final KafkaStreams streams, final Set> expectedStoreContent) { - ReadOnlyKeyValueStore store = null; + ReadOnlyKeyValueStore> store = null; final long maxWaitingTime = System.currentTimeMillis() + 300000L; while (System.currentTimeMillis() < maxWaitingTime) { try { - store = streams.store(storeName, QueryableStoreTypes.keyValueStore()); + store = streams.store(storeName, QueryableStoreTypes.keyValueWithTimestampStore()); break; } catch (final InvalidStateStoreException okJustRetry) { try { @@ -760,7 +761,7 @@ private void verifyStateStore(final KafkaStreams streams, assertNotNull(store); - final KeyValueIterator it = store.all(); + final KeyValueIterator> it = store.all(); while (it.hasNext()) { assertTrue(expectedStoreContent.remove(it.next())); } diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/GlobalKTableEOSIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/GlobalKTableEOSIntegrationTest.java index 787cb2921f1e8..5ef9610495c77 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/GlobalKTableEOSIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/GlobalKTableEOSIntegrationTest.java @@ -40,6 +40,7 @@ import org.apache.kafka.streams.state.KeyValueStore; import org.apache.kafka.streams.state.QueryableStoreTypes; import org.apache.kafka.streams.state.ReadOnlyKeyValueStore; +import org.apache.kafka.streams.state.ValueAndTimestamp; import org.apache.kafka.test.IntegrationTest; import org.apache.kafka.test.TestUtils; import org.junit.After; @@ -138,11 +139,11 @@ public void shouldKStreamGlobalKTableLeftJoin() throws Exception { produceGlobalTableValues(); - final ReadOnlyKeyValueStore replicatedStore = - kafkaStreams.store(globalStore, QueryableStoreTypes.keyValueStore()); + final ReadOnlyKeyValueStore> replicatedStore = + kafkaStreams.store(globalStore, QueryableStoreTypes.keyValueWithTimestampStore()); TestUtils.waitForCondition( - () -> "J".equals(replicatedStore.get(5L)), + () -> "J".equals(replicatedStore.get(5L).value()), 30000, "waiting for data in replicated store"); @@ -182,11 +183,11 @@ public void shouldKStreamGlobalKTableJoin() throws Exception { produceGlobalTableValues(); - final ReadOnlyKeyValueStore replicatedStore = - kafkaStreams.store(globalStore, QueryableStoreTypes.keyValueStore()); + final ReadOnlyKeyValueStore> replicatedStore = + kafkaStreams.store(globalStore, QueryableStoreTypes.keyValueWithTimestampStore()); TestUtils.waitForCondition( - () -> "J".equals(replicatedStore.get(5L)), + () -> "J".equals(replicatedStore.get(5L).value()), 30000, "waiting for data in replicated store"); @@ -218,17 +219,17 @@ public void shouldRestoreTransactionalMessages() throws Exception { TestUtils.waitForCondition( () -> { - final ReadOnlyKeyValueStore store; + final ReadOnlyKeyValueStore> store; try { - store = kafkaStreams.store(globalStore, QueryableStoreTypes.keyValueStore()); + store = kafkaStreams.store(globalStore, QueryableStoreTypes.keyValueWithTimestampStore()); } catch (final InvalidStateStoreException ex) { return false; } final Map result = new HashMap<>(); - final Iterator> it = store.all(); + final Iterator>> it = store.all(); while (it.hasNext()) { - final KeyValue kv = it.next(); - result.put(kv.key, kv.value); + final KeyValue> kv = it.next(); + result.put(kv.key, kv.value.value()); } return result.equals(expected); }, @@ -252,17 +253,17 @@ public void shouldNotRestoreAbortedMessages() throws Exception { TestUtils.waitForCondition( () -> { - final ReadOnlyKeyValueStore store; + final ReadOnlyKeyValueStore> store; try { - store = kafkaStreams.store(globalStore, QueryableStoreTypes.keyValueStore()); + store = kafkaStreams.store(globalStore, QueryableStoreTypes.keyValueWithTimestampStore()); } catch (final InvalidStateStoreException ex) { return false; } final Map result = new HashMap<>(); - final Iterator> it = store.all(); + final Iterator>> it = store.all(); while (it.hasNext()) { - final KeyValue kv = it.next(); - result.put(kv.key, kv.value); + final KeyValue> kv = it.next(); + result.put(kv.key, kv.value.value()); } return result.equals(expected); }, diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/GlobalKTableIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/GlobalKTableIntegrationTest.java index 2190d7071fa80..98a0c4c86d96d 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/GlobalKTableIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/GlobalKTableIntegrationTest.java @@ -39,6 +39,7 @@ import org.apache.kafka.streams.state.QueryableStoreTypes; import org.apache.kafka.streams.state.ReadOnlyKeyValueStore; import org.apache.kafka.streams.state.Stores; +import org.apache.kafka.streams.state.ValueAndTimestamp; import org.apache.kafka.test.IntegrationTest; import org.apache.kafka.test.TestUtils; import org.junit.After; @@ -131,11 +132,11 @@ public void shouldKStreamGlobalKTableLeftJoin() throws Exception { produceGlobalTableValues(); - final ReadOnlyKeyValueStore replicatedStore = - kafkaStreams.store(globalStore, QueryableStoreTypes.keyValueStore()); + final ReadOnlyKeyValueStore> replicatedStore = + kafkaStreams.store(globalStore, QueryableStoreTypes.keyValueWithTimestampStore()); TestUtils.waitForCondition( - () -> "J".equals(replicatedStore.get(5L)), + () -> "J".equals(replicatedStore.get(5L).value()), 30000, "waiting for data in replicated store"); @@ -175,11 +176,11 @@ public void shouldKStreamGlobalKTableJoin() throws Exception { produceGlobalTableValues(); - final ReadOnlyKeyValueStore replicatedStore = - kafkaStreams.store(globalStore, QueryableStoreTypes.keyValueStore()); + final ReadOnlyKeyValueStore> replicatedStore = + kafkaStreams.store(globalStore, QueryableStoreTypes.keyValueWithTimestampStore()); TestUtils.waitForCondition( - () -> "J".equals(replicatedStore.get(5L)), + () -> "J".equals(replicatedStore.get(5L).value()), 30000, "waiting for data in replicated store"); @@ -208,7 +209,7 @@ public void shouldRestoreGlobalInMemoryKTableOnRestart() throws Exception { produceInitialGlobalTableValues(); startStreams(); - ReadOnlyKeyValueStore store = kafkaStreams.store(globalStore, QueryableStoreTypes.keyValueStore()); + ReadOnlyKeyValueStore> store = kafkaStreams.store(globalStore, QueryableStoreTypes.keyValueWithTimestampStore()); assertThat(store.approximateNumEntries(), equalTo(4L)); kafkaStreams.close(); diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/GlobalThreadShutDownOrderTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/GlobalThreadShutDownOrderTest.java index 7c16927fcacfd..580f9ed16c0f3 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/GlobalThreadShutDownOrderTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/GlobalThreadShutDownOrderTest.java @@ -14,7 +14,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.streams.integration; import kafka.utils.MockTime; @@ -34,7 +33,8 @@ import org.apache.kafka.streams.processor.ProcessorContext; import org.apache.kafka.streams.state.KeyValueStore; import org.apache.kafka.streams.state.Stores; -import org.apache.kafka.streams.state.internals.KeyValueStoreBuilder; +import org.apache.kafka.streams.state.ValueAndTimestamp; +import org.apache.kafka.streams.state.internals.KeyValueWithTimestampStoreBuilder; import org.apache.kafka.test.IntegrationTest; import org.apache.kafka.test.MockProcessorSupplier; import org.apache.kafka.test.TestUtils; @@ -93,11 +93,12 @@ public void before() throws Exception { final Consumed stringLongConsumed = Consumed.with(Serdes.String(), Serdes.Long()); - final KeyValueStoreBuilder storeBuilder = new KeyValueStoreBuilder<>( - Stores.persistentKeyValueStore(globalStore), - Serdes.String(), - Serdes.Long(), - mockTime); + final KeyValueWithTimestampStoreBuilder storeBuilder = + new KeyValueWithTimestampStoreBuilder<>( + Stores.persistentKeyValueWithTimestampStore(globalStore), + Serdes.String(), + Serdes.Long(), + mockTime); builder.addGlobalStore( storeBuilder, @@ -167,7 +168,7 @@ private void populateTopics(final String topicName) throws Exception { private class GlobalStoreProcessor extends AbstractProcessor { - private KeyValueStore store; + private KeyValueStore> store; private final String storeName; GlobalStoreProcessor(final String storeName) { @@ -178,7 +179,7 @@ private class GlobalStoreProcessor extends AbstractProcessor { @SuppressWarnings("unchecked") public void init(final ProcessorContext context) { super.init(context); - store = (KeyValueStore) context.getStateStore(storeName); + store = (KeyValueStore>) context.getStateStore(storeName); } @Override @@ -193,7 +194,7 @@ public void close() { for (final String key : keys) { // need to simulate thread slow in closing Utils.sleep(1000); - retrievedValuesList.add(store.get(key)); + retrievedValuesList.add(store.get(key).value()); } } } diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/KTableSourceTopicRestartIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/KTableSourceTopicRestartIntegrationTest.java index 32d77c0d5ae5c..fb2b11d3e6bf8 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/KTableSourceTopicRestartIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/KTableSourceTopicRestartIntegrationTest.java @@ -14,10 +14,8 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.streams.integration; - import org.apache.kafka.clients.producer.ProducerConfig; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.serialization.Serdes; diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/PurgeRepartitionTopicIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/PurgeRepartitionTopicIntegrationTest.java index 4c7859bd86f81..23a4e546fae8b 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/PurgeRepartitionTopicIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/PurgeRepartitionTopicIntegrationTest.java @@ -161,7 +161,6 @@ public void setup() { streamsConfiguration.put(StreamsConfig.topicPrefix(TopicConfig.SEGMENT_BYTES_CONFIG), PURGE_SEGMENT_BYTES); streamsConfiguration.put(StreamsConfig.producerPrefix(ProducerConfig.BATCH_SIZE_CONFIG), PURGE_SEGMENT_BYTES / 2); // we cannot allow batch size larger than segment size streamsConfiguration.put(IntegrationTestUtils.INTERNAL_LEAVE_GROUP_ON_CLOSE, true); - final StreamsBuilder builder = new StreamsBuilder(); builder.stream(INPUT_TOPIC) .groupBy(MockMapper.selectKeyKeyValueMapper()) diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/QueryableStateIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/QueryableStateIntegrationTest.java index 856a85d787141..485baabb75a80 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/QueryableStateIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/QueryableStateIntegrationTest.java @@ -50,7 +50,9 @@ import org.apache.kafka.streams.state.QueryableStoreTypes; import org.apache.kafka.streams.state.ReadOnlyKeyValueStore; import org.apache.kafka.streams.state.ReadOnlyWindowStore; +import org.apache.kafka.streams.state.Stores; import org.apache.kafka.streams.state.StreamsMetadata; +import org.apache.kafka.streams.state.ValueAndTimestamp; import org.apache.kafka.streams.state.WindowStoreIterator; import org.apache.kafka.test.IntegrationTest; import org.apache.kafka.test.MockMapper; @@ -121,6 +123,7 @@ public class QueryableStateIntegrationTest { private KafkaStreams kafkaStreams; private Comparator> stringComparator; private Comparator> stringLongComparator; + private Comparator>> stringLongTimestampComparator; private static int testNo = 0; private void createTopics() throws Exception { @@ -193,6 +196,9 @@ public void before() throws Exception { stringComparator = Comparator.comparing((KeyValue o) -> o.key).thenComparing(o -> o.value); stringLongComparator = Comparator.comparing((KeyValue o) -> o.key).thenComparingLong(o -> o.value); + stringLongTimestampComparator = Comparator.comparing((KeyValue> o) -> o.key) + .thenComparingLong(o -> o.value.value()) + .thenComparing(o -> o.value.timestamp()); inputValues = getInputValues(); inputValuesKeys = new HashSet<>(); for (final String sentence : inputValues) { @@ -301,8 +307,8 @@ private void verifyAllKVKeys(final StreamRunnable[] streamRunnables, } final int index = metadata.hostInfo().port(); final KafkaStreams streamsWithKey = streamRunnables[index].getStream(); - final ReadOnlyKeyValueStore store = - streamsWithKey.store(storeName, QueryableStoreTypes.keyValueStore()); + final ReadOnlyKeyValueStore> store = + streamsWithKey.store(storeName, QueryableStoreTypes.keyValueWithTimestampStore()); return store != null && store.get(key) != null; } catch (final IllegalStateException e) { @@ -336,8 +342,8 @@ private void verifyAllWindowedKeys(final StreamRunnable[] streamRunnables, } final int index = metadata.hostInfo().port(); final KafkaStreams streamsWithKey = streamRunnables[index].getStream(); - final ReadOnlyWindowStore store = - streamsWithKey.store(storeName, QueryableStoreTypes.windowStore()); + final ReadOnlyWindowStore> store = + streamsWithKey.store(storeName, QueryableStoreTypes.windowWithTimestampStore()); return store != null && store.fetch(key, ofEpochMilli(from), ofEpochMilli(to)) != null; } catch (final IllegalStateException e) { // Kafka Streams instance may have closed but rebalance hasn't happened @@ -455,8 +461,8 @@ public void concurrentAccesses() throws Exception { waitUntilAtLeastNumRecordProcessed(outputTopicConcurrent, numberOfWordsPerIteration); waitUntilAtLeastNumRecordProcessed(outputTopicConcurrentWindowed, numberOfWordsPerIteration); - final ReadOnlyKeyValueStore keyValueStore = - kafkaStreams.store(storeName + "-" + streamConcurrent, QueryableStoreTypes.keyValueStore()); + final ReadOnlyKeyValueStore> + keyValueWithTimestampStore = kafkaStreams.store(storeName + "-" + streamConcurrent, QueryableStoreTypes.keyValueWithTimestampStore()); final ReadOnlyWindowStore windowStore = kafkaStreams.store(windowStoreName + "-" + streamConcurrent, QueryableStoreTypes.windowStore()); @@ -465,7 +471,7 @@ public void concurrentAccesses() throws Exception { final Map expectedCount = new HashMap<>(); while (producerRunnable.getCurrIteration() < numIterations) { verifyGreaterOrEqual(inputValuesKeys.toArray(new String[0]), expectedWindowState, - expectedCount, windowStore, keyValueStore, true); + expectedCount, windowStore, keyValueWithTimestampStore, true); } } finally { producerRunnable.shutdown(); @@ -476,12 +482,22 @@ public void concurrentAccesses() throws Exception { @Test public void shouldBeAbleToQueryStateWithZeroSizedCache() throws Exception { - verifyCanQueryState(0); + verifyCanQueryState(0, false, false); } @Test public void shouldBeAbleToQueryStateWithNonZeroSizedCache() throws Exception { - verifyCanQueryState(10 * 1024 * 1024); + verifyCanQueryState(10 * 1024 * 1024, false, false); + } + + @Test + public void shouldBeAbleToQueryTimestampStoreWithoutTimestamps() throws Exception { + verifyCanQueryState(0, false, true); + } + + @Test + public void shouldBeAbleToQueryWithKeyValueStoreSupplier() throws Exception { + verifyCanQueryState(0, true, true); } @Test @@ -521,13 +537,13 @@ public void shouldBeAbleToQueryFilterState() throws Exception { waitUntilAtLeastNumRecordProcessed(outputTopic, 2); - final ReadOnlyKeyValueStore - myFilterStore = kafkaStreams.store("queryFilter", QueryableStoreTypes.keyValueStore()); - final ReadOnlyKeyValueStore - myFilterNotStore = kafkaStreams.store("queryFilterNot", QueryableStoreTypes.keyValueStore()); + final ReadOnlyKeyValueStore> + myFilterStore = kafkaStreams.store("queryFilter", QueryableStoreTypes.keyValueWithTimestampStore()); + final ReadOnlyKeyValueStore> + myFilterNotStore = kafkaStreams.store("queryFilterNot", QueryableStoreTypes.keyValueWithTimestampStore()); for (final KeyValue expectedEntry : expectedBatch1) { - TestUtils.waitForCondition(() -> expectedEntry.value.equals(myFilterStore.get(expectedEntry.key)), + TestUtils.waitForCondition(() -> expectedEntry.value.equals(myFilterStore.get(expectedEntry.key).value()), "Cannot get expected result"); } for (final KeyValue batchEntry : batch1) { @@ -543,7 +559,7 @@ public void shouldBeAbleToQueryFilterState() throws Exception { } for (final KeyValue batchEntry : batch1) { if (!expectedBatch1.contains(batchEntry)) { - TestUtils.waitForCondition(() -> batchEntry.value.equals(myFilterNotStore.get(batchEntry.key)), + TestUtils.waitForCondition(() -> batchEntry.value.equals(myFilterNotStore.get(batchEntry.key).value()), "Cannot get expected result"); } } @@ -587,10 +603,10 @@ public void shouldBeAbleToQueryMapValuesState() throws Exception { waitUntilAtLeastNumRecordProcessed(outputTopic, 5); - final ReadOnlyKeyValueStore myMapStore = - kafkaStreams.store("queryMapValues", QueryableStoreTypes.keyValueStore()); + final ReadOnlyKeyValueStore> myMapStore = + kafkaStreams.store("queryMapValues", QueryableStoreTypes.keyValueWithTimestampStore()); for (final KeyValue batchEntry : batch1) { - assertEquals(Long.valueOf(batchEntry.value), myMapStore.get(batchEntry.key)); + assertEquals(Long.valueOf(batchEntry.value), myMapStore.get(batchEntry.key).value()); } } @@ -635,11 +651,11 @@ public void shouldBeAbleToQueryMapValuesAfterFilterState() throws Exception { waitUntilAtLeastNumRecordProcessed(outputTopic, 1); - final ReadOnlyKeyValueStore + final ReadOnlyKeyValueStore> myMapStore = kafkaStreams.store("queryMapValues", - QueryableStoreTypes.keyValueStore()); + QueryableStoreTypes.keyValueWithTimestampStore()); for (final KeyValue expectedEntry : expectedBatch1) { - assertEquals(myMapStore.get(expectedEntry.key), expectedEntry.value); + assertEquals(myMapStore.get(expectedEntry.key).value(), expectedEntry.value); } for (final KeyValue batchEntry : batch1) { final KeyValue batchEntryMapValue = @@ -650,7 +666,9 @@ public void shouldBeAbleToQueryMapValuesAfterFilterState() throws Exception { } } - private void verifyCanQueryState(final int cacheSizeBytes) throws Exception { + private void verifyCanQueryState(final int cacheSizeBytes, + final boolean usePlainKeyValueStoreSupplier, + final boolean queryAsNoneTimestampStore) throws Exception { streamsConfiguration.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, cacheSizeBytes); final StreamsBuilder builder = new StreamsBuilder(); final String[] keys = {"hello", "goodbye", "welcome", "go", "kafka"}; @@ -680,10 +698,17 @@ private void verifyCanQueryState(final int cacheSizeBytes) throws Exception { final KStream s1 = builder.stream(streamOne); - // Non Windowed final String storeName = "my-count"; + final Materialized> materialized; + if (usePlainKeyValueStoreSupplier) { + materialized = Materialized.as(Stores.persistentKeyValueStore(storeName)); + } else { + materialized = Materialized.as(storeName); + } + + // Non Windowed s1.groupByKey() - .count(Materialized.as(storeName)) + .count(materialized) .toStream() .to(outputTopic, Produced.with(Serdes.String(), Serdes.Long())); @@ -696,18 +721,38 @@ private void verifyCanQueryState(final int cacheSizeBytes) throws Exception { waitUntilAtLeastNumRecordProcessed(outputTopic, 1); - final ReadOnlyKeyValueStore - myCount = kafkaStreams.store(storeName, QueryableStoreTypes.keyValueStore()); + if (!queryAsNoneTimestampStore) { + final ReadOnlyKeyValueStore> + myCount = kafkaStreams.store(storeName, QueryableStoreTypes.keyValueWithTimestampStore()); + + verifyRangeAndAll(expectedCount, myCount); + + final ReadOnlyWindowStore> windowStore = + kafkaStreams.store(windowStoreName, QueryableStoreTypes.windowWithTimestampStore()); + + verifyCanGetByKey( + keys, + expectedCount, + expectedCount, + windowStore, + myCount); + } else { + final ReadOnlyKeyValueStore + myCount = kafkaStreams.store(storeName, QueryableStoreTypes.keyValueStore()); + + verifyRangeAndAllWithoutTimestamp(expectedCount, myCount); + + final ReadOnlyWindowStore windowStore = + kafkaStreams.store(windowStoreName, QueryableStoreTypes.windowStore()); - final ReadOnlyWindowStore windowStore = - kafkaStreams.store(windowStoreName, QueryableStoreTypes.windowStore()); - verifyCanGetByKey(keys, - expectedCount, - expectedCount, - windowStore, - myCount); + verifyCanGetByKeyWithOutTimestamps( + keys, + expectedCount, + expectedCount, + windowStore, + myCount); - verifyRangeAndAll(expectedCount, myCount); + } } @Test @@ -738,11 +783,11 @@ public void shouldNotMakeStoreAvailableUntilAllStoresAvailable() throws Exceptio maxWaitMs, "waiting for store " + storeName); - final ReadOnlyKeyValueStore store = - kafkaStreams.store(storeName, QueryableStoreTypes.keyValueStore()); + final ReadOnlyKeyValueStore> store = + kafkaStreams.store(storeName, QueryableStoreTypes.keyValueWithTimestampStore()); TestUtils.waitForCondition( - () -> new Long(8).equals(store.get("hello")), + () -> new Long(8).equals(store.get("hello").value()), maxWaitMs, "wait for count to be 8"); @@ -759,7 +804,7 @@ public void shouldNotMakeStoreAvailableUntilAllStoresAvailable() throws Exceptio try { assertEquals( Long.valueOf(8L), - kafkaStreams.store(storeName, QueryableStoreTypes.keyValueStore()).get("hello")); + kafkaStreams.store(storeName, QueryableStoreTypes.keyValueWithTimestampStore()).get("hello").value()); return true; } catch (final InvalidStateStoreException ise) { return false; @@ -780,7 +825,7 @@ private class WaitForStore implements TestCondition { @Override public boolean conditionMet() { try { - kafkaStreams.store(storeName, QueryableStoreTypes.keyValueStore()); + kafkaStreams.store(storeName, QueryableStoreTypes.keyValueWithTimestampStore()); return true; } catch (final InvalidStateStoreException ise) { return false; @@ -835,13 +880,10 @@ public void shouldAllowToQueryAfterThreadDied() throws Exception { maxWaitMs, "waiting for store " + storeName); - final ReadOnlyKeyValueStore store = - kafkaStreams.store(storeName, QueryableStoreTypes.keyValueStore()); + final ReadOnlyKeyValueStore> store = + kafkaStreams.store(storeName, QueryableStoreTypes.keyValueWithTimestampStore()); - TestUtils.waitForCondition( - () -> "12".equals(store.get("a")) && "34".equals(store.get("b")), - maxWaitMs, - "wait for agg to be and "); + TestUtils.waitForCondition(() -> "12".equals(store.get("a").value()) && "34".equals(store.get("b").value()), maxWaitMs, "wait for agg to be and "); IntegrationTestUtils.produceKeyValuesSynchronously( streamOne, @@ -862,18 +904,18 @@ public void shouldAllowToQueryAfterThreadDied() throws Exception { maxWaitMs, "waiting for store " + storeName); - final ReadOnlyKeyValueStore store2 = - kafkaStreams.store(storeName, QueryableStoreTypes.keyValueStore()); + final ReadOnlyKeyValueStore> store2 = + kafkaStreams.store(storeName, QueryableStoreTypes.keyValueWithTimestampStore()); try { TestUtils.waitForCondition( - () -> ("125".equals(store2.get("a")) - || "1225".equals(store2.get("a")) - || "12125".equals(store2.get("a"))) + () -> ("125".equals(store2.get("a").value()) + || "1225".equals(store2.get("a").value()) + || "12125".equals(store2.get("a").value())) && - ("34".equals(store2.get("b")) - || "344".equals(store2.get("b")) - || "3434".equals(store2.get("b"))), + ("34".equals(store2.get("b").value()) + || "344".equals(store2.get("b").value()) + || "3434".equals(store2.get("b").value())), maxWaitMs, "wait for agg to be |||| and ||||"); } catch (final Throwable t) { @@ -882,7 +924,40 @@ public void shouldAllowToQueryAfterThreadDied() throws Exception { } private void verifyRangeAndAll(final Set> expectedCount, - final ReadOnlyKeyValueStore myCount) { + final ReadOnlyKeyValueStore> myCount) { + final Set> countRangeResults = new TreeSet<>(stringLongComparator); + final Set> countAllResults = new TreeSet<>(stringLongComparator); + final Set> + expectedRangeResults = + new TreeSet<>(stringLongComparator); + + expectedRangeResults.addAll(Arrays.asList( + new KeyValue<>("hello", 1L), + new KeyValue<>("go", 1L), + new KeyValue<>("goodbye", 1L), + new KeyValue<>("kafka", 1L) + )); + + try (final KeyValueIterator> range = myCount.range("go", "kafka")) { + while (range.hasNext()) { + final KeyValue> keyValue = range.next(); + countRangeResults.add(KeyValue.pair(keyValue.key, keyValue.value.value())); + } + } + + try (final KeyValueIterator> all = myCount.all()) { + while (all.hasNext()) { + final KeyValue> keyValue = all.next(); + countAllResults.add(KeyValue.pair(keyValue.key, keyValue.value.value())); + } + } + + assertThat(countRangeResults, equalTo(expectedRangeResults)); + assertThat(countAllResults, equalTo(expectedCount)); + } + + private void verifyRangeAndAllWithoutTimestamp(final Set> expectedCount, + final ReadOnlyKeyValueStore myCount) { final Set> countRangeResults = new TreeSet<>(stringLongComparator); final Set> countAllResults = new TreeSet<>(stringLongComparator); final Set> expectedRangeResults = new TreeSet<>(stringLongComparator); @@ -896,13 +971,15 @@ private void verifyRangeAndAll(final Set> expectedCount, try (final KeyValueIterator range = myCount.range("go", "kafka")) { while (range.hasNext()) { - countRangeResults.add(range.next()); + final KeyValue keyValue = range.next(); + countRangeResults.add(KeyValue.pair(keyValue.key, keyValue.value)); } } try (final KeyValueIterator all = myCount.all()) { while (all.hasNext()) { - countAllResults.add(all.next()); + final KeyValue keyValue = all.next(); + countAllResults.add(KeyValue.pair(keyValue.key, keyValue.value)); } } @@ -913,8 +990,35 @@ private void verifyRangeAndAll(final Set> expectedCount, private void verifyCanGetByKey(final String[] keys, final Set> expectedWindowState, final Set> expectedCount, - final ReadOnlyWindowStore windowStore, - final ReadOnlyKeyValueStore myCount) throws Exception { + final ReadOnlyWindowStore> windowStore, + final ReadOnlyKeyValueStore> myCount) throws Exception { + final Set> windowState = new TreeSet<>(stringLongComparator); + final Set> countState = new TreeSet<>(stringLongComparator); + + final long timeout = System.currentTimeMillis() + 30000; + while ((windowState.size() < keys.length || + countState.size() < keys.length) && + System.currentTimeMillis() < timeout) { + Thread.sleep(10); + for (final String key : keys) { + for (final KeyValue> keyValue : fetch(windowStore, key)) { + windowState.add(KeyValue.pair(keyValue.key, keyValue.value.value())); + } + final Long value = myCount.get(key).value(); + if (value != null) { + countState.add(KeyValue.pair(key, value)); + } + } + } + assertThat(windowState, equalTo(expectedWindowState)); + assertThat(countState, equalTo(expectedCount)); + } + + private void verifyCanGetByKeyWithOutTimestamps(final String[] keys, + final Set> expectedWindowState, + final Set> expectedCount, + final ReadOnlyWindowStore windowStore, + final ReadOnlyKeyValueStore myCount) throws Exception { final Set> windowState = new TreeSet<>(stringLongComparator); final Set> countState = new TreeSet<>(stringLongComparator); @@ -952,7 +1056,7 @@ private void verifyGreaterOrEqual(final String[] keys, final Map expectedWindowedCount, final Map expectedCount, final ReadOnlyWindowStore windowStore, - final ReadOnlyKeyValueStore keyValueStore, + final ReadOnlyKeyValueStore> keyValueStore, final boolean failIfKeyNotFound) { final Map windowState = new HashMap<>(); final Map countState = new HashMap<>(); @@ -963,7 +1067,7 @@ private void verifyGreaterOrEqual(final String[] keys, fail("Key in windowed-store not found " + key); } windowState.putAll(map); - final Long value = keyValueStore.get(key); + final Long value = keyValueStore.get(key).value(); if (value != null) { countState.put(key, value); } else if (failIfKeyNotFound) { @@ -1006,12 +1110,12 @@ private void waitUntilAtLeastNumRecordProcessed(final String topic, 120 * 1000); } - private Set> fetch(final ReadOnlyWindowStore store, - final String key) { - final WindowStoreIterator fetch = + private Set> fetch(final ReadOnlyWindowStore store, + final String key) { + final WindowStoreIterator fetch = store.fetch(key, ofEpochMilli(0), ofEpochMilli(System.currentTimeMillis())); if (fetch.hasNext()) { - final KeyValue next = fetch.next(); + final KeyValue next = fetch.next(); return Collections.singleton(KeyValue.pair(key, next.value)); } return Collections.emptySet(); diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/StoreUpgradeIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/StoreUpgradeIntegrationTest.java new file mode 100644 index 0000000000000..6f0d6096c04eb --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/streams/integration/StoreUpgradeIntegrationTest.java @@ -0,0 +1,296 @@ +/* + * 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.integration; + +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.common.serialization.IntegerSerializer; +import org.apache.kafka.common.serialization.Serdes; +import org.apache.kafka.common.utils.Bytes; +import org.apache.kafka.streams.KafkaStreams; +import org.apache.kafka.streams.KeyValue; +import org.apache.kafka.streams.StreamsBuilder; +import org.apache.kafka.streams.StreamsConfig; +import org.apache.kafka.streams.integration.utils.EmbeddedKafkaCluster; +import org.apache.kafka.streams.integration.utils.IntegrationTestUtils; +import org.apache.kafka.streams.kstream.Materialized; +import org.apache.kafka.streams.processor.Processor; +import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.state.KeyValueIterator; +import org.apache.kafka.streams.state.KeyValueStore; +import org.apache.kafka.streams.state.QueryableStoreTypes; +import org.apache.kafka.streams.state.ReadOnlyKeyValueStore; +import org.apache.kafka.streams.state.Stores; +import org.apache.kafka.streams.state.ValueAndTimestamp; +import org.apache.kafka.test.IntegrationTest; +import org.apache.kafka.test.TestUtils; +import org.junit.After; +import org.junit.Before; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +import java.time.Duration; +import java.util.Collections; +import java.util.LinkedList; +import java.util.List; +import java.util.Properties; + +@Category({IntegrationTest.class}) +public class StoreUpgradeIntegrationTest { + private static String inputStream; + private static final String STORE_NAME = "store"; + + private KafkaStreams kafkaStreams; + private static int testCounter = 0; + + @ClassRule + public static final EmbeddedKafkaCluster CLUSTER = new EmbeddedKafkaCluster(1); + + @Before + public void createTopics() throws Exception { + inputStream = "input-stream-" + testCounter; + CLUSTER.createTopic(inputStream); + } + + private Properties props() { + final Properties streamsConfiguration = new Properties(); + streamsConfiguration.put(StreamsConfig.APPLICATION_ID_CONFIG, "addId-" + testCounter++); + streamsConfiguration.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers()); + streamsConfiguration.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 0); + streamsConfiguration.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory().getPath()); + streamsConfiguration.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.Integer().getClass()); + streamsConfiguration.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.Integer().getClass()); + streamsConfiguration.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 1000); + streamsConfiguration.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); + streamsConfiguration.put(IntegrationTestUtils.INTERNAL_LEAVE_GROUP_ON_CLOSE, true); + return streamsConfiguration; + } + + @After + public void shutdown() { + if (kafkaStreams != null) { + kafkaStreams.close(Duration.ofSeconds(30)); + kafkaStreams.cleanUp(); + } + } + + @Test + public void shouldMigrateRecordFromPlainValueToValueWithTimestampStore() throws Exception { + runUpgradeTest(false); + } + + @Test + public void shouldMigrateRecordFromPlainValueToValueWithTimestampStoreWithOldSupplier() throws Exception { + runUpgradeTest(true); + } + + private void runUpgradeTest(final boolean usePlainKeyValueByteStoreSupplier) throws Exception { + final StreamsBuilder streamsBuilderForOldStore = new StreamsBuilder(); + + streamsBuilderForOldStore.addStateStore(Stores.keyValueStoreBuilder( + Stores.persistentKeyValueStore(STORE_NAME), + Serdes.Integer(), + Serdes.Long() + )); + streamsBuilderForOldStore + .stream(inputStream) + .process( + () -> new Processor() { + private KeyValueStore store; + + @SuppressWarnings("unchecked") + @Override + public void init(final ProcessorContext context) { + store = (KeyValueStore) context.getStateStore(STORE_NAME); + } + + @Override + public void process(final Integer key, final Integer value) { + final long newCount; + + final Long oldCount = store.get(key); + if (oldCount != null) { + newCount = oldCount + 1; + } else { + newCount = 1L; + } + + store.put(key, newCount); + } + + @Override + public void close() {} + }, + STORE_NAME); + + final Properties props = props(); + kafkaStreams = new KafkaStreams(streamsBuilderForOldStore.build(), props); + kafkaStreams.start(); + + final LinkedList> expectedResult = new LinkedList<>(); + + expectedResult.add(KeyValue.pair(1, 1L)); + processKeyValueAndVerifyPlainCount(1, expectedResult); + + expectedResult.remove(0); + expectedResult.add(KeyValue.pair(1, 2L)); + processKeyValueAndVerifyPlainCount(1, expectedResult); + + expectedResult.add(KeyValue.pair(2, 1L)); + processKeyValueAndVerifyPlainCount(2, expectedResult); + + expectedResult.add(KeyValue.pair(3, 1L)); + processKeyValueAndVerifyPlainCount(3, expectedResult); + + expectedResult.add(KeyValue.pair(4, 1L)); + processKeyValueAndVerifyPlainCount(4, expectedResult); + + expectedResult.remove(3); + expectedResult.add(KeyValue.pair(4, 2L)); + processKeyValueAndVerifyPlainCount(4, expectedResult); + + expectedResult.remove(3); + expectedResult.add(KeyValue.pair(4, 3L)); + processKeyValueAndVerifyPlainCount(4, expectedResult); + + kafkaStreams.close(); + kafkaStreams = null; + + + + final StreamsBuilder newStoreBuilder = new StreamsBuilder(); + + final Materialized> materialized; + if (usePlainKeyValueByteStoreSupplier) { + materialized = Materialized.as(Stores.persistentKeyValueStore(STORE_NAME)); + } else { + materialized = Materialized.as(STORE_NAME); + } + newStoreBuilder + .stream(inputStream) + .groupByKey() + .count(materialized); + + kafkaStreams = new KafkaStreams(newStoreBuilder.build(), props); + kafkaStreams.start(); + + verifyCountWithSurrogateTimestamp(1, 2L); + verifyCountWithSurrogateTimestamp(2, 1L); + verifyCountWithSurrogateTimestamp(3, 1L); + verifyCountWithSurrogateTimestamp(4, 3L); + + expectedResult.clear(); + expectedResult.add(KeyValue.pair(1, ValueAndTimestamp.make(3L, usePlainKeyValueByteStoreSupplier ? -1 : 42L))); + expectedResult.add(KeyValue.pair(2, ValueAndTimestamp.make(1L, -1))); + expectedResult.add(KeyValue.pair(3, ValueAndTimestamp.make(1L, -1))); + expectedResult.add(KeyValue.pair(4, ValueAndTimestamp.make(3L, -1))); + processKeyValueAndVerifyCountWithTimestamp(1, 42L, expectedResult); + + expectedResult.remove(1); + expectedResult.add(1, KeyValue.pair(2, ValueAndTimestamp.make(2L, usePlainKeyValueByteStoreSupplier ? -1 : 45))); + processKeyValueAndVerifyCountWithTimestamp(2, 45L, expectedResult); + + // can process "out of order" record for different key + expectedResult.remove(3); + expectedResult.add(3, KeyValue.pair(4, ValueAndTimestamp.make(4L, usePlainKeyValueByteStoreSupplier ? -1 : 21))); + processKeyValueAndVerifyCountWithTimestamp(4, 21L, expectedResult); + + expectedResult.remove(3); + expectedResult.add(3, KeyValue.pair(4, ValueAndTimestamp.make(5L, usePlainKeyValueByteStoreSupplier ? -1 : 42))); + processKeyValueAndVerifyCountWithTimestamp(4, 42L, expectedResult); + + // out of order (same key) record should not reduce result timestamp + expectedResult.remove(3); + expectedResult.add(3, KeyValue.pair(4, ValueAndTimestamp.make(6L, usePlainKeyValueByteStoreSupplier ? -1 : 42))); + processKeyValueAndVerifyCountWithTimestamp(4, 10L, expectedResult); + + kafkaStreams.close(); + } + + private void processKeyValueAndVerifyPlainCount(final K key, + final List> expectedStoreContent) throws Exception { + + IntegrationTestUtils.produceKeyValuesSynchronously(inputStream, + Collections.singletonList(KeyValue.pair(key, 0)), + TestUtils.producerConfig(CLUSTER.bootstrapServers(), + IntegerSerializer.class, + IntegerSerializer.class), + CLUSTER.time); + + TestUtils.waitForCondition(() -> { + try { + final ReadOnlyKeyValueStore store = kafkaStreams.store(STORE_NAME, QueryableStoreTypes.keyValueStore()); + try (final KeyValueIterator all = store.all()) { + final List> storeContent = new LinkedList<>(); + while (all.hasNext()) { + storeContent.add(all.next()); + } + return storeContent.equals(expectedStoreContent); + } + } catch (final Exception swallow) { + swallow.printStackTrace(); + System.err.println(swallow.getMessage()); + return false; + } + }, "Could not get expected result in time."); + } + + private void verifyCountWithSurrogateTimestamp(final K key, + final long value) throws Exception { + TestUtils.waitForCondition(() -> { + try { + final ReadOnlyKeyValueStore> store = kafkaStreams.store(STORE_NAME, QueryableStoreTypes.keyValueWithTimestampStore()); + final ValueAndTimestamp count = store.get(key); + return count.value() == value && count.timestamp() == -1L; + } catch (final Exception swallow) { + swallow.printStackTrace(); + System.err.println(swallow.getMessage()); + return false; + } + }, "Could not get expected result in time."); + + } + + private void processKeyValueAndVerifyCountWithTimestamp(final K key, + final long timestamp, + final List> expectedStoreContent) throws Exception { + + IntegrationTestUtils.produceKeyValuesSynchronouslyWithTimestamp(inputStream, + Collections.singletonList(KeyValue.pair(key, 0)), + TestUtils.producerConfig(CLUSTER.bootstrapServers(), + IntegerSerializer.class, + IntegerSerializer.class), + timestamp); + + TestUtils.waitForCondition(() -> { + try { + final ReadOnlyKeyValueStore> store = kafkaStreams.store(STORE_NAME, QueryableStoreTypes.keyValueWithTimestampStore()); + try (final KeyValueIterator> all = store.all()) { + final List>> storeContent = new LinkedList<>(); + while (all.hasNext()) { + storeContent.add(all.next()); + } + return storeContent.equals(expectedStoreContent); + } + } catch (final Exception swallow) { + swallow.printStackTrace(); + System.err.println(swallow.getMessage()); + return false; + } + }, "Could not get expected result in time."); + } +} \ No newline at end of file diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/GlobalKTableJoinsTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/GlobalKTableJoinsTest.java index c43fcd80e93f5..0ed74ff092575 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/GlobalKTableJoinsTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/GlobalKTableJoinsTest.java @@ -55,18 +55,8 @@ public void setUp() { final Consumed consumed = Consumed.with(Serdes.String(), Serdes.String()); global = builder.globalTable(globalTopic, consumed); stream = builder.stream(streamTopic, consumed); - keyValueMapper = new KeyValueMapper() { - @Override - public String apply(final String key, final String value) { - return value; - } - }; - action = new ForeachAction() { - @Override - public void apply(final String key, final String value) { - results.put(key, value); - } - }; + keyValueMapper = (key, value) -> value; + action = results::put; } @Test diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KGroupedStreamImplTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KGroupedStreamImplTest.java index 9bdea13dfeb6e..9e6087693e0ae 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KGroupedStreamImplTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KGroupedStreamImplTest.java @@ -38,6 +38,7 @@ import org.apache.kafka.streams.processor.internals.testutil.LogCaptureAppender; import org.apache.kafka.streams.state.KeyValueStore; import org.apache.kafka.streams.state.SessionStore; +import org.apache.kafka.streams.state.ValueAndTimestamp; import org.apache.kafka.streams.test.ConsumerRecordFactory; import org.apache.kafka.test.MockAggregator; import org.apache.kafka.test.MockInitializer; @@ -385,11 +386,11 @@ public void shouldCountAndMaterializeResults() { try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { processData(driver); - final KeyValueStore count = driver.getKeyValueStore("count"); + final KeyValueStore> count = driver.getKeyValueWithTimestampStore("count"); - assertThat(count.get("1"), equalTo(3L)); - assertThat(count.get("2"), equalTo(1L)); - assertThat(count.get("3"), equalTo(2L)); + assertThat(count.get("1").value(), equalTo(3L)); + assertThat(count.get("2").value(), equalTo(1L)); + assertThat(count.get("3").value(), equalTo(2L)); } } @@ -421,11 +422,11 @@ public void shouldReduceAndMaterializeResults() { try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { processData(driver); - final KeyValueStore reduced = driver.getKeyValueStore("reduce"); + final KeyValueStore> reduced = driver.getKeyValueWithTimestampStore("reduce"); - assertThat(reduced.get("1"), equalTo("A+C+D")); - assertThat(reduced.get("2"), equalTo("B")); - assertThat(reduced.get("3"), equalTo("E+F")); + assertThat(reduced.get("1").value(), equalTo("A+C+D")); + assertThat(reduced.get("2").value(), equalTo("B")); + assertThat(reduced.get("3").value(), equalTo("E+F")); } } @@ -464,11 +465,11 @@ public void shouldAggregateAndMaterializeResults() { try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { processData(driver); - final KeyValueStore aggregate = driver.getKeyValueStore("aggregate"); + final KeyValueStore> aggregate = driver.getKeyValueWithTimestampStore("aggregate"); - assertThat(aggregate.get("1"), equalTo("0+A+C+D")); - assertThat(aggregate.get("2"), equalTo("0+B")); - assertThat(aggregate.get("3"), equalTo("0+E+F")); + assertThat(aggregate.get("1").value(), equalTo("0+A+C+D")); + assertThat(aggregate.get("2").value(), equalTo("0+B")); + assertThat(aggregate.get("3").value(), equalTo("0+E+F")); } } diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KGroupedTableImplTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KGroupedTableImplTest.java index 09f93e7dfefae..a72fbdac162b7 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KGroupedTableImplTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KGroupedTableImplTest.java @@ -31,6 +31,7 @@ import org.apache.kafka.streams.kstream.KeyValueMapper; import org.apache.kafka.streams.kstream.Materialized; import org.apache.kafka.streams.state.KeyValueStore; +import org.apache.kafka.streams.state.ValueAndTimestamp; import org.apache.kafka.streams.test.ConsumerRecordFactory; import org.apache.kafka.test.MockAggregator; import org.apache.kafka.test.MockInitializer; @@ -45,9 +46,9 @@ import java.util.Properties; import static org.hamcrest.CoreMatchers.equalTo; -import static org.hamcrest.MatcherAssert.assertThat; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertThat; public class KGroupedTableImplTest { @@ -220,9 +221,9 @@ public void shouldReduceAndMaterializeResults() { final Map results = getReducedResults(reduced); try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { assertReduced(results, topic, driver); - final KeyValueStore reduce = driver.getKeyValueStore("reduce"); - assertThat(reduce.get("A"), equalTo(5)); - assertThat(reduce.get("B"), equalTo(6)); + final KeyValueStore> reduce = driver.getKeyValueWithTimestampStore("reduce"); + assertThat(reduce.get("A").value(), equalTo(5)); + assertThat(reduce.get("B").value(), equalTo(6)); } } @@ -243,9 +244,9 @@ public void shouldCountAndMaterializeResults() { try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { processData(topic, driver); - final KeyValueStore counts = driver.getKeyValueStore("count"); - assertThat(counts.get("1"), equalTo(3L)); - assertThat(counts.get("2"), equalTo(2L)); + final KeyValueStore> counts = driver.getKeyValueWithTimestampStore("count"); + assertThat(counts.get("1").value(), equalTo(3L)); + assertThat(counts.get("2").value(), equalTo(2L)); } } @@ -269,9 +270,9 @@ public void shouldAggregateAndMaterializeResults() { try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { processData(topic, driver); - final KeyValueStore aggregate = driver.getKeyValueStore("aggregate"); - assertThat(aggregate.get("1"), equalTo("0+1+1+1")); - assertThat(aggregate.get("2"), equalTo("0+2+2")); + final KeyValueStore> aggregate = driver.getKeyValueWithTimestampStore("aggregate"); + assertThat(aggregate.get("1").value(), equalTo("0+1+1+1")); + assertThat(aggregate.get("2").value(), equalTo("0+2+2")); } } diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSessionWindowAggregateProcessorTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSessionWindowAggregateProcessorTest.java index 966310c352a61..db9ba3733b684 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSessionWindowAggregateProcessorTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSessionWindowAggregateProcessorTest.java @@ -30,12 +30,14 @@ import org.apache.kafka.streams.processor.Processor; import org.apache.kafka.streams.processor.To; import org.apache.kafka.streams.processor.internals.MockStreamsMetrics; +import org.apache.kafka.streams.processor.internals.ProcessorContextImpl; import org.apache.kafka.streams.processor.internals.ProcessorRecordContext; import org.apache.kafka.streams.processor.internals.testutil.LogCaptureAppender; import org.apache.kafka.streams.state.KeyValueIterator; import org.apache.kafka.streams.state.SessionStore; import org.apache.kafka.streams.state.StoreBuilder; import org.apache.kafka.streams.state.Stores; +import org.apache.kafka.streams.state.ValueAndTimestamp; import org.apache.kafka.streams.state.internals.ThreadCache; import org.apache.kafka.test.InternalMockProcessorContext; import org.apache.kafka.test.NoOpRecordCollector; @@ -49,6 +51,7 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.List; +import java.util.Map; import static java.time.Duration.ofMillis; import static org.apache.kafka.common.utils.Utils.mkEntry; @@ -80,7 +83,7 @@ public class KStreamSessionWindowAggregateProcessorTest { private final List results = new ArrayList<>(); private final Processor processor = sessionAggregator.get(); - private SessionStore sessionStore; + private SessionStore> sessionStore; private InternalMockProcessorContext context; private Metrics metrics; @@ -109,19 +112,65 @@ public void forward(final K key, final V value, final To to) { } private void initStore(final boolean enableCaching) { - final StoreBuilder> storeBuilder = - Stores.sessionStoreBuilder( - Stores.persistentSessionStore(STORE_NAME, ofMillis(GAP_MS * 3)), - Serdes.String(), - Serdes.Long()) - .withLoggingDisabled(); + final StoreBuilder> storeBuilder = new StoreBuilder>() { + final StoreBuilder>> inner = + Stores.sessionWithTimestampStoreBuilder( + Stores.persistentSessionWithTimestampStore(STORE_NAME, ofMillis(GAP_MS * 3)), + Serdes.String(), + Serdes.Long()) + .withLoggingDisabled(); + @Override + public StoreBuilder> withCachingEnabled() { + inner.withCachingEnabled(); + return this; + } + + @Override + public StoreBuilder> withCachingDisabled() { + inner.withCachingDisabled(); + return this; + } + + @Override + public StoreBuilder> withLoggingEnabled(final Map config) { + inner.withLoggingEnabled(config); + return this; + } + + @Override + public StoreBuilder> withLoggingDisabled() { + inner.withLoggingDisabled(); + return this; + } + + @Override + public Map logConfig() { + return inner.logConfig(); + } + + @Override + public boolean loggingEnabled() { + return inner.loggingEnabled(); + } + + @Override + public String name() { + return inner.name(); + } + + @Override + public SessionWindowedKStreamImpl.SessionStoreFacade build() { + return new SessionWindowedKStreamImpl.SessionStoreFacade<>(inner.build()); + } + }; if (enableCaching) { storeBuilder.withCachingEnabled(); } - sessionStore = storeBuilder.build(); - sessionStore.init(context, sessionStore); + final SessionWindowedKStreamImpl.SessionStoreFacade outerStore = storeBuilder.build(); + sessionStore = outerStore.inner; + outerStore.init(context, new ProcessorContextImpl.SessionStoreReadWriteDecorator<>(outerStore)); } @After @@ -136,10 +185,10 @@ public void shouldCreateSingleSessionWhenWithinGap() { context.setTime(500); processor.process("john", "second"); - final KeyValueIterator, Long> values = + final KeyValueIterator, ValueAndTimestamp> values = sessionStore.findSessions("john", 0, 2000); assertTrue(values.hasNext()); - assertEquals(Long.valueOf(2), values.next().value); + assertEquals(Long.valueOf(2), values.next().value.value()); } @Test @@ -159,11 +208,11 @@ public void shouldMergeSessions() { context.setTime(GAP_MS / 2); processor.process(sessionId, "third"); - final KeyValueIterator, Long> iterator = + final KeyValueIterator, ValueAndTimestamp> iterator = sessionStore.findSessions(sessionId, 0, GAP_MS + 1); - final KeyValue, Long> kv = iterator.next(); + final KeyValue, ValueAndTimestamp> kv = iterator.next(); - assertEquals(Long.valueOf(3), kv.value); + assertEquals(Long.valueOf(3), kv.value.value()); assertFalse(iterator.hasNext()); } @@ -172,9 +221,9 @@ public void shouldUpdateSessionIfTheSameTime() { context.setTime(0); processor.process("mel", "first"); processor.process("mel", "second"); - final KeyValueIterator, Long> iterator = + final KeyValueIterator, ValueAndTimestamp> iterator = sessionStore.findSessions("mel", 0, 0); - assertEquals(Long.valueOf(2L), iterator.next().value); + assertEquals(Long.valueOf(2L), iterator.next().value.value()); assertFalse(iterator.hasNext()); } @@ -210,17 +259,17 @@ public void shouldRemoveMergedSessionsFromStateStore() { processor.process("a", "1"); // first ensure it is in the store - final KeyValueIterator, Long> a1 = + final KeyValueIterator, ValueAndTimestamp> a1 = sessionStore.findSessions("a", 0, 0); - assertEquals(KeyValue.pair(new Windowed<>("a", new SessionWindow(0, 0)), 1L), a1.next()); + assertEquals(KeyValue.pair(new Windowed<>("a", new SessionWindow(0, 0)), ValueAndTimestamp.make(1L, 0L)), a1.next()); context.setTime(100); processor.process("a", "2"); // a1 from above should have been removed // should have merged session in store - final KeyValueIterator, Long> a2 = + final KeyValueIterator, ValueAndTimestamp> a2 = sessionStore.findSessions("a", 0, 100); - assertEquals(KeyValue.pair(new Windowed<>("a", new SessionWindow(0, 100)), 2L), a2.next()); + assertEquals(KeyValue.pair(new Windowed<>("a", new SessionWindow(0, 100)), ValueAndTimestamp.make(2L, 100L)), a2.next()); assertFalse(a2.hasNext()); } @@ -265,8 +314,8 @@ public void shouldGetAggregatedValuesFromValueGetter() { context.setTime(GAP_MS + 1); processor.process("a", "1"); processor.process("a", "2"); - final long t0 = getter.get(new Windowed<>("a", new SessionWindow(0, 0))); - final long t1 = getter.get(new Windowed<>("a", new SessionWindow(GAP_MS + 1, GAP_MS + 1))); + final long t0 = getter.get(new Windowed<>("a", new SessionWindow(0, 0))).value(); + final long t1 = getter.get(new Windowed<>("a", new SessionWindow(GAP_MS + 1, GAP_MS + 1))).value(); assertEquals(1L, t0); assertEquals(2L, t1); } diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableAggregateTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableAggregateTest.java index 0452c0672bf5a..e63da14887d2e 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableAggregateTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableAggregateTest.java @@ -22,17 +22,10 @@ import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.StreamsBuilder; import org.apache.kafka.streams.integration.utils.EmbeddedKafkaCluster; -import org.apache.kafka.streams.kstream.Aggregator; import org.apache.kafka.streams.kstream.Consumed; -import org.apache.kafka.streams.kstream.ForeachAction; import org.apache.kafka.streams.kstream.Grouped; -import org.apache.kafka.streams.kstream.Initializer; import org.apache.kafka.streams.kstream.KTable; -import org.apache.kafka.streams.kstream.KeyValueMapper; import org.apache.kafka.streams.kstream.Materialized; -import org.apache.kafka.streams.kstream.Reducer; -import org.apache.kafka.streams.kstream.ValueJoiner; -import org.apache.kafka.streams.kstream.ValueMapper; import org.apache.kafka.streams.state.KeyValueStore; import org.apache.kafka.test.MockAggregator; import org.apache.kafka.test.MockInitializer; @@ -45,6 +38,7 @@ import org.junit.Test; import java.io.File; +import java.util.Collections; import java.util.HashMap; import java.util.Map; @@ -78,12 +72,13 @@ public void testAggBasic() { final KTable table1 = builder.table(topic1, consumed); - final KTable table2 = table1.groupBy(MockMapper.noOpKeyValueMapper(), - stringSerialzied - ).aggregate(MockInitializer.STRING_INIT, + final KTable table2 = table1.groupBy(MockMapper.noOpKeyValueMapper(), stringSerialzied) + .aggregate( + MockInitializer.STRING_INIT, MockAggregator.TOSTRING_ADDER, MockAggregator.TOSTRING_REMOVER, - Materialized.>as("topic1-Canonized").withValueSerde(stringSerde)); + Materialized.>as("topic1-Canonized") + .withValueSerde(stringSerde)); table2.toStream().process(supplier); @@ -106,7 +101,8 @@ public void testAggBasic() { driver.process(topic1, "C", "8"); driver.flushState(); - assertEquals(asList( + assertEquals( + asList( "A:0+1", "B:0+2", "A:0+1-1+3", @@ -114,22 +110,23 @@ public void testAggBasic() { "C:0+5", "D:0+6", "B:0+2-2+4-4+7", - "C:0+5-5+8"), supplier.theCapturedProcessor().processed); + "C:0+5-5+8"), + supplier.theCapturedProcessor().processed); } - @Test public void testAggCoalesced() { final StreamsBuilder builder = new StreamsBuilder(); final String topic1 = "topic1"; final KTable table1 = builder.table(topic1, consumed); - final KTable table2 = table1.groupBy(MockMapper.noOpKeyValueMapper(), - stringSerialzied - ).aggregate(MockInitializer.STRING_INIT, - MockAggregator.TOSTRING_ADDER, - MockAggregator.TOSTRING_REMOVER, - Materialized.>as("topic1-Canonized").withValueSerde(stringSerde)); + final KTable table2 = table1.groupBy(MockMapper.noOpKeyValueMapper(), stringSerialzied) + .aggregate( + MockInitializer.STRING_INIT, + MockAggregator.TOSTRING_ADDER, + MockAggregator.TOSTRING_REMOVER, + Materialized.>as("topic1-Canonized") + .withValueSerde(stringSerde)); table2.toStream().process(supplier); @@ -139,8 +136,9 @@ public void testAggCoalesced() { driver.process(topic1, "A", "3"); driver.process(topic1, "A", "4"); driver.flushState(); - assertEquals(asList( - "A:0+4"), supplier.theCapturedProcessor().processed); + assertEquals( + Collections.singletonList("A:0+4"), + supplier.theCapturedProcessor().processed); } @@ -150,10 +148,9 @@ public void testAggRepartition() { final String topic1 = "topic1"; final KTable table1 = builder.table(topic1, consumed); - final KTable table2 = table1.groupBy( - new KeyValueMapper>() { - @Override - public KeyValue apply(final String key, final String value) { + final KTable table2 = table1 + .groupBy( + (key, value) -> { switch (key) { case "null": return KeyValue.pair(null, value); @@ -162,11 +159,10 @@ public KeyValue apply(final String key, final String value) { default: return KeyValue.pair(value, value); } - } - }, - stringSerialzied - ) - .aggregate(MockInitializer.STRING_INIT, + }, + stringSerialzied) + .aggregate( + MockInitializer.STRING_INIT, MockAggregator.TOSTRING_ADDER, MockAggregator.TOSTRING_REMOVER, Materialized.>as("topic1-Canonized").withValueSerde(stringSerde)); @@ -192,7 +188,8 @@ public KeyValue apply(final String key, final String value) { driver.process(topic1, "B", "7"); driver.flushState(); - assertEquals(asList( + assertEquals( + asList( "1:0+1", "1:0+1-1", "1:0+1-1+1", @@ -200,11 +197,13 @@ public KeyValue apply(final String key, final String value) { //noop "2:0+2-2", "4:0+4", //noop - "4:0+4-4", "7:0+7" - ), supplier.theCapturedProcessor().processed); + "4:0+4-4", "7:0+7"), + supplier.theCapturedProcessor().processed); } - private void testCountHelper(final StreamsBuilder builder, final String input, final MockProcessorSupplier supplier) { + private void testCountHelper(final StreamsBuilder builder, + final String input, + final MockProcessorSupplier supplier) { driver.setUp(builder, stateDir); driver.process(input, "A", "green"); @@ -220,13 +219,14 @@ private void testCountHelper(final StreamsBuilder builder, final String input, f driver.flushState(); - assertEquals(asList( - "green:1", - "green:2", - "green:1", "blue:1", - "yellow:1", - "green:2" - ), supplier.theCapturedProcessor().processed); + assertEquals( + asList( + "green:1", + "green:2", + "green:1", "blue:1", + "yellow:1", + "green:2"), + supplier.theCapturedProcessor().processed); } @Test @@ -235,10 +235,10 @@ public void testCount() { final String input = "count-test-input"; builder.table(input, consumed) - .groupBy(MockMapper.selectValueKeyValueMapper(), stringSerialzied) - .count(Materialized.>as("count")) - .toStream() - .process(supplier); + .groupBy(MockMapper.selectValueKeyValueMapper(), stringSerialzied) + .count(Materialized.as("count")) + .toStream() + .process(supplier); testCountHelper(builder, input, supplier); } @@ -249,7 +249,7 @@ public void testCountWithInternalStore() { final String input = "count-test-input"; builder.table(input, consumed) - .groupBy(MockMapper.selectValueKeyValueMapper(), stringSerialzied) + .groupBy(MockMapper.selectValueKeyValueMapper(), stringSerialzied) .count() .toStream() .process(supplier); @@ -264,8 +264,8 @@ public void testCountCoalesced() { final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); builder.table(input, consumed) - .groupBy(MockMapper.selectValueKeyValueMapper(), stringSerialzied) - .count(Materialized.>as("count")) + .groupBy(MockMapper.selectValueKeyValueMapper(), stringSerialzied) + .count(Materialized.as("count")) .toStream() .process(supplier); @@ -281,11 +281,13 @@ public void testCountCoalesced() { driver.flushState(); - assertEquals(asList( - "blue:1", - "yellow:1", - "green:2" - ), proc.processed); + assertEquals( + asList( + "blue:1", + "yellow:1", + "green:2" + ), + proc.processed); } @Test @@ -295,34 +297,14 @@ public void testRemoveOldBeforeAddNew() { final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); builder.table(input, consumed) - .groupBy(new KeyValueMapper>() { - - @Override - public KeyValue apply(final String key, final String value) { - return KeyValue.pair(String.valueOf(key.charAt(0)), String.valueOf(key.charAt(1))); - } - }, stringSerialzied) - .aggregate(new Initializer() { - - @Override - public String apply() { - return ""; - } - }, new Aggregator() { - - @Override - public String apply(final String aggKey, final String value, final String aggregate) { - return aggregate + value; - } - }, new Aggregator() { - - @Override - public String apply(final String key, final String value, final String aggregate) { - return aggregate.replaceAll(value, ""); - } - }, Materialized.>as("someStore").withValueSerde(Serdes.String())) - .toStream() - .process(supplier); + .groupBy((key, value) -> KeyValue.pair(String.valueOf(key.charAt(0)), String.valueOf(key.charAt(1))), stringSerialzied) + .aggregate( + () -> "", + (aggKey, value, aggregate) -> aggregate + value, + (key, value, aggregate) -> aggregate.replaceAll(value, ""), + Materialized.>as("someStore").withValueSerde(Serdes.String())) + .toStream() + .process(supplier); driver.setUp(builder, stateDir); @@ -337,12 +319,13 @@ public String apply(final String key, final String value, final String aggregate driver.process(input, "12", "C"); driver.flushState(); - assertEquals(asList( - "1:1", - "1:12", - "1:2", - "1:2" - ), proc.processed); + assertEquals( + asList( + "1:1", + "1:12", + "1:2", + "1:2"), + proc.processed); } @Test @@ -356,44 +339,17 @@ public void shouldForwardToCorrectProcessorNodeWhenMultiCacheEvictions() { final KTable one = builder.table(tableOne, consumed); final KTable two = builder.table(tableTwo, Consumed.with(Serdes.Long(), Serdes.String())); + final KTable reduce = two + .groupBy((key, value) -> new KeyValue<>(value, key), Grouped.with(Serdes.String(), Serdes.Long())) + .reduce( + (value1, value2) -> value1 + value2, + (value1, value2) -> value1 - value2, + Materialized.as("reducer-store")); - final KTable reduce = two.groupBy(new KeyValueMapper>() { - @Override - public KeyValue apply(final Long key, final String value) { - return new KeyValue<>(value, key); - } - }, Grouped.with(Serdes.String(), Serdes.Long())) - .reduce(new Reducer() { - @Override - public Long apply(final Long value1, final Long value2) { - return value1 + value2; - } - }, new Reducer() { - @Override - public Long apply(final Long value1, final Long value2) { - return value1 - value2; - } - }, Materialized.>as("reducer-store")); - - reduce.toStream().foreach(new ForeachAction() { - @Override - public void apply(final String key, final Long value) { - reduceResults.put(key, value); - } - }); - - one.leftJoin(reduce, new ValueJoiner() { - @Override - public String apply(final String value1, final Long value2) { - return value1 + ":" + value2; - } - }) - .mapValues(new ValueMapper() { - @Override - public String apply(final String value) { - return value; - } - }); + reduce.toStream().foreach(reduceResults::put); + + one.leftJoin(reduce, (value1, value2) -> value1 + ":" + value2) + .mapValues(value -> value); driver.setUp(builder, stateDir, 111); driver.process(reduceTopic, "1", new Change<>(1L, null)); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableFilterTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableFilterTest.java index 8474565d5e76a..0df013b37feee 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableFilterTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableFilterTest.java @@ -136,30 +136,30 @@ private void doTestValueGetter(final StreamsBuilder builder, assertNull(getter2.get("B")); assertNull(getter2.get("C")); - assertEquals(1, (int) getter3.get("A")); - assertEquals(1, (int) getter3.get("B")); - assertEquals(1, (int) getter3.get("C")); + assertEquals(1, (int) getter3.get("A").value()); + assertEquals(1, (int) getter3.get("B").value()); + assertEquals(1, (int) getter3.get("C").value()); driver.pipeInput(recordFactory.create(topic1, "A", 2)); driver.pipeInput(recordFactory.create(topic1, "B", 2)); - assertEquals(2, (int) getter2.get("A")); - assertEquals(2, (int) getter2.get("B")); + assertEquals(2, (int) getter2.get("A").value()); + assertEquals(2, (int) getter2.get("B").value()); assertNull(getter2.get("C")); assertNull(getter3.get("A")); assertNull(getter3.get("B")); - assertEquals(1, (int) getter3.get("C")); + assertEquals(1, (int) getter3.get("C").value()); driver.pipeInput(recordFactory.create(topic1, "A", 3)); assertNull(getter2.get("A")); - assertEquals(2, (int) getter2.get("B")); + assertEquals(2, (int) getter2.get("B").value()); assertNull(getter2.get("C")); - assertEquals(3, (int) getter3.get("A")); + assertEquals(3, (int) getter3.get("A").value()); assertNull(getter3.get("B")); - assertEquals(1, (int) getter3.get("C")); + assertEquals(1, (int) getter3.get("C").value()); driver.pipeInput(recordFactory.create(topic1, "A", null)); driver.pipeInput(recordFactory.create(topic1, "B", null)); @@ -170,7 +170,7 @@ private void doTestValueGetter(final StreamsBuilder builder, assertNull(getter3.get("A")); assertNull(getter3.get("B")); - assertEquals(1, (int) getter3.get("C")); + assertEquals(1, (int) getter3.get("C").value()); } } diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableInnerJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableInnerJoinTest.java index 63ed53f5cf423..498dcd4a4e59e 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableInnerJoinTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableInnerJoinTest.java @@ -29,6 +29,7 @@ import org.apache.kafka.streams.processor.MockProcessorContext; import org.apache.kafka.streams.processor.Processor; import org.apache.kafka.streams.processor.internals.testutil.LogCaptureAppender; +import org.apache.kafka.streams.state.ValueAndTimestamp; import org.apache.kafka.test.MockProcessor; import org.apache.kafka.test.MockProcessorSupplier; import org.apache.kafka.test.MockValueJoiner; @@ -347,11 +348,11 @@ private KeyValue kv(final Integer key, final String value) { @SafeVarargs private final void checkJoinedValues(final KTableValueGetter getter, final KeyValue... expected) { for (final KeyValue kv : expected) { - final String value = getter.get(kv.key); + final ValueAndTimestamp value = getter.get(kv.key); if (kv.value == null) { assertNull(value); } else { - assertEquals(kv.value, value); + assertEquals(kv.value, value.value()); } } } diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableLeftJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableLeftJoinTest.java index f5d74b225f019..88bf6170289f5 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableLeftJoinTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableLeftJoinTest.java @@ -29,6 +29,7 @@ import org.apache.kafka.streams.processor.MockProcessorContext; import org.apache.kafka.streams.processor.Processor; import org.apache.kafka.streams.processor.internals.testutil.LogCaptureAppender; +import org.apache.kafka.streams.state.ValueAndTimestamp; import org.apache.kafka.test.MockProcessor; import org.apache.kafka.test.MockProcessorSupplier; import org.apache.kafka.test.MockReducer; @@ -382,7 +383,11 @@ public void shouldNotThrowIllegalStateExceptionWhenMultiCacheEvictions() { .leftJoin(eight, MockValueJoiner.TOSTRING_JOINER) .mapValues(mapper); - driver.setUp(builder, stateDir, 250); + try { + driver.setUp(builder, stateDir, 250); + } catch (final Exception e) { + e.printStackTrace(); + } final String[] values = { "a", "AA", "BBB", "CCCC", "DD", "EEEEEEEE", "F", "GGGGGGGGGGGGGGG", "HHH", "IIIIIIIIII", @@ -429,11 +434,11 @@ private KeyValue kv(final Integer key, final String value) { @SafeVarargs private final void checkJoinedValues(final KTableValueGetter getter, final KeyValue... expected) { for (final KeyValue kv : expected) { - final String value = getter.get(kv.key); + final ValueAndTimestamp value = getter.get(kv.key); if (kv.value == null) { assertNull(value); } else { - assertEquals(kv.value, value); + assertEquals(kv.value, value.value()); } } } diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableOuterJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableOuterJoinTest.java index 1500ba37e7694..e8eefd11873bb 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableOuterJoinTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableOuterJoinTest.java @@ -26,6 +26,7 @@ import org.apache.kafka.streams.processor.MockProcessorContext; import org.apache.kafka.streams.processor.Processor; import org.apache.kafka.streams.processor.internals.testutil.LogCaptureAppender; +import org.apache.kafka.streams.state.ValueAndTimestamp; import org.apache.kafka.test.MockProcessor; import org.apache.kafka.test.MockProcessorSupplier; import org.apache.kafka.test.MockValueJoiner; @@ -386,11 +387,11 @@ private KeyValue kv(final Integer key, final String value) { @SafeVarargs private final void checkJoinedValues(final KTableValueGetter getter, final KeyValue... expected) { for (final KeyValue kv : expected) { - final String value = getter.get(kv.key); + final ValueAndTimestamp value = getter.get(kv.key); if (kv.value == null) { assertNull(value); } else { - assertEquals(kv.value, value); + assertEquals(kv.value, value.value()); } } } diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableMapValuesTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableMapValuesTest.java index b16391571c9cf..787864430cb41 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableMapValuesTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableMapValuesTest.java @@ -122,44 +122,44 @@ private void doTestValueGetter(final StreamsBuilder builder, driver.pipeInput(recordFactory.create(topic1, "B", "01")); driver.pipeInput(recordFactory.create(topic1, "C", "01")); - assertEquals(new Integer(1), getter2.get("A")); - assertEquals(new Integer(1), getter2.get("B")); - assertEquals(new Integer(1), getter2.get("C")); + assertEquals(new Integer(1), getter2.get("A").value()); + assertEquals(new Integer(1), getter2.get("B").value()); + assertEquals(new Integer(1), getter2.get("C").value()); - assertEquals(new Integer(-1), getter3.get("A")); - assertEquals(new Integer(-1), getter3.get("B")); - assertEquals(new Integer(-1), getter3.get("C")); + assertEquals(new Integer(-1), getter3.get("A").value()); + assertEquals(new Integer(-1), getter3.get("B").value()); + assertEquals(new Integer(-1), getter3.get("C").value()); driver.pipeInput(recordFactory.create(topic1, "A", "02")); driver.pipeInput(recordFactory.create(topic1, "B", "02")); - assertEquals(new Integer(2), getter2.get("A")); - assertEquals(new Integer(2), getter2.get("B")); - assertEquals(new Integer(1), getter2.get("C")); + assertEquals(new Integer(2), getter2.get("A").value()); + assertEquals(new Integer(2), getter2.get("B").value()); + assertEquals(new Integer(1), getter2.get("C").value()); - assertEquals(new Integer(-2), getter3.get("A")); - assertEquals(new Integer(-2), getter3.get("B")); - assertEquals(new Integer(-1), getter3.get("C")); + assertEquals(new Integer(-2), getter3.get("A").value()); + assertEquals(new Integer(-2), getter3.get("B").value()); + assertEquals(new Integer(-1), getter3.get("C").value()); driver.pipeInput(recordFactory.create(topic1, "A", "03")); - assertEquals(new Integer(3), getter2.get("A")); - assertEquals(new Integer(2), getter2.get("B")); - assertEquals(new Integer(1), getter2.get("C")); + assertEquals(new Integer(3), getter2.get("A").value()); + assertEquals(new Integer(2), getter2.get("B").value()); + assertEquals(new Integer(1), getter2.get("C").value()); - assertEquals(new Integer(-3), getter3.get("A")); - assertEquals(new Integer(-2), getter3.get("B")); - assertEquals(new Integer(-1), getter3.get("C")); + assertEquals(new Integer(-3), getter3.get("A").value()); + assertEquals(new Integer(-2), getter3.get("B").value()); + assertEquals(new Integer(-1), getter3.get("C").value()); driver.pipeInput(recordFactory.create(topic1, "A", (String) null)); assertNull(getter2.get("A")); - assertEquals(new Integer(2), getter2.get("B")); - assertEquals(new Integer(1), getter2.get("C")); + assertEquals(new Integer(2), getter2.get("B").value()); + assertEquals(new Integer(1), getter2.get("C").value()); assertNull(getter3.get("A")); - assertEquals(new Integer(-2), getter3.get("B")); - assertEquals(new Integer(-1), getter3.get("C")); + assertEquals(new Integer(-2), getter3.get("B").value()); + assertEquals(new Integer(-1), getter3.get("C").value()); } } diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableReduceTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableReduceTest.java index 05b74dca9f45b..7f1e50d572b40 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableReduceTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableReduceTest.java @@ -18,8 +18,9 @@ import org.apache.kafka.streams.processor.Processor; import org.apache.kafka.streams.processor.internals.AbstractProcessorContext; +import org.apache.kafka.streams.processor.internals.ProcessorContextImpl; import org.apache.kafka.streams.processor.internals.ProcessorNode; -import org.apache.kafka.streams.state.internals.InMemoryKeyValueStore; +import org.apache.kafka.streams.state.internals.InMemoryKeyValueWithTimestampStore; import org.apache.kafka.test.InternalMockProcessorContext; import org.junit.Test; @@ -45,19 +46,22 @@ public void shouldAddAndSubtract() { ).get(); - final InMemoryKeyValueStore> myStore = - new InMemoryKeyValueStore<>("myStore", null, null); + final InMemoryKeyValueWithTimestampStore> myStore = + new InMemoryKeyValueWithTimestampStore<>("myStore", null, null); - context.register(myStore, null); + context.register( + new ProcessorContextImpl.KeyValueStoreReadWriteDecorator<>( + new KeyValueWithTimestampStoreMaterializer.KeyValueStoreFacade<>(myStore)), + null); reduceProcessor.init(context); context.setCurrentNode(new ProcessorNode<>("reduce", reduceProcessor, singleton("myStore"))); reduceProcessor.process("A", new Change<>(singleton("a"), null)); - assertEquals(singleton("a"), myStore.get("A")); + assertEquals(singleton("a"), myStore.get("A").value()); reduceProcessor.process("A", new Change<>(singleton("b"), singleton("a"))); - assertEquals(singleton("b"), myStore.get("A")); + assertEquals(singleton("b"), myStore.get("A").value()); reduceProcessor.process("A", new Change<>(null, singleton("b"))); - assertEquals(emptySet(), myStore.get("A")); + assertEquals(emptySet(), myStore.get("A").value()); } private Set differenceNotNullArgs(final Set left, final Set right) { diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableSourceTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableSourceTest.java index 0d42e44790aec..ab0c5a80faec4 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableSourceTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableSourceTest.java @@ -115,31 +115,30 @@ public void testValueGetter() { driver.pipeInput(recordFactory.create(topic1, "B", "01")); driver.pipeInput(recordFactory.create(topic1, "C", "01")); - assertEquals("01", getter1.get("A")); - assertEquals("01", getter1.get("B")); - assertEquals("01", getter1.get("C")); + assertEquals("01", getter1.get("A").value()); + assertEquals("01", getter1.get("B").value()); + assertEquals("01", getter1.get("C").value()); driver.pipeInput(recordFactory.create(topic1, "A", "02")); driver.pipeInput(recordFactory.create(topic1, "B", "02")); - assertEquals("02", getter1.get("A")); - assertEquals("02", getter1.get("B")); - assertEquals("01", getter1.get("C")); + assertEquals("02", getter1.get("A").value()); + assertEquals("02", getter1.get("B").value()); + assertEquals("01", getter1.get("C").value()); driver.pipeInput(recordFactory.create(topic1, "A", "03")); - assertEquals("03", getter1.get("A")); - assertEquals("02", getter1.get("B")); - assertEquals("01", getter1.get("C")); + assertEquals("03", getter1.get("A").value()); + assertEquals("02", getter1.get("B").value()); + assertEquals("01", getter1.get("C").value()); driver.pipeInput(recordFactory.create(topic1, "A", (String) null)); driver.pipeInput(recordFactory.create(topic1, "B", (String) null)); assertNull(getter1.get("A")); assertNull(getter1.get("B")); - assertEquals("01", getter1.get("C")); + assertEquals("01", getter1.get("C").value()); } - } @Test diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableTransformValuesTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableTransformValuesTest.java index 7da5077221045..8a44bbb798dec 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableTransformValuesTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableTransformValuesTest.java @@ -34,9 +34,11 @@ import org.apache.kafka.streams.processor.ProcessorContext; import org.apache.kafka.streams.processor.internals.ForwardingDisabledProcessorContext; import org.apache.kafka.streams.processor.internals.InternalProcessorContext; +import org.apache.kafka.streams.processor.internals.ProcessorContextImpl; import org.apache.kafka.streams.state.KeyValueStore; import org.apache.kafka.streams.state.StoreBuilder; import org.apache.kafka.streams.state.Stores; +import org.apache.kafka.streams.state.ValueAndTimestamp; import org.apache.kafka.streams.test.ConsumerRecordFactory; import org.apache.kafka.test.MockProcessorSupplier; import org.apache.kafka.test.MockReducer; @@ -59,6 +61,7 @@ import static org.easymock.EasyMock.expectLastCall; import static org.easymock.EasyMock.replay; import static org.easymock.EasyMock.verify; +import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.CoreMatchers.hasItems; import static org.hamcrest.CoreMatchers.is; import static org.hamcrest.CoreMatchers.isA; @@ -90,7 +93,7 @@ public class KTableTransformValuesTest { @Mock(MockType.NICE) private KTableValueGetter parentGetter; @Mock(MockType.NICE) - private KeyValueStore stateStore; + private KeyValueStore> stateStore; @Mock(MockType.NICE) private ValueTransformerWithKeySupplier mockSupplier; @Mock(MockType.NICE) @@ -203,15 +206,15 @@ public void shouldTransformOnGetIfNotMaterialized() { expect(parent.valueGetterSupplier()).andReturn(parentGetterSupplier); expect(parentGetterSupplier.get()).andReturn(parentGetter); - expect(parentGetter.get("Key")).andReturn("Value"); + expect(parentGetter.get("Key")).andReturn(ValueAndTimestamp.make("Value", 42L)); replay(parent, parentGetterSupplier, parentGetter); final KTableValueGetter getter = transformValues.view().get(); getter.init(context); - final String result = getter.get("Key"); + final ValueAndTimestamp result = getter.get("Key"); - assertThat(result, is("Key->Value!")); + assertThat(result.value(), is("Key->Value!")); } @Test @@ -219,16 +222,18 @@ public void shouldGetFromStateStoreIfMaterialized() { final KTableTransformValues transformValues = new KTableTransformValues<>(parent, new ExclamationValueTransformerSupplier(), QUERYABLE_NAME); - expect(context.getStateStore(QUERYABLE_NAME)).andReturn(stateStore); - expect(stateStore.get("Key")).andReturn("something"); + expect(context.getStateStore(QUERYABLE_NAME)).andReturn( + new ProcessorContextImpl.KeyValueStoreReadWriteDecorator<>( + new KeyValueWithTimestampStoreMaterializer.KeyValueStoreFacade<>(stateStore))); + expect(stateStore.get("Key")).andReturn(ValueAndTimestamp.make("something", 42L)); replay(context, stateStore); final KTableValueGetter getter = transformValues.view().get(); getter.init(context); - final String result = getter.get("Key"); + final ValueAndTimestamp result = getter.get("Key"); - assertThat(result, is("something")); + assertThat(result, equalTo(ValueAndTimestamp.make("something", 42L))); } @Test @@ -355,10 +360,10 @@ public void shouldTransformValuesWithKeyAndMaterialize() { assertThat(output(), hasItems("A:A->a!", "B:B->b!", "C:C->null!")); - final KeyValueStore keyValueStore = driver.getKeyValueStore(QUERYABLE_NAME); - assertThat(keyValueStore.get("A"), is("A->a!")); - assertThat(keyValueStore.get("B"), is("B->b!")); - assertThat(keyValueStore.get("C"), is("C->null!")); + final KeyValueStore> keyValueStore = driver.getKeyValueWithTimestampStore(QUERYABLE_NAME); + assertThat(keyValueStore.get("A").value(), is("A->a!")); + assertThat(keyValueStore.get("B").value(), is("B->b!")); + assertThat(keyValueStore.get("C").value(), is("C->null!")); } @Test @@ -384,8 +389,8 @@ public void shouldCalculateCorrectOldValuesIfMaterializedEvenIfStateful() { assertThat(output(), hasItems("A:1", "A:0", "A:2", "A:0", "A:3")); - final KeyValueStore keyValueStore = driver.getKeyValueStore(QUERYABLE_NAME); - assertThat(keyValueStore.get("A"), is(3)); + final KeyValueStore> keyValueStore = driver.getKeyValueWithTimestampStore(QUERYABLE_NAME); + assertThat(keyValueStore.get("A").value(), is(3)); } @Test diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/KeyValueStoreMaterializerTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KeyValueWithTimestampStoreMaterializerTest.java similarity index 56% rename from streams/src/test/java/org/apache/kafka/streams/processor/internals/KeyValueStoreMaterializerTest.java rename to streams/src/test/java/org/apache/kafka/streams/kstream/internals/KeyValueWithTimestampStoreMaterializerTest.java index 494ae02e66c51..98ca8342c29d0 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/KeyValueStoreMaterializerTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KeyValueWithTimestampStoreMaterializerTest.java @@ -14,23 +14,19 @@ * See the License for the specific language governing permissions and * limitations under the License. */ +package org.apache.kafka.streams.kstream.internals; -package org.apache.kafka.streams.processor.internals; - -import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.utils.Bytes; import org.apache.kafka.streams.kstream.Materialized; -import org.apache.kafka.streams.kstream.internals.InternalNameProvider; -import org.apache.kafka.streams.kstream.internals.KeyValueStoreMaterializer; -import org.apache.kafka.streams.kstream.internals.MaterializedInternal; import org.apache.kafka.streams.processor.StateStore; import org.apache.kafka.streams.state.KeyValueBytesStoreSupplier; import org.apache.kafka.streams.state.KeyValueStore; import org.apache.kafka.streams.state.StoreBuilder; +import org.apache.kafka.streams.state.ValueAndTimestamp; import org.apache.kafka.streams.state.internals.CachedStateStore; import org.apache.kafka.streams.state.internals.ChangeLoggingKeyValueBytesStore; -import org.apache.kafka.streams.state.internals.InMemoryKeyValueStore; -import org.apache.kafka.streams.state.internals.MeteredKeyValueStore; +import org.apache.kafka.streams.state.internals.KeyValueToKeyValueWithTimestampByteProxyStore; +import org.apache.kafka.streams.state.internals.MeteredKeyValueWithTimestampStore; import org.apache.kafka.streams.state.internals.WrappedStateStore; import org.easymock.EasyMock; import org.easymock.EasyMockRunner; @@ -45,7 +41,7 @@ import static org.hamcrest.core.IsNot.not; @RunWith(EasyMockRunner.class) -public class KeyValueStoreMaterializerTest { +public class KeyValueWithTimestampStoreMaterializerTest { private final String storePrefix = "prefix"; @Mock(type = MockType.NICE) @@ -56,12 +52,13 @@ public void shouldCreateBuilderThatBuildsMeteredStoreWithCachingAndLoggingEnable final MaterializedInternal> materialized = new MaterializedInternal<>(Materialized.as("store"), nameProvider, storePrefix); - final KeyValueStoreMaterializer materializer = new KeyValueStoreMaterializer<>(materialized); + final KeyValueWithTimestampStoreMaterializer materializer = new KeyValueWithTimestampStoreMaterializer<>(materialized); final StoreBuilder> builder = materializer.materialize(); - final KeyValueStore store = builder.build(); + final KeyValueStore> store = + ((KeyValueWithTimestampStoreMaterializer.KeyValueStoreFacade) builder.build()).inner; final WrappedStateStore caching = (WrappedStateStore) ((WrappedStateStore) store).wrappedStore(); final StateStore logging = caching.wrappedStore(); - assertThat(store, instanceOf(MeteredKeyValueStore.class)); + assertThat(store, instanceOf(MeteredKeyValueWithTimestampStore.class)); assertThat(caching, instanceOf(CachedStateStore.class)); assertThat(logging, instanceOf(ChangeLoggingKeyValueBytesStore.class)); } @@ -71,9 +68,10 @@ public void shouldCreateBuilderThatBuildsStoreWithCachingDisabled() { final MaterializedInternal> materialized = new MaterializedInternal<>( Materialized.>as("store").withCachingDisabled(), nameProvider, storePrefix ); - final KeyValueStoreMaterializer materializer = new KeyValueStoreMaterializer<>(materialized); + final KeyValueWithTimestampStoreMaterializer materializer = new KeyValueWithTimestampStoreMaterializer<>(materialized); final StoreBuilder> builder = materializer.materialize(); - final KeyValueStore store = builder.build(); + final KeyValueStore> store = + ((KeyValueWithTimestampStoreMaterializer.KeyValueStoreFacade) builder.build()).inner; final WrappedStateStore logging = (WrappedStateStore) ((WrappedStateStore) store).wrappedStore(); assertThat(logging, instanceOf(ChangeLoggingKeyValueBytesStore.class)); } @@ -83,9 +81,10 @@ public void shouldCreateBuilderThatBuildsStoreWithLoggingDisabled() { final MaterializedInternal> materialized = new MaterializedInternal<>( Materialized.>as("store").withLoggingDisabled(), nameProvider, storePrefix ); - final KeyValueStoreMaterializer materializer = new KeyValueStoreMaterializer<>(materialized); + final KeyValueWithTimestampStoreMaterializer materializer = new KeyValueWithTimestampStoreMaterializer<>(materialized); final StoreBuilder> builder = materializer.materialize(); - final KeyValueStore store = builder.build(); + final KeyValueStore> store = + ((KeyValueWithTimestampStoreMaterializer.KeyValueStoreFacade) builder.build()).inner; final WrappedStateStore caching = (WrappedStateStore) ((WrappedStateStore) store).wrappedStore(); assertThat(caching, instanceOf(CachedStateStore.class)); assertThat(caching.wrappedStore(), not(instanceOf(ChangeLoggingKeyValueBytesStore.class))); @@ -96,30 +95,52 @@ public void shouldCreateBuilderThatBuildsStoreWithCachingAndLoggingDisabled() { final MaterializedInternal> materialized = new MaterializedInternal<>( Materialized.>as("store").withCachingDisabled().withLoggingDisabled(), nameProvider, storePrefix ); - final KeyValueStoreMaterializer materializer = new KeyValueStoreMaterializer<>(materialized); + final KeyValueWithTimestampStoreMaterializer materializer = new KeyValueWithTimestampStoreMaterializer<>(materialized); final StoreBuilder> builder = materializer.materialize(); - final KeyValueStore store = builder.build(); + final KeyValueStore> store = + ((KeyValueWithTimestampStoreMaterializer.KeyValueStoreFacade) builder.build()).inner; final StateStore wrapped = ((WrappedStateStore) store).wrappedStore(); assertThat(wrapped, not(instanceOf(CachedStateStore.class))); assertThat(wrapped, not(instanceOf(ChangeLoggingKeyValueBytesStore.class))); } @Test - public void shouldCreateKeyValueStoreWithTheProvidedInnerStore() { + public void shouldCreateKeyValueStoreWithProvidedInMemoryInnerStore() { final KeyValueBytesStoreSupplier supplier = EasyMock.createNiceMock(KeyValueBytesStoreSupplier.class); - final InMemoryKeyValueStore store = new InMemoryKeyValueStore<>("name", Serdes.Bytes(), Serdes.ByteArray()); + final KeyValueStore store = EasyMock.createNiceMock(KeyValueStore.class); EasyMock.expect(supplier.name()).andReturn("name").anyTimes(); - EasyMock.expect(supplier.get()).andReturn(store); - EasyMock.replay(supplier); + EasyMock.expect(supplier.get()).andReturn(store).anyTimes(); + EasyMock.expect(store.persistent()).andReturn(false); + EasyMock.replay(supplier, store); - final MaterializedInternal> materialized = + final MaterializedInternal> materialized = new MaterializedInternal<>(Materialized.as(supplier), nameProvider, storePrefix); - final KeyValueStoreMaterializer materializer = new KeyValueStoreMaterializer<>(materialized); - final StoreBuilder> builder = materializer.materialize(); - final KeyValueStore built = builder.build(); + final KeyValueWithTimestampStoreMaterializer materializer = new KeyValueWithTimestampStoreMaterializer<>(materialized); + final StoreBuilder> builder = materializer.materialize(); + final KeyValueStore> built = + ((KeyValueWithTimestampStoreMaterializer.KeyValueStoreFacade) builder.build()).inner; final StateStore inner = ((WrappedStateStore) built).inner(); assertThat(inner, CoreMatchers.equalTo(store)); } -} \ No newline at end of file + @Test + public void shouldCreateKeyValueStoreProxyWithProvidedPersistentInnerStore() { + final KeyValueBytesStoreSupplier supplier = EasyMock.createNiceMock(KeyValueBytesStoreSupplier.class); + final KeyValueStore store = EasyMock.createNiceMock(KeyValueStore.class); + EasyMock.expect(supplier.name()).andReturn("name").anyTimes(); + EasyMock.expect(supplier.get()).andReturn(store).anyTimes(); + EasyMock.expect(store.persistent()).andReturn(true); + EasyMock.replay(supplier, store); + + final MaterializedInternal> materialized = + new MaterializedInternal<>(Materialized.as(supplier), nameProvider, storePrefix); + final KeyValueWithTimestampStoreMaterializer materializer = new KeyValueWithTimestampStoreMaterializer<>(materialized); + final StoreBuilder> builder = materializer.materialize(); + final KeyValueStore> built = + ((KeyValueWithTimestampStoreMaterializer.KeyValueStoreFacade) builder.build()).inner; + final StateStore inner = ((WrappedStateStore) built).inner(); + + assertThat(inner, instanceOf(KeyValueToKeyValueWithTimestampByteProxyStore.class)); + } +} diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/SuppressScenarioTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/SuppressScenarioTest.java index b0b87be51b5bf..f81bf03b5ccd6 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/SuppressScenarioTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/SuppressScenarioTest.java @@ -389,14 +389,14 @@ public void shouldSupportFinalResultsForTimeWindows() { new KeyValueTimestamp<>("[k1@0/2]", 2L, 1L), new KeyValueTimestamp<>("[k1@2/4]", 1L, 2L), new KeyValueTimestamp<>("[k1@0/2]", 3L, 1L), - new KeyValueTimestamp<>("[k1@0/2]", 4L, 0L), + new KeyValueTimestamp<>("[k1@0/2]", 4L, 1L), new KeyValueTimestamp<>("[k1@4/6]", 1L, 5L) ) ); verify( drainProducerRecords(driver, "output-suppressed", STRING_DESERIALIZER, LONG_DESERIALIZER), asList( - new KeyValueTimestamp<>("[k1@0/2]", 4L, 0L), + new KeyValueTimestamp<>("[k1@0/2]", 4L, 1L), new KeyValueTimestamp<>("[k1@2/4]", 1L, 2L) ) ); @@ -441,9 +441,9 @@ public void shouldSupportFinalResultsForTimeWindowsWithLargeJump() { new KeyValueTimestamp<>("[k1@0/2]", 1L, 0L), new KeyValueTimestamp<>("[k1@0/2]", 2L, 1L), new KeyValueTimestamp<>("[k1@2/4]", 1L, 2L), - new KeyValueTimestamp<>("[k1@0/2]", 3L, 0L), + new KeyValueTimestamp<>("[k1@0/2]", 3L, 1L), new KeyValueTimestamp<>("[k1@2/4]", 2L, 3L), - new KeyValueTimestamp<>("[k1@0/2]", 4L, 0L), + new KeyValueTimestamp<>("[k1@0/2]", 4L, 1L), new KeyValueTimestamp<>("[k1@4/6]", 1L, 4L), new KeyValueTimestamp<>("[k1@30/32]", 1L, 30L) ) @@ -451,7 +451,7 @@ public void shouldSupportFinalResultsForTimeWindowsWithLargeJump() { verify( drainProducerRecords(driver, "output-suppressed", STRING_DESERIALIZER, LONG_DESERIALIZER), asList( - new KeyValueTimestamp<>("[k1@0/2]", 4L, 0L), + new KeyValueTimestamp<>("[k1@0/2]", 4L, 1L), new KeyValueTimestamp<>("[k1@2/4]", 2L, 3L), new KeyValueTimestamp<>("[k1@4/6]", 1L, 4L) ) diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/GlobalStreamThreadTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/GlobalStreamThreadTest.java index c0e0de314964f..cf80f3b628d0a 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/GlobalStreamThreadTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/GlobalStreamThreadTest.java @@ -31,7 +31,7 @@ import org.apache.kafka.streams.kstream.Materialized; import org.apache.kafka.streams.kstream.internals.InternalNameProvider; import org.apache.kafka.streams.kstream.internals.KTableSource; -import org.apache.kafka.streams.kstream.internals.KeyValueStoreMaterializer; +import org.apache.kafka.streams.kstream.internals.KeyValueWithTimestampStoreMaterializer; import org.apache.kafka.streams.kstream.internals.MaterializedInternal; import org.apache.kafka.streams.processor.StateStore; import org.apache.kafka.streams.state.KeyValueStore; @@ -40,6 +40,7 @@ import org.junit.Before; import org.junit.Test; +import java.nio.ByteBuffer; import java.util.Collections; import java.util.HashMap; import java.util.List; @@ -87,7 +88,7 @@ public String newStoreName(final String prefix) { ); builder.addGlobalStore( - new KeyValueStoreMaterializer<>(materialized).materialize().withLoggingDisabled(), + new KeyValueWithTimestampStoreMaterializer<>(materialized).materialize().withLoggingDisabled(), "sourceName", null, null, @@ -243,7 +244,8 @@ public Set partitions() { } }); // feed first record for recovery - mockConsumer.addRecord(new ConsumerRecord<>(GLOBAL_STORE_TOPIC_NAME, 0, 0L, "K1".getBytes(), "V1".getBytes())); + final byte[] valueAndTimestamp = ByteBuffer.allocate(10).putLong(0L).put("V1".getBytes()).array(); + mockConsumer.addRecord(new ConsumerRecord<>(GLOBAL_STORE_TOPIC_NAME, 0, 0L, "K1".getBytes(), valueAndTimestamp)); TestUtils.waitForCondition( () -> globalStreamThread.state() == DEAD, diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StandbyTaskTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StandbyTaskTest.java index 52599c67a6d1d..001c46548b86e 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StandbyTaskTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StandbyTaskTest.java @@ -43,16 +43,18 @@ import org.apache.kafka.streams.kstream.internals.ConsumedInternal; import org.apache.kafka.streams.kstream.internals.InternalStreamsBuilder; import org.apache.kafka.streams.kstream.internals.InternalStreamsBuilderTest; +import org.apache.kafka.streams.kstream.internals.KStreamImpl; import org.apache.kafka.streams.kstream.internals.TimeWindow; import org.apache.kafka.streams.processor.TaskId; import org.apache.kafka.streams.state.KeyValueIterator; +import org.apache.kafka.streams.state.ValueAndTimestamp; import org.apache.kafka.streams.state.WindowStore; import org.apache.kafka.streams.state.internals.OffsetCheckpoint; import org.apache.kafka.streams.state.internals.WindowKeySchema; import org.apache.kafka.test.MockKeyValueStore; +import org.apache.kafka.test.MockKeyValueStoreBuilder; import org.apache.kafka.test.MockRestoreConsumer; import org.apache.kafka.test.MockStateRestoreListener; -import org.apache.kafka.test.MockKeyValueStoreBuilder; import org.apache.kafka.test.MockTimestampExtractor; import org.apache.kafka.test.TestUtils; import org.junit.After; @@ -377,18 +379,19 @@ public void shouldWriteCheckpointFile() throws IOException { } @SuppressWarnings("unchecked") - private List, Long>> getWindowedStoreContents(final String storeName, final StandbyTask task) { + private List, Long>> getWindowedStoreContents(final String storeName, + final StandbyTask task) { final StandbyContextImpl context = (StandbyContextImpl) task.context(); final List, Long>> result = new ArrayList<>(); - try (final KeyValueIterator, Long> iterator = - ((WindowStore) context.getStateMgr().getStore(storeName)).all()) { + try (final KeyValueIterator, ValueAndTimestamp> iterator = + ((KStreamImpl.WindowStoreFacade) context.getStateMgr().getStore(storeName)).inner.all()) { while (iterator.hasNext()) { - final KeyValue, Long> next = iterator.next(); + final KeyValue, ValueAndTimestamp> next = iterator.next(); final Integer deserializedKey = new IntegerDeserializer().deserialize(null, next.key.key()); - result.add(new KeyValue<>(new Windowed<>(deserializedKey, next.key.window()), next.value)); + result.add(new KeyValue<>(new Windowed<>(deserializedKey, next.key.window()), next.value.value())); } } diff --git a/streams/src/test/java/org/apache/kafka/streams/state/StoresTest.java b/streams/src/test/java/org/apache/kafka/streams/state/StoresTest.java index 1d4a849a19d5b..20602a40b74b5 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/StoresTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/StoresTest.java @@ -17,19 +17,22 @@ package org.apache.kafka.streams.state; import org.apache.kafka.common.serialization.Serdes; +import org.apache.kafka.common.utils.Bytes; import org.apache.kafka.streams.state.internals.InMemoryKeyValueStore; +import org.apache.kafka.streams.state.internals.KeyValueToKeyValueWithTimestampByteProxyStore; import org.apache.kafka.streams.state.internals.MemoryNavigableLRUCache; +import org.apache.kafka.streams.state.internals.RocksDBSegmentedBytesStore; import org.apache.kafka.streams.state.internals.RocksDBSessionStore; import org.apache.kafka.streams.state.internals.RocksDBStore; import org.apache.kafka.streams.state.internals.RocksDBWindowStore; +import org.apache.kafka.streams.state.internals.RocksDBWithTimestampStore; +import org.apache.kafka.streams.state.internals.WrappedStateStore; import org.junit.Test; import static java.time.Duration.ZERO; import static java.time.Duration.ofMillis; -import static org.hamcrest.CoreMatchers.nullValue; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.core.IsInstanceOf.instanceOf; -import static org.hamcrest.core.IsNot.not; public class StoresTest { @@ -116,6 +119,13 @@ public void shouldCreateRocksDbStore() { assertThat(Stores.persistentKeyValueStore("store").get(), instanceOf(RocksDBStore.class)); } + @Test + public void shouldCreateRocksDbWithTimestampStore() { + final KeyValueStore store = Stores.persistentKeyValueWithTimestampStore("store").get(); + assertThat(store, instanceOf(RocksDBWithTimestampStore.class)); + assertThat(store, instanceOf(RecordConverter.class)); + } + @Test public void shouldCreateRocksDbWindowStore() { assertThat(Stores.persistentWindowStore("store", ofMillis(1L), ofMillis(1L), false).get(), instanceOf(RocksDBWindowStore.class)); @@ -127,23 +137,43 @@ public void shouldCreateRocksDbSessionStore() { } @Test - public void shouldBuildWindowStore() { - final WindowStore store = Stores.windowStoreBuilder( - Stores.persistentWindowStore("store", ofMillis(3L), ofMillis(3L), true), + public void shouldBuildKeyValueStore() { + final KeyValueStore store = Stores.keyValueStoreBuilder( + Stores.persistentKeyValueStore("name"), Serdes.String(), Serdes.String() ).build(); - assertThat(store, not(nullValue())); + assertThat(((WrappedStateStore) store).inner(), instanceOf(RocksDBStore.class)); } @Test - public void shouldBuildKeyValueStore() { - final KeyValueStore store = Stores.keyValueStoreBuilder( + public void shouldBuildKeyValueWithTimestampStore() { + final KeyValueStore> store = Stores.keyValueWithTimestampStoreBuilder( + Stores.persistentKeyValueWithTimestampStore("name"), + Serdes.String(), + Serdes.String() + ).build(); + assertThat(((WrappedStateStore) store).inner(), instanceOf(RocksDBWithTimestampStore.class)); + } + + @Test + public void shouldWrapKeyValueStoreWithKeyValueKeyValueWithTimestampProxyStore() { + final KeyValueStore> store = Stores.keyValueWithTimestampStoreBuilder( Stores.persistentKeyValueStore("name"), Serdes.String(), Serdes.String() ).build(); - assertThat(store, not(nullValue())); + assertThat(((WrappedStateStore) store).inner(), instanceOf(KeyValueToKeyValueWithTimestampByteProxyStore.class)); + } + + @Test + public void shouldBuildWindowStore() { + final WindowStore store = Stores.windowStoreBuilder( + Stores.persistentWindowStore("store", ofMillis(3L), ofMillis(3L), true), + Serdes.String(), + Serdes.String() + ).build(); + assertThat(((WrappedStateStore) store).inner(), instanceOf(RocksDBSegmentedBytesStore.class)); } @Test @@ -153,6 +183,6 @@ public void shouldBuildSessionStore() { Serdes.String(), Serdes.String() ).build(); - assertThat(store, not(nullValue())); + assertThat(((WrappedStateStore) store).inner(), instanceOf(RocksDBSegmentedBytesStore.class)); } } diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingKeyValueStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingKeyValueStoreTest.java index cf15f9d060a0c..ab93056422c09 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingKeyValueStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingKeyValueStoreTest.java @@ -346,4 +346,4 @@ public void apply(final K key, forwarded.put(key, new Change<>(newValue, oldValue)); } } -} \ No newline at end of file +} diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingSessionStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingSessionStoreTest.java index f57b94a4fb239..310c61fe6f7bb 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingSessionStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingSessionStoreTest.java @@ -65,7 +65,7 @@ public class CachingSessionStoreTest { @Before public void setUp() { - final SessionKeySchema schema = new SessionKeySchema(); + final SessionKeySchema schema = new SessionKeySchema<>(); underlying = new RocksDBSegmentedBytesStore("test", "metrics-scope", 0L, SEGMENT_INTERVAL, schema); final RocksDBSessionStore sessionStore = new RocksDBSessionStore<>(underlying, Serdes.Bytes(), Serdes.ByteArray()); cachingStore = new CachingSessionStore<>(sessionStore, Serdes.String(), Serdes.String(), SEGMENT_INTERVAL); @@ -150,7 +150,8 @@ public void shouldFetchAllSessionsWithSameRecordKey() { public void shouldFlushItemsToStoreOnEviction() { final List, byte[]>> added = addSessionsUntilOverflow("a", "b", "c", "d"); assertEquals(added.size() - 1, cache.size()); - final KeyValueIterator, byte[]> iterator = cachingStore.findSessions(added.get(0).key.key(), 0, 0); + final KeyValueIterator, byte[]> iterator + = cachingStore.findSessions(added.get(0).key.key(), 0, 0); final KeyValue, byte[]> next = iterator.next(); assertEquals(added.get(0).key, next.key); assertArrayEquals(added.get(0).value, next.value); @@ -159,10 +160,11 @@ public void shouldFlushItemsToStoreOnEviction() { @Test public void shouldQueryItemsInCacheAndStore() { final List, byte[]>> added = addSessionsUntilOverflow("a"); - final KeyValueIterator, byte[]> iterator = cachingStore.findSessions( - Bytes.wrap("a".getBytes(StandardCharsets.UTF_8)), - 0, - added.size() * 10); + final KeyValueIterator, byte[]> iterator = + cachingStore.findSessions( + Bytes.wrap("a".getBytes(StandardCharsets.UTF_8)), + 0, + added.size() * 10); final List, byte[]>> actual = toList(iterator); verifyKeyValueList(added, actual); } diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingWindowStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingWindowStoreTest.java index 610fc6044e10e..ad069892e3003 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingWindowStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingWindowStoreTest.java @@ -77,11 +77,11 @@ public class CachingWindowStoreTest { private CachingKeyValueStoreTest.CacheFlushListenerStub, String> cacheListener; private ThreadCache cache; private String topic; - private WindowKeySchema keySchema; + private WindowKeySchema keySchema; @Before public void setUp() { - keySchema = new WindowKeySchema(); + keySchema = new WindowKeySchema<>(); underlying = new RocksDBSegmentedBytesStore("test", "metrics-scope", 0, SEGMENT_INTERVAL, keySchema); final RocksDBWindowStore windowStore = new RocksDBWindowStore<>( underlying, diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/SegmentIteratorTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/PlainSegmentIteratorTest.java similarity index 87% rename from streams/src/test/java/org/apache/kafka/streams/state/internals/SegmentIteratorTest.java rename to streams/src/test/java/org/apache/kafka/streams/state/internals/PlainSegmentIteratorTest.java index db5f083272962..c5a1958d61110 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/SegmentIteratorTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/PlainSegmentIteratorTest.java @@ -38,25 +38,25 @@ import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; -public class SegmentIteratorTest { +public class PlainSegmentIteratorTest { - private final Segment segmentOne = new Segment("one", "one", 0); - private final Segment segmentTwo = new Segment("two", "window", 1); + private final PlainSegment segmentOne = new PlainSegment("one", "one", 0); + private final PlainSegment segmentTwo = new PlainSegment("two", "window", 1); private final HasNextCondition hasNextCondition = Iterator::hasNext; - private SegmentIterator iterator = null; + private SegmentIterator iterator = null; @Before public void before() { final InternalMockProcessorContext context = new InternalMockProcessorContext( - TestUtils.tempDirectory(), - Serdes.String(), - Serdes.String(), - new NoOpRecordCollector(), - new ThreadCache( - new LogContext("testCache "), - 0, - new MockStreamsMetrics(new Metrics()))); + TestUtils.tempDirectory(), + Serdes.String(), + Serdes.String(), + new NoOpRecordCollector(), + new ThreadCache( + new LogContext("testCache "), + 0, + new MockStreamsMetrics(new Metrics()))); segmentOne.openDB(context); segmentTwo.openDB(context); segmentOne.put(Bytes.wrap("a".getBytes()), "1".getBytes()); @@ -77,7 +77,7 @@ public void closeSegments() { @Test public void shouldIterateOverAllSegments() { - iterator = new SegmentIterator( + iterator = new SegmentIterator<>( Arrays.asList(segmentOne, segmentTwo).iterator(), hasNextCondition, Bytes.wrap("a".getBytes()), @@ -104,7 +104,7 @@ public void shouldIterateOverAllSegments() { @Test public void shouldNotThrowExceptionOnHasNextWhenStoreClosed() { - iterator = new SegmentIterator( + iterator = new SegmentIterator<>( Collections.singletonList(segmentOne).iterator(), hasNextCondition, Bytes.wrap("a".getBytes()), @@ -117,7 +117,7 @@ public void shouldNotThrowExceptionOnHasNextWhenStoreClosed() { @Test public void shouldOnlyIterateOverSegmentsInRange() { - iterator = new SegmentIterator( + iterator = new SegmentIterator<>( Arrays.asList(segmentOne, segmentTwo).iterator(), hasNextCondition, Bytes.wrap("a".getBytes()), @@ -136,7 +136,7 @@ public void shouldOnlyIterateOverSegmentsInRange() { @Test(expected = NoSuchElementException.class) public void shouldThrowNoSuchElementOnPeekNextKeyIfNoNext() { - iterator = new SegmentIterator( + iterator = new SegmentIterator<>( Arrays.asList(segmentOne, segmentTwo).iterator(), hasNextCondition, Bytes.wrap("f".getBytes()), @@ -147,7 +147,7 @@ public void shouldThrowNoSuchElementOnPeekNextKeyIfNoNext() { @Test(expected = NoSuchElementException.class) public void shouldThrowNoSuchElementOnNextIfNoNext() { - iterator = new SegmentIterator( + iterator = new SegmentIterator<>( Arrays.asList(segmentOne, segmentTwo).iterator(), hasNextCondition, Bytes.wrap("f".getBytes()), @@ -159,4 +159,4 @@ public void shouldThrowNoSuchElementOnNextIfNoNext() { private KeyValue toStringKeyValue(final KeyValue binaryKv) { return KeyValue.pair(new String(binaryKv.key.get()), new String(binaryKv.value)); } -} \ No newline at end of file +} diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/SegmentsTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/PlainSegmentsTest.java similarity index 86% rename from streams/src/test/java/org/apache/kafka/streams/state/internals/SegmentsTest.java rename to streams/src/test/java/org/apache/kafka/streams/state/internals/PlainSegmentsTest.java index efed24f49e9fe..56df29a297795 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/SegmentsTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/PlainSegmentsTest.java @@ -41,13 +41,13 @@ import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; -public class SegmentsTest { +public class PlainSegmentsTest { private static final int NUM_SEGMENTS = 5; private static final long SEGMENT_INTERVAL = 100L; private static final long RETENTION_PERIOD = 4 * SEGMENT_INTERVAL; private InternalMockProcessorContext context; - private Segments segments; + private PlainSegments segments; private File stateDirectory; private final String storeName = "test"; @@ -61,7 +61,7 @@ public void createContext() { new NoOpRecordCollector(), new ThreadCache(new LogContext("testCache "), 0, new MockStreamsMetrics(new Metrics())) ); - segments = new Segments(storeName, RETENTION_PERIOD, SEGMENT_INTERVAL); + segments = new PlainSegments(storeName, RETENTION_PERIOD, SEGMENT_INTERVAL); } @After @@ -79,7 +79,7 @@ public void shouldGetSegmentIdsFromTimestamp() { @Test public void shouldBaseSegmentIntervalOnRetentionAndNumSegments() { - final Segments segments = new Segments("test", 8 * SEGMENT_INTERVAL, 2 * SEGMENT_INTERVAL); + final PlainSegments segments = new PlainSegments("test", 8 * SEGMENT_INTERVAL, 2 * SEGMENT_INTERVAL); assertEquals(0, segments.segmentId(0)); assertEquals(0, segments.segmentId(SEGMENT_INTERVAL)); assertEquals(1, segments.segmentId(2 * SEGMENT_INTERVAL)); @@ -94,9 +94,9 @@ public void shouldGetSegmentNameFromId() { @Test public void shouldCreateSegments() { - final Segment segment1 = segments.getOrCreateSegmentIfLive(0, context); - final Segment segment2 = segments.getOrCreateSegmentIfLive(1, context); - final Segment segment3 = segments.getOrCreateSegmentIfLive(2, context); + final PlainSegment segment1 = segments.getOrCreateSegmentIfLive(0, context); + final PlainSegment segment2 = segments.getOrCreateSegmentIfLive(1, context); + final PlainSegment segment3 = segments.getOrCreateSegmentIfLive(2, context); assertTrue(new File(context.stateDir(), "test/test.0").isDirectory()); assertTrue(new File(context.stateDir(), "test/test." + SEGMENT_INTERVAL).isDirectory()); assertTrue(new File(context.stateDir(), "test/test." + 2 * SEGMENT_INTERVAL).isDirectory()); @@ -114,10 +114,10 @@ public void shouldNotCreateSegmentThatIsAlreadyExpired() { @Test public void shouldCleanupSegmentsThatHaveExpired() { - final Segment segment1 = segments.getOrCreateSegmentIfLive(0, context); - final Segment segment2 = segments.getOrCreateSegmentIfLive(1, context); + final PlainSegment segment1 = segments.getOrCreateSegmentIfLive(0, context); + final PlainSegment segment2 = segments.getOrCreateSegmentIfLive(1, context); context.setStreamTime(SEGMENT_INTERVAL * 7); - final Segment segment3 = segments.getOrCreateSegmentIfLive(7, context); + final PlainSegment segment3 = segments.getOrCreateSegmentIfLive(7, context); assertFalse(segment1.isOpen()); assertFalse(segment2.isOpen()); assertTrue(segment3.isOpen()); @@ -128,22 +128,22 @@ public void shouldCleanupSegmentsThatHaveExpired() { @Test public void shouldGetSegmentForTimestamp() { - final Segment segment = segments.getOrCreateSegmentIfLive(0, context); + final PlainSegment segment = segments.getOrCreateSegmentIfLive(0, context); segments.getOrCreateSegmentIfLive(1, context); assertEquals(segment, segments.getSegmentForTimestamp(0L)); } @Test public void shouldGetCorrectSegmentString() { - final Segment segment = segments.getOrCreateSegmentIfLive(0, context); - assertEquals("Segment(id=0, name=test.0)", segment.toString()); + final PlainSegment segment = segments.getOrCreateSegmentIfLive(0, context); + assertEquals("PlainSegment(id=0, name=test.0)", segment.toString()); } @Test public void shouldCloseAllOpenSegments() { - final Segment first = segments.getOrCreateSegmentIfLive(0, context); - final Segment second = segments.getOrCreateSegmentIfLive(1, context); - final Segment third = segments.getOrCreateSegmentIfLive(2, context); + final PlainSegment first = segments.getOrCreateSegmentIfLive(0, context); + final PlainSegment second = segments.getOrCreateSegmentIfLive(1, context); + final PlainSegment third = segments.getOrCreateSegmentIfLive(2, context); segments.close(); assertFalse(first.isOpen()); @@ -153,7 +153,7 @@ public void shouldCloseAllOpenSegments() { @Test public void shouldOpenExistingSegments() { - segments = new Segments("test", 4, 1); + segments = new PlainSegments("test", 4, 1); segments.getOrCreateSegmentIfLive(0, context); segments.getOrCreateSegmentIfLive(1, context); segments.getOrCreateSegmentIfLive(2, context); @@ -162,7 +162,7 @@ public void shouldOpenExistingSegments() { // close existing. segments.close(); - segments = new Segments("test", 4, 1); + segments = new PlainSegments("test", 4, 1); segments.openExisting(context); assertTrue(segments.getSegmentForTimestamp(0).isOpen()); @@ -185,7 +185,7 @@ public void shouldGetSegmentsWithinTimeRange() { segments.getOrCreateSegmentIfLive(3, context); segments.getOrCreateSegmentIfLive(4, context); - final List segments = this.segments.segments(0, 2 * SEGMENT_INTERVAL); + final List segments = this.segments.segments(0, 2 * SEGMENT_INTERVAL); assertEquals(3, segments.size()); assertEquals(0, segments.get(0).id); assertEquals(1, segments.get(1).id); @@ -200,7 +200,7 @@ public void shouldGetSegmentsWithinTimeRangeOutOfOrder() { updateStreamTimeAndCreateSegment(1); updateStreamTimeAndCreateSegment(3); - final List segments = this.segments.segments(0, 2 * SEGMENT_INTERVAL); + final List segments = this.segments.segments(0, 2 * SEGMENT_INTERVAL); assertEquals(3, segments.size()); assertEquals(0, segments.get(0).id); assertEquals(1, segments.get(1).id); @@ -252,7 +252,7 @@ private void updateStreamTimeAndCreateSegment(final int segment) { public void shouldUpdateSegmentFileNameFromOldDateFormatToNewFormat() throws Exception { final long segmentInterval = 60_000L; // the old segment file's naming system maxes out at 1 minute granularity. - segments = new Segments(storeName, NUM_SEGMENTS * segmentInterval, segmentInterval); + segments = new PlainSegments(storeName, NUM_SEGMENTS * segmentInterval, segmentInterval); final String storeDirectoryPath = stateDirectory.getAbsolutePath() + File.separator + storeName; final File storeDirectory = new File(storeDirectoryPath); @@ -306,7 +306,7 @@ public void shouldClearSegmentsOnClose() { } private void verifyCorrectSegments(final long first, final int numSegments) { - final List result = this.segments.segments(0, Long.MAX_VALUE); + final List result = this.segments.segments(0, Long.MAX_VALUE); assertEquals(numSegments, result.size()); for (int i = 0; i < numSegments; i++) { assertEquals(i + first, result.get(i).id); diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBGenericOptionsToDbOptionsColumnFamilyOptionsFacadeTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBGenericOptionsToDbOptionsColumnFamilyOptionsFacadeTest.java new file mode 100644 index 0000000000000..2738ad0af9edc --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBGenericOptionsToDbOptionsColumnFamilyOptionsFacadeTest.java @@ -0,0 +1,257 @@ +/* + * 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.easymock.EasyMockRunner; +import org.easymock.Mock; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.rocksdb.AccessHint; +import org.rocksdb.BuiltinComparator; +import org.rocksdb.ColumnFamilyOptions; +import org.rocksdb.CompactionPriority; +import org.rocksdb.CompactionStyle; +import org.rocksdb.ComparatorOptions; +import org.rocksdb.CompressionType; +import org.rocksdb.DBOptions; +import org.rocksdb.Env; +import org.rocksdb.InfoLogLevel; +import org.rocksdb.LRUCache; +import org.rocksdb.Logger; +import org.rocksdb.Options; +import org.rocksdb.PlainTableConfig; +import org.rocksdb.RateLimiter; +import org.rocksdb.SstFileManager; +import org.rocksdb.StringAppendOperator; +import org.rocksdb.VectorMemTableConfig; +import org.rocksdb.WALRecoveryMode; +import org.rocksdb.util.BytewiseComparator; + +import java.lang.reflect.InvocationTargetException; +import java.lang.reflect.Method; +import java.util.ArrayList; +import java.util.LinkedList; +import java.util.List; + +import static org.easymock.EasyMock.replay; +import static org.easymock.EasyMock.reset; +import static org.easymock.EasyMock.verify; +import static org.hamcrest.Matchers.instanceOf; +import static org.hamcrest.Matchers.matchesPattern; +import static org.junit.Assert.assertThat; +import static org.junit.Assert.fail; + +@RunWith(EasyMockRunner.class) +public class RocksDBGenericOptionsToDbOptionsColumnFamilyOptionsFacadeTest { + private final List ignoreMethods = new LinkedList() { + { + add("isOwningHandle"); + add("dispose"); + add("wait"); + add("equals"); + add("getClass"); + add("hashCode"); + add("notify"); + add("notifyAll"); + add("toString"); + } + }; + + @Mock + private DBOptions dbOptions; + @Mock + private ColumnFamilyOptions columnFamilyOptions; + + @Test + public void shouldOverwriteAllOptionsMethods() throws Exception { + for (final Method method : Options.class.getMethods()) { + if (!ignoreMethods.contains(method.getName())) { + RocksDBGenericOptionsToDbOptionsColumnFamilyOptionsFacade.class + .getDeclaredMethod(method.getName(), method.getParameterTypes()); + } + } + } + + @Test + public void shouldForwardAllDbOptionsCalls() throws Exception { + for (final Method method : Options.class.getMethods()) { + if (!ignoreMethods.contains(method.getName())) { + try { + DBOptions.class.getMethod(method.getName(), method.getParameterTypes()); + verifyDBOptionsMethodCall(method); + } catch (final NoSuchMethodException expectedAndSwallow) { } + } + } + } + + private void verifyDBOptionsMethodCall(final Method method) throws Exception { + final RocksDBGenericOptionsToDbOptionsColumnFamilyOptionsFacade optionsFacadeDbOptions + = new RocksDBGenericOptionsToDbOptionsColumnFamilyOptionsFacade(dbOptions, new ColumnFamilyOptions()); + + final Object[] parameters = getDBOptionsParameters(method.getParameterTypes()); + + try { + reset(dbOptions); + replay(dbOptions); + method.invoke(optionsFacadeDbOptions, parameters); + verify(); + fail("Should have called DBOptions." + method.getName() + "()"); + } catch (final InvocationTargetException undeclaredMockMethodCall) { + assertThat(undeclaredMockMethodCall.getCause(), instanceOf(AssertionError.class)); + assertThat(undeclaredMockMethodCall.getCause().getMessage().trim(), + matchesPattern("Unexpected method call DBOptions\\." + method.getName() + "((.*\n*)*):")); + } + } + + private Object[] getDBOptionsParameters(final Class[] parameterTypes) throws Exception { + final Object[] parameters = new Object[parameterTypes.length]; + + for (int i = 0; i < parameterTypes.length; ++i) { + switch (parameterTypes[i].getName()) { + case "boolean": + parameters[i] = true; + break; + case "int": + parameters[i] = 0; + break; + case "long": + parameters[i] = 0L; + break; + case "java.util.Collection": + parameters[i] = new ArrayList<>(); + break; + case "org.rocksdb.AccessHint": + parameters[i] = AccessHint.NONE; + break; + case "org.rocksdb.Cache": + parameters[i] = new LRUCache(1L); + break; + case "org.rocksdb.Env": + parameters[i] = Env.getDefault(); + break; + case "org.rocksdb.InfoLogLevel": + parameters[i] = InfoLogLevel.FATAL_LEVEL; + break; + case "org.rocksdb.Logger": + parameters[i] = new Logger(new Options()) { + @Override + protected void log(final InfoLogLevel infoLogLevel, final String logMsg) {} + }; + break; + case "org.rocksdb.RateLimiter": + parameters[i] = new RateLimiter(1L); + break; + case "org.rocksdb.SstFileManager": + parameters[i] = new SstFileManager(Env.getDefault()); + break; + case "org.rocksdb.WALRecoveryMode": + parameters[i] = WALRecoveryMode.AbsoluteConsistency; + break; + default: + parameters[i] = parameterTypes[i].newInstance(); + } + } + + return parameters; + } + + @Test + public void shouldForwardAllColumnFamilyCalls() throws Exception { + for (final Method method : Options.class.getMethods()) { + if (!ignoreMethods.contains(method.getName())) { + try { + ColumnFamilyOptions.class.getMethod(method.getName(), method.getParameterTypes()); + verifyColumnFamilyOptionsMethodCall(method); + } catch (final NoSuchMethodException expectedAndSwallow) { } + } + } + } + + private void verifyColumnFamilyOptionsMethodCall(final Method method) throws Exception { + final RocksDBGenericOptionsToDbOptionsColumnFamilyOptionsFacade optionsFacadeColumnFamilyOptions + = new RocksDBGenericOptionsToDbOptionsColumnFamilyOptionsFacade(new DBOptions(), columnFamilyOptions); + + final Object[] parameters = getColumnFamilyOptionsParameters(method.getParameterTypes()); + + try { + reset(columnFamilyOptions); + replay(columnFamilyOptions); + method.invoke(optionsFacadeColumnFamilyOptions, parameters); + verify(); + fail("Should have called ColumnFamilyOptions." + method.getName() + "()"); + } catch (final InvocationTargetException undeclaredMockMethodCall) { + assertThat(undeclaredMockMethodCall.getCause(), instanceOf(AssertionError.class)); + assertThat(undeclaredMockMethodCall.getCause().getMessage().trim(), + matchesPattern("Unexpected method call ColumnFamilyOptions\\." + method.getName() + "(.*)")); + } + } + + private Object[] getColumnFamilyOptionsParameters(final Class[] parameterTypes) throws Exception { + final Object[] parameters = new Object[parameterTypes.length]; + + for (int i = 0; i < parameterTypes.length; ++i) { + switch (parameterTypes[i].getName()) { + case "boolean": + parameters[i] = true; + break; + case "double": + parameters[i] = 0.0d; + break; + case "int": + parameters[i] = 0; + break; + case "long": + parameters[i] = 0L; + break; + case "[I": + parameters[i] = new int[0]; + break; + case "java.util.List": + parameters[i] = new ArrayList<>(); + break; + case "org.rocksdb.AbstractComparator": + parameters[i] = new BytewiseComparator(new ComparatorOptions()); + break; + case "org.rocksdb.BuiltinComparator": + parameters[i] = BuiltinComparator.BYTEWISE_COMPARATOR; + break; + case "org.rocksdb.CompactionPriority": + parameters[i] = CompactionPriority.ByCompensatedSize; + break; + case "org.rocksdb.CompactionStyle": + parameters[i] = CompactionStyle.UNIVERSAL; + break; + case "org.rocksdb.CompressionType": + parameters[i] = CompressionType.NO_COMPRESSION; + break; + case "org.rocksdb.MemTableConfig": + parameters[i] = new VectorMemTableConfig(); + break; + case "org.rocksdb.MergeOperator": + parameters[i] = new StringAppendOperator(); + break; + case "org.rocksdb.TableFormatConfig": + parameters[i] = new PlainTableConfig(); + break; + default: + parameters[i] = parameterTypes[i].newInstance(); + } + } + + return parameters; + } +} diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBSegmentedBytesStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBSegmentedBytesStoreTest.java index 0b9d66df67bd9..94d002d26421f 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBSegmentedBytesStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBSegmentedBytesStoreTest.java @@ -83,7 +83,7 @@ public class RocksDBSegmentedBytesStoreTest { private final Window[] windows = new Window[4]; @Parameter - public SegmentedBytesStore.KeySchema schema; + public SegmentedBytesStore.KeySchema schema; @Parameters(name = "{0}") public static Object[] getKeySchemas() { @@ -178,7 +178,7 @@ public void shouldRemove() { @Test public void shouldRollSegments() { // just to validate directories - final Segments segments = new Segments(storeName, retention, segmentInterval); + final PlainSegments segments = new PlainSegments(storeName, retention, segmentInterval); final String key = "a"; bytesStore.put(serializeKey(new Windowed<>(key, windows[0])), serializeValue(50)); @@ -206,7 +206,7 @@ public void shouldRollSegments() { @Test public void shouldGetAllSegments() { // just to validate directories - final Segments segments = new Segments(storeName, retention, segmentInterval); + final PlainSegments segments = new PlainSegments(storeName, retention, segmentInterval); final String key = "a"; bytesStore.put(serializeKey(new Windowed<>(key, windows[0])), serializeValue(50L)); @@ -235,7 +235,7 @@ public void shouldGetAllSegments() { @Test public void shouldFetchAllSegments() { // just to validate directories - final Segments segments = new Segments(storeName, retention, segmentInterval); + final PlainSegments segments = new PlainSegments(storeName, retention, segmentInterval); final String key = "a"; bytesStore.put(serializeKey(new Windowed<>(key, windows[0])), serializeValue(50L)); @@ -263,7 +263,7 @@ public void shouldFetchAllSegments() { @Test public void shouldLoadSegmentsWithOldStyleDateFormattedName() { - final Segments segments = new Segments(storeName, retention, segmentInterval); + final PlainSegments segments = new PlainSegments(storeName, retention, segmentInterval); final String key = "a"; bytesStore.put(serializeKey(new Windowed<>(key, windows[0])), serializeValue(50L)); @@ -304,7 +304,7 @@ public void shouldLoadSegmentsWithOldStyleDateFormattedName() { @Test public void shouldLoadSegmentsWithOldStyleColonFormattedName() { - final Segments segments = new Segments(storeName, retention, segmentInterval); + final PlainSegments segments = new PlainSegments(storeName, retention, segmentInterval); final String key = "a"; bytesStore.put(serializeKey(new Windowed<>(key, windows[0])), serializeValue(50L)); @@ -355,7 +355,7 @@ public void shouldCreateWriteBatches() { final Collection> records = new ArrayList<>(); records.add(new KeyValue<>(serializeKey(new Windowed<>(key, windows[0])).get(), serializeValue(50L))); records.add(new KeyValue<>(serializeKey(new Windowed<>(key, windows[3])).get(), serializeValue(100L))); - final Map writeBatchMap = bytesStore.getWriteBatches(records); + final Map writeBatchMap = bytesStore.getWriteBatches(records); assertEquals(2, writeBatchMap.size()); for (final WriteBatch batch : writeBatchMap.values()) { assertEquals(1, batch.count()); @@ -376,7 +376,7 @@ public void shouldRestoreToByteStore() { assertEquals(2, bytesStore.getSegments().size()); // Bulk loading is enabled during recovery. - for (final Segment segment : bytesStore.getSegments()) { + for (final PlainSegment segment : bytesStore.getSegments()) { Assert.assertThat(segment.getOptions().level0FileNumCompactionTrigger(), equalTo(1 << 30)); } @@ -400,12 +400,12 @@ public void shouldRespectBulkLoadOptionsDuringInit() { restoreListener.onRestoreStart(null, bytesStore.name(), 0L, 0L); - for (final Segment segment : bytesStore.getSegments()) { + for (final PlainSegment segment : bytesStore.getSegments()) { Assert.assertThat(segment.getOptions().level0FileNumCompactionTrigger(), equalTo(1 << 30)); } restoreListener.onRestoreEnd(null, bytesStore.name(), 0L); - for (final Segment segment : bytesStore.getSegments()) { + for (final PlainSegment segment : bytesStore.getSegments()) { Assert.assertThat(segment.getOptions().level0FileNumCompactionTrigger(), equalTo(4)); } } diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBSessionStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBSessionStoreTest.java index f13ac0a2affa6..ac91acafa9d61 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBSessionStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBSessionStoreTest.java @@ -50,7 +50,7 @@ public class RocksDBSessionStoreTest { @Before public void before() { - final SessionKeySchema schema = new SessionKeySchema(); + final SessionKeySchema schema = new SessionKeySchema<>(); final RocksDBSegmentedBytesStore bytesStore = new RocksDBSegmentedBytesStore( "session-store", @@ -192,7 +192,7 @@ public void shouldFetchExactKeys() { "metrics-scope", 0x7a00000000000000L, 0x7a00000000000000L, - new SessionKeySchema()); + new SessionKeySchema<>()); sessionStore = new RocksDBSessionStore<>( bytesStore, diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBStoreTest.java index b98f72f58e87b..8fc6752b75b17 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBStoreTest.java @@ -98,7 +98,8 @@ public void shouldRespectBulkloadOptionsDuringInit() { assertThat(rocksDBStore.getOptions().level0FileNumCompactionTrigger(), equalTo(10)); assertThat(rocksDBStore.getOptions().level0SlowdownWritesTrigger(), equalTo(20)); - assertThat(rocksDBStore.getOptions().level0StopWritesTrigger(), equalTo(36)); } + assertThat(rocksDBStore.getOptions().level0StopWritesTrigger(), equalTo(36)); + } @Test public void shouldNotThrowExceptionOnRestoreWhenThereIsPreExistingRocksDbFiles() { diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBWindowStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBWindowStoreTest.java index c155a83420b16..4949d12c8a41c 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBWindowStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBWindowStoreTest.java @@ -74,7 +74,7 @@ public class RocksDBWindowStoreTest { private final long segmentInterval = 60_000L; private final long retentionPeriod = segmentInterval * (numSegments - 1); private final String windowName = "window"; - private final Segments segments = new Segments(windowName, retentionPeriod, segmentInterval); + private final PlainSegments segments = new PlainSegments(windowName, retentionPeriod, segmentInterval); private final StateSerdes serdes = new StateSerdes<>("", Serdes.Integer(), Serdes.String()); private final List> changeLog = new ArrayList<>(); @@ -1375,7 +1375,7 @@ public void shouldNoNullPointerWhenSerdeDoesNotHandleNull() { "metrics-scope", retentionPeriod, segmentInterval, - new WindowKeySchema()), + new WindowKeySchema<>()), Serdes.Integer(), new SerdeThatDoesntHandleNull(), false, diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBWithTimestampStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBWithTimestampStoreTest.java new file mode 100644 index 0000000000000..3487f57b4d0f7 --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBWithTimestampStoreTest.java @@ -0,0 +1,437 @@ +/* + * 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.serialization.Deserializer; +import org.apache.kafka.common.serialization.Serdes; +import org.apache.kafka.common.serialization.Serializer; +import org.apache.kafka.common.serialization.StringDeserializer; +import org.apache.kafka.common.serialization.StringSerializer; +import org.apache.kafka.common.utils.Bytes; +import org.apache.kafka.common.utils.Utils; +import org.apache.kafka.streams.KeyValue; +import org.apache.kafka.streams.StreamsConfig; +import org.apache.kafka.streams.errors.ProcessorStateException; +import org.apache.kafka.streams.processor.StateRestoreListener; +import org.apache.kafka.streams.state.KeyValueIterator; +import org.apache.kafka.streams.state.RocksDBConfigSetter; +import org.apache.kafka.test.InternalMockProcessorContext; +import org.apache.kafka.test.StreamsTestUtils; +import org.apache.kafka.test.TestUtils; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; +import org.rocksdb.Options; + +import java.io.File; +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.Set; + +import static java.nio.charset.StandardCharsets.UTF_8; +import static org.hamcrest.CoreMatchers.equalTo; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertThat; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +public class RocksDBWithTimestampStoreTest { + private final Serializer stringSerializer = new StringSerializer(); + private final Deserializer stringDeserializer = new StringDeserializer(); + private RocksDBWithTimestampStore rocksDBStore; + private InternalMockProcessorContext context; + private File dir; + + @Before + public void setUp() { + final Properties props = StreamsTestUtils.getStreamsConfig(); + props.put(StreamsConfig.ROCKSDB_CONFIG_SETTER_CLASS_CONFIG, MockRocksDbConfigSetter.class); + rocksDBStore = new RocksDBWithTimestampStore("test"); + dir = TestUtils.tempDirectory(); + context = new InternalMockProcessorContext(dir, + Serdes.String(), + Serdes.String(), + new StreamsConfig(props)); + } + + @After + public void tearDown() { + rocksDBStore.close(); + } + + @Test + public void shouldRespectBulkloadOptionsDuringInit() { + rocksDBStore.init(context, rocksDBStore); + + final StateRestoreListener restoreListener = context.getRestoreListener(rocksDBStore.name()); + + restoreListener.onRestoreStart(null, rocksDBStore.name(), 0L, 0L); + + assertThat(rocksDBStore.getOptions().level0FileNumCompactionTrigger(), equalTo(1 << 30)); + assertThat(rocksDBStore.getOptions().level0SlowdownWritesTrigger(), equalTo(1 << 30)); + assertThat(rocksDBStore.getOptions().level0StopWritesTrigger(), equalTo(1 << 30)); + + restoreListener.onRestoreEnd(null, rocksDBStore.name(), 0L); + + assertThat(rocksDBStore.getOptions().level0FileNumCompactionTrigger(), equalTo(10)); + assertThat(rocksDBStore.getOptions().level0SlowdownWritesTrigger(), equalTo(20)); + assertThat(rocksDBStore.getOptions().level0StopWritesTrigger(), equalTo(36)); + } + + @Test + public void shouldNotThrowExceptionOnRestoreWhenThereIsPreExistingRocksDbFiles() { + rocksDBStore.init(context, rocksDBStore); + + final String message = "how can a 4 ounce bird carry a 2lb coconut"; + int intKey = 1; + for (int i = 0; i < 2000000; i++) { + rocksDBStore.put(new Bytes(stringSerializer.serialize(null, "theKeyIs" + intKey++)), + stringSerializer.serialize(null, message)); + } + + final List> restoreBytes = new ArrayList<>(); + + final byte[] restoredKey = "restoredKey".getBytes(UTF_8); + final byte[] restoredValue = "restoredValue".getBytes(UTF_8); + restoreBytes.add(KeyValue.pair(restoredKey, restoredValue)); + + context.restore("test", restoreBytes); + + assertThat( + stringDeserializer.deserialize( + null, + rocksDBStore.get(new Bytes(stringSerializer.serialize(null, "restoredKey")))), + equalTo("restoredValue")); + } + + @Test + public void shouldCallRocksDbConfigSetter() { + MockRocksDbConfigSetter.called = false; + + rocksDBStore.openDB(context); + + assertTrue(MockRocksDbConfigSetter.called); + } + + @Test + public void shouldThrowProcessorStateExceptionOnOpeningReadOnlyDir() { + final File tmpDir = TestUtils.tempDirectory(); + final InternalMockProcessorContext tmpContext = new InternalMockProcessorContext(tmpDir, new StreamsConfig(StreamsTestUtils.getStreamsConfig())); + + assertTrue(tmpDir.setReadOnly()); + + try { + rocksDBStore.openDB(tmpContext); + fail("Should have thrown ProcessorStateException"); + } catch (final ProcessorStateException e) { + // this is good, do nothing + } + } + + @Test + public void shouldPutAll() { + final List> 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"))); + + rocksDBStore.init(context, rocksDBStore); + rocksDBStore.putAll(entries); + rocksDBStore.flush(); + + assertEquals( + stringDeserializer.deserialize( + null, + rocksDBStore.get(new Bytes(stringSerializer.serialize(null, "1")))), + "a"); + assertEquals( + stringDeserializer.deserialize( + null, + rocksDBStore.get(new Bytes(stringSerializer.serialize(null, "2")))), + "b"); + assertEquals( + stringDeserializer.deserialize( + null, + rocksDBStore.get(new Bytes(stringSerializer.serialize(null, "3")))), + "c"); + } + + @Test + public void shouldTogglePrepareForBulkloadSetting() { + rocksDBStore.init(context, rocksDBStore); + final RocksDBWithTimestampStore.RocksDBBatchingRestoreCallback restoreListener = + (RocksDBWithTimestampStore.RocksDBBatchingRestoreCallback) rocksDBStore.batchingStateRestoreCallback; + + restoreListener.onRestoreStart(null, null, 0, 0); + assertTrue("Should have set bulk loading to true", rocksDBStore.isPrepareForBulkload()); + + restoreListener.onRestoreEnd(null, null, 0); + assertFalse("Should have set bulk loading to false", rocksDBStore.isPrepareForBulkload()); + } + + @Test + public void shouldTogglePrepareForBulkloadSettingWhenPrexistingSstFiles() { + final List> entries = getKeyValueEntries(); + + rocksDBStore.init(context, rocksDBStore); + context.restore(rocksDBStore.name(), entries); + + final RocksDBWithTimestampStore.RocksDBBatchingRestoreCallback restoreListener = + (RocksDBWithTimestampStore.RocksDBBatchingRestoreCallback) rocksDBStore.batchingStateRestoreCallback; + + restoreListener.onRestoreStart(null, null, 0, 0); + assertTrue("Should have not set bulk loading to true", rocksDBStore.isPrepareForBulkload()); + + restoreListener.onRestoreEnd(null, null, 0); + assertFalse("Should have set bulk loading to false", rocksDBStore.isPrepareForBulkload()); + } + + @Test + public void shouldRestoreAll() { + final List> entries = getKeyValueEntries(); + + rocksDBStore.init(context, rocksDBStore); + context.restore(rocksDBStore.name(), entries); + + assertEquals( + stringDeserializer.deserialize( + null, + rocksDBStore.get(new Bytes(stringSerializer.serialize(null, "1")))), + "a"); + assertEquals( + stringDeserializer.deserialize( + null, + rocksDBStore.get(new Bytes(stringSerializer.serialize(null, "2")))), + "b"); + assertEquals( + stringDeserializer.deserialize( + null, + rocksDBStore.get(new Bytes(stringSerializer.serialize(null, "3")))), + "c"); + } + + @Test + public void shouldPutOnlyIfAbsentValue() { + rocksDBStore.init(context, rocksDBStore); + final Bytes keyBytes = new Bytes(stringSerializer.serialize(null, "one")); + final byte[] valueBytes = stringSerializer.serialize(null, "A"); + final byte[] valueBytesUpdate = stringSerializer.serialize(null, "B"); + + rocksDBStore.putIfAbsent(keyBytes, valueBytes); + rocksDBStore.putIfAbsent(keyBytes, valueBytesUpdate); + + final String retrievedValue = stringDeserializer.deserialize(null, rocksDBStore.get(keyBytes)); + assertEquals(retrievedValue, "A"); + } + + @Test + public void shouldHandleDeletesOnRestoreAll() { + final List> entries = getKeyValueEntries(); + entries.add(new KeyValue<>("1".getBytes(UTF_8), null)); + + rocksDBStore.init(context, rocksDBStore); + context.restore(rocksDBStore.name(), entries); + + final KeyValueIterator iterator = rocksDBStore.all(); + final Set keys = new HashSet<>(); + + while (iterator.hasNext()) { + keys.add(stringDeserializer.deserialize(null, iterator.next().key.get())); + } + + assertThat(keys, equalTo(Utils.mkSet("2", "3"))); + } + + @Test + public void shouldHandleDeletesAndPutbackOnRestoreAll() { + final List> entries = new ArrayList<>(); + entries.add(new KeyValue<>("1".getBytes(UTF_8), "a".getBytes(UTF_8))); + entries.add(new KeyValue<>("2".getBytes(UTF_8), "b".getBytes(UTF_8))); + // this will be deleted + entries.add(new KeyValue<>("1".getBytes(UTF_8), null)); + entries.add(new KeyValue<>("3".getBytes(UTF_8), "c".getBytes(UTF_8))); + // this will restore key "1" as WriteBatch applies updates in order + entries.add(new KeyValue<>("1".getBytes(UTF_8), "restored".getBytes(UTF_8))); + + rocksDBStore.init(context, rocksDBStore); + context.restore(rocksDBStore.name(), entries); + + final KeyValueIterator iterator = rocksDBStore.all(); + final Set keys = new HashSet<>(); + + while (iterator.hasNext()) { + keys.add(stringDeserializer.deserialize(null, iterator.next().key.get())); + } + + assertThat(keys, equalTo(Utils.mkSet("1", "2", "3"))); + + assertEquals( + stringDeserializer.deserialize( + null, + rocksDBStore.get(new Bytes(stringSerializer.serialize(null, "1")))), + "restored"); + assertEquals( + stringDeserializer.deserialize( + null, + rocksDBStore.get(new Bytes(stringSerializer.serialize(null, "2")))), + "b"); + assertEquals( + stringDeserializer.deserialize( + null, + rocksDBStore.get(new Bytes(stringSerializer.serialize(null, "3")))), + "c"); + } + + @Test + public void shouldRestoreThenDeleteOnRestoreAll() { + final List> entries = getKeyValueEntries(); + + rocksDBStore.init(context, rocksDBStore); + + context.restore(rocksDBStore.name(), entries); + + assertEquals( + stringDeserializer.deserialize( + null, + rocksDBStore.get(new Bytes(stringSerializer.serialize(null, "1")))), + "a"); + assertEquals( + stringDeserializer.deserialize( + null, + rocksDBStore.get(new Bytes(stringSerializer.serialize(null, "2")))), + "b"); + assertEquals( + stringDeserializer.deserialize( + null, + rocksDBStore.get(new Bytes(stringSerializer.serialize(null, "3")))), + "c"); + + entries.clear(); + + entries.add(new KeyValue<>("2".getBytes(UTF_8), "b".getBytes(UTF_8))); + entries.add(new KeyValue<>("3".getBytes(UTF_8), "c".getBytes(UTF_8))); + entries.add(new KeyValue<>("1".getBytes(UTF_8), null)); + + context.restore(rocksDBStore.name(), entries); + + final KeyValueIterator iterator = rocksDBStore.all(); + final Set keys = new HashSet<>(); + + while (iterator.hasNext()) { + keys.add(stringDeserializer.deserialize(null, iterator.next().key.get())); + } + + assertThat(keys, equalTo(Utils.mkSet("2", "3"))); + } + + + + @Test + public void shouldThrowNullPointerExceptionOnNullPut() { + rocksDBStore.init(context, rocksDBStore); + try { + rocksDBStore.put(null, stringSerializer.serialize(null, "someVal")); + fail("Should have thrown NullPointerException on null put()"); + } catch (final NullPointerException e) { + // this is good + } + } + + @Test + public void shouldThrowNullPointerExceptionOnNullPutAll() { + rocksDBStore.init(context, rocksDBStore); + try { + rocksDBStore.put(null, stringSerializer.serialize(null, "someVal")); + fail("Should have thrown NullPointerException on null put()"); + } catch (final NullPointerException e) { + // this is good + } + } + + @Test + public void shouldThrowNullPointerExceptionOnNullGet() { + rocksDBStore.init(context, rocksDBStore); + try { + rocksDBStore.get(null); + fail("Should have thrown NullPointerException on null get()"); + } catch (final NullPointerException e) { + // this is good + } + } + + @Test + public void shouldThrowNullPointerExceptionOnDelete() { + rocksDBStore.init(context, rocksDBStore); + try { + rocksDBStore.delete(null); + fail("Should have thrown NullPointerException on deleting null key"); + } catch (final NullPointerException e) { + // this is good + } + } + + @Test + public void shouldThrowNullPointerExceptionOnRange() { + rocksDBStore.init(context, rocksDBStore); + try { + rocksDBStore.range(null, new Bytes(stringSerializer.serialize(null, "2"))); + fail("Should have thrown NullPointerException on deleting null key"); + } catch (final NullPointerException e) { + // this is good + } + } + + @Test(expected = ProcessorStateException.class) + public void shouldThrowProcessorStateExceptionOnPutDeletedDir() throws IOException { + rocksDBStore.init(context, rocksDBStore); + Utils.delete(dir); + rocksDBStore.put( + new Bytes(stringSerializer.serialize(null, "anyKey")), + stringSerializer.serialize(null, "anyValue")); + rocksDBStore.flush(); + } + + public static class MockRocksDbConfigSetter implements RocksDBConfigSetter { + static boolean called; + + @Override + public void setConfig(final String storeName, final Options options, final Map configs) { + called = true; + + options.setLevel0FileNumCompactionTrigger(10); + } + } + + private List> getKeyValueEntries() { + final List> entries = new ArrayList<>(); + entries.add(new KeyValue<>("1".getBytes(UTF_8), "a".getBytes(UTF_8))); + entries.add(new KeyValue<>("2".getBytes(UTF_8), "b".getBytes(UTF_8))); + entries.add(new KeyValue<>("3".getBytes(UTF_8), "c".getBytes(UTF_8))); + return entries; + } +} diff --git a/streams/test-utils/src/main/java/org/apache/kafka/streams/TopologyTestDriver.java b/streams/test-utils/src/main/java/org/apache/kafka/streams/TopologyTestDriver.java index e8f4fd65d74d3..107b0cf047cee 100644 --- a/streams/test-utils/src/main/java/org/apache/kafka/streams/TopologyTestDriver.java +++ b/streams/test-utils/src/main/java/org/apache/kafka/streams/TopologyTestDriver.java @@ -41,6 +41,9 @@ import org.apache.kafka.streams.errors.LogAndContinueExceptionHandler; import org.apache.kafka.streams.errors.TopologyException; import org.apache.kafka.streams.internals.QuietStreamsConfig; +import org.apache.kafka.streams.kstream.internals.KStreamImpl; +import org.apache.kafka.streams.kstream.internals.KeyValueWithTimestampStoreMaterializer; +import org.apache.kafka.streams.kstream.internals.SessionWindowedKStreamImpl; import org.apache.kafka.streams.processor.ProcessorContext; import org.apache.kafka.streams.processor.PunctuationType; import org.apache.kafka.streams.processor.Punctuator; @@ -62,7 +65,11 @@ import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; import org.apache.kafka.streams.state.KeyValueStore; import org.apache.kafka.streams.state.SessionStore; +import org.apache.kafka.streams.state.ValueAndTimestamp; import org.apache.kafka.streams.state.WindowStore; +import org.apache.kafka.streams.state.internals.MeteredKeyValueWithTimestampStore; +import org.apache.kafka.streams.state.internals.MeteredSessionWithTimestampStore; +import org.apache.kafka.streams.state.internals.MeteredWindowWithTimestampStore; import org.apache.kafka.streams.state.internals.ThreadCache; import org.apache.kafka.streams.test.ConsumerRecordFactory; import org.apache.kafka.streams.test.OutputVerifier; @@ -564,8 +571,11 @@ public ProducerRecord readOutput(final String topic, * @return all stores my name * @see #getStateStore(String) * @see #getKeyValueStore(String) + * @see #getKeyValueWithTimestampStore(String) * @see #getWindowStore(String) + * @see #getWindowWithTimestampStore(String) * @see #getSessionStore(String) + * @see #getSessionWithTimestampStore(String) */ @SuppressWarnings("WeakerAccess") public Map getAllStateStores() { @@ -587,8 +597,11 @@ public Map getAllStateStores() { * @return the state store, or {@code null} if no store has been registered with the given name * @see #getAllStateStores() * @see #getKeyValueStore(String) + * @see #getKeyValueWithTimestampStore(String) * @see #getWindowStore(String) + * @see #getWindowWithTimestampStore(String) * @see #getSessionStore(String) + * @see #getSessionWithTimestampStore(String) */ @SuppressWarnings("WeakerAccess") public StateStore getStateStore(final String name) { @@ -621,15 +634,47 @@ public StateStore getStateStore(final String name) { * @return the key value store, or {@code null} if no {@link KeyValueStore} has been registered with the given name * @see #getAllStateStores() * @see #getStateStore(String) + * @see #getKeyValueWithTimestampStore(String) * @see #getWindowStore(String) + * @see #getWindowWithTimestampStore(String) * @see #getSessionStore(String) + * @see #getSessionWithTimestampStore(String) */ @SuppressWarnings({"unchecked", "WeakerAccess"}) public KeyValueStore getKeyValueStore(final String name) { final StateStore store = getStateStore(name); + if (store instanceof MeteredKeyValueWithTimestampStore) { + return new KeyValueWithTimestampStoreMaterializer.KeyValueStoreFacade<>((KeyValueStore>) store); + } return store instanceof KeyValueStore ? (KeyValueStore) store : null; } + /** + * Get the {@link KeyValueStore KeyValueStore<K, ValueAndTimestamp<V>>} with the given name. + * The store can be a "regular" or global store. + *

+ * This is often useful in test cases to pre-populate the store before the test case instructs the topology to + * {@link #pipeInput(ConsumerRecord) process an input message}, and/or to check the store afterward. + * + * @param name the name of the store + * @return the key value store, or {@code null} if no {@link KeyValueStore KeyValueStore<<K, ValueAndTimestamp<V>>>} has been registered with the given name + * @see #getAllStateStores() + * @see #getStateStore(String) + * @see #getKeyValueStore(String) + * @see #getWindowStore(String) + * @see #getWindowWithTimestampStore(String) + * @see #getSessionStore(String) + * @see #getSessionWithTimestampStore(String) + */ + @SuppressWarnings({"unchecked", "WeakerAccess"}) + public KeyValueStore> getKeyValueWithTimestampStore(final String name) { + final StateStore store = getStateStore(name); + if (store instanceof KeyValueWithTimestampStoreMaterializer.KeyValueStoreFacade) { + return ((KeyValueWithTimestampStoreMaterializer.KeyValueStoreFacade) store).inner; + } + return store instanceof KeyValueStore ? (KeyValueStore>) store : null; + } + /** * Get the {@link WindowStore} with the given name. * The store can be a "regular" or global store. @@ -642,14 +687,46 @@ public KeyValueStore getKeyValueStore(final String name) { * @see #getAllStateStores() * @see #getStateStore(String) * @see #getKeyValueStore(String) - * @see #getSessionStore(String) (String) + * @see #getKeyValueWithTimestampStore(String) + * @see #getWindowWithTimestampStore(String) + * @see #getSessionStore(String) + * @see #getSessionWithTimestampStore(String) */ @SuppressWarnings({"unchecked", "WeakerAccess", "unused"}) public WindowStore getWindowStore(final String name) { final StateStore store = getStateStore(name); + if (store instanceof MeteredWindowWithTimestampStore) { + return new KStreamImpl.WindowStoreFacade<>((WindowStore>) store); + } return store instanceof WindowStore ? (WindowStore) store : null; } + /** + * Get the {@link WindowStore WindowStore<K, ValueAndTimestamp<V>>} with the given name. + * The store can be a "regular" or global store. + *

+ * This is often useful in test cases to pre-populate the store before the test case instructs the topology to + * {@link #pipeInput(ConsumerRecord) process an input message}, and/or to check the store afterward. + * + * @param name the name of the store + * @return the key value store, or {@code null} if no {@link WindowStore WindowStore<K, ValueAndTimestamp<V>>} has been registered with the given name + * @see #getAllStateStores() + * @see #getStateStore(String) + * @see #getKeyValueStore(String) + * @see #getKeyValueWithTimestampStore(String) + * @see #getWindowStore(String) + * @see #getSessionStore(String) + * @see #getSessionWithTimestampStore(String) + */ + @SuppressWarnings({"unchecked", "WeakerAccess", "unused"}) + public WindowStore> getWindowWithTimestampStore(final String name) { + final StateStore store = getStateStore(name); + if (store instanceof KStreamImpl.WindowStoreFacade) { + return ((KStreamImpl.WindowStoreFacade) store).inner; + } + return store instanceof WindowStore ? (WindowStore>) store : null; + } + /** * Get the {@link SessionStore} with the given name. * The store can be a "regular" or global store. @@ -662,14 +739,46 @@ public WindowStore getWindowStore(final String name) { * @see #getAllStateStores() * @see #getStateStore(String) * @see #getKeyValueStore(String) + * @see #getKeyValueWithTimestampStore(String) * @see #getWindowStore(String) + * @see #getWindowWithTimestampStore(String) + * @see #getSessionWithTimestampStore(String) */ @SuppressWarnings({"unchecked", "WeakerAccess", "unused"}) public SessionStore getSessionStore(final String name) { final StateStore store = getStateStore(name); + if (store instanceof MeteredSessionWithTimestampStore) { + return new SessionWindowedKStreamImpl.SessionStoreFacade<>((SessionStore>) store); + } return store instanceof SessionStore ? (SessionStore) store : null; } + /** + * Get the {@link SessionStore SessionStore<K, ValueAndTimestamp<V>>} with the given name. + * The store can be a "regular" or global store. + *

+ * This is often useful in test cases to pre-populate the store before the test case instructs the topology to + * {@link #pipeInput(ConsumerRecord) process an input message}, and/or to check the store afterward. + * + * @param name the name of the store + * @return the key value store, or {@code null} if no {@link SessionStore SessionStore<K, ValueAndTimestamp<V>>} has been registered with the given name + * @see #getAllStateStores() + * @see #getStateStore(String) + * @see #getKeyValueStore(String) + * @see #getKeyValueWithTimestampStore(String) + * @see #getWindowStore(String) + * @see #getWindowWithTimestampStore(String) + * @see #getSessionStore(String) + */ + @SuppressWarnings({"unchecked", "WeakerAccess", "unused"}) + public SessionStore> getSessionWithTimestampStore(final String name) { + final StateStore store = getStateStore(name); + if (store instanceof SessionWindowedKStreamImpl.SessionStoreFacade) { + return ((SessionWindowedKStreamImpl.SessionStoreFacade) store).inner; + } + return store instanceof SessionStore ? (SessionStore>) store : null; + } + /** * Close the driver, its topology, and all processors. */ diff --git a/streams/test-utils/src/test/java/org/apache/kafka/streams/TopologyTestDriverTest.java b/streams/test-utils/src/test/java/org/apache/kafka/streams/TopologyTestDriverTest.java index 84aaa8a8e97b9..0b639eff4ac61 100644 --- a/streams/test-utils/src/test/java/org/apache/kafka/streams/TopologyTestDriverTest.java +++ b/streams/test-utils/src/test/java/org/apache/kafka/streams/TopologyTestDriverTest.java @@ -45,6 +45,7 @@ import org.apache.kafka.streams.state.KeyValueIterator; import org.apache.kafka.streams.state.KeyValueStore; import org.apache.kafka.streams.state.Stores; +import org.apache.kafka.streams.state.ValueAndTimestamp; import org.apache.kafka.streams.state.internals.KeyValueStoreBuilder; import org.apache.kafka.streams.test.ConsumerRecordFactory; import org.apache.kafka.streams.test.OutputVerifier; @@ -974,13 +975,13 @@ public void shouldFeedStoreFromGlobalKTable() { Consumed.with(Serdes.String(), Serdes.String()), Materialized.as("globalStore")); try (final TopologyTestDriver testDriver = new TopologyTestDriver(builder.build(), config)) { - final KeyValueStore globalStore = testDriver.getKeyValueStore("globalStore"); + final KeyValueStore> globalStore = testDriver.getKeyValueWithTimestampStore("globalStore"); Assert.assertNotNull(globalStore); Assert.assertNotNull(testDriver.getAllStateStores().get("globalStore")); final ConsumerRecordFactory recordFactory = new ConsumerRecordFactory<>(new StringSerializer(), new StringSerializer()); testDriver.pipeInput(recordFactory.create("topic", "k1", "value1")); // we expect to have both in the global store, the one from pipeInput and the one from the producer - Assert.assertEquals("value1", globalStore.get("k1")); + Assert.assertEquals("value1", globalStore.get("k1").value()); } }