[jira] [Commented] (KAFKA-1110) Unable to produce messages with snappy/gzip compression

2013-10-30 Thread Evan Huus (JIRA)

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

Evan Huus commented on KAFKA-1110:
--

I am the primary author of the golang library and the one quoted in the main 
description. The "[1]" link in my quote was a reference to the Snappy framing 
format spec: 
http://code.google.com/p/snappy/source/browse/trunk/framing_format.txt

The part that is confusing me the most is the 0x82 at the beginning of the 
message produced by the ConsoleConsumer, since it does not match any possible 
structure I know of. Raw snappy would expect a little-endian varint of the 
message length (which 0x82 is not), snappy framing would expect 0xff prior to 
sNaPpY (as opposed to the actual SNAPPY in the message) and a Kafka string 
would be a two-byte length.

Worth noting: recent development builds of Wireshark support decoding the Kafka 
protocol as described in the spec, so that may aid in reading the PCAP files 
for debugging. You will have to set a preference to tell it which port to 
decode on though.

> 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/.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 "" (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.
> 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)


[jira] [Commented] (KAFKA-1110) Unable to produce messages with snappy/gzip compression

2013-11-06 Thread david birdsong (JIRA)

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

david birdsong commented on KAFKA-1110:
---

Just wanted to add a me-to to this. Neither gzip nor snappy append any data to 
the topic log partitions. When I disable all off all compression, sarama 
appends data to partiions and the console consumer sees data.

Is anybody looking into this yet?

> 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/.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 "" (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)


[jira] [Commented] (KAFKA-1110) Unable to produce messages with snappy/gzip compression

2013-11-06 Thread Joe Stein (JIRA)

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

Joe Stein commented on KAFKA-1110:
--

Hi, I started to look at the ticket this morning.  I read through the github 
issues for the go lang client library too.  

if gzip is not working then snappy incompatibility between clients is not the 
issue, if gzip is working then sure compatibility between snappy could be the 
cause but we can look at that next after checking a few things first

The import for the snappy library happens here if 
https://github.com/apache/kafka/blob/0.8/core/src/main/scala/kafka/message/CompressionFactory.scala#L44
 (i know this was discounted as a reason but still wanted to link code to 
thoughts some for folks)

The compressed message contains the compression bit 
https://github.com/apache/kafka/blob/0.8/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala?source=c
 this might be what you need to modify to match not the wire protocol side.  
Also take a look at 
https://github.com/apache/kafka/blob/0.8/core/src/main/scala/kafka/message/Message.scala?source=c#L111
 you probably will implement the parts above taking it out of the bytes 
differently but maybe similarly dunno go at all.

I don't know go or its build system sorry so I can really help with the client 
library code. I am not opposed to learning Go it but will have to wait a few 
days/week at least to set some time aside to absorb it more so bare with me 
with that part of it.  

I am working on a read only user AWS lab so maybe we can try to reproduce it on 
one of those boxes together if the message byte is not the problem either

> 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/.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 "" (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

[jira] [Commented] (KAFKA-1110) Unable to produce messages with snappy/gzip compression

2013-11-06 Thread Evan Huus (JIRA)

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

Evan Huus commented on KAFKA-1110:
--

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/.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 "" (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)


[jira] [Commented] (KAFKA-1110) Unable to produce messages with snappy/gzip compression

2013-11-07 Thread Joe Stein (JIRA)

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

Joe Stein commented on KAFKA-1110:
--

Hey Evan, if you can get it to work that is a good next step, yup.  And then 
once it is working can you suggest what would be the best changes to the WIKI 
in this ticket for others that are doing what are you doing so as to-do it 
correctly for how Kafka code is written and working now? 

> 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/.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 "" (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)


[jira] [Commented] (KAFKA-1110) Unable to produce messages with snappy/gzip compression

2013-11-07 Thread Evan Huus (JIRA)

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

Evan Huus commented on KAFKA-1110:
--

I don't know how to make it work. The provided ConsoleProducer appears, for all 
intents and purposes, to be sending garbage and yet it is accepted for some 
reason. I was hoping somebody who knows the protocol could clarify the 
structure, then I could match it in Go. I really don't want to have to 
reverse-engineer the protocol from scratch.

> 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/.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 "" (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)


[jira] [Commented] (KAFKA-1110) Unable to produce messages with snappy/gzip compression

2013-11-07 Thread Joe Stein (JIRA)

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

Joe Stein commented on KAFKA-1110:
--

Hey Evan, if the issue is not occurring without compression then as I said the 
issue is not with the wire protocol.  Compression is handled in the message 
structure as I posted above.  

Is this happening with gzip also? If not then maybe the issue is in fact with 
the Go Snappy library (though I still think this is likely not the issue but if 
gzip works then maybe it is).

Maybe you can take a look at the C++ implementation from Adobe 
https://github.com/adobe-research/libkafka/blob/535860eafce32d9e7e454176770219abf473a344/lib/src/Message.cc#L129
 which is just like the Scala code I posted above.

Also, I don't know what "& 0x07" means is in your code you posted

attributes := int8(m.Codec) & 0x07
pe.putInt8(attributes)

have you thought of just trying

attributes := int8(m.Codec)
pe.putInt8(attributes)

again, I don't know Go so I am at a disadvantage to help even though I am 
trying to-do so to make it work with how it is


> 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/.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 "" (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)


[jira] [Commented] (KAFKA-1110) Unable to produce messages with snappy/gzip compression

2013-11-07 Thread Arup Malakar (JIRA)

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

Arup Malakar commented on KAFKA-1110:
-

May be Evan would be able to provide more information. But gzip is not working 
either.

> 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/.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 "" (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)


[jira] [Commented] (KAFKA-1110) Unable to produce messages with snappy/gzip compression

2013-11-08 Thread Evan Huus (JIRA)

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

Evan Huus commented on KAFKA-1110:
--

The 0x07 is the same as the CompressionCodeMask in the scala version, I just 
didn't give it a name (although I have a change pending for that now, for 
clarity's sake).

The information I really need to proceed is for somebody to explain the output 
of Kafka's ConsoleProducer. It does *not* match the wire protocol to the best 
of my knowledge, but the broker accepts it anyways.

> 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/.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 "" (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)


[jira] [Commented] (KAFKA-1110) Unable to produce messages with snappy/gzip compression

2013-11-08 Thread Joe Stein (JIRA)

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

Joe Stein commented on KAFKA-1110:
--

The CompressionCodeMask is 0x03 and comes before the codec 

val CompressionCodeMask: Int = 0x03 

attributes =  (attributes | (CompressionCodeMask & codec.codec)).toByte
buffer.put(attributes)

so maybe you need to-do

attributes := 0x03 & int8(m.Codec)
pe.putInt8(attributes)

to get it working?  again I don't know Go. 

The code is the best documentation and explanation IMHO 
https://github.com/apache/kafka/blob/0.8/core/src/main/scala/kafka/message/Message.scala?source=c
 and 
https://github.com/apache/kafka/blob/0.8/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala?source=c
 are the entry points you are need to follow in regards to this specifically.   
In regards to the producer request as a whole 
https://github.com/apache/kafka/blob/0.8/core/src/main/scala/kafka/api/ProducerRequest.scala?source=c




> 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/.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 "" (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)


[jira] [Commented] (KAFKA-1110) Unable to produce messages with snappy/gzip compression

2013-11-08 Thread Evan Huus (JIRA)

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

Evan Huus commented on KAFKA-1110:
--

I have verified that I have log4j in the path, and am getting lots of normal 
log messages but no errors.

However, I think I've figured it out. I took a wild guess based on the apparent 
structure of the messages being sent by the ConsoleProducer, and tried wrapping 
the message in a *second* message-set, and voila it appeared to be logged 
correctly.

Maybe I just got lucky, but if somebody can verify that compressed messages do 
in fact have to be wrapped twice for some reason, we're good to go.

There is still a bug in that not adding the second layer doesn't produce any 
errors. It should log an error *and* it should return an error code in the 
ProduceResponse. Also, the wiki should be updated. All of this pending 
confirmation that my fix is correct though.

> 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/.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 "" (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)


[jira] [Commented] (KAFKA-1110) Unable to produce messages with snappy/gzip compression

2013-11-08 Thread Evan Huus (JIRA)

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

Evan Huus commented on KAFKA-1110:
--

Bitwise and is commutative, so (mask & value) is always the same as (value & 
mask). According to the spec "The lowest 3 bits contain the compression codec 
used for the message" which means the mask should be 0x07, not 0x03. Either the 
spec or the code should change.

> 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/.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 "" (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)


[jira] [Commented] (KAFKA-1110) Unable to produce messages with snappy/gzip compression

2013-11-08 Thread Joe Stein (JIRA)

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

Joe Stein commented on KAFKA-1110:
--

The Wiki should change, I can grant you edit access to the Wiki if you can/want 
to update it?  Contributions are always welcome :) 

You need an ICLA on file with Apache http://www.apache.org/licenses/#clas and I 
need to know your wiki user name.

Or once you get everything working if you want to suggest all of the changes to 
the wiki in this ticket I can make them for you.

Let me know if you get everything working and if you want to update the wiki 
yourself or let me know the changes you need bad (assuming it might be more 
than just this one thing?)

> 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/.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 "" (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)


[jira] [Commented] (KAFKA-1110) Unable to produce messages with snappy/gzip compression

2013-11-08 Thread Evan Huus (JIRA)

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

Evan Huus commented on KAFKA-1110:
--

I've made changes to that page in the past but I guess the Confluence upgrade 
changed permissions. My wiki username is eapache. I *think* I already have an 
ICLA on file (given I've made changes in the past) but I'm not sure how to 
check, so I'll submit another one to be sure.

> 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/.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 "" (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)


[jira] [Commented] (KAFKA-1110) Unable to produce messages with snappy/gzip compression

2013-11-08 Thread Evan Huus (JIRA)

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

Evan Huus commented on KAFKA-1110:
--

ICLA submitted. Don't know how long it will take me to show up on the list, but 
when I do please add me to the wiki and I will make the necessary changes.

> 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/.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 "" (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)


[jira] [Commented] (KAFKA-1110) Unable to produce messages with snappy/gzip compression

2013-11-08 Thread Evan Huus (JIRA)

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

Evan Huus commented on KAFKA-1110:
--

This bug can now track the fact that a compressed message not containing a 
message-set results in no errors logged or returned.

> 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/.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 "" (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)


[jira] [Commented] (KAFKA-1110) Unable to produce messages with snappy/gzip compression

2013-11-10 Thread Evan Huus (JIRA)

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

Evan Huus commented on KAFKA-1110:
--

My ICLA has been accepted, and my name appears on the list at 
https://people.apache.org/committer-index.html#unlistedclas

> 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/.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 "" (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)