DUE TO SPAM, SIGN-UP IS DISABLED. Goto Selfserve wiki signup and request an account.
Status
Current state: Under Discussion
Discussion thread: TDB
JIRA: N/A
Please keep the discussion on the mailing list rather than commenting on the wiki (wiki discussions get unwieldy fast).
Note this KIP is the original work of Ray McDermott - Due to lack of self-service signup for wiki ID and editing permissions, we're adding Ray's text to this KIP.
Motivation
Jackdaw is a Clojure library over the Kafka Streams DSL. In its current state, Jackdaw uses wrappers around the DSL interfaces to enable function passing in Clojure as one would enjoy if using Java.
Such wrappers are verbose and add to complexity to the library. They would not be required if Clojure has first class interop for SAM interfaces.
The new Clojure version 1.12 has many updates to Java interop including SAM interfaces. Unfortunately, it does not quite deliver what we need with respect to thinning down Kafka Streams interop.
We were specifically hoping that passing (fn [] ...) to SAM interfaces would just work and thus we would no longer need to wrap the interface. This feature may eventually be added to the language but it currently only works for interfaces that have been explicitly annotated with @FunctionalInterface. See this JIRA ticket for details.
Java does not require the @FunctionalInterface annotation although it clarifies the intended interface purpose.
The Kafka Streams DSL does not have those annotations and this KIP proposes to add them.
Public Interfaces
No new interfaces or changes to the behavior of existing interfaces will be introduced in this KIP.
These annotations on the interfaces are new but do not affect client calls. Details of adding the annotation are in the next section.
Proposed Changes
Add @FunctionalInterface to the Kafka Streams DSL and the Processor API (PAPI).
The interface must conform to these rules:
- have one abstract method
- can have any number of
defaultmethods - can have any number of methods directly on
Object
- can have any number of
The details of each included / excluded interface is as follows:
DSL (org.apache.kafka.streams.kstream)
Interfaces in scope
We will add @FunctionalInterface to these interfaces:
Aggregatorapplyis the only method in the interface (SAM)
ForeachActionapplyis the only method in the interface (SAM)
GlobalKTablequeryableStoreNameis the only method in the interface (SAM)
Initializerapplyis the only method in the interface (SAM)
KeyValueMapperapplyis the only method in the interface (SAM)
Mergerapplyis the only method in the interface (SAM)
NamedOperationwithNameis the only method in the interface (SAM)
Predicatetestis the only method in the interface (SAM)
Reducerapplyis the only method in the interface (SAM)
TransformerSuppliergetis the only method in the interface (SAM)
ValueJoinerapplyis the only method in the interface (SAM)
ValueJoinerWithKeyapplyis the only method in the interface (SAM)
ValueMapperapplyis the only method in the interface (SAM)
ValueMapperWithKeyapplyis the only method in the interface (SAM)
ValueTransformerSuppliergetis the only method in the interface (SAM)
ValueTransformerWithKeySuppliergetis the only method in the interface (SAM)
PAPI (org.apache.kafka.streams.processor.api)
The PAPI already has two interfaces that have @FunctionalInterface which will be unaffected:
FixedKeyProcessorSupplierProcessorSupplier.
Interfaces in scope
We will add @FunctionalInterface to these interfaces:
FixedKeyProcessorprocessis the only non-defaultmethod
Processorprocessis the only non-defaultmethod
Compatibility, Deprecation, and Migration Plan
None required
Test Plan
Running the existing tests will be sufficient.
Rejected Alternatives
Leave off the @FunctionalInterface annotation: limits the benefits to non-Java, JVM languages.