Re: [DISCUSS] Kafka Security Specific Features

2014-07-31 Thread Rajasekar Elango
Can we get the info on targeted release dates for 0.8.2 release and 0.9 release for our planning purposes? Thanks. Raja. On Wed, Jul 30, 2014 at 7:27 PM, Joe Stein wrote: > The 0.8.2 release will not have the patch inside of it. Trunk already has > a lot inside of it as a point release. The

Re: [DISCUSS] Kafka Security Specific Features

2014-07-30 Thread Joe Stein
The 0.8.2 release will not have the patch inside of it. Trunk already has a lot inside of it as a point release. The patch also doesn't account for all of the requirements that all of the stakeholders need/want for the feature. Instead of releasing something that is useful but only for some it i

Re: [DISCUSS] Kafka Security Specific Features

2014-07-30 Thread Calvin Lei
yeah i just saw that. Looking forward to the prod release of 0.8.2 On Wed, Jul 30, 2014 at 11:01 AM, Rajasekar Elango wrote: > We implemented security features on older snapshot version of 0.8 kafka. > But Joe Stein's organization rebased it to latest version of kafka > available at https://git

Re: [DISCUSS] Kafka Security Specific Features

2014-07-30 Thread Rajasekar Elango
We implemented security features on older snapshot version of 0.8 kafka. But Joe Stein's organization rebased it to latest version of kafka available at https://github.com/stealthly/kafka/tree/v0.8.2_KAFKA-1477. Thanks, Raja. On Tue, Jul 29, 2014 at 10:54 PM, Calvin Lei wrote: > Raja, >Whi

Re: [DISCUSS] Kafka Security Specific Features

2014-07-29 Thread Calvin Lei
Raja, Which Kafka version is your security enhancement based on? thanks, Cal On Wed, Jul 23, 2014 at 5:01 PM, Chris Neal wrote: > Pramod, > > I got that same error when following the configuration from Raja's > presentation earlier in this thread. If you'll notice the usage for the > conso

Re: [DISCUSS] Kafka Security Specific Features

2014-07-23 Thread Chris Neal
Pramod, I got that same error when following the configuration from Raja's presentation earlier in this thread. If you'll notice the usage for the console_producer.sh, it is slightly different, which is also slightly different than the scala code for the ConsoleProducer. :) When I changed this:

Re: [DISCUSS] Kafka Security Specific Features

2014-07-22 Thread Pramod Deshmukh
Anyone getting this issue. Is it something related to environment or it is the code. Producer works fine when run with secure=false (no security) mode. pdeshmukh$ bin/kafka-console-producer.sh --broker-list localhost:9092:true --topic secureTopic [2014-07-18 13:12:29,817] WARN Property topic is

Re: [DISCUSS] Kafka Security Specific Features

2014-07-18 Thread Pramod Deshmukh
Thanks Joe, I don't see any Out of memory error. Now I get exception when Producer fetches metadata for a topic Here is how I created the topic and run producer pdeshmukh$ bin/kafka-topics.sh --create --zookeeper localhost:2181 --replication-factor 1 --partitions 1 --topic secureTopic Created top

Re: [DISCUSS] Kafka Security Specific Features

2014-07-18 Thread Joe Stein
Hi Pramod, Can you increase KAFKA_HEAP_OPTS to lets say -Xmx1G in the kafka-console-producer.sh to see if that gets you further along please in your testing? Thanks! /*** Joe Stein Founder, Principal Consultant Big Data Open Source Security LLC http://

Re: [DISCUSS] Kafka Security Specific Features

2014-07-18 Thread Pramod Deshmukh
Hello Raja/Joe, When I turn on security, i still get out of memory error on producer. Is this something to do with keys? Is there any other way I can connect to broker? *producer log* [2014-07-17 15:38:14,186] ERROR OOME with size 352518400 (kafka.network. BoundedByteBufferReceive) java.lang.OutOf

Re: [DISCUSS] Kafka Security Specific Features

2014-07-17 Thread Pramod Deshmukh
Correct, I don't see any exceptions when i turn off security. Consumer is able to consume the message. I still see warning for topic property. [2014-07-17 18:04:38,360] WARN Property topic is not valid (kafka.utils.VerifiableProperties) On Thu, Jul 17, 2014 at 5:49 PM, Rajasekar Elango wrot

Re: [DISCUSS] Kafka Security Specific Features

2014-07-17 Thread Rajasekar Elango
Can you try with turning off security to check if this error happens only on secure mode? Thanks, Raja. On Thu, Jul 17, 2014 at 3:51 PM, Pramod Deshmukh wrote: > Thanks Raja, it was helpful > > Now I am able to start zookeeper and broker in secure mode ready for SSL > handshake. I get *java.

Re: [DISCUSS] Kafka Security Specific Features

2014-07-17 Thread Pramod Deshmukh
Thanks Raja, it was helpful Now I am able to start zookeeper and broker in secure mode ready for SSL handshake. I get *java.lang.OutOfMemoryError: Java heap space* on producer. I using the default configuration and keystore. Is there anything missing *Start broker:* *bin/kafka-server-start.sh c

Re: [DISCUSS] Kafka Security Specific Features

2014-07-16 Thread Rajasekar Elango
Pramod, I presented secure kafka configuration and usage at last meet up. So hope this video recording would help. You can skip to about 59 min to jump to security talk. Thanks, Raja. On Wed, Jul 16, 2014 at 5:57 PM, Pramod Deshmukh wrote: > Hello Joe

Re: [DISCUSS] Kafka Security Specific Features

2014-07-16 Thread Pramod Deshmukh
Hello Joe, Is there a configuration or example to test Kafka security piece? Thanks, Pramod On Wed, Jul 16, 2014 at 5:20 PM, Pramod Deshmukh wrote: > Thanks Joe, > > This branch works. I was able to proceed. I still had to set scala version > to 2.9.2 in kafka-run-class.sh. > > > > On Wed, J

Re: [DISCUSS] Kafka Security Specific Features

2014-07-16 Thread Pramod Deshmukh
Thanks Joe, This branch works. I was able to proceed. I still had to set scala version to 2.9.2 in kafka-run-class.sh. On Wed, Jul 16, 2014 at 3:57 PM, Joe Stein wrote: > That is a very old branch. > > Here is a more up to date one > https://github.com/stealthly/kafka/tree/v0.8.2_KAFKA-1477 (

Re: [DISCUSS] Kafka Security Specific Features

2014-07-16 Thread Joe Stein
That is a very old branch. Here is a more up to date one https://github.com/stealthly/kafka/tree/v0.8.2_KAFKA-1477 (needs to be updated to latest trunk might have a chance to-do that next week). You should be using gradle now as per the README. /*** Joe S

Re: [DISCUSS] Kafka Security Specific Features

2014-07-16 Thread Pramod Deshmukh
Thanks Joe for this, I cloned this branch and tried to run zookeeper but I get Error: Could not find or load main class org.apache.zookeeper.server.quorum.QuorumPeerMain I see scala version is still set to 2.8.0 if [ -z "$SCALA_VERSION" ]; then SCALA_VERSION=2.8.0 fi Then I instal

Re: [DISCUSS] Kafka Security Specific Features

2014-06-10 Thread Robert Withers
We could extend the existing metadata to include a Kerberos-style token, whichever scheme is used. This would mean creating a producer or consumer with a security context and session negotiation would result in a token. It may be a lease. Both of our modules would authenticate and authorize t

Re: [DISCUSS] Kafka Security Specific Features

2014-06-10 Thread Todd Palino
Yes, I agree. There are definitely a variety of use cases that demand differing levels of complexity here. It comes back to enabling the development of at-rest encryption and making it as easy as possible to implement within the Kafka system. I think that this can be done with the concept of messag

Re: [DISCUSS] Kafka Security Specific Features

2014-06-10 Thread Robert Withers
What strikes me as an opportunity is to define a plug gable at-rest encryption module interface, that supports each/both of our security needs. Thanks, Rob > On Jun 10, 2014, at 4:01 PM, Todd Palino wrote: > > The situation of production before having the consumer is definitely a > good one. T

Re: [DISCUSS] Kafka Security Specific Features

2014-06-10 Thread Todd Palino
The situation of production before having the consumer is definitely a good one. That’s why I wanted to take a little time before responding. Had to think about it. I think that while we may certainly produce data before the consumer is ready, that doesn’t mean that the consumer can’t have a key p

Re: [DISCUSS] Kafka Security Specific Features

2014-06-09 Thread Robert Withers
Yes, that sounds familiar as I helped write (minimally) S/MIME in squeak (open source Smalltalk environment). This what I was thinking in my alternative here, though I have a concern... Production may occur before the consumer is coded and executed. In the analogy of mail, the mail is sent be

Re: [DISCUSS] Kafka Security Specific Features

2014-06-09 Thread Todd Palino
It’s the same method used by S/MIME and many other encryption specifications with the potential for multiple recipients. The sender generates a session key, and uses that key to encrypt the message. The session key is then encrypted once for each recipient with that recipient’s public key. All of t

Re: [DISCUSS] Kafka Security Specific Features

2014-06-08 Thread Rob Withers
I like the use of meta envelopes. We did this recently, on the job, as we have an envelope that specifies the type for decoding. We discussed adding the encodinType and you are suggesting adding encryption metadata for that msg. All good. I don't see your OTP example. Could you delve de

Re: [DISCUSS] Kafka Security Specific Features

2014-06-08 Thread Todd Palino
I’ll agree that perhaps the “absolutely not” is not quite right. There are certainly some uses for a simpler solution, but I would still say it cannot only be encryption at the broker. This would leave many use cases for at-rest encryption out of the loop (most auditing cases for SOX, PCI, HIPAA, a

Re: [DISCUSS] Kafka Security Specific Features

2014-06-07 Thread Rob Withers
At one level this makes sense to me to externalize the security issue to producers and consumers. On consideration I realized that this adds a lot of coordination requirements to the app layer across teams or even companies. Another issue I feel is that you want a specific unchanging encr

Re: [DISCUSS] Kafka Security Specific Features

2014-06-06 Thread Steven Schlansker
Hi, I’m glad there’s so much thought into getting security right! But as a user of Kafka who doesn’t need Enterprise Security sort of features, I would ask whether doing such a large project built into Kafka is the appropriate use of developer time at this point in its lifecycle. For example, eve

Re: [DISCUSS] Kafka Security Specific Features

2014-06-06 Thread Todd Palino
Yes, I realized last night that I needed to be clearer in what I was saying. Encryption should ABSOLUTELY not be handled server-side. I think it¹s a good idea to enable use of it in the consumer/producer, but doing it server side will not solve many use cases for needing encryption because the serv

Re: [DISCUSS] Kafka Security Specific Features

2014-06-06 Thread Robert Withers
It would be nice to have Alcatraz on-disk security for the discriminating client. Thanks, Rob > On Jun 6, 2014, at 11:51 AM, Pradeep Gollakota wrote: > > I'm actually not convinced that encryption needs to be handled server side > in Kafka. I think the best solution for encryption is to handle

Re: [DISCUSS] Kafka Security Specific Features

2014-06-06 Thread Pradeep Gollakota
I'm actually not convinced that encryption needs to be handled server side in Kafka. I think the best solution for encryption is to handle it producer/consumer side just like compression. This will offload key management to the users and we'll still be able to leverage the sendfile optimization for

Re: [DISCUSS] Kafka Security Specific Features

2014-06-06 Thread Rob Withers
On consideration, if we have 3 different access groups (1 for production WRITE and 2 consumers) they all need to decode the same encryption and so all need the same public/private keycerts won't work, unless you write a CertAuthority to build multiple certs with the same keys. Better s

Re: [DISCUSS] Kafka Security Specific Features

2014-06-06 Thread Rob Withers
This is quite interesting to me and it is an excelent opportunity to promote a slightly different security scheme. Object-capabilities are perfect for online security and would use ACL style authentication to gain capabilities filtered to those allowed resources for allow actions (READ/WRI

Re: [DISCUSS] Kafka Security Specific Features

2014-06-05 Thread Todd Palino
My concern is specifically around the rules for SOX compliance, or rules around PII, PCI, or HIPAA compliance. The audits get very complication, but my understanding is that the general rule is that sensitive data should be encrypted at rest and only decrypted when needed. And we don¹t just need to

Re: [DISCUSS] Kafka Security Specific Features

2014-06-05 Thread Jay Kreps
Hey Todd, Can you elaborate on this? Certainly restricting access to and modification of data is important. But this doesn't imply storing the data encrypted. Are we assuming the attacker can (1) get on the network, (2) get on the kafka server as a non-root and non-kafka user or (3) get root on th

Re: [DISCUSS] Kafka Security Specific Features

2014-06-05 Thread Todd Palino
No, at-rest encryption is definitely important. When you start talking about data that is used for financial reporting, restricting access to it (both modification and visibility) is a critical component. -Todd On 6/5/14, 2:01 PM, "Jay Kreps" wrote: >Hey Joe, > >I don't really understand the s

Re: [DISCUSS] Kafka Security Specific Features

2014-06-05 Thread Jay Kreps
Hey Joe, I don't really understand the sections you added to the wiki. Can you clarify them? Is non-repudiation what SASL would call integrity checks? If so don't SSL and and many of the SASL schemes already support this as well as on-the-wire encryption? Or are you proposing an on-disk encrypti

Re: [DISCUSS] Kafka Security Specific Features

2014-06-05 Thread Joe Stein
Raja, you need to sign an ICLA http://www.apache.org/licenses/icla.txt once that is on file your user can get permed to contribute. I think securing communication to "offset & broker management source" which can be a zookeeper implementation is important. I will elaborate more on that with the ot

Re: [DISCUSS] Kafka Security Specific Features

2014-06-05 Thread Rajasekar Elango
Hi Jay, Thanks for putting together a spec for security. Joe, Looks "Securing zookeeper.." part has been deleted from assumptions section. communication with zookeeper need to be secured as well to make entire kafka cluster secure. It may or may not require changes to kafka. But it's good to hav

Re: [DISCUSS] Kafka Security Specific Features

2014-06-04 Thread Joe Stein
I like the idea of working on the spec and prioritizing. I will update the wiki. - Joestein On Wed, Jun 4, 2014 at 1:11 PM, Jay Kreps wrote: > Hey Joe, > > Thanks for kicking this discussion off! I totally agree that for something > that acts as a central message broker security is critical fe

Re: [DISCUSS] Kafka Security Specific Features

2014-06-04 Thread Jay Kreps
Hey Joe, Thanks for kicking this discussion off! I totally agree that for something that acts as a central message broker security is critical feature. I think a number of people have been interested in this topic and several people have put effort into special purpose security efforts. Since mos

Re: [DISCUSS] Kafka Security Specific Features

2014-06-04 Thread Joe Stein
Hey Todd, I think you are right on both points. Maybe instead of modularizing authorization we could instead support some feature like being able to associate "labels" for the application specific items (topic name, reads/writes, delete topic, change config, rate limiting, etc) and then accept a

Re: [DISCUSS] Kafka Security Specific Features

2014-06-03 Thread Todd Palino
I think that¹s one option. What I would offer here is that we need to separate out the concepts of authorization and authentication. Authentication should definitely be modular, so that we can plug in appropriate schemes depending on the organization. For example, you may want client certificates,

Re: [DISCUSS] Kafka Security Specific Features

2014-06-03 Thread Robert Rodgers
... client specific presented information, signed in some way, listing topic permissions. read, write, list. TLS lends itself to client certificates. On Jun 3, 2014, at 12:57 PM, Joe Stein wrote: > 4) Authorization > > We should have a policy of "404" for data, topics, partitions (etc) if >

[DISCUSS] Kafka Security Specific Features

2014-06-03 Thread Joe Stein
Hi,I wanted to re-ignite the discussion around Apache Kafka Security. This is a huge bottleneck (non-starter in some cases) for a lot of organizations (due to regulatory, compliance and other requirements). Below are my suggestions for specific changes in Kafka to accommodate security requirements