Class KafkaFuture<T>

  • All Implemented Interfaces:
    java.util.concurrent.Future<T>

    @Evolving
    public abstract class KafkaFuture<T>
    extends java.lang.Object
    implements java.util.concurrent.Future<T>
    A flexible future which supports call chaining and other asynchronous programming patterns. This will eventually become a thin shim on top of Java 8's CompletableFuture. The API for this class is still evolving and we may break compatibility in minor releases, if necessary.
    • Nested Class Summary

      Nested Classes 
      Modifier and Type Class Description
      static interface  KafkaFuture.BaseFunction<A,​B>
      A function which takes objects of type A and returns objects of type B.
      static interface  KafkaFuture.BiConsumer<A,​B>
      A consumer of two different types of object.
      static class  KafkaFuture.Function<A,​B>
      A function which takes objects of type A and returns objects of type B.
    • Constructor Summary

      Constructors 
      Constructor Description
      KafkaFuture()  
    • Method Summary

      All Methods Static Methods Instance Methods Abstract Methods Concrete Methods 
      Modifier and Type Method Description
      protected abstract void addWaiter​(KafkaFuture.BiConsumer<? super T,​? super java.lang.Throwable> action)  
      static KafkaFuture<java.lang.Void> allOf​(KafkaFuture<?>... futures)
      Returns a new KafkaFuture that is completed when all the given futures have completed.
      abstract boolean cancel​(boolean mayInterruptIfRunning)
      If not already completed, completes this future with a CancellationException.
      protected abstract boolean complete​(T newValue)
      If not already completed, sets the value returned by get() and related methods to the given value.
      static <U> KafkaFuture<U> completedFuture​(U value)
      Returns a new KafkaFuture that is already completed with the given value.
      protected abstract boolean completeExceptionally​(java.lang.Throwable newException)
      If not already completed, causes invocations of get() and related methods to throw the given exception.
      abstract T get()
      Waits if necessary for this future to complete, and then returns its result.
      abstract T get​(long timeout, java.util.concurrent.TimeUnit unit)
      Waits if necessary for at most the given time for this future to complete, and then returns its result, if available.
      abstract T getNow​(T valueIfAbsent)
      Returns the result value (or throws any encountered exception) if completed, else returns the given valueIfAbsent.
      abstract boolean isCancelled()
      Returns true if this CompletableFuture was cancelled before it completed normally.
      abstract boolean isCompletedExceptionally()
      Returns true if this CompletableFuture completed exceptionally, in any way.
      abstract boolean isDone()
      Returns true if completed in any fashion: normally, exceptionally, or via cancellation.
      abstract <R> KafkaFuture<R> thenApply​(KafkaFuture.BaseFunction<T,​R> function)
      Returns a new KafkaFuture that, when this future completes normally, is executed with this futures's result as the argument to the supplied function.
      abstract <R> KafkaFuture<R> thenApply​(KafkaFuture.Function<T,​R> function)  
      abstract KafkaFuture<T> whenComplete​(KafkaFuture.BiConsumer<? super T,​? super java.lang.Throwable> action)
      Returns a new KafkaFuture with the same result or exception as this future, that executes the given action when this future completes.
      • Methods inherited from class java.lang.Object

        clone, equals, finalize, getClass, hashCode, notify, notifyAll, toString, wait, wait, wait
    • Constructor Detail

      • KafkaFuture

        public KafkaFuture()
    • Method Detail

      • completedFuture

        public static <U> KafkaFuture<U> completedFuture​(U value)
        Returns a new KafkaFuture that is already completed with the given value.
      • allOf

        public static KafkaFuture<java.lang.Void> allOf​(KafkaFuture<?>... futures)
        Returns a new KafkaFuture that is completed when all the given futures have completed. If any future throws an exception, the returned future returns it. If multiple futures throw an exception, which one gets returned is arbitrarily chosen.
      • thenApply

        public abstract <R> KafkaFuture<R> thenApply​(KafkaFuture.BaseFunction<T,​R> function)
        Returns a new KafkaFuture that, when this future completes normally, is executed with this futures's result as the argument to the supplied function. The function may be invoked by the thread that calls thenApply or it may be invoked by the thread that completes the future.
      • whenComplete

        public abstract KafkaFuture<T> whenComplete​(KafkaFuture.BiConsumer<? super T,​? super java.lang.Throwable> action)
        Returns a new KafkaFuture with the same result or exception as this future, that executes the given action when this future completes. When this future is done, the given action is invoked with the result (or null if none) and the exception (or null if none) of this future as arguments. The returned future is completed when the action returns. The supplied action should not throw an exception. However, if it does, the following rules apply: if this future completed normally but the supplied action throws an exception, then the returned future completes exceptionally with the supplied action's exception. Or, if this future completed exceptionally and the supplied action throws an exception, then the returned future completes exceptionally with this future's exception. The action may be invoked by the thread that calls whenComplete or it may be invoked by the thread that completes the future.
        Parameters:
        action - the action to preform
        Returns:
        the new future
      • addWaiter

        protected abstract void addWaiter​(KafkaFuture.BiConsumer<? super T,​? super java.lang.Throwable> action)
      • complete

        protected abstract boolean complete​(T newValue)
        If not already completed, sets the value returned by get() and related methods to the given value.
      • completeExceptionally

        protected abstract boolean completeExceptionally​(java.lang.Throwable newException)
        If not already completed, causes invocations of get() and related methods to throw the given exception.
      • cancel

        public abstract boolean cancel​(boolean mayInterruptIfRunning)
        If not already completed, completes this future with a CancellationException. Dependent futures that have not already completed will also complete exceptionally, with a CompletionException caused by this CancellationException.
        Specified by:
        cancel in interface java.util.concurrent.Future<T>
      • get

        public abstract T get()
                       throws java.lang.InterruptedException,
                              java.util.concurrent.ExecutionException
        Waits if necessary for this future to complete, and then returns its result.
        Specified by:
        get in interface java.util.concurrent.Future<T>
        Throws:
        java.lang.InterruptedException
        java.util.concurrent.ExecutionException
      • get

        public abstract T get​(long timeout,
                              java.util.concurrent.TimeUnit unit)
                       throws java.lang.InterruptedException,
                              java.util.concurrent.ExecutionException,
                              java.util.concurrent.TimeoutException
        Waits if necessary for at most the given time for this future to complete, and then returns its result, if available.
        Specified by:
        get in interface java.util.concurrent.Future<T>
        Throws:
        java.lang.InterruptedException
        java.util.concurrent.ExecutionException
        java.util.concurrent.TimeoutException
      • getNow

        public abstract T getNow​(T valueIfAbsent)
                          throws java.lang.InterruptedException,
                                 java.util.concurrent.ExecutionException
        Returns the result value (or throws any encountered exception) if completed, else returns the given valueIfAbsent.
        Throws:
        java.lang.InterruptedException
        java.util.concurrent.ExecutionException
      • isCancelled

        public abstract boolean isCancelled()
        Returns true if this CompletableFuture was cancelled before it completed normally.
        Specified by:
        isCancelled in interface java.util.concurrent.Future<T>
      • isCompletedExceptionally

        public abstract boolean isCompletedExceptionally()
        Returns true if this CompletableFuture completed exceptionally, in any way.
      • isDone

        public abstract boolean isDone()
        Returns true if completed in any fashion: normally, exceptionally, or via cancellation.
        Specified by:
        isDone in interface java.util.concurrent.Future<T>