guozhangwang commented on code in PR #14648: URL: https://github.com/apache/kafka/pull/14648#discussion_r1398306353
########## streams/src/main/java/org/apache/kafka/streams/kstream/internals/OuterStreamJoinStoreFactory.java: ########## @@ -0,0 +1,188 @@ +/* + * 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 static org.apache.kafka.streams.internals.ApiUtils.prepareMillisCheckFailMsgPrefix; +import static org.apache.kafka.streams.internals.ApiUtils.validateMillisecondDuration; + +import java.time.Duration; +import java.util.Map; +import org.apache.kafka.common.utils.Time; +import org.apache.kafka.streams.kstream.JoinWindows; +import org.apache.kafka.streams.processor.StateStore; +import org.apache.kafka.streams.processor.internals.StoreFactory; +import org.apache.kafka.streams.state.DslKeyValueParams; +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; +import org.apache.kafka.streams.state.internals.InMemoryWindowBytesStoreSupplier; +import org.apache.kafka.streams.state.internals.LeftOrRightValue; +import org.apache.kafka.streams.state.internals.LeftOrRightValueSerde; +import org.apache.kafka.streams.state.internals.ListValueStoreBuilder; +import org.apache.kafka.streams.state.internals.TimestampedKeyAndJoinSide; +import org.apache.kafka.streams.state.internals.TimestampedKeyAndJoinSideSerde; + +public class OuterStreamJoinStoreFactory<K, V1, V2> extends AbstractConfigurableStoreFactory { + + private final String name; + private final StreamJoinedInternal<K, V1, V2> streamJoined; + private final JoinWindows windows; + private boolean loggingEnabled; + + public enum Type { + RIGHT, + LEFT + } + + public OuterStreamJoinStoreFactory( + final String name, + final StreamJoinedInternal<K, V1, V2> streamJoined, + final JoinWindows windows, + final Type type + ) { + super(streamJoined.dslStoreSuppliers()); + this.name = buildOuterJoinWindowStoreName(streamJoined, name, type) + "-store"; + this.streamJoined = streamJoined; + this.windows = windows; + this.loggingEnabled = streamJoined.loggingEnabled(); + } + + @Override + public StateStore build() { + final Duration retentionPeriod = Duration.ofMillis(retentionPeriod()); + final Duration windowSize = Duration.ofMillis(windows.size()); + final String rpMsgPrefix = prepareMillisCheckFailMsgPrefix(retentionPeriod, "retentionPeriod"); + final long retentionMs = validateMillisecondDuration(retentionPeriod, rpMsgPrefix); + final String wsMsgPrefix = prepareMillisCheckFailMsgPrefix(windowSize, "windowSize"); + final long windowSizeMs = validateMillisecondDuration(windowSize, wsMsgPrefix); + + if (retentionMs < 0L) { + throw new IllegalArgumentException("retentionPeriod cannot be negative"); + } + if (windowSizeMs < 0L) { + throw new IllegalArgumentException("windowSize cannot be negative"); + } + if (windowSizeMs > retentionMs) { + throw new IllegalArgumentException("The retention period of the window store " + + name + " must be no smaller than its window size. Got size=[" + + windowSizeMs + "], retention=[" + retentionMs + "]"); + } + + final TimestampedKeyAndJoinSideSerde<K> timestampedKeyAndJoinSideSerde = new TimestampedKeyAndJoinSideSerde<>(streamJoined.keySerde()); + final LeftOrRightValueSerde<V1, V2> leftOrRightValueSerde = new LeftOrRightValueSerde<>(streamJoined.valueSerde(), streamJoined.otherValueSerde()); + + // TODO: we should allow for configuration of this store explicitly instead of assuming that it should + // share the same type of store as thisStoreSupplier + final boolean useInMemoryStore = streamJoined.thisStoreSupplier() != null + && streamJoined.thisStoreSupplier() instanceof InMemoryWindowBytesStoreSupplier; + final KeyValueBytesStoreSupplier supplier = useInMemoryStore + ? Stores.inMemoryKeyValueStore(name) + : dslStoreSuppliers().keyValueStore(new DslKeyValueParams(name)); Review Comment: +1 ! Likes the proposed ordering. ########## streams/src/main/java/org/apache/kafka/streams/TopologyConfig.java: ########## @@ -216,12 +232,38 @@ public TopologyConfig(final String topologyName, final StreamsConfig globalAppCo } else { storeType = globalAppConfigs.getString(DEFAULT_DSL_STORE_CONFIG); } + + if (isTopologyOverride(DSL_STORE_SUPPLIERS_CLASS_CONFIG, topologyOverrides)) { + dslStoreSuppliers = getString(DSL_STORE_SUPPLIERS_CLASS_CONFIG); + log.info("Topology {} is overriding {} to {}", topologyName, DSL_STORE_SUPPLIERS_CLASS_CONFIG, dslStoreSuppliers); + } else { + dslStoreSuppliers = globalAppConfigs.getString(DSL_STORE_SUPPLIERS_CLASS_CONFIG); + } } + @Deprecated public Materialized.StoreType parseStoreType() { return MaterializedInternal.parse(storeType); } + /** + * @return the DslStoreSuppliers if the value was explicitly configured (either by + * {@link StreamsConfig#DEFAULT_DSL_STORE} or {@link StreamsConfig#DSL_STORE_SUPPLIERS_CLASS_CONFIG}) + */ + public Optional<DslStoreSuppliers> resolveDslStoreSuppliers() { Review Comment: I'm a bit inclined with adding such an `resolveDslStoreSuppliers` to `StreamsConfig` directly as long as we label it EVOLVING to leave us options to delete it (of course this would need get other's opinion as we update the KIP DISCUSS thread), until we have a clear clue on what we want to expose eventually. -- 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