reswqa commented on code in PR #24422: URL: https://github.com/apache/flink/pull/24422#discussion_r1543999261
########## flink-datastream/src/main/java/org/apache/flink/datastream/impl/stream/KeyedPartitionStreamImpl.java: ########## @@ -0,0 +1,389 @@ +/* + * 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.flink.datastream.impl.stream; + +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.dag.Transformation; +import org.apache.flink.api.java.functions.KeySelector; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.api.java.typeutils.TypeExtractor; +import org.apache.flink.datastream.api.function.OneInputStreamProcessFunction; +import org.apache.flink.datastream.api.function.TwoInputBroadcastStreamProcessFunction; +import org.apache.flink.datastream.api.function.TwoInputNonBroadcastStreamProcessFunction; +import org.apache.flink.datastream.api.function.TwoOutputStreamProcessFunction; +import org.apache.flink.datastream.api.stream.BroadcastStream; +import org.apache.flink.datastream.api.stream.GlobalStream; +import org.apache.flink.datastream.api.stream.KeyedPartitionStream; +import org.apache.flink.datastream.api.stream.NonKeyedPartitionStream; +import org.apache.flink.datastream.api.stream.NonKeyedPartitionStream.TwoNonKeyedPartitionStreams; +import org.apache.flink.datastream.impl.operators.KeyedProcessOperator; +import org.apache.flink.datastream.impl.operators.KeyedTwoInputBroadcastProcessOperator; +import org.apache.flink.datastream.impl.operators.KeyedTwoInputNonBroadcastProcessOperator; +import org.apache.flink.datastream.impl.operators.KeyedTwoOutputProcessOperator; +import org.apache.flink.datastream.impl.stream.NonKeyedPartitionStreamImpl.TwoNonKeyedPartitionStreamsImpl; +import org.apache.flink.datastream.impl.utils.StreamUtils; +import org.apache.flink.streaming.api.graph.StreamGraphGenerator; +import org.apache.flink.streaming.api.transformations.PartitionTransformation; +import org.apache.flink.streaming.runtime.partitioner.GlobalPartitioner; +import org.apache.flink.streaming.runtime.partitioner.KeyGroupStreamPartitioner; +import org.apache.flink.streaming.runtime.partitioner.ShufflePartitioner; +import org.apache.flink.util.OutputTag; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** The implementation of {@link KeyedPartitionStream}. */ +public class KeyedPartitionStreamImpl<K, V> extends AbstractDataStream<V> + implements KeyedPartitionStream<K, V> { + + /** + * The key selector that can get the key by which the stream if partitioned from the elements. + */ + private final KeySelector<V, K> keySelector; + + /** The type of the key by which the stream is partitioned. */ + private final TypeInformation<K> keyType; + + public KeyedPartitionStreamImpl( + AbstractDataStream<V> dataStream, KeySelector<V, K> keySelector) { + this( + dataStream, + keySelector, + TypeExtractor.getKeySelectorTypes(keySelector, dataStream.getType())); + } + + public KeyedPartitionStreamImpl( + AbstractDataStream<V> dataStream, + KeySelector<V, K> keySelector, + TypeInformation<K> keyType) { + this( + dataStream, + new PartitionTransformation<>( + dataStream.getTransformation(), + new KeyGroupStreamPartitioner<>( + keySelector, + StreamGraphGenerator.DEFAULT_LOWER_BOUND_MAX_PARALLELISM)), + keySelector, + keyType); + } + + /** + * This can construct a keyed stream directly without partitionTransformation to avoid shuffle. + */ + public KeyedPartitionStreamImpl( + AbstractDataStream<V> dataStream, + Transformation<V> partitionTransformation, + KeySelector<V, K> keySelector, + TypeInformation<K> keyType) { + super(dataStream.getEnvironment(), partitionTransformation); + this.keySelector = keySelector; + this.keyType = keyType; + } + + @Override + public <OUT> NonKeyedPartitionStream<OUT> process( + OneInputStreamProcessFunction<V, OUT> processFunction) { + TypeInformation<OUT> outType; + outType = StreamUtils.getOutputTypeForOneInputProcessFunction(processFunction, getType()); + + KeyedProcessOperator<K, V, OUT> operator = new KeyedProcessOperator<>(processFunction); + Transformation<OUT> transform = + StreamUtils.getOneInputKeyedTransformation( + "KeyedProcess", this, outType, operator, keySelector, keyType); + environment.addOperator(transform); + return new NonKeyedPartitionStreamImpl<>(environment, transform); + } + + @Override + public <OUT> KeyedPartitionStream<K, OUT> process( + OneInputStreamProcessFunction<V, OUT> processFunction, + KeySelector<OUT, K> newKeySelector) { + TypeInformation<OUT> outType = + StreamUtils.getOutputTypeForOneInputProcessFunction(processFunction, getType()); + KeyedProcessOperator<K, V, OUT> operator = + new KeyedProcessOperator<>(processFunction, checkNotNull(newKeySelector)); + Transformation<OUT> transform = + StreamUtils.getOneInputKeyedTransformation( + "KeyedProcess", this, outType, operator, keySelector, keyType); + NonKeyedPartitionStreamImpl<OUT> outputStream = + new NonKeyedPartitionStreamImpl<>(environment, transform); + environment.addOperator(transform); + // Construct a keyed stream directly without partitionTransformation to avoid shuffle. + return new KeyedPartitionStreamImpl<>( + outputStream, + transform, + newKeySelector, + TypeExtractor.getKeySelectorTypes(newKeySelector, outputStream.getType())); + } + + @Override + public <OUT1, OUT2> TwoKeyedPartitionStreams<K, OUT1, OUT2> process( + TwoOutputStreamProcessFunction<V, OUT1, OUT2> processFunction, + KeySelector<OUT1, K> keySelector1, + KeySelector<OUT2, K> keySelector2) { + Tuple2<TypeInformation<OUT1>, TypeInformation<OUT2>> twoOutputType = + StreamUtils.getOutputTypesForTwoOutputProcessFunction(processFunction, getType()); + TypeInformation<OUT1> firstOutputType = twoOutputType.f0; + TypeInformation<OUT2> secondTOutputType = twoOutputType.f1; + OutputTag<OUT2> secondOutputTag = new OutputTag<>("Second-Output", secondTOutputType); Review Comment: The `OutputTag` has two constructor indeed, one with just a string(i.e. id) and one with type information. I think It only needs to be an inner class if we don't pass in type information, because type erasure is really annoying. If the type information is passed explicitly, it should not need to be an anonymous inner class: Take the following code from flink code base as example: https://github.com/apache/flink/blob/bf60c8813598d3119375cec057930240642699d4/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/PatternStream.java#L327-L328 As for the reuse, I'm not sure if this can be done because typeInformation is not always the same. 🤔 -- 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]
