pnowojski commented on code in PR #22523:
URL: https://github.com/apache/flink/pull/22523#discussion_r1186153030


##########
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/AsyncWaitOperator.java:
##########
@@ -652,4 +632,34 @@ private void timerTriggered() throws Exception {
             }
         }
     }
+
+    /** A {@link ResultFuture} that can be completed only once. */
+    private static class SafeResultFuture<OUT> implements ResultFuture<OUT> {
+
+        private final AtomicBoolean complete = new AtomicBoolean(false);
+
+        private final ResultFuture<OUT> delegate;
+
+        private SafeResultFuture(ResultFuture<OUT> delegate) {
+            this.delegate = delegate;
+        }
+
+        @Override
+        public void complete(Collection<OUT> result) {
+            if (complete.compareAndSet(false, true)) {
+                delegate.complete(result);
+            } else {
+                LOG.warn("The result future has been completed before.");
+            }
+        }
+
+        @Override
+        public void completeExceptionally(Throwable error) {
+            if (complete.compareAndSet(false, true)) {
+                delegate.completeExceptionally(error);
+            } else {
+                LOG.warn("The result future has been completed before.");
+            }
+        }
+    }

Review Comment:
   1. Why do we need another wrapper? Can not this be squashed with 
`RetryableResultHandlerDelegator`?
   2. What's the difference between this, and `ResultHandler#completed` or 
`RetryableResultHandlerDelegator#retryAwaiting`? Both are `AtomicBoolean`s 
whose javadocs are suggesting they are trying to do the same thing as yours 
`SafeResultFuture#complete`.



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