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

Guozhang Wang commented on KAFKA-1011:
--------------------------------------

Thank Jun, you are right. The message metadata is thrown away when the producer 
of the MirrorMaker sends it to the target partition, and the new metadata will 
be added at the producer side. And after thinking about this I believe the 
viable solution would be a producer API change, sending MessageAndMetadata 
instead of KeyedMessage so that the metadata will not be dropped if there is 
any.

That said, I agree that this would not be an 0.8 fix but rather goes to trunk. 
And it may also need to align with the client-redesign stuff:

https://cwiki.apache.org/confluence/display/KAFKA/Client+Rewrite


                
> Decompression and re-compression on MirrorMaker could result in messages 
> being dropped in the pipeline
> ------------------------------------------------------------------------------------------------------
>
>                 Key: KAFKA-1011
>                 URL: https://issues.apache.org/jira/browse/KAFKA-1011
>             Project: Kafka
>          Issue Type: Bug
>            Reporter: Guozhang Wang
>            Assignee: Guozhang Wang
>             Fix For: 0.8.1
>
>         Attachments: KAFKA-1011.v1.patch
>
>
> The way MirrorMaker works today is that its consumers could use deep iterator 
> to decompress messages received from the source brokers and its producers 
> could re-compress the messages while sending them to the target brokers. 
> Since MirrorMakers use a centralized data channel for its consumers to pipe 
> messages to its producers, and since producers would compress messages with 
> the same topic within a batch as a single produce request, this could result 
> in messages accepted at the front end of the pipeline being dropped at the 
> target brokers of the MirrorMaker due to MesageSizeTooLargeException if it 
> happens that one batch of messages contain too many messages of the same 
> topic in MirrorMaker's producer. If we can use shallow iterator at the 
> MirrorMaker's consumer side to directly pipe compressed messages this issue 
> can be fixed. 
> Also as Swapnil pointed out, currently if the MirrorMaker lags and there are 
> large messages in the MirrorMaker queue (large after decompression), it can 
> run into an OutOfMemoryException. Shallow iteration will be very helpful in 
> avoiding this exception.
> The proposed solution of this issue is also related to KAFKA-527.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira

Reply via email to