mjsax commented on a change in pull request #11368: URL: https://github.com/apache/kafka/pull/11368#discussion_r740505996
########## File path: streams/src/main/java/org/apache/kafka/streams/kstream/KTable.java ########## @@ -2117,12 +2118,37 @@ * @param <KO> the key type of the other {@code KTable} * @param <VO> the value type of the other {@code KTable} * @return a {@code KTable} that contains the result of joining this table with {@code other} + * + * @deprecated since 3.1, removal planned in 4.0. Use {@link #join(KTable, Function, ValueJoiner, TableJoined)} instead. */ + @Deprecated <VR, KO, VO> KTable<K, VR> join(final KTable<KO, VO> other, final Function<V, KO> foreignKeyExtractor, final ValueJoiner<V, VO, VR> joiner, final Named named); + /** + * Join records of this {@code KTable} with another {@code KTable} using non-windowed inner join, + * using the {@link TableJoined} instance for configuration of the {@link StreamPartitioner this table's + * key serde} and {@link StreamPartitioner the other table's key serde}. Review comment: ``` configuration of the {@link StreamPartitioner this table's key serde} and {@link StreamPartitioner the other table's key serde}. ``` This will render as `configuration of the this table's key serde and the other table's key serde`. `the this` does not sound right. Also, `key serde` (2x)? Should it be `partitioner` (also without "key")? ########## File path: streams/src/main/java/org/apache/kafka/streams/kstream/KTable.java ########## @@ -2117,12 +2118,37 @@ * @param <KO> the key type of the other {@code KTable} * @param <VO> the value type of the other {@code KTable} * @return a {@code KTable} that contains the result of joining this table with {@code other} + * + * @deprecated since 3.1, removal planned in 4.0. Use {@link #join(KTable, Function, ValueJoiner, TableJoined)} instead. Review comment: nit: `planned in` -> `planned for` ? (similar below) ########## File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java ########## @@ -995,7 +1075,8 @@ boolean sendingOldValueEnabled() { //This occurs whenever the extracted foreignKey changes values. enableSendingOldValues(true); - final NamedInternal renamed = new NamedInternal(joinName); + final TableJoinedInternal<K, KO> tableJoinedInternal = new TableJoinedInternal<>(tableJoined); + final NamedInternal renamed = new NamedInternal(tableJoinedInternal.name()); Review comment: Thanks. I think the current code make sense. Because both `TableJoinedInternal` and `NamedInternal` are classes, we cannot extend both and it's clumsy to embed a `NamedInternal` within `TableJoinedInternal`, too. ########## File path: streams/src/main/java/org/apache/kafka/streams/kstream/TableJoined.java ########## @@ -0,0 +1,119 @@ +/* + * 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; + +import org.apache.kafka.streams.processor.StreamPartitioner; + +import java.util.function.Function; + +/** + * The {@code TableJoined} class represents optional params that can be passed to + * {@link KTable#join(KTable, Function, ValueJoiner, TableJoined) KTable#join(KTable,Function,...)} and + * {@link KTable#leftJoin(KTable, Function, ValueJoiner, TableJoined) KTable#leftJoin(KTable,Function,...)} + * operations, for foreign key joins. + * @param <K> this key type ; key type for the left (primary) table + * @param <KO> other key type ; key type for the right (foreign key) table + */ +public class TableJoined<K, KO> implements NamedOperation<TableJoined<K, KO>> { + + protected final StreamPartitioner<K, Void> partitioner; + protected final StreamPartitioner<KO, Void> otherPartitioner; + protected final String name; + + private TableJoined(final StreamPartitioner<K, Void> partitioner, + final StreamPartitioner<KO, Void> otherPartitioner, + final String name) { + this.partitioner = partitioner; + this.otherPartitioner = otherPartitioner; + this.name = name; + } + + protected TableJoined(final TableJoined<K, KO> tableJoined) { + this(tableJoined.partitioner, tableJoined.otherPartitioner, tableJoined.name); + } + + /** + * Create an instance of {@code TableJoined} with partitioner and otherPartitioner {@link StreamPartitioner} instances. Review comment: Reads great! -- 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