1u0 commented on a change in pull request #8467: [FLINK-12535][network] Make 
CheckpointBarrierHandler non-blocking
URL: https://github.com/apache/flink/pull/8467#discussion_r287809858
 
 

 ##########
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/io/AvailabilityListener.java
 ##########
 @@ -0,0 +1,61 @@
+/*
+ * 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.runtime.io;
+
+import org.apache.flink.annotation.Internal;
+
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * Interface defining couple of essential methods for listening on data 
availability using
+ * {@link CompletableFuture}. For usage check out for example {@link 
AsyncDataInput}.
+ */
+@Internal
+public interface AvailabilityListener {
+       /**
+        * Constant that allows to avoid volatile checks {@link 
CompletableFuture#isDone()}. Check
+        * {@link #isAvailable()} for more explanation.
+        */
+       CompletableFuture<?> AVAILABLE = 
CompletableFuture.completedFuture(null);
+
+       /**
+        * @return true if is finished and for example end of input was 
reached, false otherwise.
+        */
+       boolean isFinished();
+
+       /**
+        * Check if this instance is available for further processing.
+        *
+        * <p>When hot looping to avoid volatile access in {@link 
CompletableFuture#isDone()} user of
+        * this method should do the following check:
+        * <pre>
+        * {@code
+        *      AvailabilityListener input = ...;
+        *      if (input.isAvailable() == AvailabilityListener.AVAILABLE || 
input.isAvailable().isDone()) {
+        *              // do something;
+        *      }
+        * }
+        * </pre>
+        *
+        * @return a future that is completed if there are more records 
available. If there are more
+        * records available immediately, {@link #AVAILABLE} should be 
returned. Previously returned
+        * not completed futures should become completed once there is more 
input available or if
+        * the input {@link #isFinished()}.
+        */
+       CompletableFuture<?> isAvailable();
 
 Review comment:
   Maybe you can simplify the contract of this interface, by having only one 
method that returns 3 possible states of an underlying data stream? In 
nutshell, implementations of this interface have three situations:
    * stream has finished;
    * stream has not finished, and there is an immediate data ready to be 
processed;
    * stream has not finished, but the caller needs to wait when data is ready.
   
   One way, you can apply the same trick as with `CompletableFuture<?> 
AVAILABLE = CompletableFuture.completedFuture(null);`.
   Basically, having pre-defined different constant dummy `CompletableFuture` 
that would be marker of `isFinished`:
   
   ```
   public interface AvailabilityListener {
       CompletableFuture<?> FINISHED = CompletableFuture.completedFuture(null);
       CompletableFuture<?> AVAILABLE = CompletableFuture.completedFuture(null);
   
       CompletableFuture<?> getStatusFuture();
   }
   ```

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
[email protected]


With regards,
Apache Git Services

Reply via email to