reswqa commented on code in PR #24398:
URL: https://github.com/apache/flink/pull/24398#discussion_r1519327670


##########
flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/KeyedPartitionWindowedStreamITCase.java:
##########
@@ -0,0 +1,521 @@
+/*
+ * 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.test.streaming.runtime;
+
+import org.apache.flink.api.common.functions.AggregateFunction;
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.common.functions.MapPartitionFunction;
+import org.apache.flink.api.common.functions.ReduceFunction;
+import org.apache.flink.api.common.operators.Order;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.streaming.api.datastream.DataStreamSource;
+import org.apache.flink.streaming.api.datastream.KeyedPartitionWindowedStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.util.CloseableIterator;
+import org.apache.flink.util.Collector;
+
+import org.apache.flink.shaded.guava31.com.google.common.collect.Lists;
+
+import org.junit.jupiter.api.Test;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
+/** Integration tests for {@link KeyedPartitionWindowedStream}. */
+public class KeyedPartitionWindowedStreamITCase {
+
+    private static final int EVENT_NUMBER = 100;
+
+    private static final String TEST_EVENT = "Test";
+
+    @Test
+    public void testMapPartition() throws Exception {

Review Comment:
   ```suggestion
   void testMapPartition() throws Exception {
   ```
   
   Please let it package-private for junit5 backed testing class and suite. We 
need to review all newly introduced tests.



##########
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/MapPartitionOperator.java:
##########
@@ -0,0 +1,77 @@
+/*
+ * 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.streaming.api.operators;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.common.functions.MapPartitionFunction;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.util.ExceptionUtils;
+
+/**
+ * The {@link MapPartitionOperator} is used to process all records in each 
partition on non-keyed
+ * stream. Each partition contains all records of a subtask.
+ */
+@Internal
+public class MapPartitionOperator<IN, OUT>
+        extends AbstractUdfStreamOperator<OUT, MapPartitionFunction<IN, OUT>>
+        implements OneInputStreamOperator<IN, OUT>, BoundedOneInput {
+
+    private final MapPartitionFunction<IN, OUT> function;
+
+    private MapPartitionIterator<IN> iterator;
+
+    public MapPartitionOperator(MapPartitionFunction<IN, OUT> function) {
+        super(function);
+        this.function = function;
+        // This operator is set to be non-chained as it doesn't use task main 
thread to write
+        // records to output, which may introduce risks to downstream chained 
operators.
+        this.chainingStrategy = ChainingStrategy.NEVER;

Review Comment:
   IIUC, this prevents this operator from being chained with the previous 
operator, we do not want `mapPartition` to introduce shuffle stage here. If 
this is the case, we may need a new chaining strategy(Not thinking it through). 



##########
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/PartitionAggregateOperator.java:
##########
@@ -0,0 +1,66 @@
+/*
+ * 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.streaming.api.operators;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.common.functions.AggregateFunction;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * The {@link PartitionAggregateOperator} is used to apply the aggregate 
transformation on all
+ * records of each partition. Each partition contains all records of a subtask.
+ */
+@Internal
+public class PartitionAggregateOperator<IN, ACC, OUT>
+        extends AbstractUdfStreamOperator<OUT, AggregateFunction<IN, ACC, OUT>>
+        implements OneInputStreamOperator<IN, OUT>, BoundedOneInput {
+
+    private final AggregateFunction<IN, ACC, OUT> aggregateFunction;
+
+    private ACC currentAccumulator = null;
+
+    public PartitionAggregateOperator(AggregateFunction<IN, ACC, OUT> 
aggregateFunction) {
+        super(aggregateFunction);
+        this.aggregateFunction = aggregateFunction;
+    }
+
+    @Override
+    public void open() throws Exception {
+        super.open();
+        this.currentAccumulator = 
checkNotNull(aggregateFunction.createAccumulator());
+    }
+
+    @Override
+    public void processElement(StreamRecord<IN> element) throws Exception {
+        aggregateFunction.add(element.getValue(), currentAccumulator);
+    }
+
+    @Override
+    public void endInput() throws Exception {
+        TimestampedCollector<OUT> outputCollector = new 
TimestampedCollector<>(output);

Review Comment:
   Why do we need to wrap `output` here?



##########
flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/MapPartitionIteratorTest.java:
##########
@@ -0,0 +1,86 @@
+/*
+ * 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.streaming.api.operators;
+
+import org.junit.jupiter.api.Test;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+
+import static org.assertj.core.api.AssertionsForClassTypes.assertThat;
+import static org.assertj.core.api.AssertionsForClassTypes.assertThatThrownBy;
+
+/** Unit test for {@link MapPartitionIterator}. */
+class MapPartitionIteratorTest {

Review Comment:
   I believe there are too many cases we haven't been tested for, especially 
since this data structure is unsafe-prone.



##########
flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/KeyedPartitionWindowedStreamITCase.java:
##########
@@ -0,0 +1,521 @@
+/*
+ * 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.test.streaming.runtime;
+
+import org.apache.flink.api.common.functions.AggregateFunction;
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.common.functions.MapPartitionFunction;
+import org.apache.flink.api.common.functions.ReduceFunction;
+import org.apache.flink.api.common.operators.Order;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.streaming.api.datastream.DataStreamSource;
+import org.apache.flink.streaming.api.datastream.KeyedPartitionWindowedStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.util.CloseableIterator;
+import org.apache.flink.util.Collector;
+
+import org.apache.flink.shaded.guava31.com.google.common.collect.Lists;
+
+import org.junit.jupiter.api.Test;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;

Review Comment:
   AssertJ is the only legitimate assertion.



##########
flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/NonKeyedPartitionWindowedStreamITCase.java:
##########
@@ -0,0 +1,366 @@
+/*
+ * 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.test.streaming.runtime;
+
+import org.apache.flink.api.common.functions.AggregateFunction;
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.common.functions.MapPartitionFunction;
+import org.apache.flink.api.common.functions.ReduceFunction;
+import org.apache.flink.api.common.operators.Order;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.streaming.api.datastream.DataStreamSource;
+import 
org.apache.flink.streaming.api.datastream.NonKeyedPartitionWindowedStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.util.CloseableIterator;
+import org.apache.flink.util.Collector;
+
+import org.apache.flink.shaded.guava31.com.google.common.collect.Lists;
+
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
+/** Integration tests for {@link NonKeyedPartitionWindowedStream}. */
+public class NonKeyedPartitionWindowedStreamITCase {

Review Comment:
   ```suggestion
   class NonKeyedPartitionWindowedStreamITCase {
   ```



##########
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/sortpartition/FixedLengthByteKeyAndValueComparator.java:
##########
@@ -0,0 +1,176 @@
+/*
+ * 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.streaming.api.operators.sortpartition;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.common.typeutils.TypeComparator;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.core.memory.DataInputView;
+import org.apache.flink.core.memory.DataOutputView;
+import org.apache.flink.core.memory.MemorySegment;
+
+import java.io.IOException;
+import java.util.Arrays;
+
+/**
+ * The {@link FixedLengthByteKeyAndValueComparator} is used by {@link 
KeyedSortPartitionOperator} to
+ * compare records according to both the record key and record value. The 
length of record key must
+ * be fixed and will be initialized when the {@link 
FixedLengthByteKeyAndValueComparator} is
+ * created.
+ */
+@Internal
+public class FixedLengthByteKeyAndValueComparator<INPUT>
+        extends TypeComparator<Tuple2<byte[], INPUT>> {
+
+    private final int serializedKeyLength;
+
+    private final TypeComparator<INPUT> valueComparator;
+
+    private byte[] keyReference;
+
+    private INPUT valueReference;
+
+    FixedLengthByteKeyAndValueComparator(
+            int serializedKeyLength, TypeComparator<INPUT> valueComparator) {
+        this.serializedKeyLength = serializedKeyLength;
+        this.valueComparator = valueComparator;
+    }
+
+    @Override
+    public int hash(Tuple2<byte[], INPUT> record) {
+        return record.hashCode();
+    }
+
+    @Override
+    public void setReference(Tuple2<byte[], INPUT> toCompare) {
+        this.keyReference = toCompare.f0;
+        this.valueReference = toCompare.f1;
+    }
+
+    @Override
+    public boolean equalToReference(Tuple2<byte[], INPUT> candidate) {
+        return Arrays.equals(keyReference, candidate.f0) && valueReference == 
candidate.f1;
+    }
+
+    @Override
+    public int compareToReference(TypeComparator<Tuple2<byte[], INPUT>> 
referencedComparator) {
+        byte[] otherKey =
+                ((FixedLengthByteKeyAndValueComparator<INPUT>) 
referencedComparator).keyReference;
+        INPUT otherValue =
+                ((FixedLengthByteKeyAndValueComparator<INPUT>) 
referencedComparator).valueReference;
+        int keyCmp = compare(otherKey, this.keyReference);
+        if (keyCmp != 0) {
+            return keyCmp;
+        }
+        return valueComparator.compare(this.valueReference, otherValue);
+    }
+
+    @Override
+    public int compare(Tuple2<byte[], INPUT> first, Tuple2<byte[], INPUT> 
second) {
+        int keyCmp = compare(first.f0, second.f0);
+        if (keyCmp != 0) {
+            return keyCmp;
+        }
+        return valueComparator.compare(first.f1, second.f1);
+    }
+
+    private int compare(byte[] first, byte[] second) {

Review Comment:
   ```suggestion
       private int compareKey(byte[] first, byte[] second) {
   ```



##########
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/MapPartitionOperator.java:
##########
@@ -0,0 +1,77 @@
+/*
+ * 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.streaming.api.operators;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.common.functions.MapPartitionFunction;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.util.ExceptionUtils;
+
+/**
+ * The {@link MapPartitionOperator} is used to process all records in each 
partition on non-keyed
+ * stream. Each partition contains all records of a subtask.
+ */
+@Internal
+public class MapPartitionOperator<IN, OUT>
+        extends AbstractUdfStreamOperator<OUT, MapPartitionFunction<IN, OUT>>
+        implements OneInputStreamOperator<IN, OUT>, BoundedOneInput {
+
+    private final MapPartitionFunction<IN, OUT> function;
+
+    private MapPartitionIterator<IN> iterator;

Review Comment:
   ```suggestion
       private transient MapPartitionIterator<IN> iterator;
   ```



##########
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/sortpartition/FixedLengthByteKeyAndValueComparator.java:
##########
@@ -0,0 +1,176 @@
+/*
+ * 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.streaming.api.operators.sortpartition;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.common.typeutils.TypeComparator;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.core.memory.DataInputView;
+import org.apache.flink.core.memory.DataOutputView;
+import org.apache.flink.core.memory.MemorySegment;
+
+import java.io.IOException;
+import java.util.Arrays;
+
+/**
+ * The {@link FixedLengthByteKeyAndValueComparator} is used by {@link 
KeyedSortPartitionOperator} to
+ * compare records according to both the record key and record value. The 
length of record key must
+ * be fixed and will be initialized when the {@link 
FixedLengthByteKeyAndValueComparator} is
+ * created.
+ */
+@Internal
+public class FixedLengthByteKeyAndValueComparator<INPUT>
+        extends TypeComparator<Tuple2<byte[], INPUT>> {
+
+    private final int serializedKeyLength;
+
+    private final TypeComparator<INPUT> valueComparator;
+
+    private byte[] keyReference;
+
+    private INPUT valueReference;
+
+    FixedLengthByteKeyAndValueComparator(
+            int serializedKeyLength, TypeComparator<INPUT> valueComparator) {
+        this.serializedKeyLength = serializedKeyLength;
+        this.valueComparator = valueComparator;
+    }
+
+    @Override
+    public int hash(Tuple2<byte[], INPUT> record) {
+        return record.hashCode();
+    }
+
+    @Override
+    public void setReference(Tuple2<byte[], INPUT> toCompare) {
+        this.keyReference = toCompare.f0;
+        this.valueReference = toCompare.f1;
+    }
+
+    @Override
+    public boolean equalToReference(Tuple2<byte[], INPUT> candidate) {
+        return Arrays.equals(keyReference, candidate.f0) && valueReference == 
candidate.f1;
+    }
+
+    @Override
+    public int compareToReference(TypeComparator<Tuple2<byte[], INPUT>> 
referencedComparator) {
+        byte[] otherKey =
+                ((FixedLengthByteKeyAndValueComparator<INPUT>) 
referencedComparator).keyReference;
+        INPUT otherValue =
+                ((FixedLengthByteKeyAndValueComparator<INPUT>) 
referencedComparator).valueReference;
+        int keyCmp = compare(otherKey, this.keyReference);
+        if (keyCmp != 0) {
+            return keyCmp;
+        }
+        return valueComparator.compare(this.valueReference, otherValue);
+    }
+
+    @Override
+    public int compare(Tuple2<byte[], INPUT> first, Tuple2<byte[], INPUT> 
second) {
+        int keyCmp = compare(first.f0, second.f0);
+        if (keyCmp != 0) {
+            return keyCmp;
+        }
+        return valueComparator.compare(first.f1, second.f1);
+    }
+
+    private int compare(byte[] first, byte[] second) {
+        for (int i = 0; i < serializedKeyLength; i++) {
+            int cmp = Byte.compare(first[i], second[i]);
+            if (cmp != 0) {
+                return cmp < 0 ? -1 : 1;
+            }
+        }
+        return 0;
+    }
+
+    @Override
+    public int compareSerialized(DataInputView firstSource, DataInputView 
secondSource)
+            throws IOException {
+        int minCount = serializedKeyLength;
+        while (minCount-- > 0) {
+            byte firstValue = firstSource.readByte();
+            byte secondValue = secondSource.readByte();
+            int cmp = Byte.compare(firstValue, secondValue);
+            if (cmp != 0) {
+                return cmp < 0 ? -1 : 1;
+            }
+        }
+        return valueComparator.compareSerialized(firstSource, secondSource);
+    }
+
+    @Override
+    public boolean supportsNormalizedKey() {

Review Comment:
   I'm kind of curious: What is `NormalizedKey`?



##########
flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/PartitionAggregateOperatorTest.java:
##########
@@ -0,0 +1,128 @@
+/*
+ * 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.streaming.api.operators;
+
+import org.apache.flink.api.common.functions.AggregateFunction;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.execution.Environment;
+import org.apache.flink.runtime.operators.testutils.MockEnvironment;
+import org.apache.flink.streaming.api.graph.StreamConfig;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.streaming.runtime.tasks.StreamTask;
+import org.apache.flink.streaming.util.MockOutput;
+import org.apache.flink.streaming.util.MockStreamConfig;
+
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.assertj.core.api.AssertionsForClassTypes.assertThat;
+import static org.junit.jupiter.api.Assertions.assertDoesNotThrow;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
+/** Unit test for {@link PartitionAggregateOperator}. */
+class PartitionAggregateOperatorTest {
+
+    /** The test environment. */
+    private Environment environment;
+
+    /** The test stream task. */
+    private StreamTask<?, ?> containingTask;
+
+    /** The test stream config. */
+    private StreamConfig config;
+
+    @BeforeEach
+    void before() throws Exception {
+        environment = MockEnvironment.builder().build();
+        containingTask =
+                new StreamTask<Object, StreamOperator<Object>>(environment) {
+                    @Override
+                    protected void init() {}
+                };
+        config = new MockStreamConfig(new Configuration(), 1);
+    }
+
+    @Test
+    void testOpen() {
+        PartitionAggregateOperator<Integer, TestAccumulator, String> 
partitionAggregateOperator =
+                createPartitionAggregateOperator();
+        MockOutput<String> output = new MockOutput<>(new ArrayList<>());
+        partitionAggregateOperator.setup(containingTask, config, output);
+        assertDoesNotThrow(partitionAggregateOperator::open);
+    }
+
+    @Test
+    void testProcessElement() throws Exception {

Review Comment:
   Could we rewrite all `xxxOperatorTests` with the help of 
`OneInputStreamOperatorTestHarness`?
   
   



##########
flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/NonKeyedPartitionWindowedStreamITCase.java:
##########
@@ -0,0 +1,366 @@
+/*
+ * 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.test.streaming.runtime;
+
+import org.apache.flink.api.common.functions.AggregateFunction;
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.common.functions.MapPartitionFunction;
+import org.apache.flink.api.common.functions.ReduceFunction;
+import org.apache.flink.api.common.operators.Order;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.streaming.api.datastream.DataStreamSource;
+import 
org.apache.flink.streaming.api.datastream.NonKeyedPartitionWindowedStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.util.CloseableIterator;
+import org.apache.flink.util.Collector;
+
+import org.apache.flink.shaded.guava31.com.google.common.collect.Lists;
+
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
+/** Integration tests for {@link NonKeyedPartitionWindowedStream}. */
+public class NonKeyedPartitionWindowedStreamITCase {
+
+    private static final int EVENT_NUMBER = 100;
+
+    private static final String TEST_EVENT = "Test";
+
+    @Test
+    public void testMapPartition() throws Exception {

Review Comment:
   ```suggestion
   void testMapPartition() throws Exception {
   ```



##########
flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/KeyedPartitionWindowedStreamITCase.java:
##########
@@ -0,0 +1,521 @@
+/*
+ * 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.test.streaming.runtime;
+
+import org.apache.flink.api.common.functions.AggregateFunction;
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.common.functions.MapPartitionFunction;
+import org.apache.flink.api.common.functions.ReduceFunction;
+import org.apache.flink.api.common.operators.Order;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.streaming.api.datastream.DataStreamSource;
+import org.apache.flink.streaming.api.datastream.KeyedPartitionWindowedStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.util.CloseableIterator;
+import org.apache.flink.util.Collector;
+
+import org.apache.flink.shaded.guava31.com.google.common.collect.Lists;
+
+import org.junit.jupiter.api.Test;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
+/** Integration tests for {@link KeyedPartitionWindowedStream}. */
+public class KeyedPartitionWindowedStreamITCase {

Review Comment:
   ```suggestion 
   class KeyedPartitionWindowedStreamITCase {
   ```



##########
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/MapPartitionOperator.java:
##########
@@ -0,0 +1,77 @@
+/*
+ * 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.streaming.api.operators;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.common.functions.MapPartitionFunction;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.util.ExceptionUtils;
+
+/**
+ * The {@link MapPartitionOperator} is used to process all records in each 
partition on non-keyed
+ * stream. Each partition contains all records of a subtask.
+ */
+@Internal
+public class MapPartitionOperator<IN, OUT>
+        extends AbstractUdfStreamOperator<OUT, MapPartitionFunction<IN, OUT>>
+        implements OneInputStreamOperator<IN, OUT>, BoundedOneInput {
+
+    private final MapPartitionFunction<IN, OUT> function;
+
+    private MapPartitionIterator<IN> iterator;
+
+    public MapPartitionOperator(MapPartitionFunction<IN, OUT> function) {
+        super(function);
+        this.function = function;
+        // This operator is set to be non-chained as it doesn't use task main 
thread to write
+        // records to output, which may introduce risks to downstream chained 
operators.
+        this.chainingStrategy = ChainingStrategy.NEVER;
+    }
+
+    @Override
+    public void open() throws Exception {
+        super.open();
+        this.iterator = new MapPartitionIterator<>();

Review Comment:
   Can we make the UDF a parameter of the constructor of `MapPartitionIterator` 
and get rid of `registerUDF`.



##########
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/sortpartition/FixedLengthByteKeyAndValueComparator.java:
##########
@@ -0,0 +1,176 @@
+/*
+ * 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.streaming.api.operators.sortpartition;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.common.typeutils.TypeComparator;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.core.memory.DataInputView;
+import org.apache.flink.core.memory.DataOutputView;
+import org.apache.flink.core.memory.MemorySegment;
+
+import java.io.IOException;
+import java.util.Arrays;
+
+/**
+ * The {@link FixedLengthByteKeyAndValueComparator} is used by {@link 
KeyedSortPartitionOperator} to
+ * compare records according to both the record key and record value. The 
length of record key must
+ * be fixed and will be initialized when the {@link 
FixedLengthByteKeyAndValueComparator} is
+ * created.
+ */
+@Internal
+public class FixedLengthByteKeyAndValueComparator<INPUT>
+        extends TypeComparator<Tuple2<byte[], INPUT>> {
+
+    private final int serializedKeyLength;
+
+    private final TypeComparator<INPUT> valueComparator;
+
+    private byte[] keyReference;
+
+    private INPUT valueReference;
+
+    FixedLengthByteKeyAndValueComparator(
+            int serializedKeyLength, TypeComparator<INPUT> valueComparator) {
+        this.serializedKeyLength = serializedKeyLength;
+        this.valueComparator = valueComparator;
+    }
+
+    @Override
+    public int hash(Tuple2<byte[], INPUT> record) {
+        return record.hashCode();
+    }
+
+    @Override
+    public void setReference(Tuple2<byte[], INPUT> toCompare) {
+        this.keyReference = toCompare.f0;
+        this.valueReference = toCompare.f1;
+    }
+
+    @Override
+    public boolean equalToReference(Tuple2<byte[], INPUT> candidate) {
+        return Arrays.equals(keyReference, candidate.f0) && valueReference == 
candidate.f1;

Review Comment:
   I'm not sure if `==` or `equals` should be used here for `valueReference`, 
can you explain it?



##########
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/MapPartitionOperator.java:
##########
@@ -0,0 +1,77 @@
+/*
+ * 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.streaming.api.operators;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.common.functions.MapPartitionFunction;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.util.ExceptionUtils;
+
+/**
+ * The {@link MapPartitionOperator} is used to process all records in each 
partition on non-keyed
+ * stream. Each partition contains all records of a subtask.
+ */
+@Internal
+public class MapPartitionOperator<IN, OUT>
+        extends AbstractUdfStreamOperator<OUT, MapPartitionFunction<IN, OUT>>
+        implements OneInputStreamOperator<IN, OUT>, BoundedOneInput {
+
+    private final MapPartitionFunction<IN, OUT> function;
+
+    private MapPartitionIterator<IN> iterator;
+
+    public MapPartitionOperator(MapPartitionFunction<IN, OUT> function) {
+        super(function);
+        this.function = function;
+        // This operator is set to be non-chained as it doesn't use task main 
thread to write
+        // records to output, which may introduce risks to downstream chained 
operators.
+        this.chainingStrategy = ChainingStrategy.NEVER;
+    }
+
+    @Override
+    public void open() throws Exception {
+        super.open();
+        this.iterator = new MapPartitionIterator<>();
+        this.iterator.registerUDF(
+                iterator -> {
+                    TimestampedCollector<OUT> outputCollector = new 
TimestampedCollector<>(output);
+                    try {
+                        function.mapPartition(() -> iterator, outputCollector);
+                    } catch (Exception e) {
+                        ExceptionUtils.rethrow(e);
+                    }
+                });
+    }
+
+    @Override
+    public void processElement(StreamRecord<IN> element) throws Exception {
+        iterator.addRecord(element.getValue());
+    }
+
+    @Override
+    public void endInput() throws Exception {
+        iterator.close();
+        output.close();

Review Comment:
   Why does this output need to close manually at `endInput`.



##########
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/MapPartitionIterator.java:
##########
@@ -0,0 +1,215 @@
+/*
+ * 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.streaming.api.operators;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.common.functions.MapPartitionFunction;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.concurrent.ExecutorThreadFactory;
+
+import javax.annotation.concurrent.GuardedBy;
+
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.Queue;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Executors;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.function.Consumer;
+import java.util.function.Supplier;
+
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * The {@link MapPartitionIterator} is an iterator used in the {@link 
MapPartitionOperator}.The task
+ * main thread will add records to it. It will set itself as the input 
parameter of {@link
+ * MapPartitionFunction} and execute the function.
+ */
+@Internal
+public class MapPartitionIterator<IN> implements Iterator<IN> {
+
+    /**
+     * Max number of caches.
+     *
+     * <p>The constant defines the maximum number of caches that can be 
created. Its value is set to
+     * 100, which is considered sufficient for most parallel jobs. Each cache 
is a record and
+     * occupies a minimal amount of memory so the value is not excessively 
large.
+     */
+    private static final int DEFAULT_MAX_CACHE_NUM = 100;
+
+    /** The lock to ensure consistency between task main thread and udf 
executor. */
+    private final Lock lock = new ReentrantLock();
+
+    /** The queue to store record caches. */
+    @GuardedBy("lock")
+    private final Queue<IN> cacheQueue = new LinkedList<>();
+
+    /** The condition to indicate the cache queue is not empty. */
+    private final Condition cacheNotEmpty = lock.newCondition();
+
+    /** The condition to indicate the cache queue is not full. */
+    private final Condition cacheNotFull = lock.newCondition();
+
+    /** The condition to indicate the udf is finished. */
+    private final Condition udfFinish = lock.newCondition();
+
+    /** The task udf executor. */
+    private final Executor udfExecutor =

Review Comment:
   This executor is never shutdown, resources are leaking all the time 😞 



-- 
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: issues-unsubscr...@flink.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


Reply via email to