ableegoldman commented on code in PR #14708: URL: https://github.com/apache/kafka/pull/14708#discussion_r1388791856
########## streams/src/main/java/org/apache/kafka/streams/kstream/internals/TimeWindowStoreMaterializer.java: ########## @@ -0,0 +1,133 @@ +/* + * 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 java.time.Duration; +import org.apache.kafka.common.utils.Bytes; +import org.apache.kafka.streams.kstream.EmitStrategy; +import org.apache.kafka.streams.kstream.Windows; +import org.apache.kafka.streams.processor.StateStore; +import org.apache.kafka.streams.state.StoreBuilder; +import org.apache.kafka.streams.state.Stores; +import org.apache.kafka.streams.state.TimestampedWindowStore; +import org.apache.kafka.streams.state.WindowBytesStoreSupplier; +import org.apache.kafka.streams.state.WindowStore; +import org.apache.kafka.streams.state.internals.RocksDbIndexedTimeOrderedWindowBytesStoreSupplier; + +public class TimeWindowStoreMaterializer<K, V> extends MaterializedStoreFactory<K, V, WindowStore<Bytes, byte[]>> { + + private final Windows<?> windows; + private final EmitStrategy emitStrategy; + + public TimeWindowStoreMaterializer( + final MaterializedInternal<K, V, WindowStore<Bytes, byte[]>> materialized, + final Windows<?> windows, + final EmitStrategy emitStrategy + ) { + super(materialized); + this.windows = windows; + this.emitStrategy = emitStrategy; + } + + @Override + public StateStore build() { + WindowBytesStoreSupplier supplier = (WindowBytesStoreSupplier) materialized.storeSupplier(); + if (supplier == null) { + final long retentionPeriod = retentionPeriod(); + + if ((windows.size() + windows.gracePeriodMs()) > retentionPeriod) { + throw new IllegalArgumentException("The retention period of the window store " + + materialized.storeName() + " must be no smaller than its window size plus the grace period." + + " Got size=[" + windows.size() + "]," + + " grace=[" + windows.gracePeriodMs() + "]," + + " retention=[" + retentionPeriod + "]"); + } + + switch (defaultStoreType) { + case IN_MEMORY: + supplier = Stores.inMemoryWindowStore( + materialized.storeName(), + Duration.ofMillis(retentionPeriod), + Duration.ofMillis(windows.size()), + false + ); + break; + case ROCKS_DB: + supplier = emitStrategy.type() == EmitStrategy.StrategyType.ON_WINDOW_CLOSE ? + RocksDbIndexedTimeOrderedWindowBytesStoreSupplier.create( + materialized.storeName(), + Duration.ofMillis(retentionPeriod), + Duration.ofMillis(windows.size()), + false, + false + ) : + Stores.persistentTimestampedWindowStore( + materialized.storeName(), + Duration.ofMillis(retentionPeriod), + Duration.ofMillis(windows.size()), + false + ); + break; + default: + throw new IllegalStateException("Unknown store type: " + materialized.storeType()); + } + } + + final StoreBuilder<TimestampedWindowStore<K, V>> builder = Stores.timestampedWindowStoreBuilder( + supplier, + materialized.keySerde(), + materialized.valueSerde() + ); + + if (materialized.loggingEnabled()) { + builder.withLoggingEnabled(materialized.logConfig()); + } else { + builder.withLoggingDisabled(); + } + + // TODO(agavra): remove before merging, should we do what we do with other stores + // and disable caching in the case StrategyType.ON_WINDOW_CLOSE is used? Review Comment: Yeah it's not essential to have a comment, certainly shouldn't block this PR, but there is clearly a lot of subtle and case-by-case handling here that seems all too easy to mess up by accident in the future. Like that comment you just linked Matthias -- hadn't seen that before, is it saying we handle the cache differently depending on if it's persistent or not? I know the in-memory stores have a different layout than the rocksdb ones, but that's what the `SegmentedCacheFunction` is for in the MergedSortedCacheXXXIterators -- to account for the segmentation of rocksdb stores I'm just wondering if we're doing something weird based on what's returned by `StateStore#persistent` -- we have a lot of what I would consider "bugs" in Streams due to assuming that `persistent` means RocksDBStore and `!persistent` means InMemoryStore. We don't always account for custom stores. -- 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: jira-unsubscr...@kafka.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org