lihaosky commented on code in PR #12030: URL: https://github.com/apache/kafka/pull/12030#discussion_r848733142
########## streams/src/main/java/org/apache/kafka/streams/state/internals/TimeOrderedCachingWindowStore.java: ########## @@ -0,0 +1,694 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.streams.state.internals; + +import java.util.function.Function; +import org.apache.kafka.common.header.internals.RecordHeaders; +import org.apache.kafka.common.serialization.Serdes; +import org.apache.kafka.common.utils.Bytes; +import org.apache.kafka.streams.KeyValue; +import org.apache.kafka.streams.StreamsConfig; +import org.apache.kafka.streams.kstream.Windowed; +import org.apache.kafka.streams.kstream.internals.Change; +import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.processor.StateStore; +import org.apache.kafka.streams.processor.StateStoreContext; +import org.apache.kafka.streams.processor.api.Record; +import org.apache.kafka.streams.processor.internals.InternalProcessorContext; +import org.apache.kafka.streams.processor.internals.ProcessorRecordContext; +import org.apache.kafka.streams.processor.internals.ProcessorStateManager; +import org.apache.kafka.streams.processor.internals.RecordQueue; +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; +import org.apache.kafka.streams.state.internals.MergedSortedCacheWindowStoreKeyValueIterator.StoreKeyToWindowKey; +import org.apache.kafka.streams.state.internals.MergedSortedCacheWindowStoreKeyValueIterator.WindowKeyToBytes; +import org.apache.kafka.streams.state.internals.PrefixedWindowKeySchemas.KeyFirstWindowKeySchema; +import org.apache.kafka.streams.state.internals.PrefixedWindowKeySchemas.TimeFirstWindowKeySchema; +import org.apache.kafka.streams.state.internals.SegmentedBytesStore.KeySchema; +import org.apache.kafka.streams.state.internals.ThreadCache.DirtyEntry; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.LinkedList; +import java.util.NoSuchElementException; +import java.util.concurrent.atomic.AtomicLong; + +import static org.apache.kafka.streams.processor.internals.ProcessorContextUtils.asInternalProcessorContext; +import static org.apache.kafka.streams.state.internals.ExceptionUtils.executeAll; +import static org.apache.kafka.streams.state.internals.ExceptionUtils.throwSuppressed; + +class TimeOrderedCachingWindowStore + extends WrappedStateStore<WindowStore<Bytes, byte[]>, byte[], byte[]> + implements WindowStore<Bytes, byte[]>, CachedStateStore<byte[], byte[]> { + + private static final Logger LOG = LoggerFactory.getLogger(TimeOrderedCachingWindowStore.class); + + private final long windowSize; + private final SegmentedCacheFunction baseKeyCacheFunction; + private final SegmentedCacheFunction indexKeyCacheFunction; + private final TimeFirstWindowKeySchema baseKeySchema = new TimeFirstWindowKeySchema(); + private final KeyFirstWindowKeySchema indexKeySchema = new KeyFirstWindowKeySchema(); + + private String cacheName; + private boolean hasIndex; + private boolean sendOldValues; + private InternalProcessorContext<?, ?> context; + private StateSerdes<Bytes, byte[]> bytesSerdes; + private CacheFlushListener<byte[], byte[]> flushListener; + + private final AtomicLong maxObservedTimestamp; Review Comment: `maxObservedTimestamp` is used in `put` and iterator (read). Maybe need atomic? BTW, most of these code is adapted from `CachingWindowStore.java` ########## streams/src/main/java/org/apache/kafka/streams/state/internals/TimeOrderedCachingWindowStore.java: ########## @@ -0,0 +1,694 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.streams.state.internals; + +import java.util.function.Function; +import org.apache.kafka.common.header.internals.RecordHeaders; +import org.apache.kafka.common.serialization.Serdes; +import org.apache.kafka.common.utils.Bytes; +import org.apache.kafka.streams.KeyValue; +import org.apache.kafka.streams.StreamsConfig; +import org.apache.kafka.streams.kstream.Windowed; +import org.apache.kafka.streams.kstream.internals.Change; +import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.processor.StateStore; +import org.apache.kafka.streams.processor.StateStoreContext; +import org.apache.kafka.streams.processor.api.Record; +import org.apache.kafka.streams.processor.internals.InternalProcessorContext; +import org.apache.kafka.streams.processor.internals.ProcessorRecordContext; +import org.apache.kafka.streams.processor.internals.ProcessorStateManager; +import org.apache.kafka.streams.processor.internals.RecordQueue; +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; +import org.apache.kafka.streams.state.internals.MergedSortedCacheWindowStoreKeyValueIterator.StoreKeyToWindowKey; +import org.apache.kafka.streams.state.internals.MergedSortedCacheWindowStoreKeyValueIterator.WindowKeyToBytes; +import org.apache.kafka.streams.state.internals.PrefixedWindowKeySchemas.KeyFirstWindowKeySchema; +import org.apache.kafka.streams.state.internals.PrefixedWindowKeySchemas.TimeFirstWindowKeySchema; +import org.apache.kafka.streams.state.internals.SegmentedBytesStore.KeySchema; +import org.apache.kafka.streams.state.internals.ThreadCache.DirtyEntry; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.LinkedList; +import java.util.NoSuchElementException; +import java.util.concurrent.atomic.AtomicLong; + +import static org.apache.kafka.streams.processor.internals.ProcessorContextUtils.asInternalProcessorContext; +import static org.apache.kafka.streams.state.internals.ExceptionUtils.executeAll; +import static org.apache.kafka.streams.state.internals.ExceptionUtils.throwSuppressed; + +class TimeOrderedCachingWindowStore + extends WrappedStateStore<WindowStore<Bytes, byte[]>, byte[], byte[]> + implements WindowStore<Bytes, byte[]>, CachedStateStore<byte[], byte[]> { + + private static final Logger LOG = LoggerFactory.getLogger(TimeOrderedCachingWindowStore.class); + + private final long windowSize; + private final SegmentedCacheFunction baseKeyCacheFunction; + private final SegmentedCacheFunction indexKeyCacheFunction; + private final TimeFirstWindowKeySchema baseKeySchema = new TimeFirstWindowKeySchema(); + private final KeyFirstWindowKeySchema indexKeySchema = new KeyFirstWindowKeySchema(); + + private String cacheName; + private boolean hasIndex; + private boolean sendOldValues; + private InternalProcessorContext<?, ?> context; + private StateSerdes<Bytes, byte[]> bytesSerdes; + private CacheFlushListener<byte[], byte[]> flushListener; + + private final AtomicLong maxObservedTimestamp; + + TimeOrderedCachingWindowStore(final WindowStore<Bytes, byte[]> underlying, + final long windowSize, + final long segmentInterval) { + super(underlying); + this.windowSize = windowSize; + this.baseKeyCacheFunction = new SegmentedCacheFunction(baseKeySchema, segmentInterval); + this.indexKeyCacheFunction = new SegmentedCacheFunction(indexKeySchema, segmentInterval); + this.maxObservedTimestamp = new AtomicLong(RecordQueue.UNKNOWN); + enforceWrappedStore(underlying); + } + + private void enforceWrappedStore(final WindowStore<Bytes, byte[]> underlying) { + final RocksDBTimeOrderedWindowStore timeOrderedWindowStore = getWrappedStore(underlying); + if (timeOrderedWindowStore == null) { + throw new IllegalArgumentException("TimeOrderedCachingWindowStore only supports RocksDBTimeOrderedWindowStore backed store"); + } + + hasIndex = timeOrderedWindowStore.hasIndex(); + } + + private RocksDBTimeOrderedWindowStore getWrappedStore(final StateStore wrapped) { + if (wrapped instanceof RocksDBTimeOrderedWindowStore) { + return (RocksDBTimeOrderedWindowStore) wrapped; + } + if (wrapped instanceof WrappedStateStore) { + return getWrappedStore(((WrappedStateStore) wrapped).wrapped()); + } + return null; + } + + @Deprecated + @Override + public void init(final ProcessorContext context, final StateStore root) { + initInternal(asInternalProcessorContext(context)); + super.init(context, root); + } + + @Override + public void init(final StateStoreContext context, final StateStore root) { + initInternal(asInternalProcessorContext(context)); + super.init(context, root); + } + + private void initInternal(final InternalProcessorContext<?, ?> context) { + final String prefix = StreamsConfig.InternalConfig.getString( + context.appConfigs(), + StreamsConfig.InternalConfig.TOPIC_PREFIX_ALTERNATIVE, Review Comment: Again copied from `CachingWindowStore` and the prefix is introduced here: https://github.com/apache/kafka/pull/11611. But I'm not sure if it's needed in `CachingWindowStore` or here since topic does nothing in `StateSerdes`? @wcarlson5 , @guozhangwang , do you have more context why we need the prefix here? -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
