Re: Quick question regarding kafka broker security

2012-12-07 Thread Jay Kreps
We would be interested in doing a similar thing--basically implementing ssl for socket connections. These might be on a special SSL port, and would begin with the SSL handshake. We haven't done any real research on how this might work. Be aware, also, that without zk being secure there are other p

RE: Quick question regarding kafka broker security

2012-12-07 Thread Subhash Agrawal
No. we are not planning to expose Kafka broker to public. It will be inside the fireball but we still want to authenticate the connection. I noticed that there is connection timeout to zookeeper, probably similar connection timeout from producer to broker. Does producer keeps the connection alive

Re: Quick question regarding kafka broker security

2012-12-07 Thread Jonathan Creasy
Are you planning to directly publicly expose a Kafka Broker? -Jonathan On Fri, Dec 7, 2012 at 1:53 PM, Subhash Agrawal wrote: > Hi All, > > I am new to Kafka broker and realized that Kafka broker does not enforce > client authentication at connection or message level. > To avoid DOS attack, we a

Quick question regarding kafka broker security

2012-12-07 Thread Subhash Agrawal
Hi All, I am new to Kafka broker and realized that Kafka broker does not enforce client authentication at connection or message level. To avoid DOS attack, we are planning to implement security certificate at client connection level, not at message level, so that we can authenticate client conn

Re: 0.8 producer -- many questions

2012-12-07 Thread Jay Kreps
I agree that BNFs are not the most readable thing in the world, but the fact is that our protocol has variable length fields and lots of repeated structures which are pretty confusing in a positional form. In practice I think the code one writes looks a lot like the BNF (e.g. readTopicMetadataReque

Re: 0.8 producer -- many questions

2012-12-07 Thread Jay Kreps
WRT the version I don't think it will really help to catch bugs. It is definitely possible for us to have protocol bugs, but I don't think that that version would catch the problem since the person breaking the protocol would probably not change the version number. I also think it is confusing for

Re: 0.8 producer -- many questions

2012-12-07 Thread Jun Rao
Ben, It doesn't look like that you are subscribing to our mailing list. As a result, your post doesn't automatically show up in the mailing list and has to be manually approved. Please subscribe to our mailing list by emailing to the following addresses. dev-subscr...@kafka.apache.org users-sub

Re: 0.8 producer -- many questions

2012-12-07 Thread ben fleis
First off, let me amend my previous votes: +1 for both ApiKey and ApiVersion in all responses, even @ 4 bytes. The number of bytes is insignificant, useful in the debugging and multiplex scenarios, even if the underlying service doesn't yet support it. (The protocol still can.) CorrelationId can

Re: 0.8 producer -- many questions

2012-12-07 Thread ben fleis
+1 on keeping the field for sanity checking (it's only 1 byte) +1 on each API/protocol having a singular version, ie Request+Response are a single moving version, even if one side changes w/o the other changing b