[ https://issues.apache.org/jira/browse/KAFKA-8396?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16891913#comment-16891913 ]
John Roesler commented on KAFKA-8396: ------------------------------------- >From the discussion of KIP-478, Guozhang had the following comments relevant >to this ticket: {quote}Hi John, Just a wild thought about Transformer: now with the new Processor<KIn, KOut, VIn, VOut>#init(ProcessorContext<KOut, VOut>), do we still need a Transformer (and even ValueTransformer / ValueTransformerWithKey)? What if: * We just make KStream#transform to get a ProcessorSupplier as well, and inside `process()` we check that at most one `context.forward()` is called, and then take it as the return value. * We would still use ValueTransformer for KStream#transformValue, or we can also use a `ProcessorSupplier where we allow at most one `context.forward()` AND we ignore whatever passed in as key but just use the original key. Guozhang{quote} > Clean up Transformer API > ------------------------ > > Key: KAFKA-8396 > URL: https://issues.apache.org/jira/browse/KAFKA-8396 > Project: Kafka > Issue Type: Improvement > Components: streams > Reporter: John Roesler > Priority: Major > Labels: needs-kip, user-experience > > Currently, KStream operators transformValues and flatTransformValues disable > context forwarding, and force operators to just return the new values. > The reason is that we wanted to prevent the key from changing, since the > whole point of a `xValues` transformation is that we _do not_ change the key, > and hence don't need to repartition. > However, the chosen mechanism has some drawbacks: The Transform concept is > basically a way to plug in a custom Processor within the Streams DSL, but > these restrictions make it more like a MapValues with access to the context. > For example, even though you can still schedule punctuations, there's no way > to forward values as a result of them. So, as a user, it's hard to build a > mental model of how to use a TransformValues (because it's not quite a > Transformer and not quite a Mapper). > Also, logically, a Transformer can call forward as much as it wants, so a > Transformer and a FlatTransformer are effectively the same thing. Then, we > also have TransformValues and FlatTransformValues that are also two more > versions of the same thing, just to implement the key restrictions. > Internally, some of these can send downstream by returning OR forwarding, and > others can only return. It's a lot for users to keep in mind. > We can clean up this API significantly by just allowing all transformers to > call `forward`. In the `Values` case, we can wrap the ProcessorContext in one > that checks the key is `equal` to the one that got passed in (i.e., saves a > reference and enforces equality with that reference in any call to > `forward`). Then, we can actually deprecate the `*ValueTransformer*` > interfaces and remove the restriction about calling forward. > We can consider a further cleanup (TBD) to deprecate the existing Transformer > interface entirely, and replace it with one with a `void` return type. Then, > the Transform and FlatTransform cases collapse together, and we just need > Transform and TransformValues. -- This message was sent by Atlassian JIRA (v7.6.14#76016)