Versions Compared

Key

  • This line was added.
  • This line was removed.
  • Formatting was changed.

Table of Contents

Status

Current state: "Under Discussion", "Accepted", "Rejected"

Discussion thread: here

Vote thread: here

JIRA:

Jira
serverASF JIRA
columnskey,summary,type,created,updated,due,assignee,reporter,priority,status,resolution
serverId5aa69414-a9e9-3523-82ec-879b028fb15b
keyKAFKA-6018

Pull request: PR#4033

Released: 1.1.0

Motivation

KafkaFuture.Function and KafkaFuture.BiCosumer are currently empty public abstract classes with only one abstract method.

This means you cannot implement use them with a java 8 lambda, as that only works for functional interfaces.

Replacing those classes with interfaces makes them lambda compatible.

Public Interfaces

From In the public interfcateinterface, only org.apache.kafka.common.KafkaFuture has such helper classes which can better be expressed as functional interfaces.

Proposed Changes

The current proposal implemented in PR#4033 makes from those abstract classes interfaces.

This is a binary incompatible change.  Which might be acceptable as KafkaFuture is annotated as InterfaceStability.Evolving, where compatibility may be broken at minor release.

This change also helps to move KafkaFuture closer to the interface of a java 8 CompletableFuture.  As KafkaFuture's javadoc states:

...

KIP introduces a FunctionInterface which defines the new functional interface.  The old Function class is kept for backwards compatibility reasons, but documented to be a candidate for deprecation or removal.  The @Deprecated was not yet added to this class. This was also done for the old signature of the thenApply function (the only public function which was using this Function object).

As the Biconsumer class was not used in the public interface, it was changed from a class to an interface.

To make it easier to work with the KafkaFuture the new public function whenComplete is introduced.  This allows the user of KafkaFuture to execute an action after the future is completed and it matches the signature and the behaviour of CompletableFuture#whenComplete.

Code Block
java
java
/**
 * 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 {@code whenComplete} or it may be invoked by the thread that
 * completes the future.
 *
 * @param action the action to preform
 * @return the new future
 */
public abstract KafkaFuture<T> whenComplete(BiConsumer<? super T, ? super Throwable> action);

 

Compatibility, Deprecation, and Migration Plan

When the change is implemented like proposed, everybody using this api will need to implement instead of extend their implementations of KafkaFuture.Function and KafkaFuture.Biconsumer.

...

In a future release the KafkaFuture.Function and thenApply function which uses this class can be marked as @Deprecated.  Later on they can be removed.

When KIP-118 is implemented, KafkaFuture can extend CompletableFuture or implement CompletionStage.

Rejected Alternatives

  • Keep the current interface as is, as a nicer syntax to address this api might not be worth breaking backwards compatibility
  • Wait for a kafka release which will not support java 7 anymore.  A backwards incompatible change can then be done to replace KafkaFuture with CompletableFutureSpend more effort and update the definition of KafkaFuture in such a way that it gradually switches to CompletableFuture.  By following an approach which looks like this:
  • Create new interface versions of KafkaFuture.Function and KafkaFuture.BiConsumer, create new methods which handle those functional interface and preferably line their names up with those of CompletableFuture.  So for example KafkaFuture.addWaiter(BiConsumer) becomes KafkaFuture.whenComplete(BiConsumerInterface).  Mark the old methods as @Deprecated.
  • When kafka does not support java 7 anymore, let KafkaFuture extend CompletableFuture (or even CompletionStage)
  • Later on the deprecated methods can be removed, and KafkaFuture becomes a shim on Completablefuture or CompletionStage.And replace or extend KafkaFuture with the java 8 CompletionStage. This KIP can then become part of KIP-118: Drop Support for Java 7