Table of Contents |
---|
Status
Current state: "Under Discussion", "Accepted", "Rejected"
Discussion thread: here
Vote thread: here
JIRA:
Jira | ||||||||
---|---|---|---|---|---|---|---|---|
|
Pull request: PR#4033
Released: 1.1.0
Motivation
KafkaFuture.Function
and KafkaFuture.BiCosumer
are currently public abstract classes with only one abstract method.
...
In the public interface, only org.apache.kafka.common.KafkaFuture
has helper classes which can better be expressed as functional interfaces.
Proposed Changes
The current proposal implemented in PR#4033 KIP introduces a FunctionInterface
which defines the new functional interface. The old Function
class is kept and marked as @Deprecated
.This class is there to guarantee binary compatibility, also for 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) the old signature is kept next to the new one.
As the Biconsumer
class was not used in the public interface, I decided to stick there with a backwards incompatible change.
See the pull request for more details.
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 | ||||
---|---|---|---|---|
| ||||
/**
* 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); |
I was also requested to make the addWaiter method public as it is a very usefull method for example to implement KIP-183.
Compatibility, Deprecation, and Migration Plan
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
When the change is implemented like proposed, everybody using KafkaFuture.Function
will get a deprecation warning.
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. And replace or extend
KafkaFuture
with the java 8CompletionStage
. This KIP can then become part of KIP-118: Drop Support for Java 7 in Kafka 0.11