Interface AsyncFunction<IN,OUT>

Type Parameters:
IN - The type of the input elements.
OUT - The type of the returned elements.
All Superinterfaces:
org.apache.flink.api.common.functions.Function, Serializable
All Known Implementing Classes:
RichAsyncFunction

@PublicEvolving public interface AsyncFunction<IN,OUT> extends org.apache.flink.api.common.functions.Function, Serializable
A function to trigger Async I/O operation.

For each #asyncInvoke, an async io operation can be triggered, and once it has been done, the result can be collected by calling ResultFuture.complete(java.util.Collection<OUT>). For each async operation, its context is stored in the operator immediately after invoking #asyncInvoke, avoiding blocking for each stream input as long as the internal buffer is not full.

ResultFuture can be passed into callbacks or futures to collect the result data. An error can also be propagate to the async IO operator by ResultFuture.completeExceptionally(Throwable).

Callback example usage:


 public class HBaseAsyncFunc implements AsyncFunction<String, String> {

   public void asyncInvoke(String row, ResultFuture<String> result) throws Exception {
     HBaseCallback cb = new HBaseCallback(result);
     Get get = new Get(Bytes.toBytes(row));
     hbase.asyncGet(get, cb);
   }
 }
 

Future example usage:


 public class HBaseAsyncFunc implements AsyncFunction<String, String> {

   public void asyncInvoke(String row, final ResultFuture<String> result) throws Exception {
     Get get = new Get(Bytes.toBytes(row));
     ListenableFuture<Result> future = hbase.asyncGet(get);
     Futures.addCallback(future, new FutureCallback<Result>() {
       public void onSuccess(Result result) {
         List<String> ret = process(result);
         result.complete(ret);
       }
       public void onFailure(Throwable thrown) {
         result.completeExceptionally(thrown);
       }
     });
   }
 }
 
  • Method Details

    • asyncInvoke

      void asyncInvoke(IN input, ResultFuture<OUT> resultFuture) throws Exception
      Trigger async operation for each stream input.
      Parameters:
      input - element coming from an upstream task
      resultFuture - to be completed with the result data
      Throws:
      Exception - in case of a user code error. An exception will make the task fail and trigger fail-over process.
    • timeout

      default void timeout(IN input, ResultFuture<OUT> resultFuture) throws Exception
      asyncInvoke(IN, org.apache.flink.streaming.api.functions.async.ResultFuture<OUT>) timeout occurred. By default, the result future is exceptionally completed with a timeout exception.
      Parameters:
      input - element coming from an upstream task
      resultFuture - to be completed with the result data
      Throws:
      Exception