You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by pn...@apache.org on 2022/02/04 11:42:08 UTC
[flink] 03/04: [FLINK-25827][task] Move and rename MultipleInputAvailabilityHelper
This is an automated email from the ASF dual-hosted git repository.
pnowojski pushed a commit to branch release-1.14
in repository https://gitbox.apache.org/repos/asf/flink.git
commit 9bef4bc340078e38d4b33a1a2642cd8fe5052236
Author: Piotr Nowojski <pi...@gmail.com>
AuthorDate: Mon Jan 31 17:40:32 2022 +0100
[FLINK-25827][task] Move and rename MultipleInputAvailabilityHelper
---
.../io/MultipleFuturesAvailabilityHelper.java | 80 ++++++++++++++++++++++
.../runtime/io/StreamMultipleInputProcessor.java | 61 +----------------
2 files changed, 82 insertions(+), 59 deletions(-)
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/MultipleFuturesAvailabilityHelper.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/MultipleFuturesAvailabilityHelper.java
new file mode 100644
index 0000000..df3f93f
--- /dev/null
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/MultipleFuturesAvailabilityHelper.java
@@ -0,0 +1,80 @@
+/*
+ * 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.runtime.io;
+
+import org.apache.flink.annotation.Internal;
+
+import java.util.concurrent.CompletableFuture;
+
+import static org.apache.flink.util.concurrent.FutureUtils.assertNoException;
+
+/**
+ * This class is semi-thread safe. Only method {@link #notifyCompletion()} is allowed to be executed
+ * from an outside of the task thread.
+ *
+ * <p>It solves a problem of a potential memory leak as described in FLINK-25728. In short we have
+ * to ensure, that if there is one input (future) that rarely (or never) completes, that such future
+ * would not prevent previously returned combined futures (like {@link
+ * CompletableFuture#anyOf(CompletableFuture[])} from being garbage collected. Additionally, we
+ * don't want to accumulate more and more completion stages on such rarely completed future, so we
+ * are registering {@link CompletableFuture#thenRun(Runnable)} only if it has not already been done.
+ *
+ * <p>Note {@link #resetToUnAvailable()} doesn't de register previously registered futures. If
+ * future was registered in the past, but for whatever reason now it is not, such future can still
+ * complete the newly created future.
+ *
+ * <p>It might be no longer needed after upgrading to JDK9
+ * (https://bugs.openjdk.java.net/browse/JDK-8160402).
+ */
+@Internal
+public class MultipleFuturesAvailabilityHelper {
+ private final CompletableFuture<?>[] futuresToCombine;
+
+ private volatile CompletableFuture<?> availableFuture = new CompletableFuture<>();
+
+ public MultipleFuturesAvailabilityHelper(int size) {
+ futuresToCombine = new CompletableFuture[size];
+ }
+
+ /** @return combined future using anyOf logic */
+ public CompletableFuture<?> getAvailableFuture() {
+ return availableFuture;
+ }
+
+ public void resetToUnAvailable() {
+ if (availableFuture.isDone()) {
+ availableFuture = new CompletableFuture<>();
+ }
+ }
+
+ private void notifyCompletion() {
+ availableFuture.complete(null);
+ }
+
+ /**
+ * Combine {@code availabilityFuture} using anyOf logic with other previously registered
+ * futures.
+ */
+ public void anyOf(final int idx, CompletableFuture<?> availabilityFuture) {
+ if (futuresToCombine[idx] == null || futuresToCombine[idx].isDone()) {
+ futuresToCombine[idx] = availabilityFuture;
+ assertNoException(availabilityFuture.thenRun(this::notifyCompletion));
+ }
+ }
+}
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamMultipleInputProcessor.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamMultipleInputProcessor.java
index ba7e46e..1405912 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamMultipleInputProcessor.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamMultipleInputProcessor.java
@@ -28,8 +28,6 @@ import org.apache.flink.util.ExceptionUtils;
import java.io.IOException;
import java.util.concurrent.CompletableFuture;
-import static org.apache.flink.util.concurrent.FutureUtils.assertNoException;
-
/** Input processor for {@link MultipleInputStreamOperator}. */
@Internal
public final class StreamMultipleInputProcessor implements StreamInputProcessor {
@@ -38,7 +36,7 @@ public final class StreamMultipleInputProcessor implements StreamInputProcessor
private final StreamOneInputProcessor<?>[] inputProcessors;
- private final MultipleInputAvailabilityHelper availabilityHelper;
+ private final MultipleFuturesAvailabilityHelper availabilityHelper;
/** Always try to read from the first input. */
private int lastReadInputIndex = 1;
@@ -49,7 +47,7 @@ public final class StreamMultipleInputProcessor implements StreamInputProcessor
StreamOneInputProcessor<?>[] inputProcessors) {
this.inputSelectionHandler = inputSelectionHandler;
this.inputProcessors = inputProcessors;
- this.availabilityHelper = new MultipleInputAvailabilityHelper(inputProcessors.length);
+ this.availabilityHelper = new MultipleFuturesAvailabilityHelper(inputProcessors.length);
}
@Override
@@ -158,59 +156,4 @@ public final class StreamMultipleInputProcessor implements StreamInputProcessor
}
}
}
-
- /**
- * This class is semi-thread safe. Only method {@link #notifyCompletion()} is allowed to be
- * executed from an outside of the task thread.
- *
- * <p>It solves a problem of a potential memory leak as described in FLINK-25728. In short we
- * have to ensure, that if there is one input (future) that rarely (or never) completes, that
- * such future would not prevent previously returned combined futures (like {@link
- * CompletableFuture#anyOf(CompletableFuture[])} from being garbage collected. Additionally, we
- * don't want to accumulate more and more completion stages on such rarely completed future, so
- * we are registering {@link CompletableFuture#thenRun(Runnable)} only if it has not already
- * been done.
- *
- * <p>Note {@link #resetToUnAvailable()} doesn't de register previously registered futures. If
- * future was registered in the past, but for whatever reason now it is not, such future can
- * still complete the newly created future.
- *
- * <p>It might be no longer needed after upgrading to JDK9
- * (https://bugs.openjdk.java.net/browse/JDK-8160402).
- */
- private static class MultipleInputAvailabilityHelper {
- private final CompletableFuture<?>[] futuresToCombine;
-
- private volatile CompletableFuture<?> availableFuture = new CompletableFuture<>();
-
- public MultipleInputAvailabilityHelper(int inputSize) {
- futuresToCombine = new CompletableFuture[inputSize];
- }
-
- /** @return combined future using anyOf logic */
- public CompletableFuture<?> getAvailableFuture() {
- return availableFuture;
- }
-
- public void resetToUnAvailable() {
- if (availableFuture.isDone()) {
- availableFuture = new CompletableFuture<>();
- }
- }
-
- private void notifyCompletion() {
- availableFuture.complete(null);
- }
-
- /**
- * Combine {@code availabilityFuture} using anyOf logic with other previously registered
- * futures.
- */
- public void anyOf(final int idx, CompletableFuture<?> availabilityFuture) {
- if (futuresToCombine[idx] == null || futuresToCombine[idx].isDone()) {
- futuresToCombine[idx] = availabilityFuture;
- assertNoException(availabilityFuture.thenRun(this::notifyCompletion));
- }
- }
- }
}