[ https://issues.apache.org/jira/browse/KAFKA-5684?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16110636#comment-16110636 ]
Paolo Patierno commented on KAFKA-5684: --------------------------------------- So it seems to me that the serdes parameters aren't related to the processor node anymore (the {{KStreamPrintProcessor}} in this case) but are related to the mapper. Referring on the two above cases from the {{print}} method point of view : a) the mapper is just provided to the {{PrintForeachAction}} and serdes aren ignored b) we have to build a mapper which use the provided serdes (or default ones) and then passing it to {{PrintForeachAction}}. Today there is a {{defaultKeyValueMapper}} but it doesn't fit well in this case because it's not aware of serdes At same time if serdes parameters are not related to the processor node, the {{KStreamPrintProcessor}} is really like the {{KStreamPeekProcessor}} with "forwardDownStream = false". The only difference is on the {{close}} method which is needed to flush the {{PrintForeachAction}}. In any case I think that {{KStreamPrintProcessor}} can be refactored as derived from {{KStreamPeekProcessor}}. What do you think [~guozhang] [~james.c] ? I'm starting to develop a proposal PR for that. Thanks, Paolo. > KStreamPrintProcessor as customized KStreamPeekProcessor > -------------------------------------------------------- > > Key: KAFKA-5684 > URL: https://issues.apache.org/jira/browse/KAFKA-5684 > Project: Kafka > Issue Type: Improvement > Components: streams > Reporter: Paolo Patierno > Assignee: Paolo Patierno > Priority: Minor > > Hi, > the {{KStreamPrintProcessor}} is implemented from scratch (from the > {{AbstractProcessor}}) and the same for the related supplier. > It looks to me that it's just a special {{KStreamPeekProcessor}} with > forwardDownStream to false and that allows the possibility to specify Serdes > instances used if key/values are bytes. > At same time used by a {{print()}} method it provides a fast way to print > data flowing through the pipeline (while using just {{peek()}} you need to > write the code). > I think that it could be useful to refactoring the {{KStreamPrintProcessor}} > as derived from the {{KStreamPeekProcessor}} customizing its behavior. > Thanks, > Paolo. -- This message was sent by Atlassian JIRA (v6.4.14#64029)