[ https://issues.apache.org/jira/browse/KAFKA-1110?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13815127#comment-13815127 ]
Evan Huus edited comment on KAFKA-1110 at 11/6/13 6:31 PM: ----------------------------------------------------------- Hi Joe, as verified with the network trace already attached, Sarama is setting the compression bit correctly (it happens at https://github.com/Shopify/sarama/blob/master/message.go#L36-L37). I am perfectly happy to do the coding work as necessary to match the protocol. The problem is that based on the network traces and the wiki page I *am* matching the protocol, and it still doesn't work. Moreover, the ConsoleConsumer, to the best of my knowledge is *not* matching the protocol, and yet it works. Maybe just the protocol spec (https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol) needs updating? was (Author: eapache): Hi Joe, as verified with the network trace, Sarama is setting the compression bit correctly (it happens at https://github.com/Shopify/sarama/blob/master/message.go#L36-L37). I am perfectly happy to do the coding work as necessary to match the protocol. The problem is that based on the network traces and the wiki page I *am* matching the protocol, and it still doesn't work. Moreover, the ConsoleConsumer, to the best of my knowledge is *not* matching the protocol, and yet it works. Maybe just the protocol spec (https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol) needs updating? > Unable to produce messages with snappy/gzip compression > ------------------------------------------------------- > > Key: KAFKA-1110 > URL: https://issues.apache.org/jira/browse/KAFKA-1110 > Project: Kafka > Issue Type: Bug > Affects Versions: 0.8 > Environment: Kafka version: kafka-0.8.0-beta1 > OS version: Darwin 12.4.1 Darwin Kernel Version 12.4.1: Tue May 21 17:04:50 > PDT 2013; root:xnu-2050.40.51~1/RELEASE_X86_64 x86_64 > Reporter: Arup Malakar > Attachments: kafka_producer_snappy_pkt_63.pcapng, > sarama_producer_snappy_pkt_1.pcapng > > > Sarama[1] (A golang kafka library: https://github.com/Shopify/sarama) is > following the specs as defined in: > https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol > but messages are not getting into the kafka log file and consumers never see > them when gzip/snappy is used. Without compression it works fine though. > Few observations we made: > 1. Kafka service does have required jars to be able to interpret snappy > messages. When I modify ConsoleProducer to produce messages using > SnappyCompressionCodec instead of default GZip codec. I was able to > produce/consume messages. Looking at the kafka log files I see that Snappy > Compression was indeed getting used: > % bin/kafka-run-class.sh kafka.tools.DumpLogSegments --files > /tmp/kafka-logs/aruptest-0/00000000000000000000.log | tail -1 > offset: 15 position: 18763 isvalid: true payloadsize: 52 magic: 0 > compresscodec: SnappyCompressionCodec crc: 1602790249 > So I don't think it would be a case of missing jars in kafka server. > 2. Kafka doesn't return any error if the message doesn't make it to the log > file. This seems pretty serious, as I would expect kafka to throw an error if > I am using WaitForLocal/WaitForAll. > 3. We did an inspection of the tcp packet to see the difference between what > ConsoleProducer sends vs what sarama sends > (Following is a copy/paste from a github issue): > [~eapache] : "So I have no idea what the ConsoleProducer is actually sending > in this case. The outer protocol layers in both cases look identical, but if > you compare the actual message value: > a. Sarama sends two bytes of snappy header and then "<msg-payload>" (since > Snappy decides it's too short to properly encode, so makes it a literal). > Pretty straightforward. > b. ConsoleProducer sends 0x82 then the string literal SNAPPY\0 then what > appears to be a complete embedded produce request without any compression. > This is neither valid snappy nor valid Kafka according to anything I've seen, > so I'm pretty confused. It looks almost like an incorrect version of [1] but > it's missing several key fields and the case of the identifying string is > wrong. > 1: http://code.google.com/p/snappy/source/browse/trunk/framing_format.txt " > Let us know if recent changes in the codebase makes the protocol page > obsolete, in that case if the protocol page is updated we could update our > client to use the new spec. > More information could be found in the following github issue: > https://github.com/Shopify/sarama/issues/32 -- This message was sent by Atlassian JIRA (v6.1#6144)