Status
Current state: Accepted
Discussion thread: here
Vote thread: here
JIRA:
Pull request: PR#4033
Motivation
KafkaFuture.Function
and KafkaFuture.BiCosumer
are currently public abstract classes with only one abstract method.
This means you cannot 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
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 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 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.
I was also requested to make the addWaiter method public as it is a very useful method for example to implement KIP-183. It was decided to not directly expose the addWaiter method but provide the implementation of the whenComplete of CompletableFuture. As it makes error handling more straight forward.
Compatibility, Deprecation, and Migration Plan
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