reswqa commented on code in PR #24398: URL: https://github.com/apache/flink/pull/24398#discussion_r1514029928
########## flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/PartitionWindowedStream.java: ########## @@ -0,0 +1,219 @@ +/* + * 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.datastream; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.api.common.functions.AggregateFunction; +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.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.functions.KeySelector; +import org.apache.flink.api.java.tuple.Tuple; +import org.apache.flink.api.java.typeutils.PojoTypeInfo; +import org.apache.flink.api.java.typeutils.TypeExtractor; +import org.apache.flink.core.memory.ManagedMemoryUseCase; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.operators.MapPartitionOperator; +import org.apache.flink.streaming.api.operators.PartitionAggregateOperator; +import org.apache.flink.streaming.api.operators.PartitionReduceOperator; +import org.apache.flink.streaming.api.operators.sortpartition.SortPartitionOperator; +import org.apache.flink.streaming.runtime.partitioner.ForwardPartitioner; + +import static org.apache.flink.streaming.api.operators.sortpartition.SortPartitionOperator.DEFAULT_SORTPARTITION_MANAGE_MEMORY_WEIGHT; + +/** + * {@link PartitionWindowedStream} represents a data stream that collects records of each partition + * into a separate full window. Each partition contains all records of a subtask. The window + * emission will be triggered at the end of input. + * + * @param <T> The type of the elements in this stream. + */ +@PublicEvolving +public class PartitionWindowedStream<T> { + + private final StreamExecutionEnvironment environment; + + private final DataStream<T> input; + + public PartitionWindowedStream(StreamExecutionEnvironment environment, DataStream<T> input) { + this.environment = environment; + this.input = input; + } + + /** + * Process the records of the window by {@link MapPartitionFunction}. + * + * @param mapPartitionFunction The map partition function. + * @param <R> The type of map partition result. + * @return The data stream with map partition result. + */ + public <R> SingleOutputStreamOperator<R> mapPartition( + MapPartitionFunction<T, R> mapPartitionFunction) { + if (mapPartitionFunction == null) { + throw new NullPointerException("The map partition function must not be null."); + } Review Comment: `checkNotNull` would be more better. ########## flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InternalAsyncIteratorImpl.java: ########## @@ -0,0 +1,196 @@ +/* + * 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.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 default implementation of {@link InternalAsyncIterator}. */ +@Internal +public class InternalAsyncIteratorImpl<IN> implements InternalAsyncIterator<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 condition of lock. */ + private final Condition condition = lock.newCondition(); + + /** The task udf executor. */ + private final Executor udfExecutor = + Executors.newFixedThreadPool(1, new ExecutorThreadFactory("TaskUDFExecutor")); + + /** The queue to store record caches. */ + @GuardedBy("lock") + private final Queue<IN> recordCaches = new LinkedList<>(); + + /** The flag to represent the finished state of udf. */ + @GuardedBy("lock") + private boolean isUDFFinished = false; + + /** The flag to represent the closed state of this iterator. */ + @GuardedBy("lock") + private boolean isClosed = false; + + @Override + public boolean hasNext() { + return supplyWithLock( + () -> { + if (recordCaches.size() > 0) { + return true; + } else if (isClosed) { + return false; + } else { + waitToGetStatus(); + return hasNext(); + } + }); + } + + @Override + public IN next() { + return supplyWithLock( + () -> { + IN record; + if (recordCaches.size() > 0) { + record = recordCaches.poll(); + if (!isClosed) { + notifyStatus(); + } + return record; + } + waitToGetStatus(); + if (recordCaches.size() == 0) { + checkState(isClosed); + return null; + } + return recordCaches.poll(); + }); + } + + @Override + public void registerUDF(Consumer<Iterator<IN>> udf) { + checkState(!isClosed); Review Comment: Why this line is special(without lock but this field is guarded by lock)? ########## flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/MapPartitionOperator.java: ########## @@ -0,0 +1,95 @@ +/* + * 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.api.graph.StreamConfig; +import org.apache.flink.streaming.api.watermark.Watermark; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.streaming.runtime.tasks.StreamTask; +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 long lastWatermarkTimestamp = Long.MIN_VALUE; + + private InternalAsyncIterator<IN> iterator; + + public MapPartitionOperator(MapPartitionFunction<IN, OUT> function) { + super(function); + this.function = function; + } + + @Override + public void setup( + StreamTask<?, ?> containingTask, + StreamConfig config, + Output<StreamRecord<OUT>> output) { + super.setup(containingTask, config, output); + this.iterator = new InternalAsyncIteratorImpl<>(); + this.iterator.registerUDF( + iterator -> { + TimestampedCollector<OUT> outputCollector = new TimestampedCollector<>(output); + try { + function.mapPartition(() -> iterator, outputCollector); Review Comment: How do you guarantee emit data in the mailbox thread? Otherwise there are some risk, especially for chained operators. ########## flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InternalAsyncIteratorImpl.java: ########## @@ -0,0 +1,196 @@ +/* + * 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.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 default implementation of {@link InternalAsyncIterator}. */ +@Internal +public class InternalAsyncIteratorImpl<IN> implements InternalAsyncIterator<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 condition of lock. */ + private final Condition condition = lock.newCondition(); + + /** The task udf executor. */ + private final Executor udfExecutor = + Executors.newFixedThreadPool(1, new ExecutorThreadFactory("TaskUDFExecutor")); Review Comment: Why not use `Executors.newSingleThreadExecutor(new ExecutorThreadFactory("TaskUDFExecutor"))` here? ########## flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InternalAsyncIterator.java: ########## @@ -0,0 +1,48 @@ +/* + * 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 java.util.Iterator; +import java.util.function.Consumer; + +/** + * The {@link InternalAsyncIterator} is an iterator used in the function with {@link Iterator} input + * parameter (such as {@link MapPartitionFunction}). The task main thread will add records to it. It + * will set itself as the input parameter of udf and execute the udf. + */ +@Internal +public interface InternalAsyncIterator<IN> extends Iterator<IN> { Review Comment: I kind of feel like this is not a well-defined interface, why isn't it just an implementation class? ########## flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InternalAsyncIteratorImpl.java: ########## @@ -0,0 +1,196 @@ +/* + * 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.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 default implementation of {@link InternalAsyncIterator}. */ +@Internal +public class InternalAsyncIteratorImpl<IN> implements InternalAsyncIterator<IN> { Review Comment: I don't think it's a good idea to implement a thread-safe queue ourselves. In particular, maintaining condition variables. ########## flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/MapPartitionOperator.java: ########## @@ -0,0 +1,95 @@ +/* + * 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.api.graph.StreamConfig; +import org.apache.flink.streaming.api.watermark.Watermark; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.streaming.runtime.tasks.StreamTask; +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 long lastWatermarkTimestamp = Long.MIN_VALUE; + + private InternalAsyncIterator<IN> iterator; + + public MapPartitionOperator(MapPartitionFunction<IN, OUT> function) { + super(function); + this.function = function; + } + + @Override + public void setup( Review Comment: Could you explain why we registerUDF in `setup` instead of `open`. -- 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