Status
Current state: "Accepted"
Discussion thread: n/a – skipped discussion and called for a vote directly
Voting thread: https://lists.apache.org/thread/ht0685l0ypg47onj90kxpzbw76vtv6ob
JIRA: - KAFKA-9738Getting issue details... STATUS
Released: 4.0
Motivation
MockProcessorContext
(packed kafka-streams-test-utils
) is used to test the custom processors which implement the old and already deprecated Processor
interface. Both Processor
and MockProcessorContext
are already replace with new versions api.Processor
and api.MockProcessorContext
. Given that Processor
is already deprecated, we should also deprecate MockProcessorContext
.
Furthermore, Transformer
, TransformerSupplier
, ValueTransformer
, and ValueTransformerSupplier
, are already replaces with api.[FixedKey]Processor[Supplier]
and should be deprecated to cleanup the old Processor API because KStream#transform()
and KStream#transformValue()
are already deprecated, too.
Public Interfaces
org.apache.kafka.streams.processor.MockProcessorContext
from package kafka-stream-test-utils
.
org.apache.kafka.streams.kstream.Transformer
org.apache.kafka.streams.kstream.TransformerSupplier
org.apache.kafka.streams.kstream.ValueTransformer
org.apache.kafka.streams.kstream.ValueTransformerSupplier
Proposed Changes
We propose to deprecate MockProcessorContext
, Transformer
, TransformerSupplier
, ValueTransformer
, and ValueTransformerSupplier
, so we can remove thme in a future major release (most likely 5.0).
Compatibility, Deprecation, and Migration Plan
None. We only deprecate a class for future removal in a major release.
Test Plan
n/a
Documentation Plan
Regular JavaDocs and webpage docs updates.
Rejected Alternatives
n/a