Sxnan commented on code in PR #24422:
URL: https://github.com/apache/flink/pull/24422#discussion_r1540748320


##########
flink-datastream-api/src/main/java/org/apache/flink/datastream/api/ExecutionEnvironment.java:
##########
@@ -0,0 +1,53 @@
+/*
+ * 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.api;
+
+import org.apache.flink.annotation.Experimental;
+import org.apache.flink.api.common.RuntimeExecutionMode;
+
+/**
+ * This is the context in which a program is executed.
+ *
+ * <p>The environment provides methods to create a DataStream and control the 
job execution.
+ */
+@Experimental
+public interface ExecutionEnvironment {
+    /**
+     * Get the execution environment instance.
+     *
+     * @return A {@link ExecutionEnvironment} instance.
+     */
+    static ExecutionEnvironment getInstance() throws 
ReflectiveOperationException {

Review Comment:
   This should be `getExecutionEnvironment` according to the FLIP.



##########
flink-datastream/src/main/java/org/apache/flink/datastream/impl/ExecutionEnvironmentFactory.java:
##########
@@ -0,0 +1,33 @@
+/*
+ * 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;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.datastream.api.ExecutionEnvironment;
+
+/** Factory class for execution environments. */
+@FunctionalInterface
+public interface ExecutionEnvironmentFactory {
+    /**
+     * Creates a StreamExecutionEnvironment from this factory.
+     *
+     * @return A StreamExecutionEnvironment.

Review Comment:
   `StreamExecutionEnvironment` should be `ExecutionEnvironment`



##########
flink-datastream/src/main/java/org/apache/flink/datastream/impl/common/TimestampCollector.java:
##########
@@ -0,0 +1,43 @@
+/*
+ * 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.common;
+
+import org.apache.flink.datastream.api.common.Collector;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+
+/** The base {@link Collector} which take care of records timestamp. */
+public abstract class TimestampCollector<OUT> implements Collector<OUT> {
+    protected final StreamRecord<OUT> reuse = new StreamRecord<>(null);
+
+    public void setTimestamp(StreamRecord<?> timestampBase) {
+        if (timestampBase.hasTimestamp()) {
+            setAbsoluteTimestamp(timestampBase.getTimestamp());
+        } else {
+            eraseTimestamp();
+        }
+    }
+
+    public void setAbsoluteTimestamp(long timestamp) {

Review Comment:
   The name `absoluteTimestamp` is a little bit confusing to me. Maybe just 
name it `setTimestamp`. 
   
   And we may rename the `setTimestamp(StreamRecord)` to something like 
`setTiemstampFromStreamRecord`. 



##########
flink-datastream/src/main/java/org/apache/flink/datastream/impl/operators/KeyedTwoOutputProcessOperator.java:
##########
@@ -0,0 +1,109 @@
+/*
+ * 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.operators;
+
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.datastream.api.function.TwoOutputStreamProcessFunction;
+import org.apache.flink.datastream.impl.common.OutputCollector;
+import org.apache.flink.datastream.impl.common.TimestampCollector;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.OutputTag;
+
+import javax.annotation.Nullable;
+
+/** */
+public class KeyedTwoOutputProcessOperator<KEY, IN, OUT_MAIN, OUT_SIDE>
+        extends TwoOutputProcessOperator<IN, OUT_MAIN, OUT_SIDE> {
+
+    @Nullable private final KeySelector<OUT_MAIN, KEY> mainOutKeySelector;
+
+    @Nullable private final KeySelector<OUT_SIDE, KEY> sideOutKeySelector;
+
+    public KeyedTwoOutputProcessOperator(
+            TwoOutputStreamProcessFunction<IN, OUT_MAIN, OUT_SIDE> 
userFunction,
+            OutputTag<OUT_SIDE> outputTag) {
+        this(userFunction, outputTag, null, null);
+    }
+
+    public KeyedTwoOutputProcessOperator(
+            TwoOutputStreamProcessFunction<IN, OUT_MAIN, OUT_SIDE> 
userFunction,
+            OutputTag<OUT_SIDE> outputTag,
+            @Nullable KeySelector<OUT_MAIN, KEY> mainOutKeySelector,
+            @Nullable KeySelector<OUT_SIDE, KEY> sideOutKeySelector) {
+        super(userFunction, outputTag);
+        this.mainOutKeySelector = mainOutKeySelector;
+        this.sideOutKeySelector = sideOutKeySelector;
+    }
+
+    @Override
+    protected TimestampCollector<OUT_MAIN> getMainCollector() {
+        return mainOutKeySelector != null && sideOutKeySelector != null
+                ? new KeyCheckedOutputCollector<>(new 
OutputCollector<>(output), mainOutKeySelector)
+                : new OutputCollector<>(output);
+    }
+
+    @Override
+    public TimestampCollector<OUT_SIDE> getSideCollector() {
+        return mainOutKeySelector != null && sideOutKeySelector != null
+                ? new KeyCheckedOutputCollector<>(
+                        new SideOutputCollector(output), sideOutKeySelector)
+                : new SideOutputCollector(output);
+    }
+
+    private class KeyCheckedOutputCollector<T> extends TimestampCollector<T> {

Review Comment:
   Can we reuse the 
`org.apache.flink.datastream.impl.common.KeyCheckedOutputCollector`?



##########
flink-datastream/src/main/java/org/apache/flink/datastream/impl/ExecutionContextEnvironment.java:
##########
@@ -0,0 +1,111 @@
+/*
+ * 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;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.DeploymentOptions;
+import org.apache.flink.core.execution.PipelineExecutorServiceLoader;
+import org.apache.flink.datastream.api.ExecutionEnvironment;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+
+/**
+ * Special {@link ExecutionEnvironment} that will be used in cases where the 
CLI client or testing
+ * utilities create a {@link ExecutionEnvironment} that should be used when 
{@link
+ * ExecutionEnvironment#getInstance()} ()} is called.
+ */
+public class ExecutionContextEnvironment extends ExecutionEnvironmentImpl {
+    private final boolean suppressSysout;
+
+    private final boolean enforceSingleJobExecution;
+    private final Configuration clusterConfiguration;
+
+    private final boolean programConfigEnabled;
+    private final Collection<String> programConfigWildcards;

Review Comment:
   Currently, none of these are used. Do we have a plan to use these fields? Or 
should we add them when they are needed in the future?



##########
flink-datastream/src/main/java/org/apache/flink/datastream/impl/operators/KeyedTwoOutputProcessOperator.java:
##########
@@ -0,0 +1,109 @@
+/*
+ * 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.operators;
+
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.datastream.api.function.TwoOutputStreamProcessFunction;
+import org.apache.flink.datastream.impl.common.OutputCollector;
+import org.apache.flink.datastream.impl.common.TimestampCollector;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.OutputTag;
+
+import javax.annotation.Nullable;
+
+/** */
+public class KeyedTwoOutputProcessOperator<KEY, IN, OUT_MAIN, OUT_SIDE>
+        extends TwoOutputProcessOperator<IN, OUT_MAIN, OUT_SIDE> {
+
+    @Nullable private final KeySelector<OUT_MAIN, KEY> mainOutKeySelector;
+
+    @Nullable private final KeySelector<OUT_SIDE, KEY> sideOutKeySelector;
+
+    public KeyedTwoOutputProcessOperator(
+            TwoOutputStreamProcessFunction<IN, OUT_MAIN, OUT_SIDE> 
userFunction,
+            OutputTag<OUT_SIDE> outputTag) {
+        this(userFunction, outputTag, null, null);
+    }
+
+    public KeyedTwoOutputProcessOperator(
+            TwoOutputStreamProcessFunction<IN, OUT_MAIN, OUT_SIDE> 
userFunction,
+            OutputTag<OUT_SIDE> outputTag,
+            @Nullable KeySelector<OUT_MAIN, KEY> mainOutKeySelector,
+            @Nullable KeySelector<OUT_SIDE, KEY> sideOutKeySelector) {
+        super(userFunction, outputTag);
+        this.mainOutKeySelector = mainOutKeySelector;
+        this.sideOutKeySelector = sideOutKeySelector;
+    }
+
+    @Override
+    protected TimestampCollector<OUT_MAIN> getMainCollector() {
+        return mainOutKeySelector != null && sideOutKeySelector != null

Review Comment:
   IIUC, we require that the two key selectors must be both null or both 
not-null. Should we check that?



##########
flink-datastream/src/main/java/org/apache/flink/datastream/impl/ExecutionContextEnvironment.java:
##########
@@ -0,0 +1,111 @@
+/*
+ * 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;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.DeploymentOptions;
+import org.apache.flink.core.execution.PipelineExecutorServiceLoader;
+import org.apache.flink.datastream.api.ExecutionEnvironment;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+
+/**
+ * Special {@link ExecutionEnvironment} that will be used in cases where the 
CLI client or testing
+ * utilities create a {@link ExecutionEnvironment} that should be used when 
{@link
+ * ExecutionEnvironment#getInstance()} ()} is called.
+ */
+public class ExecutionContextEnvironment extends ExecutionEnvironmentImpl {
+    private final boolean suppressSysout;
+
+    private final boolean enforceSingleJobExecution;
+    private final Configuration clusterConfiguration;
+
+    private final boolean programConfigEnabled;
+    private final Collection<String> programConfigWildcards;
+
+    public ExecutionContextEnvironment(

Review Comment:
   The constructor is not used. Maybe we can remove it to keep it simple.



##########
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:
   We need to define the OutputTag as an anonymous inner class, i.e. `new 
OutputTag<>("Second-Output", secondTOutputType){}`. Maybe, we can reuse the 
same OutputTag for the second output of all the two-output operations.



##########
flink-datastream/src/main/java/org/apache/flink/datastream/impl/operators/KeyedTwoOutputProcessOperator.java:
##########
@@ -0,0 +1,109 @@
+/*
+ * 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.operators;
+
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.datastream.api.function.TwoOutputStreamProcessFunction;
+import org.apache.flink.datastream.impl.common.OutputCollector;
+import org.apache.flink.datastream.impl.common.TimestampCollector;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.OutputTag;
+
+import javax.annotation.Nullable;
+
+/** */
+public class KeyedTwoOutputProcessOperator<KEY, IN, OUT_MAIN, OUT_SIDE>
+        extends TwoOutputProcessOperator<IN, OUT_MAIN, OUT_SIDE> {
+
+    @Nullable private final KeySelector<OUT_MAIN, KEY> mainOutKeySelector;
+
+    @Nullable private final KeySelector<OUT_SIDE, KEY> sideOutKeySelector;
+
+    public KeyedTwoOutputProcessOperator(
+            TwoOutputStreamProcessFunction<IN, OUT_MAIN, OUT_SIDE> 
userFunction,
+            OutputTag<OUT_SIDE> outputTag) {
+        this(userFunction, outputTag, null, null);
+    }
+
+    public KeyedTwoOutputProcessOperator(
+            TwoOutputStreamProcessFunction<IN, OUT_MAIN, OUT_SIDE> 
userFunction,
+            OutputTag<OUT_SIDE> outputTag,
+            @Nullable KeySelector<OUT_MAIN, KEY> mainOutKeySelector,
+            @Nullable KeySelector<OUT_SIDE, KEY> sideOutKeySelector) {
+        super(userFunction, outputTag);
+        this.mainOutKeySelector = mainOutKeySelector;
+        this.sideOutKeySelector = sideOutKeySelector;
+    }
+
+    @Override
+    protected TimestampCollector<OUT_MAIN> getMainCollector() {
+        return mainOutKeySelector != null && sideOutKeySelector != null
+                ? new KeyCheckedOutputCollector<>(new 
OutputCollector<>(output), mainOutKeySelector)
+                : new OutputCollector<>(output);
+    }
+
+    @Override
+    public TimestampCollector<OUT_SIDE> getSideCollector() {
+        return mainOutKeySelector != null && sideOutKeySelector != null
+                ? new KeyCheckedOutputCollector<>(
+                        new SideOutputCollector(output), sideOutKeySelector)
+                : new SideOutputCollector(output);
+    }
+
+    private class KeyCheckedOutputCollector<T> extends TimestampCollector<T> {
+
+        private final TimestampCollector<T> outputCollector;
+
+        private final KeySelector<T, KEY> outputKeySelector;
+
+        private KeyCheckedOutputCollector(
+                TimestampCollector<T> outputCollector, KeySelector<T, KEY> 
outputKeySelector) {
+            this.outputCollector = outputCollector;
+            this.outputKeySelector = outputKeySelector;
+        }
+
+        @Override
+        public void collect(T outputRecord) {
+            checkOutputKey(outputRecord);
+            this.outputCollector.collect(outputRecord);
+        }
+
+        @Override
+        public void collectAndOverwriteTimestamp(T outputRecord, long 
timestamp) {
+            checkOutputKey(outputRecord);
+            this.outputCollector.collectAndOverwriteTimestamp(outputRecord, 
timestamp);
+        }
+
+        @SuppressWarnings("unchecked")
+        private void checkOutputKey(T outputRecord) {
+            try {
+                KEY currentKey = (KEY) getCurrentKey();
+                KEY outputKey = this.outputKeySelector.getKey(outputRecord);
+                if (!outputKey.equals(currentKey)) {
+                    throw new IllegalStateException(
+                            "Output key must equals to input key if you want 
the produced stream "
+                                    + "is keyed. ");
+                }
+            } catch (Exception e) {
+                // TODO Change Consumer to ThrowingConsumer.

Review Comment:
   I am not sure what the TODO means. 



##########
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.

Review Comment:
   The only difference with the `process` without `keySelector` is that we 
construct a keyed stream directly. Maybe we can extract a common method to be 
reused by the two methods.
   
   Same for the other methods



##########
flink-core/src/main/java/org/apache/flink/api/connector/dsv2/DataStreamV2SourceUtils.java:
##########
@@ -0,0 +1,52 @@
+/*
+ * 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.api.connector.dsv2;
+
+import org.apache.flink.annotation.Experimental;
+import org.apache.flink.util.Preconditions;
+
+import java.util.Collection;
+
+/** Utils to create the DataStream V2 supported {@link Source}. */
+@Experimental
+public final class DataStreamV2SourceUtils {
+    /**
+     * Wrap a FLIP-27 based source to a DataStream V2 supported source.
+     *
+     * @param source The FLIP-27 based source to wrap.
+     * @return The DataStream V2 supported source.
+     */
+    public static <T> Source<T> wrapSource(

Review Comment:
   The method is not used.



-- 
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]


Reply via email to