[ 
https://issues.apache.org/jira/browse/KAFKA-15912?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17795926#comment-17795926
 ] 

Vojtech Juranek commented on KAFKA-15912:
-----------------------------------------

I'd be careful to do the parallelization per SMT/converter as moving data 
between threads maybe be in result more expensive, as you already mentioned. 
Also, if there is some bottleneck, e.g. value converter, running it in a single 
thread won't give any significant speed up. And this is actually what we 
(Debezium project) observe, either in our perf. tests or reported by the users 
(e.g. KAFKA-15996, resp. [DBZ-7240|https://issues.redhat.com/browse/DBZ-7240]). 
It would be IMHO more useful, if possible, to form record pipelines and run in 
parallel records through these pipelines in parallel. With this approach, 
thread safety can be solved e.g. by creating SMT/convertors copies for each 
processing pipeline. The issue with stateful transformation however remains. 
Also there is an issue with records ordering, however quite easily solvable 
when processing is done in batches. I'm currently doing some experiments with 
Debezium server, if such pipelines are possible there and if it gives any 
significant performance boost (still WIP, no results yet). So I'm wondering if 
doing something similar for Kafka Connect make sense for you or this seems to 
be too much complicated to worth the effort/possible backward compatibility 
issues/etc?

> Parallelize conversion and transformation steps in Connect
> ----------------------------------------------------------
>
>                 Key: KAFKA-15912
>                 URL: https://issues.apache.org/jira/browse/KAFKA-15912
>             Project: Kafka
>          Issue Type: Improvement
>          Components: connect
>            Reporter: Mickael Maison
>            Priority: Major
>
> In busy Connect pipelines, the conversion and transformation steps can 
> sometimes have a very significant impact on performance. This is especially 
> true with large records with complex schemas, for example with CDC connectors 
> like Debezium.
> Today in order to always preserve ordering, converters and transformations 
> are called on one record at a time in a single thread in the Connect worker. 
> As Connect usually handles records in batches (up to max.poll.records in sink 
> pipelines, for source pipelines while it really depends on the connector, 
> most connectors I've seen still tend to return multiple records each loop), 
> it could be highly beneficial to attempt running the converters and 
> transformation chain in parallel by a pool a processing threads.
> It should be possible to do some of these steps in parallel and still keep 
> exact ordering. I'm even considering whether an option to lose ordering but 
> allow even faster processing would make sense.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

Reply via email to