You are viewing a plain text version of this content. The canonical link for it is here.
Posted to users@kafka.apache.org by Joe Stein <jo...@stealth.ly> on 2014/06/03 21:57:15 UTC

[DISCUSS] Kafka Security Specific Features

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.  This comes from what folks are doing "in the wild" to
workaround and implement security with Kafka as it is today and also what I
have discovered from organizations about their blockers. It also picks up
from the wiki (which I should have time to update later in the week based
on the below and feedback from the thread).

1) Transport Layer Security (i.e. SSL)

This also includes client authentication in addition to in-transit security
layer.  This work has been picked up here
https://issues.apache.org/jira/browse/KAFKA-1477 and do appreciate any
thoughts, comments, feedback, tomatoes, whatever for this patch.  It is a
pickup from the fork of the work first done here
https://github.com/relango/kafka/tree/kafka_security.

2) Data encryption at rest.

This is very important and something that can be facilitated within the
wire protocol. It requires an additional map data structure for the
"encrypted [data encryption key]". With this map (either in your object or
in the wire protocol) you can store the dynamically generated symmetric key
(for each message) and then encrypt the data using that dynamically
generated key.  You then encrypt the encryption key using each public key
for whom is expected to be able to decrypt the encryption key to then
decrypt the message.  For each public key encrypted symmetric key (which is
now the "encrypted [data encryption key]" along with which public key it
was encrypted with for (so a map of [publicKey] =
encryptedDataEncryptionKey) as a chain.   Other patterns can be implemented
but this is a pretty standard digital enveloping [0] pattern with only 1
field added. Other patterns should be able to use that field to-do their
implementation too.

3) Non-repudiation and long term non-repudiation.

Non-repudiation is proving data hasn't changed.  This is often (if not
always) done with x509 public certificates (chained to a certificate
authority).

Long term non-repudiation is what happens when the certificates of the
certificate authority are expired (or revoked) and everything ever signed
(ever) with that certificate's public key then becomes "no longer provable
as ever being authentic".  That is where RFC3126 [1] and RFC3161 [2] come
in (or worm drives [hardware], etc).

For either (or both) of these it is an operation of the encryptor to
sign/hash the data (with or without third party trusted timestap of the
signing event) and encrypt that with their own private key and distribute
the results (before and after encrypting if required) along with their
public key. This structure is a bit more complex but feasible, it is a map
of digital signature formats and the chain of dig sig attestations.  The
map's key being the method (i.e. CRC32, PKCS7 [3], XmlDigSig [4]) and then
a list of map where that key is "purpose" of signature (what your attesting
too).  As a sibling field to the list another field for "the attester" as
bytes (e.g. their PKCS12 [5] for the map of PKCS7 signatures).

4) Authorization

We should have a policy of "404" for data, topics, partitions (etc) if
authenticated connections do not have access.  In "secure mode" any non
authenticated connections should get a "404" type message on everything.
Knowing "something is there" is a security risk in many uses cases.  So if
you don't have access you don't even see it.  Baking "that" into Kafka
along with some interface for entitlement (access management) systems
(pretty standard) is all that I think needs to be done to the core project.
 I want to tackle item later in the year after summer after the other three
are complete.

I look forward to thoughts on this and anyone else interested in working
with us on these items.

[0]
http://www.emc.com/emc-plus/rsa-labs/standards-initiatives/what-is-a-digital-envelope.htm
[1] http://tools.ietf.org/html/rfc3126
[2] http://tools.ietf.org/html/rfc3161
[3]
http://www.emc.com/emc-plus/rsa-labs/standards-initiatives/pkcs-7-cryptographic-message-syntax-standar.htm
[4] http://en.wikipedia.org/wiki/XML_Signature
[5] http://en.wikipedia.org/wiki/PKCS_12

/*******************************************
 Joe Stein
 Founder, Principal Consultant
 Big Data Open Source Security LLC
 http://www.stealth.ly
 Twitter: @allthingshadoop <http://www.twitter.com/allthingshadoop>
********************************************/

Re: [DISCUSS] Kafka Security Specific Features

Posted by Todd Palino <tp...@linkedin.com.INVALID>.
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 be concerned about a malicious user. Consider a ³typical²
technology environment where many people have administrative access to
systems. This is the one where you need to not have the data visible to
anyone unless they have a specific use for it, which means having it
encrypted. In almost any audit situation, you need to be able to show a
trail of exactly who modified the data, and exactly who viewed the data.

Now, I do agree that not everything has to be done within Kafka, and the
producers and consumers can coordinate their own encryption. But I think
it¹s useful to have the concept of an envelope for a message within Kafka.
This can be used to hold all sorts of useful information, such as hashes
of the encryption keys that were used to encrypt a message, or the
signature of the message itself (so that you can have both confidentiality
and integrity). It can also be used to hold things like the time a message
was received into your infrastructure, or the specific Kafka cluster it
was stored in. A special consumer and producer, such as the mirror maker,
would be able to preserve this envelope across clusters.

-Todd


On 6/5/14, 2:18 PM, "Jay Kreps" <ja...@gmail.com> wrote:

>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 the Kafka
>server? If we assume (3) then it seems we are in a pretty bad state as
>almost any facility Kafka provides can be subverted by the root user just
>changing the Kafka code to not enforce that facility. Which of these
>levels
>of access are we assuming?
>
>Also which things actually need to be done inside Kafka and which can be
>done externally? Nothing prevents users from encrypting data they put into
>Kafka today, it is just that Kafka doesn't do this for you. But is there a
>reason you want Kafka to do this?
>
>The reason I am pushing on these things a bit is because I want to make
>sure we don't end up with a set of requirements so broad we can never
>really get them implemented...
>
>-Jay
>
>
>
>
>On Thu, Jun 5, 2014 at 2:05 PM, Todd Palino <tp...@linkedin.com.invalid>
>wrote:
>
>> 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" <ja...@gmail.com> wrote:
>>
>> >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 encryption scheme? Is this actually
>> >needed?
>> >Isn't a on-the-wire encryption when combined with mutual authentication
>> >and
>> >permissions sufficient for most uses?
>> >
>> >On-disk encryption seems unnecessary because if an attacker can get
>>root
>> >on
>> >the kafka boxes it can potentially modify Kafka to do anything he or
>>she
>> >wants with data. So this seems to break any security model.
>> >
>> >I understand the problem of a large organization not really having a
>> >trusted network and wanting to secure data transfer and limit and audit
>> >data access. The uses for these other things I don't totally
>>understand.
>> >
>> >Also it would be worth understanding the state of other messaging and
>> >storage systems (Hadoop, dbs, etc). What features do they support. I
>>think
>> >there is a sense in which you don't have to run faster than the bear,
>>but
>> >only faster then your friends. :-)
>> >
>> >-Jay
>> >
>> >
>> >On Wed, Jun 4, 2014 at 5:57 PM, Joe Stein <jo...@stealth.ly> wrote:
>> >
>> >> 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 <ja...@gmail.com>
>>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
>>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 most the LinkedIn folks are working on the consumer right
>>now I
>> >> think
>> >> > this would be a great project for any other interested people to
>>take
>> >>on.
>> >> > There are some challenges in doing these things distributed but it
>>can
>> >> also
>> >> > be a lot of fun.
>> >> >
>> >> > I think a good first step would be to get a written plan we can all
>> >>agree
>> >> > on for how things should work. Then we can break things down into
>> >>chunks
>> >> > that can be done independently while still aiming at a good end
>>state.
>> >> >
>> >> > I had tried to write up some notes that summarized at least the
>> >>thoughts
>> >> I
>> >> > had had on security:
>> >> > https://cwiki.apache.org/confluence/display/KAFKA/Security
>> >> >
>> >> > What do you think of that?
>> >> >
>> >> > One assumption I had (which may be incorrect) is that although we
>>want
>> >> all
>> >> > the things in your list, the two most pressing would be
>>authentication
>> >> and
>> >> > authorization, and that was all that write up covered. You have
>>more
>> >> > experience in this domain, so I wonder how you would prioritize?
>> >> >
>> >> > Those notes are really sketchy, so I think the first goal I would
>>have
>> >> > would be to get to a real spec we can all agree on and discuss. A
>>lot
>> >>of
>> >> > the security stuff has a high human interaction element and needs
>>to
>> >>work
>> >> > in pretty different domains and different companies so getting this
>> >>kind
>> >> of
>> >> > review is important.
>> >> >
>> >> > -Jay
>> >> >
>> >> >
>> >> > On Tue, Jun 3, 2014 at 12:57 PM, Joe Stein <jo...@stealth.ly>
>> >>wrote:
>> >> >
>> >> > > 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.  This comes from what folks are doing "in the
>>wild" to
>> >> > > workaround and implement security with Kafka as it is today and
>>also
>> >> > what I
>> >> > > have discovered from organizations about their blockers. It also
>> >>picks
>> >> up
>> >> > > from the wiki (which I should have time to update later in the
>>week
>> >> based
>> >> > > on the below and feedback from the thread).
>> >> > >
>> >> > > 1) Transport Layer Security (i.e. SSL)
>> >> > >
>> >> > > This also includes client authentication in addition to
>>in-transit
>> >> > security
>> >> > > layer.  This work has been picked up here
>> >> > > https://issues.apache.org/jira/browse/KAFKA-1477 and do
>>appreciate
>> >>any
>> >> > > thoughts, comments, feedback, tomatoes, whatever for this patch.
>> It
>> >> is a
>> >> > > pickup from the fork of the work first done here
>> >> > > https://github.com/relango/kafka/tree/kafka_security.
>> >> > >
>> >> > > 2) Data encryption at rest.
>> >> > >
>> >> > > This is very important and something that can be facilitated
>>within
>> >>the
>> >> > > wire protocol. It requires an additional map data structure for
>>the
>> >> > > "encrypted [data encryption key]". With this map (either in your
>> >>object
>> >> > or
>> >> > > in the wire protocol) you can store the dynamically generated
>> >>symmetric
>> >> > key
>> >> > > (for each message) and then encrypt the data using that
>>dynamically
>> >> > > generated key.  You then encrypt the encryption key using each
>> >>public
>> >> key
>> >> > > for whom is expected to be able to decrypt the encryption key to
>> >>then
>> >> > > decrypt the message.  For each public key encrypted symmetric key
>> >> (which
>> >> > is
>> >> > > now the "encrypted [data encryption key]" along with which public
>> >>key
>> >> it
>> >> > > was encrypted with for (so a map of [publicKey] =
>> >> > > encryptedDataEncryptionKey) as a chain.   Other patterns can be
>> >> > implemented
>> >> > > but this is a pretty standard digital enveloping [0] pattern with
>> >>only
>> >> 1
>> >> > > field added. Other patterns should be able to use that field
>>to-do
>> >> their
>> >> > > implementation too.
>> >> > >
>> >> > > 3) Non-repudiation and long term non-repudiation.
>> >> > >
>> >> > > Non-repudiation is proving data hasn't changed.  This is often
>>(if
>> >>not
>> >> > > always) done with x509 public certificates (chained to a
>>certificate
>> >> > > authority).
>> >> > >
>> >> > > Long term non-repudiation is what happens when the certificates
>>of
>> >>the
>> >> > > certificate authority are expired (or revoked) and everything
>>ever
>> >> signed
>> >> > > (ever) with that certificate's public key then becomes "no longer
>> >> > provable
>> >> > > as ever being authentic".  That is where RFC3126 [1] and RFC3161
>>[2]
>> >> come
>> >> > > in (or worm drives [hardware], etc).
>> >> > >
>> >> > > For either (or both) of these it is an operation of the
>>encryptor to
>> >> > > sign/hash the data (with or without third party trusted timestap
>>of
>> >>the
>> >> > > signing event) and encrypt that with their own private key and
>> >> distribute
>> >> > > the results (before and after encrypting if required) along with
>> >>their
>> >> > > public key. This structure is a bit more complex but feasible, it
>> >>is a
>> >> > map
>> >> > > of digital signature formats and the chain of dig sig
>>attestations.
>> >>  The
>> >> > > map's key being the method (i.e. CRC32, PKCS7 [3], XmlDigSig [4])
>> >>and
>> >> > then
>> >> > > a list of map where that key is "purpose" of signature (what your
>> >> > attesting
>> >> > > too).  As a sibling field to the list another field for "the
>> >>attester"
>> >> as
>> >> > > bytes (e.g. their PKCS12 [5] for the map of PKCS7 signatures).
>> >> > >
>> >> > > 4) Authorization
>> >> > >
>> >> > > We should have a policy of "404" for data, topics, partitions
>>(etc)
>> >>if
>> >> > > authenticated connections do not have access.  In "secure mode"
>>any
>> >>non
>> >> > > authenticated connections should get a "404" type message on
>> >> everything.
>> >> > > Knowing "something is there" is a security risk in many uses
>>cases.
>> >> So
>> >> > if
>> >> > > you don't have access you don't even see it.  Baking "that" into
>> >>Kafka
>> >> > > along with some interface for entitlement (access management)
>> >>systems
>> >> > > (pretty standard) is all that I think needs to be done to the
>>core
>> >> > project.
>> >> > >  I want to tackle item later in the year after summer after the
>> >>other
>> >> > three
>> >> > > are complete.
>> >> > >
>> >> > > I look forward to thoughts on this and anyone else interested in
>> >> working
>> >> > > with us on these items.
>> >> > >
>> >> > > [0]
>> >> > >
>> >> > >
>> >> >
>> >>
>> >>
>> 
>>http://www.emc.com/emc-plus/rsa-labs/standards-initiatives/what-is-a-digi
>> >>tal-envelope.htm
>> >> > > [1] http://tools.ietf.org/html/rfc3126
>> >> > > [2] http://tools.ietf.org/html/rfc3161
>> >> > > [3]
>> >> > >
>> >> > >
>> >> >
>> >>
>> >>
>> 
>>http://www.emc.com/emc-plus/rsa-labs/standards-initiatives/pkcs-7-cryptog
>> >>raphic-message-syntax-standar.htm
>> >> > > [4] http://en.wikipedia.org/wiki/XML_Signature
>> >> > > [5] http://en.wikipedia.org/wiki/PKCS_12
>> >> > >
>> >> > > /*******************************************
>> >> > >  Joe Stein
>> >> > >  Founder, Principal Consultant
>> >> > >  Big Data Open Source Security LLC
>> >> > >  http://www.stealth.ly
>> >> > >  Twitter: @allthingshadoop
>><http://www.twitter.com/allthingshadoop>
>> >> > > ********************************************/
>> >> > >
>> >> >
>> >>
>>
>>


Re: [DISCUSS] Kafka Security Specific Features

Posted by Todd Palino <tp...@linkedin.com.INVALID>.
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 be concerned about a malicious user. Consider a ³typical²
technology environment where many people have administrative access to
systems. This is the one where you need to not have the data visible to
anyone unless they have a specific use for it, which means having it
encrypted. In almost any audit situation, you need to be able to show a
trail of exactly who modified the data, and exactly who viewed the data.

Now, I do agree that not everything has to be done within Kafka, and the
producers and consumers can coordinate their own encryption. But I think
it¹s useful to have the concept of an envelope for a message within Kafka.
This can be used to hold all sorts of useful information, such as hashes
of the encryption keys that were used to encrypt a message, or the
signature of the message itself (so that you can have both confidentiality
and integrity). It can also be used to hold things like the time a message
was received into your infrastructure, or the specific Kafka cluster it
was stored in. A special consumer and producer, such as the mirror maker,
would be able to preserve this envelope across clusters.

-Todd


On 6/5/14, 2:18 PM, "Jay Kreps" <ja...@gmail.com> wrote:

>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 the Kafka
>server? If we assume (3) then it seems we are in a pretty bad state as
>almost any facility Kafka provides can be subverted by the root user just
>changing the Kafka code to not enforce that facility. Which of these
>levels
>of access are we assuming?
>
>Also which things actually need to be done inside Kafka and which can be
>done externally? Nothing prevents users from encrypting data they put into
>Kafka today, it is just that Kafka doesn't do this for you. But is there a
>reason you want Kafka to do this?
>
>The reason I am pushing on these things a bit is because I want to make
>sure we don't end up with a set of requirements so broad we can never
>really get them implemented...
>
>-Jay
>
>
>
>
>On Thu, Jun 5, 2014 at 2:05 PM, Todd Palino <tp...@linkedin.com.invalid>
>wrote:
>
>> 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" <ja...@gmail.com> wrote:
>>
>> >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 encryption scheme? Is this actually
>> >needed?
>> >Isn't a on-the-wire encryption when combined with mutual authentication
>> >and
>> >permissions sufficient for most uses?
>> >
>> >On-disk encryption seems unnecessary because if an attacker can get
>>root
>> >on
>> >the kafka boxes it can potentially modify Kafka to do anything he or
>>she
>> >wants with data. So this seems to break any security model.
>> >
>> >I understand the problem of a large organization not really having a
>> >trusted network and wanting to secure data transfer and limit and audit
>> >data access. The uses for these other things I don't totally
>>understand.
>> >
>> >Also it would be worth understanding the state of other messaging and
>> >storage systems (Hadoop, dbs, etc). What features do they support. I
>>think
>> >there is a sense in which you don't have to run faster than the bear,
>>but
>> >only faster then your friends. :-)
>> >
>> >-Jay
>> >
>> >
>> >On Wed, Jun 4, 2014 at 5:57 PM, Joe Stein <jo...@stealth.ly> wrote:
>> >
>> >> 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 <ja...@gmail.com>
>>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
>>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 most the LinkedIn folks are working on the consumer right
>>now I
>> >> think
>> >> > this would be a great project for any other interested people to
>>take
>> >>on.
>> >> > There are some challenges in doing these things distributed but it
>>can
>> >> also
>> >> > be a lot of fun.
>> >> >
>> >> > I think a good first step would be to get a written plan we can all
>> >>agree
>> >> > on for how things should work. Then we can break things down into
>> >>chunks
>> >> > that can be done independently while still aiming at a good end
>>state.
>> >> >
>> >> > I had tried to write up some notes that summarized at least the
>> >>thoughts
>> >> I
>> >> > had had on security:
>> >> > https://cwiki.apache.org/confluence/display/KAFKA/Security
>> >> >
>> >> > What do you think of that?
>> >> >
>> >> > One assumption I had (which may be incorrect) is that although we
>>want
>> >> all
>> >> > the things in your list, the two most pressing would be
>>authentication
>> >> and
>> >> > authorization, and that was all that write up covered. You have
>>more
>> >> > experience in this domain, so I wonder how you would prioritize?
>> >> >
>> >> > Those notes are really sketchy, so I think the first goal I would
>>have
>> >> > would be to get to a real spec we can all agree on and discuss. A
>>lot
>> >>of
>> >> > the security stuff has a high human interaction element and needs
>>to
>> >>work
>> >> > in pretty different domains and different companies so getting this
>> >>kind
>> >> of
>> >> > review is important.
>> >> >
>> >> > -Jay
>> >> >
>> >> >
>> >> > On Tue, Jun 3, 2014 at 12:57 PM, Joe Stein <jo...@stealth.ly>
>> >>wrote:
>> >> >
>> >> > > 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.  This comes from what folks are doing "in the
>>wild" to
>> >> > > workaround and implement security with Kafka as it is today and
>>also
>> >> > what I
>> >> > > have discovered from organizations about their blockers. It also
>> >>picks
>> >> up
>> >> > > from the wiki (which I should have time to update later in the
>>week
>> >> based
>> >> > > on the below and feedback from the thread).
>> >> > >
>> >> > > 1) Transport Layer Security (i.e. SSL)
>> >> > >
>> >> > > This also includes client authentication in addition to
>>in-transit
>> >> > security
>> >> > > layer.  This work has been picked up here
>> >> > > https://issues.apache.org/jira/browse/KAFKA-1477 and do
>>appreciate
>> >>any
>> >> > > thoughts, comments, feedback, tomatoes, whatever for this patch.
>> It
>> >> is a
>> >> > > pickup from the fork of the work first done here
>> >> > > https://github.com/relango/kafka/tree/kafka_security.
>> >> > >
>> >> > > 2) Data encryption at rest.
>> >> > >
>> >> > > This is very important and something that can be facilitated
>>within
>> >>the
>> >> > > wire protocol. It requires an additional map data structure for
>>the
>> >> > > "encrypted [data encryption key]". With this map (either in your
>> >>object
>> >> > or
>> >> > > in the wire protocol) you can store the dynamically generated
>> >>symmetric
>> >> > key
>> >> > > (for each message) and then encrypt the data using that
>>dynamically
>> >> > > generated key.  You then encrypt the encryption key using each
>> >>public
>> >> key
>> >> > > for whom is expected to be able to decrypt the encryption key to
>> >>then
>> >> > > decrypt the message.  For each public key encrypted symmetric key
>> >> (which
>> >> > is
>> >> > > now the "encrypted [data encryption key]" along with which public
>> >>key
>> >> it
>> >> > > was encrypted with for (so a map of [publicKey] =
>> >> > > encryptedDataEncryptionKey) as a chain.   Other patterns can be
>> >> > implemented
>> >> > > but this is a pretty standard digital enveloping [0] pattern with
>> >>only
>> >> 1
>> >> > > field added. Other patterns should be able to use that field
>>to-do
>> >> their
>> >> > > implementation too.
>> >> > >
>> >> > > 3) Non-repudiation and long term non-repudiation.
>> >> > >
>> >> > > Non-repudiation is proving data hasn't changed.  This is often
>>(if
>> >>not
>> >> > > always) done with x509 public certificates (chained to a
>>certificate
>> >> > > authority).
>> >> > >
>> >> > > Long term non-repudiation is what happens when the certificates
>>of
>> >>the
>> >> > > certificate authority are expired (or revoked) and everything
>>ever
>> >> signed
>> >> > > (ever) with that certificate's public key then becomes "no longer
>> >> > provable
>> >> > > as ever being authentic".  That is where RFC3126 [1] and RFC3161
>>[2]
>> >> come
>> >> > > in (or worm drives [hardware], etc).
>> >> > >
>> >> > > For either (or both) of these it is an operation of the
>>encryptor to
>> >> > > sign/hash the data (with or without third party trusted timestap
>>of
>> >>the
>> >> > > signing event) and encrypt that with their own private key and
>> >> distribute
>> >> > > the results (before and after encrypting if required) along with
>> >>their
>> >> > > public key. This structure is a bit more complex but feasible, it
>> >>is a
>> >> > map
>> >> > > of digital signature formats and the chain of dig sig
>>attestations.
>> >>  The
>> >> > > map's key being the method (i.e. CRC32, PKCS7 [3], XmlDigSig [4])
>> >>and
>> >> > then
>> >> > > a list of map where that key is "purpose" of signature (what your
>> >> > attesting
>> >> > > too).  As a sibling field to the list another field for "the
>> >>attester"
>> >> as
>> >> > > bytes (e.g. their PKCS12 [5] for the map of PKCS7 signatures).
>> >> > >
>> >> > > 4) Authorization
>> >> > >
>> >> > > We should have a policy of "404" for data, topics, partitions
>>(etc)
>> >>if
>> >> > > authenticated connections do not have access.  In "secure mode"
>>any
>> >>non
>> >> > > authenticated connections should get a "404" type message on
>> >> everything.
>> >> > > Knowing "something is there" is a security risk in many uses
>>cases.
>> >> So
>> >> > if
>> >> > > you don't have access you don't even see it.  Baking "that" into
>> >>Kafka
>> >> > > along with some interface for entitlement (access management)
>> >>systems
>> >> > > (pretty standard) is all that I think needs to be done to the
>>core
>> >> > project.
>> >> > >  I want to tackle item later in the year after summer after the
>> >>other
>> >> > three
>> >> > > are complete.
>> >> > >
>> >> > > I look forward to thoughts on this and anyone else interested in
>> >> working
>> >> > > with us on these items.
>> >> > >
>> >> > > [0]
>> >> > >
>> >> > >
>> >> >
>> >>
>> >>
>> 
>>http://www.emc.com/emc-plus/rsa-labs/standards-initiatives/what-is-a-digi
>> >>tal-envelope.htm
>> >> > > [1] http://tools.ietf.org/html/rfc3126
>> >> > > [2] http://tools.ietf.org/html/rfc3161
>> >> > > [3]
>> >> > >
>> >> > >
>> >> >
>> >>
>> >>
>> 
>>http://www.emc.com/emc-plus/rsa-labs/standards-initiatives/pkcs-7-cryptog
>> >>raphic-message-syntax-standar.htm
>> >> > > [4] http://en.wikipedia.org/wiki/XML_Signature
>> >> > > [5] http://en.wikipedia.org/wiki/PKCS_12
>> >> > >
>> >> > > /*******************************************
>> >> > >  Joe Stein
>> >> > >  Founder, Principal Consultant
>> >> > >  Big Data Open Source Security LLC
>> >> > >  http://www.stealth.ly
>> >> > >  Twitter: @allthingshadoop
>><http://www.twitter.com/allthingshadoop>
>> >> > > ********************************************/
>> >> > >
>> >> >
>> >>
>>
>>


Re: [DISCUSS] Kafka Security Specific Features

Posted by Jay Kreps <ja...@gmail.com>.
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 the Kafka
server? If we assume (3) then it seems we are in a pretty bad state as
almost any facility Kafka provides can be subverted by the root user just
changing the Kafka code to not enforce that facility. Which of these levels
of access are we assuming?

Also which things actually need to be done inside Kafka and which can be
done externally? Nothing prevents users from encrypting data they put into
Kafka today, it is just that Kafka doesn't do this for you. But is there a
reason you want Kafka to do this?

The reason I am pushing on these things a bit is because I want to make
sure we don't end up with a set of requirements so broad we can never
really get them implemented...

-Jay




On Thu, Jun 5, 2014 at 2:05 PM, Todd Palino <tp...@linkedin.com.invalid>
wrote:

> 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" <ja...@gmail.com> wrote:
>
> >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 encryption scheme? Is this actually
> >needed?
> >Isn't a on-the-wire encryption when combined with mutual authentication
> >and
> >permissions sufficient for most uses?
> >
> >On-disk encryption seems unnecessary because if an attacker can get root
> >on
> >the kafka boxes it can potentially modify Kafka to do anything he or she
> >wants with data. So this seems to break any security model.
> >
> >I understand the problem of a large organization not really having a
> >trusted network and wanting to secure data transfer and limit and audit
> >data access. The uses for these other things I don't totally understand.
> >
> >Also it would be worth understanding the state of other messaging and
> >storage systems (Hadoop, dbs, etc). What features do they support. I think
> >there is a sense in which you don't have to run faster than the bear, but
> >only faster then your friends. :-)
> >
> >-Jay
> >
> >
> >On Wed, Jun 4, 2014 at 5:57 PM, Joe Stein <jo...@stealth.ly> wrote:
> >
> >> 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 <ja...@gmail.com> 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 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 most the LinkedIn folks are working on the consumer right now I
> >> think
> >> > this would be a great project for any other interested people to take
> >>on.
> >> > There are some challenges in doing these things distributed but it can
> >> also
> >> > be a lot of fun.
> >> >
> >> > I think a good first step would be to get a written plan we can all
> >>agree
> >> > on for how things should work. Then we can break things down into
> >>chunks
> >> > that can be done independently while still aiming at a good end state.
> >> >
> >> > I had tried to write up some notes that summarized at least the
> >>thoughts
> >> I
> >> > had had on security:
> >> > https://cwiki.apache.org/confluence/display/KAFKA/Security
> >> >
> >> > What do you think of that?
> >> >
> >> > One assumption I had (which may be incorrect) is that although we want
> >> all
> >> > the things in your list, the two most pressing would be authentication
> >> and
> >> > authorization, and that was all that write up covered. You have more
> >> > experience in this domain, so I wonder how you would prioritize?
> >> >
> >> > Those notes are really sketchy, so I think the first goal I would have
> >> > would be to get to a real spec we can all agree on and discuss. A lot
> >>of
> >> > the security stuff has a high human interaction element and needs to
> >>work
> >> > in pretty different domains and different companies so getting this
> >>kind
> >> of
> >> > review is important.
> >> >
> >> > -Jay
> >> >
> >> >
> >> > On Tue, Jun 3, 2014 at 12:57 PM, Joe Stein <jo...@stealth.ly>
> >>wrote:
> >> >
> >> > > 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.  This comes from what folks are doing "in the wild" to
> >> > > workaround and implement security with Kafka as it is today and also
> >> > what I
> >> > > have discovered from organizations about their blockers. It also
> >>picks
> >> up
> >> > > from the wiki (which I should have time to update later in the week
> >> based
> >> > > on the below and feedback from the thread).
> >> > >
> >> > > 1) Transport Layer Security (i.e. SSL)
> >> > >
> >> > > This also includes client authentication in addition to in-transit
> >> > security
> >> > > layer.  This work has been picked up here
> >> > > https://issues.apache.org/jira/browse/KAFKA-1477 and do appreciate
> >>any
> >> > > thoughts, comments, feedback, tomatoes, whatever for this patch.  It
> >> is a
> >> > > pickup from the fork of the work first done here
> >> > > https://github.com/relango/kafka/tree/kafka_security.
> >> > >
> >> > > 2) Data encryption at rest.
> >> > >
> >> > > This is very important and something that can be facilitated within
> >>the
> >> > > wire protocol. It requires an additional map data structure for the
> >> > > "encrypted [data encryption key]". With this map (either in your
> >>object
> >> > or
> >> > > in the wire protocol) you can store the dynamically generated
> >>symmetric
> >> > key
> >> > > (for each message) and then encrypt the data using that dynamically
> >> > > generated key.  You then encrypt the encryption key using each
> >>public
> >> key
> >> > > for whom is expected to be able to decrypt the encryption key to
> >>then
> >> > > decrypt the message.  For each public key encrypted symmetric key
> >> (which
> >> > is
> >> > > now the "encrypted [data encryption key]" along with which public
> >>key
> >> it
> >> > > was encrypted with for (so a map of [publicKey] =
> >> > > encryptedDataEncryptionKey) as a chain.   Other patterns can be
> >> > implemented
> >> > > but this is a pretty standard digital enveloping [0] pattern with
> >>only
> >> 1
> >> > > field added. Other patterns should be able to use that field to-do
> >> their
> >> > > implementation too.
> >> > >
> >> > > 3) Non-repudiation and long term non-repudiation.
> >> > >
> >> > > Non-repudiation is proving data hasn't changed.  This is often (if
> >>not
> >> > > always) done with x509 public certificates (chained to a certificate
> >> > > authority).
> >> > >
> >> > > Long term non-repudiation is what happens when the certificates of
> >>the
> >> > > certificate authority are expired (or revoked) and everything ever
> >> signed
> >> > > (ever) with that certificate's public key then becomes "no longer
> >> > provable
> >> > > as ever being authentic".  That is where RFC3126 [1] and RFC3161 [2]
> >> come
> >> > > in (or worm drives [hardware], etc).
> >> > >
> >> > > For either (or both) of these it is an operation of the encryptor to
> >> > > sign/hash the data (with or without third party trusted timestap of
> >>the
> >> > > signing event) and encrypt that with their own private key and
> >> distribute
> >> > > the results (before and after encrypting if required) along with
> >>their
> >> > > public key. This structure is a bit more complex but feasible, it
> >>is a
> >> > map
> >> > > of digital signature formats and the chain of dig sig attestations.
> >>  The
> >> > > map's key being the method (i.e. CRC32, PKCS7 [3], XmlDigSig [4])
> >>and
> >> > then
> >> > > a list of map where that key is "purpose" of signature (what your
> >> > attesting
> >> > > too).  As a sibling field to the list another field for "the
> >>attester"
> >> as
> >> > > bytes (e.g. their PKCS12 [5] for the map of PKCS7 signatures).
> >> > >
> >> > > 4) Authorization
> >> > >
> >> > > We should have a policy of "404" for data, topics, partitions (etc)
> >>if
> >> > > authenticated connections do not have access.  In "secure mode" any
> >>non
> >> > > authenticated connections should get a "404" type message on
> >> everything.
> >> > > Knowing "something is there" is a security risk in many uses cases.
> >> So
> >> > if
> >> > > you don't have access you don't even see it.  Baking "that" into
> >>Kafka
> >> > > along with some interface for entitlement (access management)
> >>systems
> >> > > (pretty standard) is all that I think needs to be done to the core
> >> > project.
> >> > >  I want to tackle item later in the year after summer after the
> >>other
> >> > three
> >> > > are complete.
> >> > >
> >> > > I look forward to thoughts on this and anyone else interested in
> >> working
> >> > > with us on these items.
> >> > >
> >> > > [0]
> >> > >
> >> > >
> >> >
> >>
> >>
> http://www.emc.com/emc-plus/rsa-labs/standards-initiatives/what-is-a-digi
> >>tal-envelope.htm
> >> > > [1] http://tools.ietf.org/html/rfc3126
> >> > > [2] http://tools.ietf.org/html/rfc3161
> >> > > [3]
> >> > >
> >> > >
> >> >
> >>
> >>
> http://www.emc.com/emc-plus/rsa-labs/standards-initiatives/pkcs-7-cryptog
> >>raphic-message-syntax-standar.htm
> >> > > [4] http://en.wikipedia.org/wiki/XML_Signature
> >> > > [5] http://en.wikipedia.org/wiki/PKCS_12
> >> > >
> >> > > /*******************************************
> >> > >  Joe Stein
> >> > >  Founder, Principal Consultant
> >> > >  Big Data Open Source Security LLC
> >> > >  http://www.stealth.ly
> >> > >  Twitter: @allthingshadoop <http://www.twitter.com/allthingshadoop>
> >> > > ********************************************/
> >> > >
> >> >
> >>
>
>

Re: [DISCUSS] Kafka Security Specific Features

Posted by Jay Kreps <ja...@gmail.com>.
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 the Kafka
server? If we assume (3) then it seems we are in a pretty bad state as
almost any facility Kafka provides can be subverted by the root user just
changing the Kafka code to not enforce that facility. Which of these levels
of access are we assuming?

Also which things actually need to be done inside Kafka and which can be
done externally? Nothing prevents users from encrypting data they put into
Kafka today, it is just that Kafka doesn't do this for you. But is there a
reason you want Kafka to do this?

The reason I am pushing on these things a bit is because I want to make
sure we don't end up with a set of requirements so broad we can never
really get them implemented...

-Jay




On Thu, Jun 5, 2014 at 2:05 PM, Todd Palino <tp...@linkedin.com.invalid>
wrote:

> 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" <ja...@gmail.com> wrote:
>
> >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 encryption scheme? Is this actually
> >needed?
> >Isn't a on-the-wire encryption when combined with mutual authentication
> >and
> >permissions sufficient for most uses?
> >
> >On-disk encryption seems unnecessary because if an attacker can get root
> >on
> >the kafka boxes it can potentially modify Kafka to do anything he or she
> >wants with data. So this seems to break any security model.
> >
> >I understand the problem of a large organization not really having a
> >trusted network and wanting to secure data transfer and limit and audit
> >data access. The uses for these other things I don't totally understand.
> >
> >Also it would be worth understanding the state of other messaging and
> >storage systems (Hadoop, dbs, etc). What features do they support. I think
> >there is a sense in which you don't have to run faster than the bear, but
> >only faster then your friends. :-)
> >
> >-Jay
> >
> >
> >On Wed, Jun 4, 2014 at 5:57 PM, Joe Stein <jo...@stealth.ly> wrote:
> >
> >> 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 <ja...@gmail.com> 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 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 most the LinkedIn folks are working on the consumer right now I
> >> think
> >> > this would be a great project for any other interested people to take
> >>on.
> >> > There are some challenges in doing these things distributed but it can
> >> also
> >> > be a lot of fun.
> >> >
> >> > I think a good first step would be to get a written plan we can all
> >>agree
> >> > on for how things should work. Then we can break things down into
> >>chunks
> >> > that can be done independently while still aiming at a good end state.
> >> >
> >> > I had tried to write up some notes that summarized at least the
> >>thoughts
> >> I
> >> > had had on security:
> >> > https://cwiki.apache.org/confluence/display/KAFKA/Security
> >> >
> >> > What do you think of that?
> >> >
> >> > One assumption I had (which may be incorrect) is that although we want
> >> all
> >> > the things in your list, the two most pressing would be authentication
> >> and
> >> > authorization, and that was all that write up covered. You have more
> >> > experience in this domain, so I wonder how you would prioritize?
> >> >
> >> > Those notes are really sketchy, so I think the first goal I would have
> >> > would be to get to a real spec we can all agree on and discuss. A lot
> >>of
> >> > the security stuff has a high human interaction element and needs to
> >>work
> >> > in pretty different domains and different companies so getting this
> >>kind
> >> of
> >> > review is important.
> >> >
> >> > -Jay
> >> >
> >> >
> >> > On Tue, Jun 3, 2014 at 12:57 PM, Joe Stein <jo...@stealth.ly>
> >>wrote:
> >> >
> >> > > 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.  This comes from what folks are doing "in the wild" to
> >> > > workaround and implement security with Kafka as it is today and also
> >> > what I
> >> > > have discovered from organizations about their blockers. It also
> >>picks
> >> up
> >> > > from the wiki (which I should have time to update later in the week
> >> based
> >> > > on the below and feedback from the thread).
> >> > >
> >> > > 1) Transport Layer Security (i.e. SSL)
> >> > >
> >> > > This also includes client authentication in addition to in-transit
> >> > security
> >> > > layer.  This work has been picked up here
> >> > > https://issues.apache.org/jira/browse/KAFKA-1477 and do appreciate
> >>any
> >> > > thoughts, comments, feedback, tomatoes, whatever for this patch.  It
> >> is a
> >> > > pickup from the fork of the work first done here
> >> > > https://github.com/relango/kafka/tree/kafka_security.
> >> > >
> >> > > 2) Data encryption at rest.
> >> > >
> >> > > This is very important and something that can be facilitated within
> >>the
> >> > > wire protocol. It requires an additional map data structure for the
> >> > > "encrypted [data encryption key]". With this map (either in your
> >>object
> >> > or
> >> > > in the wire protocol) you can store the dynamically generated
> >>symmetric
> >> > key
> >> > > (for each message) and then encrypt the data using that dynamically
> >> > > generated key.  You then encrypt the encryption key using each
> >>public
> >> key
> >> > > for whom is expected to be able to decrypt the encryption key to
> >>then
> >> > > decrypt the message.  For each public key encrypted symmetric key
> >> (which
> >> > is
> >> > > now the "encrypted [data encryption key]" along with which public
> >>key
> >> it
> >> > > was encrypted with for (so a map of [publicKey] =
> >> > > encryptedDataEncryptionKey) as a chain.   Other patterns can be
> >> > implemented
> >> > > but this is a pretty standard digital enveloping [0] pattern with
> >>only
> >> 1
> >> > > field added. Other patterns should be able to use that field to-do
> >> their
> >> > > implementation too.
> >> > >
> >> > > 3) Non-repudiation and long term non-repudiation.
> >> > >
> >> > > Non-repudiation is proving data hasn't changed.  This is often (if
> >>not
> >> > > always) done with x509 public certificates (chained to a certificate
> >> > > authority).
> >> > >
> >> > > Long term non-repudiation is what happens when the certificates of
> >>the
> >> > > certificate authority are expired (or revoked) and everything ever
> >> signed
> >> > > (ever) with that certificate's public key then becomes "no longer
> >> > provable
> >> > > as ever being authentic".  That is where RFC3126 [1] and RFC3161 [2]
> >> come
> >> > > in (or worm drives [hardware], etc).
> >> > >
> >> > > For either (or both) of these it is an operation of the encryptor to
> >> > > sign/hash the data (with or without third party trusted timestap of
> >>the
> >> > > signing event) and encrypt that with their own private key and
> >> distribute
> >> > > the results (before and after encrypting if required) along with
> >>their
> >> > > public key. This structure is a bit more complex but feasible, it
> >>is a
> >> > map
> >> > > of digital signature formats and the chain of dig sig attestations.
> >>  The
> >> > > map's key being the method (i.e. CRC32, PKCS7 [3], XmlDigSig [4])
> >>and
> >> > then
> >> > > a list of map where that key is "purpose" of signature (what your
> >> > attesting
> >> > > too).  As a sibling field to the list another field for "the
> >>attester"
> >> as
> >> > > bytes (e.g. their PKCS12 [5] for the map of PKCS7 signatures).
> >> > >
> >> > > 4) Authorization
> >> > >
> >> > > We should have a policy of "404" for data, topics, partitions (etc)
> >>if
> >> > > authenticated connections do not have access.  In "secure mode" any
> >>non
> >> > > authenticated connections should get a "404" type message on
> >> everything.
> >> > > Knowing "something is there" is a security risk in many uses cases.
> >> So
> >> > if
> >> > > you don't have access you don't even see it.  Baking "that" into
> >>Kafka
> >> > > along with some interface for entitlement (access management)
> >>systems
> >> > > (pretty standard) is all that I think needs to be done to the core
> >> > project.
> >> > >  I want to tackle item later in the year after summer after the
> >>other
> >> > three
> >> > > are complete.
> >> > >
> >> > > I look forward to thoughts on this and anyone else interested in
> >> working
> >> > > with us on these items.
> >> > >
> >> > > [0]
> >> > >
> >> > >
> >> >
> >>
> >>
> http://www.emc.com/emc-plus/rsa-labs/standards-initiatives/what-is-a-digi
> >>tal-envelope.htm
> >> > > [1] http://tools.ietf.org/html/rfc3126
> >> > > [2] http://tools.ietf.org/html/rfc3161
> >> > > [3]
> >> > >
> >> > >
> >> >
> >>
> >>
> http://www.emc.com/emc-plus/rsa-labs/standards-initiatives/pkcs-7-cryptog
> >>raphic-message-syntax-standar.htm
> >> > > [4] http://en.wikipedia.org/wiki/XML_Signature
> >> > > [5] http://en.wikipedia.org/wiki/PKCS_12
> >> > >
> >> > > /*******************************************
> >> > >  Joe Stein
> >> > >  Founder, Principal Consultant
> >> > >  Big Data Open Source Security LLC
> >> > >  http://www.stealth.ly
> >> > >  Twitter: @allthingshadoop <http://www.twitter.com/allthingshadoop>
> >> > > ********************************************/
> >> > >
> >> >
> >>
>
>

Re: [DISCUSS] Kafka Security Specific Features

Posted by Todd Palino <tp...@linkedin.com.INVALID>.
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" <ja...@gmail.com> wrote:

>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 encryption scheme? Is this actually
>needed?
>Isn't a on-the-wire encryption when combined with mutual authentication
>and
>permissions sufficient for most uses?
>
>On-disk encryption seems unnecessary because if an attacker can get root
>on
>the kafka boxes it can potentially modify Kafka to do anything he or she
>wants with data. So this seems to break any security model.
>
>I understand the problem of a large organization not really having a
>trusted network and wanting to secure data transfer and limit and audit
>data access. The uses for these other things I don't totally understand.
>
>Also it would be worth understanding the state of other messaging and
>storage systems (Hadoop, dbs, etc). What features do they support. I think
>there is a sense in which you don't have to run faster than the bear, but
>only faster then your friends. :-)
>
>-Jay
>
>
>On Wed, Jun 4, 2014 at 5:57 PM, Joe Stein <jo...@stealth.ly> wrote:
>
>> 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 <ja...@gmail.com> 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 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 most the LinkedIn folks are working on the consumer right now I
>> think
>> > this would be a great project for any other interested people to take
>>on.
>> > There are some challenges in doing these things distributed but it can
>> also
>> > be a lot of fun.
>> >
>> > I think a good first step would be to get a written plan we can all
>>agree
>> > on for how things should work. Then we can break things down into
>>chunks
>> > that can be done independently while still aiming at a good end state.
>> >
>> > I had tried to write up some notes that summarized at least the
>>thoughts
>> I
>> > had had on security:
>> > https://cwiki.apache.org/confluence/display/KAFKA/Security
>> >
>> > What do you think of that?
>> >
>> > One assumption I had (which may be incorrect) is that although we want
>> all
>> > the things in your list, the two most pressing would be authentication
>> and
>> > authorization, and that was all that write up covered. You have more
>> > experience in this domain, so I wonder how you would prioritize?
>> >
>> > Those notes are really sketchy, so I think the first goal I would have
>> > would be to get to a real spec we can all agree on and discuss. A lot
>>of
>> > the security stuff has a high human interaction element and needs to
>>work
>> > in pretty different domains and different companies so getting this
>>kind
>> of
>> > review is important.
>> >
>> > -Jay
>> >
>> >
>> > On Tue, Jun 3, 2014 at 12:57 PM, Joe Stein <jo...@stealth.ly>
>>wrote:
>> >
>> > > 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.  This comes from what folks are doing "in the wild" to
>> > > workaround and implement security with Kafka as it is today and also
>> > what I
>> > > have discovered from organizations about their blockers. It also
>>picks
>> up
>> > > from the wiki (which I should have time to update later in the week
>> based
>> > > on the below and feedback from the thread).
>> > >
>> > > 1) Transport Layer Security (i.e. SSL)
>> > >
>> > > This also includes client authentication in addition to in-transit
>> > security
>> > > layer.  This work has been picked up here
>> > > https://issues.apache.org/jira/browse/KAFKA-1477 and do appreciate
>>any
>> > > thoughts, comments, feedback, tomatoes, whatever for this patch.  It
>> is a
>> > > pickup from the fork of the work first done here
>> > > https://github.com/relango/kafka/tree/kafka_security.
>> > >
>> > > 2) Data encryption at rest.
>> > >
>> > > This is very important and something that can be facilitated within
>>the
>> > > wire protocol. It requires an additional map data structure for the
>> > > "encrypted [data encryption key]". With this map (either in your
>>object
>> > or
>> > > in the wire protocol) you can store the dynamically generated
>>symmetric
>> > key
>> > > (for each message) and then encrypt the data using that dynamically
>> > > generated key.  You then encrypt the encryption key using each
>>public
>> key
>> > > for whom is expected to be able to decrypt the encryption key to
>>then
>> > > decrypt the message.  For each public key encrypted symmetric key
>> (which
>> > is
>> > > now the "encrypted [data encryption key]" along with which public
>>key
>> it
>> > > was encrypted with for (so a map of [publicKey] =
>> > > encryptedDataEncryptionKey) as a chain.   Other patterns can be
>> > implemented
>> > > but this is a pretty standard digital enveloping [0] pattern with
>>only
>> 1
>> > > field added. Other patterns should be able to use that field to-do
>> their
>> > > implementation too.
>> > >
>> > > 3) Non-repudiation and long term non-repudiation.
>> > >
>> > > Non-repudiation is proving data hasn't changed.  This is often (if
>>not
>> > > always) done with x509 public certificates (chained to a certificate
>> > > authority).
>> > >
>> > > Long term non-repudiation is what happens when the certificates of
>>the
>> > > certificate authority are expired (or revoked) and everything ever
>> signed
>> > > (ever) with that certificate's public key then becomes "no longer
>> > provable
>> > > as ever being authentic".  That is where RFC3126 [1] and RFC3161 [2]
>> come
>> > > in (or worm drives [hardware], etc).
>> > >
>> > > For either (or both) of these it is an operation of the encryptor to
>> > > sign/hash the data (with or without third party trusted timestap of
>>the
>> > > signing event) and encrypt that with their own private key and
>> distribute
>> > > the results (before and after encrypting if required) along with
>>their
>> > > public key. This structure is a bit more complex but feasible, it
>>is a
>> > map
>> > > of digital signature formats and the chain of dig sig attestations.
>>  The
>> > > map's key being the method (i.e. CRC32, PKCS7 [3], XmlDigSig [4])
>>and
>> > then
>> > > a list of map where that key is "purpose" of signature (what your
>> > attesting
>> > > too).  As a sibling field to the list another field for "the
>>attester"
>> as
>> > > bytes (e.g. their PKCS12 [5] for the map of PKCS7 signatures).
>> > >
>> > > 4) Authorization
>> > >
>> > > We should have a policy of "404" for data, topics, partitions (etc)
>>if
>> > > authenticated connections do not have access.  In "secure mode" any
>>non
>> > > authenticated connections should get a "404" type message on
>> everything.
>> > > Knowing "something is there" is a security risk in many uses cases.
>> So
>> > if
>> > > you don't have access you don't even see it.  Baking "that" into
>>Kafka
>> > > along with some interface for entitlement (access management)
>>systems
>> > > (pretty standard) is all that I think needs to be done to the core
>> > project.
>> > >  I want to tackle item later in the year after summer after the
>>other
>> > three
>> > > are complete.
>> > >
>> > > I look forward to thoughts on this and anyone else interested in
>> working
>> > > with us on these items.
>> > >
>> > > [0]
>> > >
>> > >
>> >
>> 
>>http://www.emc.com/emc-plus/rsa-labs/standards-initiatives/what-is-a-digi
>>tal-envelope.htm
>> > > [1] http://tools.ietf.org/html/rfc3126
>> > > [2] http://tools.ietf.org/html/rfc3161
>> > > [3]
>> > >
>> > >
>> >
>> 
>>http://www.emc.com/emc-plus/rsa-labs/standards-initiatives/pkcs-7-cryptog
>>raphic-message-syntax-standar.htm
>> > > [4] http://en.wikipedia.org/wiki/XML_Signature
>> > > [5] http://en.wikipedia.org/wiki/PKCS_12
>> > >
>> > > /*******************************************
>> > >  Joe Stein
>> > >  Founder, Principal Consultant
>> > >  Big Data Open Source Security LLC
>> > >  http://www.stealth.ly
>> > >  Twitter: @allthingshadoop <http://www.twitter.com/allthingshadoop>
>> > > ********************************************/
>> > >
>> >
>>


Re: [DISCUSS] Kafka Security Specific Features

Posted by Robert Withers <ro...@gmail.com>.
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 <pr...@gmail.com> 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 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 better performance.
> 
> 
> On Fri, Jun 6, 2014 at 10:48 AM, Rob Withers <ro...@gmail.com>
> wrote:
> 
>> 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 key....certs won't work, unless you write
>> a CertAuthority to build multiple certs with the same keys.  Better seems
>> to not use certs and wrap the encryption specification with an ACL
>> capabilities for each group of access.
>> 
>> 
>> On Jun 6, 2014, at 11:43 AM, Rob Withers wrote:
>> 
>> 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/WRITE/DELETE/LIST/SCAN).  Erights.org has the quitenscential (??)
>>> object capabilities model and capnproto is impleemting this for C++.  I
>>> have a java implementation at http://github.com/pauwau/pauwau but the
>>> master is broken.  0.2 works, basically.  B asically a TLS connection with
>>> no certificate server, it is peer to peer.  It has some advanced features,
>>> but the lining of capabilities with authorization so that you can only
>>> invoke correct services is extended to the secure user.
>>> 
>>> Regarding non-repudiation, on disk, why not prepend a CRC?
>>> 
>>> Regarding on-disk encryption, multiple users/groups may need to access,
>>> with different capabilities.  Sounds like zookeeper needs to store a cert
>>> for each class of access so that a group member can access the decrypted
>>> data from disk.  Use cert-based async decryption.  The only isue is storing
>>> the private key in zookeeper.  Perhaps some hash magic could be used.
>>> 
>>> Thanks for kafka,
>>> Rob
>>> 
>>> On Jun 5, 2014, at 3:01 PM, Jay Kreps wrote:
>>> 
>>> 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 encryption scheme? Is this actually
>>>> needed?
>>>> Isn't a on-the-wire encryption when combined with mutual authentication
>>>> and
>>>> permissions sufficient for most uses?
>>>> 
>>>> On-disk encryption seems unnecessary because if an attacker can get root
>>>> on
>>>> the kafka boxes it can potentially modify Kafka to do anything he or she
>>>> wants with data. So this seems to break any security model.
>>>> 
>>>> I understand the problem of a large organization not really having a
>>>> trusted network and wanting to secure data transfer and limit and audit
>>>> data access. The uses for these other things I don't totally understand.
>>>> 
>>>> Also it would be worth understanding the state of other messaging and
>>>> storage systems (Hadoop, dbs, etc). What features do they support. I
>>>> think
>>>> there is a sense in which you don't have to run faster than the bear, but
>>>> only faster then your friends. :-)
>>>> 
>>>> -Jay
>>>> 
>>>> 
>>>> On Wed, Jun 4, 2014 at 5:57 PM, Joe Stein <jo...@stealth.ly> wrote:
>>>> 
>>>> 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 <ja...@gmail.com> 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 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 most the LinkedIn folks are working on the consumer right now I
>>>>> think
>>>>> 
>>>>>> this would be a great project for any other interested people to take
>>>>>> on.
>>>>>> There are some challenges in doing these things distributed but it can
>>>>> also
>>>>> 
>>>>>> be a lot of fun.
>>>>>> 
>>>>>> I think a good first step would be to get a written plan we can all
>>>>>> agree
>>>>>> on for how things should work. Then we can break things down into
>>>>>> chunks
>>>>>> that can be done independently while still aiming at a good end state.
>>>>>> 
>>>>>> I had tried to write up some notes that summarized at least the
>>>>>> thoughts
>>>>> I
>>>>> 
>>>>>> had had on security:
>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/Security
>>>>>> 
>>>>>> What do you think of that?
>>>>>> 
>>>>>> One assumption I had (which may be incorrect) is that although we want
>>>>> all
>>>>> 
>>>>>> the things in your list, the two most pressing would be authentication
>>>>> and
>>>>> 
>>>>>> authorization, and that was all that write up covered. You have more
>>>>>> experience in this domain, so I wonder how you would prioritize?
>>>>>> 
>>>>>> Those notes are really sketchy, so I think the first goal I would have
>>>>>> would be to get to a real spec we can all agree on and discuss. A lot
>>>>>> of
>>>>>> the security stuff has a high human interaction element and needs to
>>>>>> work
>>>>>> in pretty different domains and different companies so getting this
>>>>>> kind
>>>>> of
>>>>> 
>>>>>> review is important.
>>>>>> 
>>>>>> -Jay
>>>>>> 
>>>>>> 
>>>>>> On Tue, Jun 3, 2014 at 12:57 PM, Joe Stein <jo...@stealth.ly>
>>>>>> wrote:
>>>>>> 
>>>>>> 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.  This comes from what folks are doing "in the wild" to
>>>>>>> workaround and implement security with Kafka as it is today and also
>>>>>> what I
>>>>>> 
>>>>>>> have discovered from organizations about their blockers. It also picks
>>>>>> up
>>>>> 
>>>>>> from the wiki (which I should have time to update later in the week
>>>>>> based
>>>>> 
>>>>>> on the below and feedback from the thread).
>>>>>>> 
>>>>>>> 1) Transport Layer Security (i.e. SSL)
>>>>>>> 
>>>>>>> This also includes client authentication in addition to in-transit
>>>>>> security
>>>>>> 
>>>>>>> layer.  This work has been picked up here
>>>>>>> https://issues.apache.org/jira/browse/KAFKA-1477 and do appreciate
>>>>>>> any
>>>>>>> thoughts, comments, feedback, tomatoes, whatever for this patch.  It
>>>>>> is a
>>>>> 
>>>>>> pickup from the fork of the work first done here
>>>>>>> https://github.com/relango/kafka/tree/kafka_security.
>>>>>>> 
>>>>>>> 2) Data encryption at rest.
>>>>>>> 
>>>>>>> This is very important and something that can be facilitated within
>>>>>>> the
>>>>>>> wire protocol. It requires an additional map data structure for the
>>>>>>> "encrypted [data encryption key]". With this map (either in your
>>>>>>> object
>>>>>> or
>>>>>> 
>>>>>>> in the wire protocol) you can store the dynamically generated
>>>>>>> symmetric
>>>>>> key
>>>>>> 
>>>>>>> (for each message) and then encrypt the data using that dynamically
>>>>>>> generated key.  You then encrypt the encryption key using each public
>>>>>> key
>>>>> 
>>>>>> for whom is expected to be able to decrypt the encryption key to then
>>>>>>> decrypt the message.  For each public key encrypted symmetric key
>>>>>> (which
>>>>> 
>>>>>> is
>>>>>> 
>>>>>>> now the "encrypted [data encryption key]" along with which public key
>>>>>> it
>>>>> 
>>>>>> was encrypted with for (so a map of [publicKey] =
>>>>>>> encryptedDataEncryptionKey) as a chain.   Other patterns can be
>>>>>> implemented
>>>>>> 
>>>>>>> but this is a pretty standard digital enveloping [0] pattern with only
>>>>>> 1
>>>>> 
>>>>>> field added. Other patterns should be able to use that field to-do
>>>>>> their
>>>>> 
>>>>>> implementation too.
>>>>>>> 
>>>>>>> 3) Non-repudiation and long term non-repudiation.
>>>>>>> 
>>>>>>> Non-repudiation is proving data hasn't changed.  This is often (if not
>>>>>>> always) done with x509 public certificates (chained to a certificate
>>>>>>> authority).
>>>>>>> 
>>>>>>> Long term non-repudiation is what happens when the certificates of the
>>>>>>> certificate authority are expired (or revoked) and everything ever
>>>>>> signed
>>>>> 
>>>>>> (ever) with that certificate's public key then becomes "no longer
>>>>>> provable
>>>>>> 
>>>>>>> as ever being authentic".  That is where RFC3126 [1] and RFC3161 [2]
>>>>>> come
>>>>> 
>>>>>> in (or worm drives [hardware], etc).
>>>>>>> 
>>>>>>> For either (or both) of these it is an operation of the encryptor to
>>>>>>> sign/hash the data (with or without third party trusted timestap of
>>>>>>> the
>>>>>>> signing event) and encrypt that with their own private key and
>>>>>> distribute
>>>>> 
>>>>>> the results (before and after encrypting if required) along with their
>>>>>>> public key. This structure is a bit more complex but feasible, it is a
>>>>>> map
>>>>>> 
>>>>>>> of digital signature formats and the chain of dig sig attestations.
>>>>>> The
>>>>> 
>>>>>> map's key being the method (i.e. CRC32, PKCS7 [3], XmlDigSig [4]) and
>>>>>> then
>>>>>> 
>>>>>>> a list of map where that key is "purpose" of signature (what your
>>>>>> attesting
>>>>>> 
>>>>>>> too).  As a sibling field to the list another field for "the attester"
>>>>>> as
>>>>> 
>>>>>> bytes (e.g. their PKCS12 [5] for the map of PKCS7 signatures).
>>>>>>> 
>>>>>>> 4) Authorization
>>>>>>> 
>>>>>>> We should have a policy of "404" for data, topics, partitions (etc) if
>>>>>>> authenticated connections do not have access.  In "secure mode" any
>>>>>>> non
>>>>>>> authenticated connections should get a "404" type message on
>>>>>> everything.
>>>>> 
>>>>>> Knowing "something is there" is a security risk in many uses cases.  So
>>>>>> if
>>>>>> 
>>>>>>> you don't have access you don't even see it.  Baking "that" into Kafka
>>>>>>> along with some interface for entitlement (access management) systems
>>>>>>> (pretty standard) is all that I think needs to be done to the core
>>>>>> project.
>>>>>> 
>>>>>>> I want to tackle item later in the year after summer after the other
>>>>>> three
>>>>>> 
>>>>>>> are complete.
>>>>>>> 
>>>>>>> I look forward to thoughts on this and anyone else interested in
>>>>>> working
>>>>> 
>>>>>> with us on these items.
>>>>>>> 
>>>>>>> [0]
>>>>>> http://www.emc.com/emc-plus/rsa-labs/standards-
>>>>> initiatives/what-is-a-digital-envelope.htm
>>>>> 
>>>>>> [1] http://tools.ietf.org/html/rfc3126
>>>>>>> [2] http://tools.ietf.org/html/rfc3161
>>>>>>> [3]
>>>>>> http://www.emc.com/emc-plus/rsa-labs/standards-initiatives/pkcs-7-
>>>>> cryptographic-message-syntax-standar.htm
>>>>> 
>>>>>> [4] http://en.wikipedia.org/wiki/XML_Signature
>>>>>>> [5] http://en.wikipedia.org/wiki/PKCS_12
>>>>>>> 
>>>>>>> /*******************************************
>>>>>>> Joe Stein
>>>>>>> Founder, Principal Consultant
>>>>>>> Big Data Open Source Security LLC
>>>>>>> http://www.stealth.ly
>>>>>>> Twitter: @allthingshadoop <http://www.twitter.com/allthingshadoop>
>>>>>>> ********************************************/
>> 

Re: [DISCUSS] Kafka Security Specific Features

Posted by Robert Withers <ro...@gmail.com>.
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 the token, then do de/encryption, each our own way.

Thanks,
Rob

> On Jun 10, 2014, at 4:38 PM, Todd Palino <tp...@linkedin.com.INVALID> wrote:
> 
> 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 message metadata that can be preserved across clusters, which
> is separate from the message itself (so it’s not an overlay of a schema on
> top of the message, but rather a separate structure entirely that is
> stored with the message).
> 
> -Todd
> 
>> On 6/10/14, 3:26 PM, "Robert Withers" <ro...@gmail.com> wrote:
>> 
>> 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 <tp...@linkedin.com.INVALID>
>>> wrote:
>>> 
>>> 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 pair
>>> generated
>>> for it already, so the producer could start encrypting for that consumer
>>> before it exists. This would probably work fine for lower retention
>>> periods (a week or two), but could be a little more difficult to manage
>>> if
>>> you are keeping data in Kafka longer than that. My gut reaction is that
>>> it’s better to handle it that way and keep the key pair and session key
>>> handling simple. The more we can do that, the more we can leave key
>>> management as a separate component that can be swapped out so the user
>>> can
>>> decide how it should be done.
>>> 
>>> -Todd
>>> 
>>> 
>>>> On 6/9/14, 8:16 AM, "Robert Withers" <ro...@gmail.com>
>>>> wrote:
>>>> 
>>>> 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 before the complete recipient list is
>>>> known.
>>>> 
>>>> This seems to mean that the private key (cert or OTP) must be stored
>>>> and
>>>> interacted with.  My feeling is that key metadata are in a system
>>>> encrypted Hbase store (session key store), for low latency reads,
>>>> rather
>>>> than a topic requiring scanning.  Store the private keys and then give
>>>> client access (producers/consumers) with the hash of the OTP.  A new
>>>> consumer comes along, create a new cert encoding the OTP hash.
>>>> 
>>>> On write, use the producer cert to send a topic hash with the msg which
>>>> would allow the broker to reuse or generate an OTP, stored in the
>>>> session
>>>> key store.
>>>> 
>>>> On read (consumer), if we have a previously run reader, use the
>>>> encrypted
>>>> hash.  If new, create consumer cert and encrypt the hash for that
>>>> session.
>>>> 
>>>> The reader/writer will pass a cert encrypted session hash.  The trick
>>>> seems to be converting hash to PK to encrypt/decrypt.  Given Kafka
>>>> resource distribution, we need system encryption for metadata and
>>>> cert-based key exchange.  This seems to mean triple encryption:
>>>> 1) client to/from broker
>>>> 2) system key/hash  mgmt/translation
>>>> 3) at-rest encryption
>>>> 
>>>> Thanks,
>>>> Rob
>>>> 
>>>>> On Jun 9, 2014, at 7:57 AM, Todd Palino <tp...@linkedin.com.INVALID>
>>>>> wrote:
>>>>> 
>>>>> 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 the encrypted copies of the session key
>>>>> are
>>>>> then included with the encrypted message. This way, you avoid having
>>>>> to
>>>>> encrypt the message multiple times (this assumes, of course, that the
>>>>> message itself is larger than the key).
>>>>> 
>>>>> In our case, we have some options available to us. We could do that,
>>>>> and
>>>>> put all the encrypted keys in the message metadata. Or we could treat
>>>>> it
>>>>> more like a session and have the encrypted session keys in a special
>>>>> topic
>>>>> (e.g. __session_keys), much like offsets are now. When the producer
>>>>> starts
>>>>> up, they create a session key and encrypt it for each consumer with
>>>>> the
>>>>> current consumer key. The producer publishes the bundle of encrypted
>>>>> keys
>>>>> into __session_keys as a single message. The producer then publishes
>>>>> messages to the normal topic encrypted with the session key. The
>>>>> metadata
>>>>> for each of those messages would contain something the offset into
>>>>> __session_keys to identify the bundle. This has the added benefit of
>>>>> not
>>>>> increasing the per-message data size too much.
>>>>> 
>>>>> Whenever a consumer key is invalidated, or however often the session
>>>>> key
>>>>> should be rotated, the producer would publish a new bundle. This
>>>>> maintains
>>>>> a history of session keys that can be used to decrypt any messages, so
>>>>> the
>>>>> retention on __session_keys must be at least as long as any topic
>>>>> which
>>>>> may potentially contain encrypted data. Past that point, it’s up to
>>>>> the
>>>>> consumer what they want to do with the data. A consumer like Hadoop
>>>>> might
>>>>> re-encrypt it for local storage, or store it in plaintext (depending
>>>>> on
>>>>> the security and requirements of that system).
>>>>> 
>>>>> -Todd
>>>>> 
>>>>>> On 6/8/14, 2:33 PM, "Rob Withers" <ro...@gmail.com> wrote:
>>>>>> 
>>>>>> 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 deeper for me, please?
>>>>>> The model I envision is internal OTP, with access to decryption
>>>>>> accessed by cert.  A double layer of security, with the internal at-
>>>>>> rest encryption being an unchanging OTP with ACL access to it as the
>>>>>> upper layer.  Are you saying it is possible to re-encrypt with new
>>>>>> keys or that there is a chain of keys over time?
>>>>>> 
>>>>>> Thanks,
>>>>>> Rob
>>>>>> 
>>>>>>> On Jun 8, 2014, at 3:06 PM, Todd Palino wrote:
>>>>>>> 
>>>>>>> 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, and other PII standards). Yes, it does add external overhead
>>>>>>> that
>>>>>>> must be managed, but it’s just the nature of the beast. We can’t
>>>>>>> solve all
>>>>>>> of the external infrastructure needed for this, but we can make it
>>>>>>> easier
>>>>>>> to use for consumers and producers by adding metadata.
>>>>>>> 
>>>>>>> There’s no need for unchanging encryption, and that’s specifically
>>>>>>> why I
>>>>>>> want to see a message envelope that will help consumers determine
>>>>>>> the
>>>>>>> encryption uses for a particular message.  You can definitely still
>>>>>>> expire
>>>>>>> keys, you just have to keep the expired keys around as long as the
>>>>>>> encrypted data stays around, and your endpoints need to know when
>>>>>>> they are
>>>>>>> decrypting data with an expired key (you might want to throw up a
>>>>>>> warning,
>>>>>>> or do something else to let the users know that it’s happening). And
>>>>>>> as
>>>>>>> someone else mentioned, there are solutions for encrypting data for
>>>>>>> multiple consumers. You can encrypt the data with an OTP, and then
>>>>>>> multiply encrypt the OTP once for each consumer and store those
>>>>>>> encrypted
>>>>>>> strings in the envelope.
>>>>>>> 
>>>>>>> -Todd
>>>>>>> 
>>>>>>>> On 6/7/14, 12:25 PM, "Rob Withers" <ro...@gmail.com>
>>>>>>>> wrote:
>>>>>>>> 
>>>>>>>> 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 encryption for the data and the clients (producers/
>>>>>>>> consumers) will need to be able to decode frozen data.  If certs
>>>>>>>> are
>>>>>>>> used they cannot expire.  Also, different clients would need to use
>>>>>>>> the same cert.
>>>>>>>> 
>>>>>>>> So, you statement that it should ABSOLUTELY not include internal
>>>>>>>> encryption rings seems misplaced.  There are some customers of
>>>>>>>> kafka
>>>>>>>> that would opt to encrypt the on-disk data and key management is a
>>>>>>>> significant issue.  This is best handled internally, with key
>>>>>>>> management stored in either ZK or in a topic.  Truly, perhaps
>>>>>>>> annealing Hadoop/HBASE as a metadata store seems applicable.
>>>>>>>> 
>>>>>>>> Thanks, another 2 cents,
>>>>>>>> Rob
>>>>>>>> 
>>>>>>>>> On Jun 6, 2014, at 12:15 PM, Todd Palino wrote:
>>>>>>>>> 
>>>>>>>>> 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 server then has access to all the keys. You could say
>>>>>>>>> that
>>>>>>>>> this eliminates the need for TLS, but TLS is pretty low-hanging
>>>>>>>>> fruit, and
>>>>>>>>> there¹s definitely a need for encryption of the traffic across the
>>>>>>>>> network
>>>>>>>>> even if you don¹t need at-rest encryption as well.
>>>>>>>>> 
>>>>>>>>> And as you mentioned, something needs to be done about key
>>>>>>>>> management.
>>>>>>>>> Storing information with the message about which key(s) was used
>>>>>>>>> is
>>>>>>>>> a good
>>>>>>>>> idea, because it allows you to know when a producer has switched
>>>>>>>>> keys.
>>>>>>>>> There are definitely some alternative solutions to that as well.
>>>>>>>>> But
>>>>>>>>> storing the keys in the broker, Zookeeper, or other systems like
>>>>>>>>> that are
>>>>>>>>> not. There needs to be a system used where the keys are only
>>>>>>>>> available to
>>>>>>>>> the producers and consumers that need them, and they only get
>>>>>>>>> access
>>>>>>>>> to
>>>>>>>>> the appropriate part of the key pair.  Even as the guy running
>>>>>>>>> Kafka
>>>>>>>>> and
>>>>>>>>> Zookeeper, I should not have access to the keys being used, and if
>>>>>>>>> data is
>>>>>>>>> encrypted I should not be able to see the cleartext.
>>>>>>>>> 
>>>>>>>>> And even if we decide not to put anything about at-rest encryption
>>>>>>>>> in the
>>>>>>>>> consumer/producer clients directly, and leave it for an exercise
>>>>>>>>> above
>>>>>>>>> that level (you have to pass the ciphertext as the message to the
>>>>>>>>> client),
>>>>>>>>> I still think there is a good case for implementing a message
>>>>>>>>> envelope
>>>>>>>>> that can store the information about which key was used, and other
>>>>>>>>> pertinent metadata, and have the ability for special applications
>>>>>>>>> like
>>>>>>>>> mirror maker to be able to preserve it across clusters. This still
>>>>>>>>> helps
>>>>>>>>> to enable the use of encryption and other features (like auditing)
>>>>>>>>> even if
>>>>>>>>> we decide it¹s too large a scope to fully implement.
>>>>>>>>> 
>>>>>>>>> -Todd
>>>>>>>>> 
>>>>>>>>> On 6/6/14, 10:51 AM, "Pradeep Gollakota" <pr...@gmail.com>
>>>>>>>>> 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
>>>>>>>>>> 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 better performance.
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> On Fri, Jun 6, 2014 at 10:48 AM, Rob Withers
>>>>>>>>>> <robert.w.withers@gmail.com
>>>>>>>>>> wrote:
>>>>>>>>>> 
>>>>>>>>>>> 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 key....certs won't work, unless
>>>>>>>>>>> you
>>>>>>>>>>> write
>>>>>>>>>>> a CertAuthority to build multiple certs with the same keys.
>>>>>>>>>>> Better
>>>>>>>>>>> seems
>>>>>>>>>>> to not use certs and wrap the encryption specification with an
>>>>>>>>>>> ACL
>>>>>>>>>>> capabilities for each group of access.
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> On Jun 6, 2014, at 11:43 AM, Rob Withers wrote:
>>>>>>>>>>> 
>>>>>>>>>>> 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/WRITE/DELETE/LIST/SCAN).  Erights.org has the
>>>>>>>>>>>> quitenscential (??)
>>>>>>>>>>>> object capabilities model and capnproto is impleemting this for
>>>>>>>>>>>> C+
>>>>>>>>>>>> +.  I
>>>>>>>>>>>> have a java implementation at http://github.com/pauwau/pauwau
>>>>>>>>>>>> but
>>>>>>>>>>>> the
>>>>>>>>>>>> master is broken.  0.2 works, basically.  B asically a TLS
>>>>>>>>>>>> connection
>>>>>>>>>>>> with
>>>>>>>>>>>> no certificate server, it is peer to peer.  It has some
>>>>>>>>>>>> advanced
>>>>>>>>>>>> features,
>>>>>>>>>>>> but the lining of capabilities with authorization so that you
>>>>>>>>>>>> can
>>>>>>>>>>>> only
>>>>>>>>>>>> invoke correct services is extended to the secure user.
>>>>>>>>>>>> 
>>>>>>>>>>>> Regarding non-repudiation, on disk, why not prepend a CRC?
>>>>>>>>>>>> 
>>>>>>>>>>>> Regarding on-disk encryption, multiple users/groups may need to
>>>>>>>>>>>> access,
>>>>>>>>>>>> with different capabilities.  Sounds like zookeeper needs to
>>>>>>>>>>>> store a
>>>>>>>>>>>> cert
>>>>>>>>>>>> for each class of access so that a group member can access the
>>>>>>>>>>>> decrypted
>>>>>>>>>>>> data from disk.  Use cert-based async decryption.  The only
>>>>>>>>>>>> isue is
>>>>>>>>>>>> storing
>>>>>>>>>>>> the private key in zookeeper.  Perhaps some hash magic could be
>>>>>>>>>>>> used.
>>>>>>>>>>>> 
>>>>>>>>>>>> Thanks for kafka,
>>>>>>>>>>>> Rob
>>>>>>>>>>>> 
>>>>>>>>>>>> On Jun 5, 2014, at 3:01 PM, Jay Kreps wrote:
>>>>>>>>>>>> 
>>>>>>>>>>>> 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 encryption scheme? Is this
>>>>>>>>>>>>> actually
>>>>>>>>>>>>> needed?
>>>>>>>>>>>>> Isn't a on-the-wire encryption when combined with mutual
>>>>>>>>>>>>> authentication
>>>>>>>>>>>>> and
>>>>>>>>>>>>> permissions sufficient for most uses?
>>>>>>>>>>>>> 
>>>>>>>>>>>>> On-disk encryption seems unnecessary because if an attacker
>>>>>>>>>>>>> can
>>>>>>>>>>>>> get
>>>>>>>>>>>>> root
>>>>>>>>>>>>> on
>>>>>>>>>>>>> the kafka boxes it can potentially modify Kafka to do anything
>>>>>>>>>>>>> he or
>>>>>>>>>>>>> she
>>>>>>>>>>>>> wants with data. So this seems to break any security model.
>>>>>>>>>>>>> 
>>>>>>>>>>>>> I understand the problem of a large organization not really
>>>>>>>>>>>>> having a
>>>>>>>>>>>>> trusted network and wanting to secure data transfer and limit
>>>>>>>>>>>>> and
>>>>>>>>>>>>> audit
>>>>>>>>>>>>> data access. The uses for these other things I don't totally
>>>>>>>>>>>>> understand.
>>>>>>>>>>>>> 
>>>>>>>>>>>>> Also it would be worth understanding the state of other
>>>>>>>>>>>>> messaging and
>>>>>>>>>>>>> storage systems (Hadoop, dbs, etc). What features do they
>>>>>>>>>>>>> support. I
>>>>>>>>>>>>> think
>>>>>>>>>>>>> there is a sense in which you don't have to run faster than
>>>>>>>>>>>>> the
>>>>>>>>>>>>> bear,
>>>>>>>>>>>>> but
>>>>>>>>>>>>> only faster then your friends. :-)
>>>>>>>>>>>>> 
>>>>>>>>>>>>> -Jay
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> On Wed, Jun 4, 2014 at 5:57 PM, Joe Stein
>>>>>>>>>>>>> <jo...@stealth.ly>
>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 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
>>>>>>>>>>>>>> <ja...@gmail.com>
>>>>>>>>>>>>>> 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
>>>>>>>>>>>>>>> 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 most the LinkedIn folks are working on the consumer
>>>>>>>>>>>>>>> right now
>>>>>>>>>>>>>>> I
>>>>>>>>>>>>>> think
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> this would be a great project for any other interested
>>>>>>>>>>>>>>> people to
>>>>>>>>>>>>>>> take
>>>>>>>>>>>>>>> on.
>>>>>>>>>>>>>>> There are some challenges in doing these things distributed
>>>>>>>>>>>>>>> but it
>>>>>>>>>>>>>>> can
>>>>>>>>>>>>>> also
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> be a lot of fun.
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> I think a good first step would be to get a written plan we
>>>>>>>>>>>>>>> can all
>>>>>>>>>>>>>>> agree
>>>>>>>>>>>>>>> on for how things should work. Then we can break things down
>>>>>>>>>>>>>>> into
>>>>>>>>>>>>>>> chunks
>>>>>>>>>>>>>>> that can be done independently while still aiming at a good
>>>>>>>>>>>>>>> end
>>>>>>>>>>>>>>> state.
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> I had tried to write up some notes that summarized at least
>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>> thoughts
>>>>>>>>>>>>>> I
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> had had on security:
>>>>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/Security
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> What do you think of that?
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> One assumption I had (which may be incorrect) is that
>>>>>>>>>>>>>>> although
>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>> want
>>>>>>>>>>>>>> all
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> the things in your list, the two most pressing would be
>>>>>>>>>>>>>>> authentication
>>>>>>>>>>>>>> and
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> authorization, and that was all that write up covered. You
>>>>>>>>>>>>>>> have more
>>>>>>>>>>>>>>> experience in this domain, so I wonder how you would
>>>>>>>>>>>>>>> prioritize?
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> Those notes are really sketchy, so I think the first goal I
>>>>>>>>>>>>>>> would
>>>>>>>>>>>>>>> have
>>>>>>>>>>>>>>> would be to get to a real spec we can all agree on and
>>>>>>>>>>>>>>> discuss. A
>>>>>>>>>>>>>>> lot
>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>> the security stuff has a high human interaction element and
>>>>>>>>>>>>>>> needs to
>>>>>>>>>>>>>>> work
>>>>>>>>>>>>>>> in pretty different domains and different companies so
>>>>>>>>>>>>>>> getting
>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>> kind
>>>>>>>>>>>>>> of
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> review is important.
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> -Jay
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> On Tue, Jun 3, 2014 at 12:57 PM, Joe Stein
>>>>>>>>>>>>>>> <jo...@stealth.ly>
>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 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.  This comes from what folks are doing "in the
>>>>>>>>>>>>>>>> wild"
>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>> workaround and implement security with Kafka as it is today
>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>> also
>>>>>>>>>>>>>>> what I
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> have discovered from organizations about their blockers. It
>>>>>>>>>>>>>>>> also
>>>>>>>>>>>>>>>> picks
>>>>>>>>>>>>>>> up
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> from the wiki (which I should have time to update later in
>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>> week
>>>>>>>>>>>>>>> based
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> on the below and feedback from the thread).
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 1) Transport Layer Security (i.e. SSL)
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> This also includes client authentication in addition to in-
>>>>>>>>>>>>>>>> transit
>>>>>>>>>>>>>>> security
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> layer.  This work has been picked up here
>>>>>>>>>>>>>>>> https://issues.apache.org/jira/browse/KAFKA-1477 and do
>>>>>>>>>>>>>>>> appreciate
>>>>>>>>>>>>>>>> any
>>>>>>>>>>>>>>>> thoughts, comments, feedback, tomatoes, whatever for this
>>>>>>>>>>>>>>>> patch.
>>>>>>>>>>>>>>>> It
>>>>>>>>>>>>>>> is a
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> pickup from the fork of the work first done here
>>>>>>>>>>>>>>>> https://github.com/relango/kafka/tree/kafka_security.
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 2) Data encryption at rest.
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> This is very important and something that can be
>>>>>>>>>>>>>>>> facilitated
>>>>>>>>>>>>>>>> within
>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>> wire protocol. It requires an additional map data structure
>>>>>>>>>>>>>>>> for the
>>>>>>>>>>>>>>>> "encrypted [data encryption key]". With this map (either in
>>>>>>>>>>>>>>>> your
>>>>>>>>>>>>>>>> object
>>>>>>>>>>>>>>> or
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> in the wire protocol) you can store the dynamically
>>>>>>>>>>>>>>>> generated
>>>>>>>>>>>>>>>> symmetric
>>>>>>>>>>>>>>> key
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> (for each message) and then encrypt the data using that
>>>>>>>>>>>>>>>> dynamically
>>>>>>>>>>>>>>>> generated key.  You then encrypt the encryption key using
>>>>>>>>>>>>>>>> each
>>>>>>>>>>>>>>>> public
>>>>>>>>>>>>>>> key
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> for whom is expected to be able to decrypt the encryption
>>>>>>>>>>>>>>> key to
>>>>>>>>>>>>>>> then
>>>>>>>>>>>>>>>> decrypt the message.  For each public key encrypted
>>>>>>>>>>>>>>>> symmetric
>>>>>>>>>>>>>>>> key
>>>>>>>>>>>>>>> (which
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> now the "encrypted [data encryption key]" along with which
>>>>>>>>>>>>>>>> public
>>>>>>>>>>>>>>>> key
>>>>>>>>>>>>>>> it
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> was encrypted with for (so a map of [publicKey] =
>>>>>>>>>>>>>>>> encryptedDataEncryptionKey) as a chain.   Other patterns
>>>>>>>>>>>>>>>> can be
>>>>>>>>>>>>>>> implemented
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> but this is a pretty standard digital enveloping [0]
>>>>>>>>>>>>>>>> pattern
>>>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>> only
>>>>>>>>>>>>>>> 1
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> field added. Other patterns should be able to use that field
>>>>>>>>>>>>>>> to-do
>>>>>>>>>>>>>>> their
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> implementation too.
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 3) Non-repudiation and long term non-repudiation.
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> Non-repudiation is proving data hasn't changed.  This is
>>>>>>>>>>>>>>>> often (if
>>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>> always) done with x509 public certificates (chained to a
>>>>>>>>>>>>>>>> certificate
>>>>>>>>>>>>>>>> authority).
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> Long term non-repudiation is what happens when the
>>>>>>>>>>>>>>>> certificates of
>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>> certificate authority are expired (or revoked) and
>>>>>>>>>>>>>>>> everything
>>>>>>>>>>>>>>>> ever
>>>>>>>>>>>>>>> signed
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> (ever) with that certificate's public key then becomes "no
>>>>>>>>>>>>>>> longer
>>>>>>>>>>>>>>> provable
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> as ever being authentic".  That is where RFC3126 [1] and
>>>>>>>>>>>>>>>> RFC3161
>>>>>>>>>>>>>>>> [2]
>>>>>>>>>>>>>>> come
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> in (or worm drives [hardware], etc).
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> For either (or both) of these it is an operation of the
>>>>>>>>>>>>>>>> encryptor
>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>> sign/hash the data (with or without third party trusted
>>>>>>>>>>>>>>>> timestap of
>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>> signing event) and encrypt that with their own private key
>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>> distribute
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> the results (before and after encrypting if required) along
>>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>> their
>>>>>>>>>>>>>>>> public key. This structure is a bit more complex but
>>>>>>>>>>>>>>>> feasible, it
>>>>>>>>>>>>>>>> is a
>>>>>>>>>>>>>>> map
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> of digital signature formats and the chain of dig sig
>>>>>>>>>>>>>>>> attestations.
>>>>>>>>>>>>>>> The
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> map's key being the method (i.e. CRC32, PKCS7 [3], XmlDigSig
>>>>>>>>>>>>>>> [4])
>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>> then
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> a list of map where that key is "purpose" of signature
>>>>>>>>>>>>>>>> (what
>>>>>>>>>>>>>>>> your
>>>>>>>>>>>>>>> attesting
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> too).  As a sibling field to the list another field for
>>>>>>>>>>>>>>>> "the
>>>>>>>>>>>>>>>> attester"
>>>>>>>>>>>>>>> as
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> bytes (e.g. their PKCS12 [5] for the map of PKCS7
>>>>>>>>>>>>>>> signatures).
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 4) Authorization
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> We should have a policy of "404" for data, topics,
>>>>>>>>>>>>>>>> partitions
>>>>>>>>>>>>>>>> (etc) if
>>>>>>>>>>>>>>>> authenticated connections do not have access.  In "secure
>>>>>>>>>>>>>>>> mode" any
>>>>>>>>>>>>>>>> non
>>>>>>>>>>>>>>>> authenticated connections should get a "404" type message
>>>>>>>>>>>>>>>> on
>>>>>>>>>>>>>>> everything.
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> Knowing "something is there" is a security risk in many uses
>>>>>>>>>>>>>>> cases.
>>>>>>>>>>>>>>> So
>>>>>>>>>>>>>>> if
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> you don't have access you don't even see it.  Baking "that"
>>>>>>>>>>>>>>>> into
>>>>>>>>>>>>>>>> Kafka
>>>>>>>>>>>>>>>> along with some interface for entitlement (access
>>>>>>>>>>>>>>>> management)
>>>>>>>>>>>>>>>> systems
>>>>>>>>>>>>>>>> (pretty standard) is all that I think needs to be done to
>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>> core
>>>>>>>>>>>>>>> project.
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> I want to tackle item later in the year after summer after
>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>> other
>>>>>>>>>>>>>>> three
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> are complete.
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> I look forward to thoughts on this and anyone else
>>>>>>>>>>>>>>>> interested
>>>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>> working
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> with us on these items.
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> [0]
>>>>>>>>>>>>>>> http://www.emc.com/emc-plus/rsa-labs/standards-
>>>>>>>>>>>>>> initiatives/what-is-a-digital-envelope.htm
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> [1] http://tools.ietf.org/html/rfc3126
>>>>>>>>>>>>>>>> [2] http://tools.ietf.org/html/rfc3161
>>>>>>>>>>>>>>>> [3]
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> http://www.emc.com/emc-plus/rsa-labs/standards-initiatives/pk
>>>>>>>>>>>>>>> cs
>>>>>>>>>>>>>>> -7
>>>>>>>>>>>>>>> -
>>>>>>>>>>>>>> cryptographic-message-syntax-standar.htm
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> [4] http://en.wikipedia.org/wiki/XML_Signature
>>>>>>>>>>>>>>>> [5] http://en.wikipedia.org/wiki/PKCS_12
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> /*******************************************
>>>>>>>>>>>>>>>> Joe Stein
>>>>>>>>>>>>>>>> Founder, Principal Consultant
>>>>>>>>>>>>>>>> Big Data Open Source Security LLC
>>>>>>>>>>>>>>>> http://www.stealth.ly
>>>>>>>>>>>>>>>> Twitter: @allthingshadoop
>>>>>>>>>>>>>>>> <http://www.twitter.com/allthingshadoop
>>>>>>>>>>>>>>>> ********************************************/
> 

Re: [DISCUSS] Kafka Security Specific Features

Posted by Todd Palino <tp...@linkedin.com.INVALID>.
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 message metadata that can be preserved across clusters, which
is separate from the message itself (so it’s not an overlay of a schema on
top of the message, but rather a separate structure entirely that is
stored with the message).

-Todd

On 6/10/14, 3:26 PM, "Robert Withers" <ro...@gmail.com> wrote:

>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 <tp...@linkedin.com.INVALID>
>>wrote:
>> 
>> 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 pair
>>generated
>> for it already, so the producer could start encrypting for that consumer
>> before it exists. This would probably work fine for lower retention
>> periods (a week or two), but could be a little more difficult to manage
>>if
>> you are keeping data in Kafka longer than that. My gut reaction is that
>> it’s better to handle it that way and keep the key pair and session key
>> handling simple. The more we can do that, the more we can leave key
>> management as a separate component that can be swapped out so the user
>>can
>> decide how it should be done.
>> 
>> -Todd
>> 
>> 
>>> On 6/9/14, 8:16 AM, "Robert Withers" <ro...@gmail.com>
>>>wrote:
>>> 
>>> 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 before the complete recipient list is
>>> known.
>>> 
>>> This seems to mean that the private key (cert or OTP) must be stored
>>>and
>>> interacted with.  My feeling is that key metadata are in a system
>>> encrypted Hbase store (session key store), for low latency reads,
>>>rather
>>> than a topic requiring scanning.  Store the private keys and then give
>>> client access (producers/consumers) with the hash of the OTP.  A new
>>> consumer comes along, create a new cert encoding the OTP hash.
>>> 
>>> On write, use the producer cert to send a topic hash with the msg which
>>> would allow the broker to reuse or generate an OTP, stored in the
>>>session
>>> key store.
>>> 
>>> On read (consumer), if we have a previously run reader, use the
>>>encrypted
>>> hash.  If new, create consumer cert and encrypt the hash for that
>>>session.
>>> 
>>> The reader/writer will pass a cert encrypted session hash.  The trick
>>> seems to be converting hash to PK to encrypt/decrypt.  Given Kafka
>>> resource distribution, we need system encryption for metadata and
>>> cert-based key exchange.  This seems to mean triple encryption:
>>> 1) client to/from broker
>>> 2) system key/hash  mgmt/translation
>>> 3) at-rest encryption
>>> 
>>> Thanks,
>>> Rob
>>> 
>>>> On Jun 9, 2014, at 7:57 AM, Todd Palino <tp...@linkedin.com.INVALID>
>>>> wrote:
>>>> 
>>>> 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 the encrypted copies of the session key
>>>> are
>>>> then included with the encrypted message. This way, you avoid having
>>>>to
>>>> encrypt the message multiple times (this assumes, of course, that the
>>>> message itself is larger than the key).
>>>> 
>>>> In our case, we have some options available to us. We could do that,
>>>>and
>>>> put all the encrypted keys in the message metadata. Or we could treat
>>>>it
>>>> more like a session and have the encrypted session keys in a special
>>>> topic
>>>> (e.g. __session_keys), much like offsets are now. When the producer
>>>> starts
>>>> up, they create a session key and encrypt it for each consumer with
>>>>the
>>>> current consumer key. The producer publishes the bundle of encrypted
>>>> keys
>>>> into __session_keys as a single message. The producer then publishes
>>>> messages to the normal topic encrypted with the session key. The
>>>> metadata
>>>> for each of those messages would contain something the offset into
>>>> __session_keys to identify the bundle. This has the added benefit of
>>>>not
>>>> increasing the per-message data size too much.
>>>> 
>>>> Whenever a consumer key is invalidated, or however often the session
>>>>key
>>>> should be rotated, the producer would publish a new bundle. This
>>>> maintains
>>>> a history of session keys that can be used to decrypt any messages, so
>>>> the
>>>> retention on __session_keys must be at least as long as any topic
>>>>which
>>>> may potentially contain encrypted data. Past that point, it’s up to
>>>>the
>>>> consumer what they want to do with the data. A consumer like Hadoop
>>>> might
>>>> re-encrypt it for local storage, or store it in plaintext (depending
>>>>on
>>>> the security and requirements of that system).
>>>> 
>>>> -Todd
>>>> 
>>>>> On 6/8/14, 2:33 PM, "Rob Withers" <ro...@gmail.com> wrote:
>>>>> 
>>>>> 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 deeper for me, please?
>>>>> The model I envision is internal OTP, with access to decryption
>>>>> accessed by cert.  A double layer of security, with the internal at-
>>>>> rest encryption being an unchanging OTP with ACL access to it as the
>>>>> upper layer.  Are you saying it is possible to re-encrypt with new
>>>>> keys or that there is a chain of keys over time?
>>>>> 
>>>>> Thanks,
>>>>> Rob
>>>>> 
>>>>>> On Jun 8, 2014, at 3:06 PM, Todd Palino wrote:
>>>>>> 
>>>>>> 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, and other PII standards). Yes, it does add external overhead
>>>>>> that
>>>>>> must be managed, but it’s just the nature of the beast. We can’t
>>>>>> solve all
>>>>>> of the external infrastructure needed for this, but we can make it
>>>>>> easier
>>>>>> to use for consumers and producers by adding metadata.
>>>>>> 
>>>>>> There’s no need for unchanging encryption, and that’s specifically
>>>>>> why I
>>>>>> want to see a message envelope that will help consumers determine
>>>>>>the
>>>>>> encryption uses for a particular message.  You can definitely still
>>>>>> expire
>>>>>> keys, you just have to keep the expired keys around as long as the
>>>>>> encrypted data stays around, and your endpoints need to know when
>>>>>> they are
>>>>>> decrypting data with an expired key (you might want to throw up a
>>>>>> warning,
>>>>>> or do something else to let the users know that it’s happening). And
>>>>>> as
>>>>>> someone else mentioned, there are solutions for encrypting data for
>>>>>> multiple consumers. You can encrypt the data with an OTP, and then
>>>>>> multiply encrypt the OTP once for each consumer and store those
>>>>>> encrypted
>>>>>> strings in the envelope.
>>>>>> 
>>>>>> -Todd
>>>>>> 
>>>>>>> On 6/7/14, 12:25 PM, "Rob Withers" <ro...@gmail.com>
>>>>>>> wrote:
>>>>>>> 
>>>>>>> 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 encryption for the data and the clients (producers/
>>>>>>> consumers) will need to be able to decode frozen data.  If certs
>>>>>>>are
>>>>>>> used they cannot expire.  Also, different clients would need to use
>>>>>>> the same cert.
>>>>>>> 
>>>>>>> So, you statement that it should ABSOLUTELY not include internal
>>>>>>> encryption rings seems misplaced.  There are some customers of
>>>>>>>kafka
>>>>>>> that would opt to encrypt the on-disk data and key management is a
>>>>>>> significant issue.  This is best handled internally, with key
>>>>>>> management stored in either ZK or in a topic.  Truly, perhaps
>>>>>>> annealing Hadoop/HBASE as a metadata store seems applicable.
>>>>>>> 
>>>>>>> Thanks, another 2 cents,
>>>>>>> Rob
>>>>>>> 
>>>>>>>> On Jun 6, 2014, at 12:15 PM, Todd Palino wrote:
>>>>>>>> 
>>>>>>>> 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 server then has access to all the keys. You could say
>>>>>>>> that
>>>>>>>> this eliminates the need for TLS, but TLS is pretty low-hanging
>>>>>>>> fruit, and
>>>>>>>> there¹s definitely a need for encryption of the traffic across the
>>>>>>>> network
>>>>>>>> even if you don¹t need at-rest encryption as well.
>>>>>>>> 
>>>>>>>> And as you mentioned, something needs to be done about key
>>>>>>>> management.
>>>>>>>> Storing information with the message about which key(s) was used
>>>>>>>>is
>>>>>>>> a good
>>>>>>>> idea, because it allows you to know when a producer has switched
>>>>>>>> keys.
>>>>>>>> There are definitely some alternative solutions to that as well.
>>>>>>>>But
>>>>>>>> storing the keys in the broker, Zookeeper, or other systems like
>>>>>>>> that are
>>>>>>>> not. There needs to be a system used where the keys are only
>>>>>>>> available to
>>>>>>>> the producers and consumers that need them, and they only get
>>>>>>>>access
>>>>>>>> to
>>>>>>>> the appropriate part of the key pair.  Even as the guy running
>>>>>>>>Kafka
>>>>>>>> and
>>>>>>>> Zookeeper, I should not have access to the keys being used, and if
>>>>>>>> data is
>>>>>>>> encrypted I should not be able to see the cleartext.
>>>>>>>> 
>>>>>>>> And even if we decide not to put anything about at-rest encryption
>>>>>>>> in the
>>>>>>>> consumer/producer clients directly, and leave it for an exercise
>>>>>>>> above
>>>>>>>> that level (you have to pass the ciphertext as the message to the
>>>>>>>> client),
>>>>>>>> I still think there is a good case for implementing a message
>>>>>>>> envelope
>>>>>>>> that can store the information about which key was used, and other
>>>>>>>> pertinent metadata, and have the ability for special applications
>>>>>>>> like
>>>>>>>> mirror maker to be able to preserve it across clusters. This still
>>>>>>>> helps
>>>>>>>> to enable the use of encryption and other features (like auditing)
>>>>>>>> even if
>>>>>>>> we decide it¹s too large a scope to fully implement.
>>>>>>>> 
>>>>>>>> -Todd
>>>>>>>> 
>>>>>>>> On 6/6/14, 10:51 AM, "Pradeep Gollakota" <pr...@gmail.com>
>>>>>>>> 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
>>>>>>>>>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 better performance.
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> On Fri, Jun 6, 2014 at 10:48 AM, Rob Withers
>>>>>>>>> <robert.w.withers@gmail.com
>>>>>>>>> wrote:
>>>>>>>>> 
>>>>>>>>>> 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 key....certs won't work, unless
>>>>>>>>>> you
>>>>>>>>>> write
>>>>>>>>>> a CertAuthority to build multiple certs with the same keys.
>>>>>>>>>> Better
>>>>>>>>>> seems
>>>>>>>>>> to not use certs and wrap the encryption specification with an
>>>>>>>>>>ACL
>>>>>>>>>> capabilities for each group of access.
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> On Jun 6, 2014, at 11:43 AM, Rob Withers wrote:
>>>>>>>>>> 
>>>>>>>>>> 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/WRITE/DELETE/LIST/SCAN).  Erights.org has the
>>>>>>>>>>> quitenscential (??)
>>>>>>>>>>> object capabilities model and capnproto is impleemting this for
>>>>>>>>>>> C+
>>>>>>>>>>> +.  I
>>>>>>>>>>> have a java implementation at http://github.com/pauwau/pauwau
>>>>>>>>>>>but
>>>>>>>>>>> the
>>>>>>>>>>> master is broken.  0.2 works, basically.  B asically a TLS
>>>>>>>>>>> connection
>>>>>>>>>>> with
>>>>>>>>>>> no certificate server, it is peer to peer.  It has some
>>>>>>>>>>>advanced
>>>>>>>>>>> features,
>>>>>>>>>>> but the lining of capabilities with authorization so that you
>>>>>>>>>>>can
>>>>>>>>>>> only
>>>>>>>>>>> invoke correct services is extended to the secure user.
>>>>>>>>>>> 
>>>>>>>>>>> Regarding non-repudiation, on disk, why not prepend a CRC?
>>>>>>>>>>> 
>>>>>>>>>>> Regarding on-disk encryption, multiple users/groups may need to
>>>>>>>>>>> access,
>>>>>>>>>>> with different capabilities.  Sounds like zookeeper needs to
>>>>>>>>>>> store a
>>>>>>>>>>> cert
>>>>>>>>>>> for each class of access so that a group member can access the
>>>>>>>>>>> decrypted
>>>>>>>>>>> data from disk.  Use cert-based async decryption.  The only
>>>>>>>>>>> isue is
>>>>>>>>>>> storing
>>>>>>>>>>> the private key in zookeeper.  Perhaps some hash magic could be
>>>>>>>>>>> used.
>>>>>>>>>>> 
>>>>>>>>>>> Thanks for kafka,
>>>>>>>>>>> Rob
>>>>>>>>>>> 
>>>>>>>>>>> On Jun 5, 2014, at 3:01 PM, Jay Kreps wrote:
>>>>>>>>>>> 
>>>>>>>>>>> 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 encryption scheme? Is this
>>>>>>>>>>>> actually
>>>>>>>>>>>> needed?
>>>>>>>>>>>> Isn't a on-the-wire encryption when combined with mutual
>>>>>>>>>>>> authentication
>>>>>>>>>>>> and
>>>>>>>>>>>> permissions sufficient for most uses?
>>>>>>>>>>>> 
>>>>>>>>>>>> On-disk encryption seems unnecessary because if an attacker
>>>>>>>>>>>>can
>>>>>>>>>>>> get
>>>>>>>>>>>> root
>>>>>>>>>>>> on
>>>>>>>>>>>> the kafka boxes it can potentially modify Kafka to do anything
>>>>>>>>>>>> he or
>>>>>>>>>>>> she
>>>>>>>>>>>> wants with data. So this seems to break any security model.
>>>>>>>>>>>> 
>>>>>>>>>>>> I understand the problem of a large organization not really
>>>>>>>>>>>> having a
>>>>>>>>>>>> trusted network and wanting to secure data transfer and limit
>>>>>>>>>>>> and
>>>>>>>>>>>> audit
>>>>>>>>>>>> data access. The uses for these other things I don't totally
>>>>>>>>>>>> understand.
>>>>>>>>>>>> 
>>>>>>>>>>>> Also it would be worth understanding the state of other
>>>>>>>>>>>> messaging and
>>>>>>>>>>>> storage systems (Hadoop, dbs, etc). What features do they
>>>>>>>>>>>> support. I
>>>>>>>>>>>> think
>>>>>>>>>>>> there is a sense in which you don't have to run faster than
>>>>>>>>>>>>the
>>>>>>>>>>>> bear,
>>>>>>>>>>>> but
>>>>>>>>>>>> only faster then your friends. :-)
>>>>>>>>>>>> 
>>>>>>>>>>>> -Jay
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> On Wed, Jun 4, 2014 at 5:57 PM, Joe Stein
>>>>>>>>>>>><jo...@stealth.ly>
>>>>>>>>>>>> wrote:
>>>>>>>>>>>> 
>>>>>>>>>>>> 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
>>>>>>>>>>>>><ja...@gmail.com>
>>>>>>>>>>>>> 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
>>>>>>>>>>>>>> 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 most the LinkedIn folks are working on the consumer
>>>>>>>>>>>>>> right now
>>>>>>>>>>>>>> I
>>>>>>>>>>>>> think
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> this would be a great project for any other interested
>>>>>>>>>>>>>> people to
>>>>>>>>>>>>>> take
>>>>>>>>>>>>>> on.
>>>>>>>>>>>>>> There are some challenges in doing these things distributed
>>>>>>>>>>>>>> but it
>>>>>>>>>>>>>> can
>>>>>>>>>>>>> also
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> be a lot of fun.
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> I think a good first step would be to get a written plan we
>>>>>>>>>>>>>> can all
>>>>>>>>>>>>>> agree
>>>>>>>>>>>>>> on for how things should work. Then we can break things down
>>>>>>>>>>>>>> into
>>>>>>>>>>>>>> chunks
>>>>>>>>>>>>>> that can be done independently while still aiming at a good
>>>>>>>>>>>>>> end
>>>>>>>>>>>>>> state.
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> I had tried to write up some notes that summarized at least
>>>>>>>>>>>>>> the
>>>>>>>>>>>>>> thoughts
>>>>>>>>>>>>> I
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> had had on security:
>>>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/Security
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> What do you think of that?
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> One assumption I had (which may be incorrect) is that
>>>>>>>>>>>>>>although
>>>>>>>>>>>>>> we
>>>>>>>>>>>>>> want
>>>>>>>>>>>>> all
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> the things in your list, the two most pressing would be
>>>>>>>>>>>>>> authentication
>>>>>>>>>>>>> and
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> authorization, and that was all that write up covered. You
>>>>>>>>>>>>>> have more
>>>>>>>>>>>>>> experience in this domain, so I wonder how you would
>>>>>>>>>>>>>> prioritize?
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> Those notes are really sketchy, so I think the first goal I
>>>>>>>>>>>>>> would
>>>>>>>>>>>>>> have
>>>>>>>>>>>>>> would be to get to a real spec we can all agree on and
>>>>>>>>>>>>>> discuss. A
>>>>>>>>>>>>>> lot
>>>>>>>>>>>>>> of
>>>>>>>>>>>>>> the security stuff has a high human interaction element and
>>>>>>>>>>>>>> needs to
>>>>>>>>>>>>>> work
>>>>>>>>>>>>>> in pretty different domains and different companies so
>>>>>>>>>>>>>>getting
>>>>>>>>>>>>>> this
>>>>>>>>>>>>>> kind
>>>>>>>>>>>>> of
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> review is important.
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> -Jay
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> On Tue, Jun 3, 2014 at 12:57 PM, Joe Stein
>>>>>>>>>>>>>> <jo...@stealth.ly>
>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 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.  This comes from what folks are doing "in the
>>>>>>>>>>>>>>> wild"
>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>> workaround and implement security with Kafka as it is today
>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>> also
>>>>>>>>>>>>>> what I
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> have discovered from organizations about their blockers. It
>>>>>>>>>>>>>>> also
>>>>>>>>>>>>>>> picks
>>>>>>>>>>>>>> up
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> from the wiki (which I should have time to update later in
>>>>>>>>>>>>>>the
>>>>>>>>>>>>>> week
>>>>>>>>>>>>>> based
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> on the below and feedback from the thread).
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 1) Transport Layer Security (i.e. SSL)
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> This also includes client authentication in addition to in-
>>>>>>>>>>>>>>> transit
>>>>>>>>>>>>>> security
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> layer.  This work has been picked up here
>>>>>>>>>>>>>>> https://issues.apache.org/jira/browse/KAFKA-1477 and do
>>>>>>>>>>>>>>> appreciate
>>>>>>>>>>>>>>> any
>>>>>>>>>>>>>>> thoughts, comments, feedback, tomatoes, whatever for this
>>>>>>>>>>>>>>> patch.
>>>>>>>>>>>>>>> It
>>>>>>>>>>>>>> is a
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> pickup from the fork of the work first done here
>>>>>>>>>>>>>>> https://github.com/relango/kafka/tree/kafka_security.
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 2) Data encryption at rest.
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> This is very important and something that can be
>>>>>>>>>>>>>>>facilitated
>>>>>>>>>>>>>>> within
>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>> wire protocol. It requires an additional map data structure
>>>>>>>>>>>>>>> for the
>>>>>>>>>>>>>>> "encrypted [data encryption key]". With this map (either in
>>>>>>>>>>>>>>> your
>>>>>>>>>>>>>>> object
>>>>>>>>>>>>>> or
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> in the wire protocol) you can store the dynamically
>>>>>>>>>>>>>>>generated
>>>>>>>>>>>>>>> symmetric
>>>>>>>>>>>>>> key
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> (for each message) and then encrypt the data using that
>>>>>>>>>>>>>>> dynamically
>>>>>>>>>>>>>>> generated key.  You then encrypt the encryption key using
>>>>>>>>>>>>>>> each
>>>>>>>>>>>>>>> public
>>>>>>>>>>>>>> key
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> for whom is expected to be able to decrypt the encryption
>>>>>>>>>>>>>> key to
>>>>>>>>>>>>>> then
>>>>>>>>>>>>>>> decrypt the message.  For each public key encrypted
>>>>>>>>>>>>>>>symmetric
>>>>>>>>>>>>>>> key
>>>>>>>>>>>>>> (which
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> is
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> now the "encrypted [data encryption key]" along with which
>>>>>>>>>>>>>>> public
>>>>>>>>>>>>>>> key
>>>>>>>>>>>>>> it
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> was encrypted with for (so a map of [publicKey] =
>>>>>>>>>>>>>>> encryptedDataEncryptionKey) as a chain.   Other patterns
>>>>>>>>>>>>>>> can be
>>>>>>>>>>>>>> implemented
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> but this is a pretty standard digital enveloping [0]
>>>>>>>>>>>>>>>pattern
>>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>> only
>>>>>>>>>>>>>> 1
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> field added. Other patterns should be able to use that field
>>>>>>>>>>>>>> to-do
>>>>>>>>>>>>>> their
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> implementation too.
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 3) Non-repudiation and long term non-repudiation.
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> Non-repudiation is proving data hasn't changed.  This is
>>>>>>>>>>>>>>> often (if
>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>> always) done with x509 public certificates (chained to a
>>>>>>>>>>>>>>> certificate
>>>>>>>>>>>>>>> authority).
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> Long term non-repudiation is what happens when the
>>>>>>>>>>>>>>> certificates of
>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>> certificate authority are expired (or revoked) and
>>>>>>>>>>>>>>>everything
>>>>>>>>>>>>>>> ever
>>>>>>>>>>>>>> signed
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> (ever) with that certificate's public key then becomes "no
>>>>>>>>>>>>>> longer
>>>>>>>>>>>>>> provable
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> as ever being authentic".  That is where RFC3126 [1] and
>>>>>>>>>>>>>>> RFC3161
>>>>>>>>>>>>>>> [2]
>>>>>>>>>>>>>> come
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> in (or worm drives [hardware], etc).
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> For either (or both) of these it is an operation of the
>>>>>>>>>>>>>>> encryptor
>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>> sign/hash the data (with or without third party trusted
>>>>>>>>>>>>>>> timestap of
>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>> signing event) and encrypt that with their own private key
>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>> distribute
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> the results (before and after encrypting if required) along
>>>>>>>>>>>>>> with
>>>>>>>>>>>>>> their
>>>>>>>>>>>>>>> public key. This structure is a bit more complex but
>>>>>>>>>>>>>>> feasible, it
>>>>>>>>>>>>>>> is a
>>>>>>>>>>>>>> map
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> of digital signature formats and the chain of dig sig
>>>>>>>>>>>>>>> attestations.
>>>>>>>>>>>>>> The
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> map's key being the method (i.e. CRC32, PKCS7 [3], XmlDigSig
>>>>>>>>>>>>>> [4])
>>>>>>>>>>>>>> and
>>>>>>>>>>>>>> then
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> a list of map where that key is "purpose" of signature
>>>>>>>>>>>>>>>(what
>>>>>>>>>>>>>>> your
>>>>>>>>>>>>>> attesting
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> too).  As a sibling field to the list another field for
>>>>>>>>>>>>>>>"the
>>>>>>>>>>>>>>> attester"
>>>>>>>>>>>>>> as
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> bytes (e.g. their PKCS12 [5] for the map of PKCS7
>>>>>>>>>>>>>>signatures).
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 4) Authorization
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> We should have a policy of "404" for data, topics,
>>>>>>>>>>>>>>>partitions
>>>>>>>>>>>>>>> (etc) if
>>>>>>>>>>>>>>> authenticated connections do not have access.  In "secure
>>>>>>>>>>>>>>> mode" any
>>>>>>>>>>>>>>> non
>>>>>>>>>>>>>>> authenticated connections should get a "404" type message
>>>>>>>>>>>>>>>on
>>>>>>>>>>>>>> everything.
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> Knowing "something is there" is a security risk in many uses
>>>>>>>>>>>>>> cases.
>>>>>>>>>>>>>> So
>>>>>>>>>>>>>> if
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> you don't have access you don't even see it.  Baking "that"
>>>>>>>>>>>>>>> into
>>>>>>>>>>>>>>> Kafka
>>>>>>>>>>>>>>> along with some interface for entitlement (access
>>>>>>>>>>>>>>>management)
>>>>>>>>>>>>>>> systems
>>>>>>>>>>>>>>> (pretty standard) is all that I think needs to be done to
>>>>>>>>>>>>>>>the
>>>>>>>>>>>>>>> core
>>>>>>>>>>>>>> project.
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> I want to tackle item later in the year after summer after
>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>> other
>>>>>>>>>>>>>> three
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> are complete.
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> I look forward to thoughts on this and anyone else
>>>>>>>>>>>>>>>interested
>>>>>>>>>>>>>>> in
>>>>>>>>>>>>>> working
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> with us on these items.
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> [0]
>>>>>>>>>>>>>> http://www.emc.com/emc-plus/rsa-labs/standards-
>>>>>>>>>>>>> initiatives/what-is-a-digital-envelope.htm
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> [1] http://tools.ietf.org/html/rfc3126
>>>>>>>>>>>>>>> [2] http://tools.ietf.org/html/rfc3161
>>>>>>>>>>>>>>> [3]
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>http://www.emc.com/emc-plus/rsa-labs/standards-initiatives/pk
>>>>>>>>>>>>>>cs
>>>>>>>>>>>>>> -7
>>>>>>>>>>>>>> -
>>>>>>>>>>>>> cryptographic-message-syntax-standar.htm
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> [4] http://en.wikipedia.org/wiki/XML_Signature
>>>>>>>>>>>>>>> [5] http://en.wikipedia.org/wiki/PKCS_12
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> /*******************************************
>>>>>>>>>>>>>>> Joe Stein
>>>>>>>>>>>>>>> Founder, Principal Consultant
>>>>>>>>>>>>>>> Big Data Open Source Security LLC
>>>>>>>>>>>>>>> http://www.stealth.ly
>>>>>>>>>>>>>>> Twitter: @allthingshadoop
>>>>>>>>>>>>>>> <http://www.twitter.com/allthingshadoop
>>>>>>>>>>>>>>> ********************************************/
>> 


Re: [DISCUSS] Kafka Security Specific Features

Posted by Robert Withers <ro...@gmail.com>.
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 <tp...@linkedin.com.INVALID> wrote:
> 
> 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 pair generated
> for it already, so the producer could start encrypting for that consumer
> before it exists. This would probably work fine for lower retention
> periods (a week or two), but could be a little more difficult to manage if
> you are keeping data in Kafka longer than that. My gut reaction is that
> it’s better to handle it that way and keep the key pair and session key
> handling simple. The more we can do that, the more we can leave key
> management as a separate component that can be swapped out so the user can
> decide how it should be done.
> 
> -Todd
> 
> 
>> On 6/9/14, 8:16 AM, "Robert Withers" <ro...@gmail.com> wrote:
>> 
>> 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 before the complete recipient list is
>> known.
>> 
>> This seems to mean that the private key (cert or OTP) must be stored and
>> interacted with.  My feeling is that key metadata are in a system
>> encrypted Hbase store (session key store), for low latency reads, rather
>> than a topic requiring scanning.  Store the private keys and then give
>> client access (producers/consumers) with the hash of the OTP.  A new
>> consumer comes along, create a new cert encoding the OTP hash.
>> 
>> On write, use the producer cert to send a topic hash with the msg which
>> would allow the broker to reuse or generate an OTP, stored in the session
>> key store.
>> 
>> On read (consumer), if we have a previously run reader, use the encrypted
>> hash.  If new, create consumer cert and encrypt the hash for that session.
>> 
>> The reader/writer will pass a cert encrypted session hash.  The trick
>> seems to be converting hash to PK to encrypt/decrypt.  Given Kafka
>> resource distribution, we need system encryption for metadata and
>> cert-based key exchange.  This seems to mean triple encryption:
>> 1) client to/from broker
>> 2) system key/hash  mgmt/translation
>> 3) at-rest encryption
>> 
>> Thanks,
>> Rob
>> 
>>> On Jun 9, 2014, at 7:57 AM, Todd Palino <tp...@linkedin.com.INVALID>
>>> wrote:
>>> 
>>> 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 the encrypted copies of the session key
>>> are
>>> then included with the encrypted message. This way, you avoid having to
>>> encrypt the message multiple times (this assumes, of course, that the
>>> message itself is larger than the key).
>>> 
>>> In our case, we have some options available to us. We could do that, and
>>> put all the encrypted keys in the message metadata. Or we could treat it
>>> more like a session and have the encrypted session keys in a special
>>> topic
>>> (e.g. __session_keys), much like offsets are now. When the producer
>>> starts
>>> up, they create a session key and encrypt it for each consumer with the
>>> current consumer key. The producer publishes the bundle of encrypted
>>> keys
>>> into __session_keys as a single message. The producer then publishes
>>> messages to the normal topic encrypted with the session key. The
>>> metadata
>>> for each of those messages would contain something the offset into
>>> __session_keys to identify the bundle. This has the added benefit of not
>>> increasing the per-message data size too much.
>>> 
>>> Whenever a consumer key is invalidated, or however often the session key
>>> should be rotated, the producer would publish a new bundle. This
>>> maintains
>>> a history of session keys that can be used to decrypt any messages, so
>>> the
>>> retention on __session_keys must be at least as long as any topic which
>>> may potentially contain encrypted data. Past that point, it’s up to the
>>> consumer what they want to do with the data. A consumer like Hadoop
>>> might
>>> re-encrypt it for local storage, or store it in plaintext (depending on
>>> the security and requirements of that system).
>>> 
>>> -Todd
>>> 
>>>> On 6/8/14, 2:33 PM, "Rob Withers" <ro...@gmail.com> wrote:
>>>> 
>>>> 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 deeper for me, please?
>>>> The model I envision is internal OTP, with access to decryption
>>>> accessed by cert.  A double layer of security, with the internal at-
>>>> rest encryption being an unchanging OTP with ACL access to it as the
>>>> upper layer.  Are you saying it is possible to re-encrypt with new
>>>> keys or that there is a chain of keys over time?
>>>> 
>>>> Thanks,
>>>> Rob
>>>> 
>>>>> On Jun 8, 2014, at 3:06 PM, Todd Palino wrote:
>>>>> 
>>>>> 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, and other PII standards). Yes, it does add external overhead
>>>>> that
>>>>> must be managed, but it’s just the nature of the beast. We can’t
>>>>> solve all
>>>>> of the external infrastructure needed for this, but we can make it
>>>>> easier
>>>>> to use for consumers and producers by adding metadata.
>>>>> 
>>>>> There’s no need for unchanging encryption, and that’s specifically
>>>>> why I
>>>>> want to see a message envelope that will help consumers determine the
>>>>> encryption uses for a particular message.  You can definitely still
>>>>> expire
>>>>> keys, you just have to keep the expired keys around as long as the
>>>>> encrypted data stays around, and your endpoints need to know when
>>>>> they are
>>>>> decrypting data with an expired key (you might want to throw up a
>>>>> warning,
>>>>> or do something else to let the users know that it’s happening). And
>>>>> as
>>>>> someone else mentioned, there are solutions for encrypting data for
>>>>> multiple consumers. You can encrypt the data with an OTP, and then
>>>>> multiply encrypt the OTP once for each consumer and store those
>>>>> encrypted
>>>>> strings in the envelope.
>>>>> 
>>>>> -Todd
>>>>> 
>>>>>> On 6/7/14, 12:25 PM, "Rob Withers" <ro...@gmail.com>
>>>>>> wrote:
>>>>>> 
>>>>>> 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 encryption for the data and the clients (producers/
>>>>>> consumers) will need to be able to decode frozen data.  If certs are
>>>>>> used they cannot expire.  Also, different clients would need to use
>>>>>> the same cert.
>>>>>> 
>>>>>> So, you statement that it should ABSOLUTELY not include internal
>>>>>> encryption rings seems misplaced.  There are some customers of kafka
>>>>>> that would opt to encrypt the on-disk data and key management is a
>>>>>> significant issue.  This is best handled internally, with key
>>>>>> management stored in either ZK or in a topic.  Truly, perhaps
>>>>>> annealing Hadoop/HBASE as a metadata store seems applicable.
>>>>>> 
>>>>>> Thanks, another 2 cents,
>>>>>> Rob
>>>>>> 
>>>>>>> On Jun 6, 2014, at 12:15 PM, Todd Palino wrote:
>>>>>>> 
>>>>>>> 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 server then has access to all the keys. You could say
>>>>>>> that
>>>>>>> this eliminates the need for TLS, but TLS is pretty low-hanging
>>>>>>> fruit, and
>>>>>>> there¹s definitely a need for encryption of the traffic across the
>>>>>>> network
>>>>>>> even if you don¹t need at-rest encryption as well.
>>>>>>> 
>>>>>>> And as you mentioned, something needs to be done about key
>>>>>>> management.
>>>>>>> Storing information with the message about which key(s) was used is
>>>>>>> a good
>>>>>>> idea, because it allows you to know when a producer has switched
>>>>>>> keys.
>>>>>>> There are definitely some alternative solutions to that as well. But
>>>>>>> storing the keys in the broker, Zookeeper, or other systems like
>>>>>>> that are
>>>>>>> not. There needs to be a system used where the keys are only
>>>>>>> available to
>>>>>>> the producers and consumers that need them, and they only get access
>>>>>>> to
>>>>>>> the appropriate part of the key pair.  Even as the guy running Kafka
>>>>>>> and
>>>>>>> Zookeeper, I should not have access to the keys being used, and if
>>>>>>> data is
>>>>>>> encrypted I should not be able to see the cleartext.
>>>>>>> 
>>>>>>> And even if we decide not to put anything about at-rest encryption
>>>>>>> in the
>>>>>>> consumer/producer clients directly, and leave it for an exercise
>>>>>>> above
>>>>>>> that level (you have to pass the ciphertext as the message to the
>>>>>>> client),
>>>>>>> I still think there is a good case for implementing a message
>>>>>>> envelope
>>>>>>> that can store the information about which key was used, and other
>>>>>>> pertinent metadata, and have the ability for special applications
>>>>>>> like
>>>>>>> mirror maker to be able to preserve it across clusters. This still
>>>>>>> helps
>>>>>>> to enable the use of encryption and other features (like auditing)
>>>>>>> even if
>>>>>>> we decide it¹s too large a scope to fully implement.
>>>>>>> 
>>>>>>> -Todd
>>>>>>> 
>>>>>>> On 6/6/14, 10:51 AM, "Pradeep Gollakota" <pr...@gmail.com>
>>>>>>> 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 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 better performance.
>>>>>>>> 
>>>>>>>> 
>>>>>>>> On Fri, Jun 6, 2014 at 10:48 AM, Rob Withers
>>>>>>>> <robert.w.withers@gmail.com
>>>>>>>> wrote:
>>>>>>>> 
>>>>>>>>> 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 key....certs won't work, unless
>>>>>>>>> you
>>>>>>>>> write
>>>>>>>>> a CertAuthority to build multiple certs with the same keys.
>>>>>>>>> Better
>>>>>>>>> seems
>>>>>>>>> to not use certs and wrap the encryption specification with an ACL
>>>>>>>>> capabilities for each group of access.
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> On Jun 6, 2014, at 11:43 AM, Rob Withers wrote:
>>>>>>>>> 
>>>>>>>>> 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/WRITE/DELETE/LIST/SCAN).  Erights.org has the
>>>>>>>>>> quitenscential (??)
>>>>>>>>>> object capabilities model and capnproto is impleemting this for
>>>>>>>>>> C+
>>>>>>>>>> +.  I
>>>>>>>>>> have a java implementation at http://github.com/pauwau/pauwau but
>>>>>>>>>> the
>>>>>>>>>> master is broken.  0.2 works, basically.  B asically a TLS
>>>>>>>>>> connection
>>>>>>>>>> with
>>>>>>>>>> no certificate server, it is peer to peer.  It has some advanced
>>>>>>>>>> features,
>>>>>>>>>> but the lining of capabilities with authorization so that you can
>>>>>>>>>> only
>>>>>>>>>> invoke correct services is extended to the secure user.
>>>>>>>>>> 
>>>>>>>>>> Regarding non-repudiation, on disk, why not prepend a CRC?
>>>>>>>>>> 
>>>>>>>>>> Regarding on-disk encryption, multiple users/groups may need to
>>>>>>>>>> access,
>>>>>>>>>> with different capabilities.  Sounds like zookeeper needs to
>>>>>>>>>> store a
>>>>>>>>>> cert
>>>>>>>>>> for each class of access so that a group member can access the
>>>>>>>>>> decrypted
>>>>>>>>>> data from disk.  Use cert-based async decryption.  The only
>>>>>>>>>> isue is
>>>>>>>>>> storing
>>>>>>>>>> the private key in zookeeper.  Perhaps some hash magic could be
>>>>>>>>>> used.
>>>>>>>>>> 
>>>>>>>>>> Thanks for kafka,
>>>>>>>>>> Rob
>>>>>>>>>> 
>>>>>>>>>> On Jun 5, 2014, at 3:01 PM, Jay Kreps wrote:
>>>>>>>>>> 
>>>>>>>>>> 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 encryption scheme? Is this
>>>>>>>>>>> actually
>>>>>>>>>>> needed?
>>>>>>>>>>> Isn't a on-the-wire encryption when combined with mutual
>>>>>>>>>>> authentication
>>>>>>>>>>> and
>>>>>>>>>>> permissions sufficient for most uses?
>>>>>>>>>>> 
>>>>>>>>>>> On-disk encryption seems unnecessary because if an attacker can
>>>>>>>>>>> get
>>>>>>>>>>> root
>>>>>>>>>>> on
>>>>>>>>>>> the kafka boxes it can potentially modify Kafka to do anything
>>>>>>>>>>> he or
>>>>>>>>>>> she
>>>>>>>>>>> wants with data. So this seems to break any security model.
>>>>>>>>>>> 
>>>>>>>>>>> I understand the problem of a large organization not really
>>>>>>>>>>> having a
>>>>>>>>>>> trusted network and wanting to secure data transfer and limit
>>>>>>>>>>> and
>>>>>>>>>>> audit
>>>>>>>>>>> data access. The uses for these other things I don't totally
>>>>>>>>>>> understand.
>>>>>>>>>>> 
>>>>>>>>>>> Also it would be worth understanding the state of other
>>>>>>>>>>> messaging and
>>>>>>>>>>> storage systems (Hadoop, dbs, etc). What features do they
>>>>>>>>>>> support. I
>>>>>>>>>>> think
>>>>>>>>>>> there is a sense in which you don't have to run faster than the
>>>>>>>>>>> bear,
>>>>>>>>>>> but
>>>>>>>>>>> only faster then your friends. :-)
>>>>>>>>>>> 
>>>>>>>>>>> -Jay
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> On Wed, Jun 4, 2014 at 5:57 PM, Joe Stein <jo...@stealth.ly>
>>>>>>>>>>> wrote:
>>>>>>>>>>> 
>>>>>>>>>>> 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 <ja...@gmail.com>
>>>>>>>>>>>> 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
>>>>>>>>>>>>> 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 most the LinkedIn folks are working on the consumer
>>>>>>>>>>>>> right now
>>>>>>>>>>>>> I
>>>>>>>>>>>> think
>>>>>>>>>>>> 
>>>>>>>>>>>>> this would be a great project for any other interested
>>>>>>>>>>>>> people to
>>>>>>>>>>>>> take
>>>>>>>>>>>>> on.
>>>>>>>>>>>>> There are some challenges in doing these things distributed
>>>>>>>>>>>>> but it
>>>>>>>>>>>>> can
>>>>>>>>>>>> also
>>>>>>>>>>>> 
>>>>>>>>>>>>> be a lot of fun.
>>>>>>>>>>>>> 
>>>>>>>>>>>>> I think a good first step would be to get a written plan we
>>>>>>>>>>>>> can all
>>>>>>>>>>>>> agree
>>>>>>>>>>>>> on for how things should work. Then we can break things down
>>>>>>>>>>>>> into
>>>>>>>>>>>>> chunks
>>>>>>>>>>>>> that can be done independently while still aiming at a good
>>>>>>>>>>>>> end
>>>>>>>>>>>>> state.
>>>>>>>>>>>>> 
>>>>>>>>>>>>> I had tried to write up some notes that summarized at least
>>>>>>>>>>>>> the
>>>>>>>>>>>>> thoughts
>>>>>>>>>>>> I
>>>>>>>>>>>> 
>>>>>>>>>>>>> had had on security:
>>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/Security
>>>>>>>>>>>>> 
>>>>>>>>>>>>> What do you think of that?
>>>>>>>>>>>>> 
>>>>>>>>>>>>> One assumption I had (which may be incorrect) is that although
>>>>>>>>>>>>> we
>>>>>>>>>>>>> want
>>>>>>>>>>>> all
>>>>>>>>>>>> 
>>>>>>>>>>>>> the things in your list, the two most pressing would be
>>>>>>>>>>>>> authentication
>>>>>>>>>>>> and
>>>>>>>>>>>> 
>>>>>>>>>>>>> authorization, and that was all that write up covered. You
>>>>>>>>>>>>> have more
>>>>>>>>>>>>> experience in this domain, so I wonder how you would
>>>>>>>>>>>>> prioritize?
>>>>>>>>>>>>> 
>>>>>>>>>>>>> Those notes are really sketchy, so I think the first goal I
>>>>>>>>>>>>> would
>>>>>>>>>>>>> have
>>>>>>>>>>>>> would be to get to a real spec we can all agree on and
>>>>>>>>>>>>> discuss. A
>>>>>>>>>>>>> lot
>>>>>>>>>>>>> of
>>>>>>>>>>>>> the security stuff has a high human interaction element and
>>>>>>>>>>>>> needs to
>>>>>>>>>>>>> work
>>>>>>>>>>>>> in pretty different domains and different companies so getting
>>>>>>>>>>>>> this
>>>>>>>>>>>>> kind
>>>>>>>>>>>> of
>>>>>>>>>>>> 
>>>>>>>>>>>>> review is important.
>>>>>>>>>>>>> 
>>>>>>>>>>>>> -Jay
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> On Tue, Jun 3, 2014 at 12:57 PM, Joe Stein
>>>>>>>>>>>>> <jo...@stealth.ly>
>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 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.  This comes from what folks are doing "in the
>>>>>>>>>>>>>> wild"
>>>>>>>>>>>>>> to
>>>>>>>>>>>>>> workaround and implement security with Kafka as it is today
>>>>>>>>>>>>>> and
>>>>>>>>>>>>>> also
>>>>>>>>>>>>> what I
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> have discovered from organizations about their blockers. It
>>>>>>>>>>>>>> also
>>>>>>>>>>>>>> picks
>>>>>>>>>>>>> up
>>>>>>>>>>>> 
>>>>>>>>>>>>> from the wiki (which I should have time to update later in the
>>>>>>>>>>>>> week
>>>>>>>>>>>>> based
>>>>>>>>>>>> 
>>>>>>>>>>>>> on the below and feedback from the thread).
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 1) Transport Layer Security (i.e. SSL)
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> This also includes client authentication in addition to in-
>>>>>>>>>>>>>> transit
>>>>>>>>>>>>> security
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> layer.  This work has been picked up here
>>>>>>>>>>>>>> https://issues.apache.org/jira/browse/KAFKA-1477 and do
>>>>>>>>>>>>>> appreciate
>>>>>>>>>>>>>> any
>>>>>>>>>>>>>> thoughts, comments, feedback, tomatoes, whatever for this
>>>>>>>>>>>>>> patch.
>>>>>>>>>>>>>> It
>>>>>>>>>>>>> is a
>>>>>>>>>>>> 
>>>>>>>>>>>>> pickup from the fork of the work first done here
>>>>>>>>>>>>>> https://github.com/relango/kafka/tree/kafka_security.
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 2) Data encryption at rest.
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> This is very important and something that can be facilitated
>>>>>>>>>>>>>> within
>>>>>>>>>>>>>> the
>>>>>>>>>>>>>> wire protocol. It requires an additional map data structure
>>>>>>>>>>>>>> for the
>>>>>>>>>>>>>> "encrypted [data encryption key]". With this map (either in
>>>>>>>>>>>>>> your
>>>>>>>>>>>>>> object
>>>>>>>>>>>>> or
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> in the wire protocol) you can store the dynamically generated
>>>>>>>>>>>>>> symmetric
>>>>>>>>>>>>> key
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> (for each message) and then encrypt the data using that
>>>>>>>>>>>>>> dynamically
>>>>>>>>>>>>>> generated key.  You then encrypt the encryption key using
>>>>>>>>>>>>>> each
>>>>>>>>>>>>>> public
>>>>>>>>>>>>> key
>>>>>>>>>>>> 
>>>>>>>>>>>>> for whom is expected to be able to decrypt the encryption
>>>>>>>>>>>>> key to
>>>>>>>>>>>>> then
>>>>>>>>>>>>>> decrypt the message.  For each public key encrypted symmetric
>>>>>>>>>>>>>> key
>>>>>>>>>>>>> (which
>>>>>>>>>>>> 
>>>>>>>>>>>>> is
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> now the "encrypted [data encryption key]" along with which
>>>>>>>>>>>>>> public
>>>>>>>>>>>>>> key
>>>>>>>>>>>>> it
>>>>>>>>>>>> 
>>>>>>>>>>>>> was encrypted with for (so a map of [publicKey] =
>>>>>>>>>>>>>> encryptedDataEncryptionKey) as a chain.   Other patterns
>>>>>>>>>>>>>> can be
>>>>>>>>>>>>> implemented
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> but this is a pretty standard digital enveloping [0] pattern
>>>>>>>>>>>>>> with
>>>>>>>>>>>>>> only
>>>>>>>>>>>>> 1
>>>>>>>>>>>> 
>>>>>>>>>>>>> field added. Other patterns should be able to use that field
>>>>>>>>>>>>> to-do
>>>>>>>>>>>>> their
>>>>>>>>>>>> 
>>>>>>>>>>>>> implementation too.
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 3) Non-repudiation and long term non-repudiation.
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> Non-repudiation is proving data hasn't changed.  This is
>>>>>>>>>>>>>> often (if
>>>>>>>>>>>>>> not
>>>>>>>>>>>>>> always) done with x509 public certificates (chained to a
>>>>>>>>>>>>>> certificate
>>>>>>>>>>>>>> authority).
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> Long term non-repudiation is what happens when the
>>>>>>>>>>>>>> certificates of
>>>>>>>>>>>>>> the
>>>>>>>>>>>>>> certificate authority are expired (or revoked) and everything
>>>>>>>>>>>>>> ever
>>>>>>>>>>>>> signed
>>>>>>>>>>>> 
>>>>>>>>>>>>> (ever) with that certificate's public key then becomes "no
>>>>>>>>>>>>> longer
>>>>>>>>>>>>> provable
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> as ever being authentic".  That is where RFC3126 [1] and
>>>>>>>>>>>>>> RFC3161
>>>>>>>>>>>>>> [2]
>>>>>>>>>>>>> come
>>>>>>>>>>>> 
>>>>>>>>>>>>> in (or worm drives [hardware], etc).
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> For either (or both) of these it is an operation of the
>>>>>>>>>>>>>> encryptor
>>>>>>>>>>>>>> to
>>>>>>>>>>>>>> sign/hash the data (with or without third party trusted
>>>>>>>>>>>>>> timestap of
>>>>>>>>>>>>>> the
>>>>>>>>>>>>>> signing event) and encrypt that with their own private key
>>>>>>>>>>>>>> and
>>>>>>>>>>>>> distribute
>>>>>>>>>>>> 
>>>>>>>>>>>>> the results (before and after encrypting if required) along
>>>>>>>>>>>>> with
>>>>>>>>>>>>> their
>>>>>>>>>>>>>> public key. This structure is a bit more complex but
>>>>>>>>>>>>>> feasible, it
>>>>>>>>>>>>>> is a
>>>>>>>>>>>>> map
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> of digital signature formats and the chain of dig sig
>>>>>>>>>>>>>> attestations.
>>>>>>>>>>>>> The
>>>>>>>>>>>> 
>>>>>>>>>>>>> map's key being the method (i.e. CRC32, PKCS7 [3], XmlDigSig
>>>>>>>>>>>>> [4])
>>>>>>>>>>>>> and
>>>>>>>>>>>>> then
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> a list of map where that key is "purpose" of signature (what
>>>>>>>>>>>>>> your
>>>>>>>>>>>>> attesting
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> too).  As a sibling field to the list another field for "the
>>>>>>>>>>>>>> attester"
>>>>>>>>>>>>> as
>>>>>>>>>>>> 
>>>>>>>>>>>>> bytes (e.g. their PKCS12 [5] for the map of PKCS7 signatures).
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 4) Authorization
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> We should have a policy of "404" for data, topics, partitions
>>>>>>>>>>>>>> (etc) if
>>>>>>>>>>>>>> authenticated connections do not have access.  In "secure
>>>>>>>>>>>>>> mode" any
>>>>>>>>>>>>>> non
>>>>>>>>>>>>>> authenticated connections should get a "404" type message on
>>>>>>>>>>>>> everything.
>>>>>>>>>>>> 
>>>>>>>>>>>>> Knowing "something is there" is a security risk in many uses
>>>>>>>>>>>>> cases.
>>>>>>>>>>>>> So
>>>>>>>>>>>>> if
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> you don't have access you don't even see it.  Baking "that"
>>>>>>>>>>>>>> into
>>>>>>>>>>>>>> Kafka
>>>>>>>>>>>>>> along with some interface for entitlement (access management)
>>>>>>>>>>>>>> systems
>>>>>>>>>>>>>> (pretty standard) is all that I think needs to be done to the
>>>>>>>>>>>>>> core
>>>>>>>>>>>>> project.
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> I want to tackle item later in the year after summer after
>>>>>>>>>>>>>> the
>>>>>>>>>>>>>> other
>>>>>>>>>>>>> three
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> are complete.
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> I look forward to thoughts on this and anyone else interested
>>>>>>>>>>>>>> in
>>>>>>>>>>>>> working
>>>>>>>>>>>> 
>>>>>>>>>>>>> with us on these items.
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> [0]
>>>>>>>>>>>>> http://www.emc.com/emc-plus/rsa-labs/standards-
>>>>>>>>>>>> initiatives/what-is-a-digital-envelope.htm
>>>>>>>>>>>> 
>>>>>>>>>>>>> [1] http://tools.ietf.org/html/rfc3126
>>>>>>>>>>>>>> [2] http://tools.ietf.org/html/rfc3161
>>>>>>>>>>>>>> [3]
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> http://www.emc.com/emc-plus/rsa-labs/standards-initiatives/pkcs
>>>>>>>>>>>>> -7
>>>>>>>>>>>>> -
>>>>>>>>>>>> cryptographic-message-syntax-standar.htm
>>>>>>>>>>>> 
>>>>>>>>>>>>> [4] http://en.wikipedia.org/wiki/XML_Signature
>>>>>>>>>>>>>> [5] http://en.wikipedia.org/wiki/PKCS_12
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> /*******************************************
>>>>>>>>>>>>>> Joe Stein
>>>>>>>>>>>>>> Founder, Principal Consultant
>>>>>>>>>>>>>> Big Data Open Source Security LLC
>>>>>>>>>>>>>> http://www.stealth.ly
>>>>>>>>>>>>>> Twitter: @allthingshadoop
>>>>>>>>>>>>>> <http://www.twitter.com/allthingshadoop
>>>>>>>>>>>>>> ********************************************/
> 

Re: [DISCUSS] Kafka Security Specific Features

Posted by Todd Palino <tp...@linkedin.com.INVALID>.
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 pair generated
for it already, so the producer could start encrypting for that consumer
before it exists. This would probably work fine for lower retention
periods (a week or two), but could be a little more difficult to manage if
you are keeping data in Kafka longer than that. My gut reaction is that
it’s better to handle it that way and keep the key pair and session key
handling simple. The more we can do that, the more we can leave key
management as a separate component that can be swapped out so the user can
decide how it should be done.

-Todd


On 6/9/14, 8:16 AM, "Robert Withers" <ro...@gmail.com> wrote:

>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 before the complete recipient list is
>known.
>
>This seems to mean that the private key (cert or OTP) must be stored and
>interacted with.  My feeling is that key metadata are in a system
>encrypted Hbase store (session key store), for low latency reads, rather
>than a topic requiring scanning.  Store the private keys and then give
>client access (producers/consumers) with the hash of the OTP.  A new
>consumer comes along, create a new cert encoding the OTP hash.
>
>On write, use the producer cert to send a topic hash with the msg which
>would allow the broker to reuse or generate an OTP, stored in the session
>key store.
>
>On read (consumer), if we have a previously run reader, use the encrypted
>hash.  If new, create consumer cert and encrypt the hash for that session.
>
>The reader/writer will pass a cert encrypted session hash.  The trick
>seems to be converting hash to PK to encrypt/decrypt.  Given Kafka
>resource distribution, we need system encryption for metadata and
>cert-based key exchange.  This seems to mean triple encryption:
>1) client to/from broker
>2) system key/hash  mgmt/translation
>3) at-rest encryption
>
>Thanks,
>Rob
>
>> On Jun 9, 2014, at 7:57 AM, Todd Palino <tp...@linkedin.com.INVALID>
>>wrote:
>> 
>> 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 the encrypted copies of the session key
>>are
>> then included with the encrypted message. This way, you avoid having to
>> encrypt the message multiple times (this assumes, of course, that the
>> message itself is larger than the key).
>> 
>> In our case, we have some options available to us. We could do that, and
>> put all the encrypted keys in the message metadata. Or we could treat it
>> more like a session and have the encrypted session keys in a special
>>topic
>> (e.g. __session_keys), much like offsets are now. When the producer
>>starts
>> up, they create a session key and encrypt it for each consumer with the
>> current consumer key. The producer publishes the bundle of encrypted
>>keys
>> into __session_keys as a single message. The producer then publishes
>> messages to the normal topic encrypted with the session key. The
>>metadata
>> for each of those messages would contain something the offset into
>> __session_keys to identify the bundle. This has the added benefit of not
>> increasing the per-message data size too much.
>> 
>> Whenever a consumer key is invalidated, or however often the session key
>> should be rotated, the producer would publish a new bundle. This
>>maintains
>> a history of session keys that can be used to decrypt any messages, so
>>the
>> retention on __session_keys must be at least as long as any topic which
>> may potentially contain encrypted data. Past that point, it’s up to the
>> consumer what they want to do with the data. A consumer like Hadoop
>>might
>> re-encrypt it for local storage, or store it in plaintext (depending on
>> the security and requirements of that system).
>> 
>> -Todd
>> 
>>> On 6/8/14, 2:33 PM, "Rob Withers" <ro...@gmail.com> wrote:
>>> 
>>> 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 deeper for me, please?
>>> The model I envision is internal OTP, with access to decryption
>>> accessed by cert.  A double layer of security, with the internal at-
>>> rest encryption being an unchanging OTP with ACL access to it as the
>>> upper layer.  Are you saying it is possible to re-encrypt with new
>>> keys or that there is a chain of keys over time?
>>> 
>>> Thanks,
>>> Rob
>>> 
>>>> On Jun 8, 2014, at 3:06 PM, Todd Palino wrote:
>>>> 
>>>> 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, and other PII standards). Yes, it does add external overhead
>>>> that
>>>> must be managed, but it’s just the nature of the beast. We can’t
>>>> solve all
>>>> of the external infrastructure needed for this, but we can make it
>>>> easier
>>>> to use for consumers and producers by adding metadata.
>>>> 
>>>> There’s no need for unchanging encryption, and that’s specifically
>>>> why I
>>>> want to see a message envelope that will help consumers determine the
>>>> encryption uses for a particular message.  You can definitely still
>>>> expire
>>>> keys, you just have to keep the expired keys around as long as the
>>>> encrypted data stays around, and your endpoints need to know when
>>>> they are
>>>> decrypting data with an expired key (you might want to throw up a
>>>> warning,
>>>> or do something else to let the users know that it’s happening). And
>>>> as
>>>> someone else mentioned, there are solutions for encrypting data for
>>>> multiple consumers. You can encrypt the data with an OTP, and then
>>>> multiply encrypt the OTP once for each consumer and store those
>>>> encrypted
>>>> strings in the envelope.
>>>> 
>>>> -Todd
>>>> 
>>>>> On 6/7/14, 12:25 PM, "Rob Withers" <ro...@gmail.com>
>>>>>wrote:
>>>>> 
>>>>> 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 encryption for the data and the clients (producers/
>>>>> consumers) will need to be able to decode frozen data.  If certs are
>>>>> used they cannot expire.  Also, different clients would need to use
>>>>> the same cert.
>>>>> 
>>>>> So, you statement that it should ABSOLUTELY not include internal
>>>>> encryption rings seems misplaced.  There are some customers of kafka
>>>>> that would opt to encrypt the on-disk data and key management is a
>>>>> significant issue.  This is best handled internally, with key
>>>>> management stored in either ZK or in a topic.  Truly, perhaps
>>>>> annealing Hadoop/HBASE as a metadata store seems applicable.
>>>>> 
>>>>> Thanks, another 2 cents,
>>>>> Rob
>>>>> 
>>>>>> On Jun 6, 2014, at 12:15 PM, Todd Palino wrote:
>>>>>> 
>>>>>> 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 server then has access to all the keys. You could say
>>>>>> that
>>>>>> this eliminates the need for TLS, but TLS is pretty low-hanging
>>>>>> fruit, and
>>>>>> there¹s definitely a need for encryption of the traffic across the
>>>>>> network
>>>>>> even if you don¹t need at-rest encryption as well.
>>>>>> 
>>>>>> And as you mentioned, something needs to be done about key
>>>>>> management.
>>>>>> Storing information with the message about which key(s) was used is
>>>>>> a good
>>>>>> idea, because it allows you to know when a producer has switched
>>>>>> keys.
>>>>>> There are definitely some alternative solutions to that as well. But
>>>>>> storing the keys in the broker, Zookeeper, or other systems like
>>>>>> that are
>>>>>> not. There needs to be a system used where the keys are only
>>>>>> available to
>>>>>> the producers and consumers that need them, and they only get access
>>>>>> to
>>>>>> the appropriate part of the key pair.  Even as the guy running Kafka
>>>>>> and
>>>>>> Zookeeper, I should not have access to the keys being used, and if
>>>>>> data is
>>>>>> encrypted I should not be able to see the cleartext.
>>>>>> 
>>>>>> And even if we decide not to put anything about at-rest encryption
>>>>>> in the
>>>>>> consumer/producer clients directly, and leave it for an exercise
>>>>>> above
>>>>>> that level (you have to pass the ciphertext as the message to the
>>>>>> client),
>>>>>> I still think there is a good case for implementing a message
>>>>>> envelope
>>>>>> that can store the information about which key was used, and other
>>>>>> pertinent metadata, and have the ability for special applications
>>>>>> like
>>>>>> mirror maker to be able to preserve it across clusters. This still
>>>>>> helps
>>>>>> to enable the use of encryption and other features (like auditing)
>>>>>> even if
>>>>>> we decide it¹s too large a scope to fully implement.
>>>>>> 
>>>>>> -Todd
>>>>>> 
>>>>>> On 6/6/14, 10:51 AM, "Pradeep Gollakota" <pr...@gmail.com>
>>>>>> 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 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 better performance.
>>>>>>> 
>>>>>>> 
>>>>>>> On Fri, Jun 6, 2014 at 10:48 AM, Rob Withers
>>>>>>> <robert.w.withers@gmail.com
>>>>>>> wrote:
>>>>>>> 
>>>>>>>> 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 key....certs won't work, unless
>>>>>>>> you
>>>>>>>> write
>>>>>>>> a CertAuthority to build multiple certs with the same keys.
>>>>>>>> Better
>>>>>>>> seems
>>>>>>>> to not use certs and wrap the encryption specification with an ACL
>>>>>>>> capabilities for each group of access.
>>>>>>>> 
>>>>>>>> 
>>>>>>>> On Jun 6, 2014, at 11:43 AM, Rob Withers wrote:
>>>>>>>> 
>>>>>>>> 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/WRITE/DELETE/LIST/SCAN).  Erights.org has the
>>>>>>>>> quitenscential (??)
>>>>>>>>> object capabilities model and capnproto is impleemting this for
>>>>>>>>> C+
>>>>>>>>> +.  I
>>>>>>>>> have a java implementation at http://github.com/pauwau/pauwau but
>>>>>>>>> the
>>>>>>>>> master is broken.  0.2 works, basically.  B asically a TLS
>>>>>>>>> connection
>>>>>>>>> with
>>>>>>>>> no certificate server, it is peer to peer.  It has some advanced
>>>>>>>>> features,
>>>>>>>>> but the lining of capabilities with authorization so that you can
>>>>>>>>> only
>>>>>>>>> invoke correct services is extended to the secure user.
>>>>>>>>> 
>>>>>>>>> Regarding non-repudiation, on disk, why not prepend a CRC?
>>>>>>>>> 
>>>>>>>>> Regarding on-disk encryption, multiple users/groups may need to
>>>>>>>>> access,
>>>>>>>>> with different capabilities.  Sounds like zookeeper needs to
>>>>>>>>> store a
>>>>>>>>> cert
>>>>>>>>> for each class of access so that a group member can access the
>>>>>>>>> decrypted
>>>>>>>>> data from disk.  Use cert-based async decryption.  The only
>>>>>>>>> isue is
>>>>>>>>> storing
>>>>>>>>> the private key in zookeeper.  Perhaps some hash magic could be
>>>>>>>>> used.
>>>>>>>>> 
>>>>>>>>> Thanks for kafka,
>>>>>>>>> Rob
>>>>>>>>> 
>>>>>>>>> On Jun 5, 2014, at 3:01 PM, Jay Kreps wrote:
>>>>>>>>> 
>>>>>>>>> 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 encryption scheme? Is this
>>>>>>>>>> actually
>>>>>>>>>> needed?
>>>>>>>>>> Isn't a on-the-wire encryption when combined with mutual
>>>>>>>>>> authentication
>>>>>>>>>> and
>>>>>>>>>> permissions sufficient for most uses?
>>>>>>>>>> 
>>>>>>>>>> On-disk encryption seems unnecessary because if an attacker can
>>>>>>>>>> get
>>>>>>>>>> root
>>>>>>>>>> on
>>>>>>>>>> the kafka boxes it can potentially modify Kafka to do anything
>>>>>>>>>> he or
>>>>>>>>>> she
>>>>>>>>>> wants with data. So this seems to break any security model.
>>>>>>>>>> 
>>>>>>>>>> I understand the problem of a large organization not really
>>>>>>>>>> having a
>>>>>>>>>> trusted network and wanting to secure data transfer and limit
>>>>>>>>>> and
>>>>>>>>>> audit
>>>>>>>>>> data access. The uses for these other things I don't totally
>>>>>>>>>> understand.
>>>>>>>>>> 
>>>>>>>>>> Also it would be worth understanding the state of other
>>>>>>>>>> messaging and
>>>>>>>>>> storage systems (Hadoop, dbs, etc). What features do they
>>>>>>>>>> support. I
>>>>>>>>>> think
>>>>>>>>>> there is a sense in which you don't have to run faster than the
>>>>>>>>>> bear,
>>>>>>>>>> but
>>>>>>>>>> only faster then your friends. :-)
>>>>>>>>>> 
>>>>>>>>>> -Jay
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> On Wed, Jun 4, 2014 at 5:57 PM, Joe Stein <jo...@stealth.ly>
>>>>>>>>>> wrote:
>>>>>>>>>> 
>>>>>>>>>> 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 <ja...@gmail.com>
>>>>>>>>>>> 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
>>>>>>>>>>>> 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 most the LinkedIn folks are working on the consumer
>>>>>>>>>>>> right now
>>>>>>>>>>>> I
>>>>>>>>>>> think
>>>>>>>>>>> 
>>>>>>>>>>>> this would be a great project for any other interested
>>>>>>>>>>>> people to
>>>>>>>>>>>> take
>>>>>>>>>>>> on.
>>>>>>>>>>>> There are some challenges in doing these things distributed
>>>>>>>>>>>> but it
>>>>>>>>>>>> can
>>>>>>>>>>> also
>>>>>>>>>>> 
>>>>>>>>>>>> be a lot of fun.
>>>>>>>>>>>> 
>>>>>>>>>>>> I think a good first step would be to get a written plan we
>>>>>>>>>>>> can all
>>>>>>>>>>>> agree
>>>>>>>>>>>> on for how things should work. Then we can break things down
>>>>>>>>>>>> into
>>>>>>>>>>>> chunks
>>>>>>>>>>>> that can be done independently while still aiming at a good
>>>>>>>>>>>> end
>>>>>>>>>>>> state.
>>>>>>>>>>>> 
>>>>>>>>>>>> I had tried to write up some notes that summarized at least
>>>>>>>>>>>> the
>>>>>>>>>>>> thoughts
>>>>>>>>>>> I
>>>>>>>>>>> 
>>>>>>>>>>>> had had on security:
>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/Security
>>>>>>>>>>>> 
>>>>>>>>>>>> What do you think of that?
>>>>>>>>>>>> 
>>>>>>>>>>>> One assumption I had (which may be incorrect) is that although
>>>>>>>>>>>> we
>>>>>>>>>>>> want
>>>>>>>>>>> all
>>>>>>>>>>> 
>>>>>>>>>>>> the things in your list, the two most pressing would be
>>>>>>>>>>>> authentication
>>>>>>>>>>> and
>>>>>>>>>>> 
>>>>>>>>>>>> authorization, and that was all that write up covered. You
>>>>>>>>>>>> have more
>>>>>>>>>>>> experience in this domain, so I wonder how you would
>>>>>>>>>>>> prioritize?
>>>>>>>>>>>> 
>>>>>>>>>>>> Those notes are really sketchy, so I think the first goal I
>>>>>>>>>>>> would
>>>>>>>>>>>> have
>>>>>>>>>>>> would be to get to a real spec we can all agree on and
>>>>>>>>>>>> discuss. A
>>>>>>>>>>>> lot
>>>>>>>>>>>> of
>>>>>>>>>>>> the security stuff has a high human interaction element and
>>>>>>>>>>>> needs to
>>>>>>>>>>>> work
>>>>>>>>>>>> in pretty different domains and different companies so getting
>>>>>>>>>>>> this
>>>>>>>>>>>> kind
>>>>>>>>>>> of
>>>>>>>>>>> 
>>>>>>>>>>>> review is important.
>>>>>>>>>>>> 
>>>>>>>>>>>> -Jay
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> On Tue, Jun 3, 2014 at 12:57 PM, Joe Stein
>>>>>>>>>>>> <jo...@stealth.ly>
>>>>>>>>>>>> wrote:
>>>>>>>>>>>> 
>>>>>>>>>>>> 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.  This comes from what folks are doing "in the
>>>>>>>>>>>>> wild"
>>>>>>>>>>>>> to
>>>>>>>>>>>>> workaround and implement security with Kafka as it is today
>>>>>>>>>>>>> and
>>>>>>>>>>>>> also
>>>>>>>>>>>> what I
>>>>>>>>>>>> 
>>>>>>>>>>>>> have discovered from organizations about their blockers. It
>>>>>>>>>>>>> also
>>>>>>>>>>>>> picks
>>>>>>>>>>>> up
>>>>>>>>>>> 
>>>>>>>>>>>> from the wiki (which I should have time to update later in the
>>>>>>>>>>>> week
>>>>>>>>>>>> based
>>>>>>>>>>> 
>>>>>>>>>>>> on the below and feedback from the thread).
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 1) Transport Layer Security (i.e. SSL)
>>>>>>>>>>>>> 
>>>>>>>>>>>>> This also includes client authentication in addition to in-
>>>>>>>>>>>>> transit
>>>>>>>>>>>> security
>>>>>>>>>>>> 
>>>>>>>>>>>>> layer.  This work has been picked up here
>>>>>>>>>>>>> https://issues.apache.org/jira/browse/KAFKA-1477 and do
>>>>>>>>>>>>> appreciate
>>>>>>>>>>>>> any
>>>>>>>>>>>>> thoughts, comments, feedback, tomatoes, whatever for this
>>>>>>>>>>>>> patch.
>>>>>>>>>>>>> It
>>>>>>>>>>>> is a
>>>>>>>>>>> 
>>>>>>>>>>>> pickup from the fork of the work first done here
>>>>>>>>>>>>> https://github.com/relango/kafka/tree/kafka_security.
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 2) Data encryption at rest.
>>>>>>>>>>>>> 
>>>>>>>>>>>>> This is very important and something that can be facilitated
>>>>>>>>>>>>> within
>>>>>>>>>>>>> the
>>>>>>>>>>>>> wire protocol. It requires an additional map data structure
>>>>>>>>>>>>> for the
>>>>>>>>>>>>> "encrypted [data encryption key]". With this map (either in
>>>>>>>>>>>>> your
>>>>>>>>>>>>> object
>>>>>>>>>>>> or
>>>>>>>>>>>> 
>>>>>>>>>>>>> in the wire protocol) you can store the dynamically generated
>>>>>>>>>>>>> symmetric
>>>>>>>>>>>> key
>>>>>>>>>>>> 
>>>>>>>>>>>>> (for each message) and then encrypt the data using that
>>>>>>>>>>>>> dynamically
>>>>>>>>>>>>> generated key.  You then encrypt the encryption key using
>>>>>>>>>>>>> each
>>>>>>>>>>>>> public
>>>>>>>>>>>> key
>>>>>>>>>>> 
>>>>>>>>>>>> for whom is expected to be able to decrypt the encryption
>>>>>>>>>>>> key to
>>>>>>>>>>>> then
>>>>>>>>>>>>> decrypt the message.  For each public key encrypted symmetric
>>>>>>>>>>>>> key
>>>>>>>>>>>> (which
>>>>>>>>>>> 
>>>>>>>>>>>> is
>>>>>>>>>>>> 
>>>>>>>>>>>>> now the "encrypted [data encryption key]" along with which
>>>>>>>>>>>>> public
>>>>>>>>>>>>> key
>>>>>>>>>>>> it
>>>>>>>>>>> 
>>>>>>>>>>>> was encrypted with for (so a map of [publicKey] =
>>>>>>>>>>>>> encryptedDataEncryptionKey) as a chain.   Other patterns
>>>>>>>>>>>>> can be
>>>>>>>>>>>> implemented
>>>>>>>>>>>> 
>>>>>>>>>>>>> but this is a pretty standard digital enveloping [0] pattern
>>>>>>>>>>>>> with
>>>>>>>>>>>>> only
>>>>>>>>>>>> 1
>>>>>>>>>>> 
>>>>>>>>>>>> field added. Other patterns should be able to use that field
>>>>>>>>>>>> to-do
>>>>>>>>>>>> their
>>>>>>>>>>> 
>>>>>>>>>>>> implementation too.
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 3) Non-repudiation and long term non-repudiation.
>>>>>>>>>>>>> 
>>>>>>>>>>>>> Non-repudiation is proving data hasn't changed.  This is
>>>>>>>>>>>>> often (if
>>>>>>>>>>>>> not
>>>>>>>>>>>>> always) done with x509 public certificates (chained to a
>>>>>>>>>>>>> certificate
>>>>>>>>>>>>> authority).
>>>>>>>>>>>>> 
>>>>>>>>>>>>> Long term non-repudiation is what happens when the
>>>>>>>>>>>>> certificates of
>>>>>>>>>>>>> the
>>>>>>>>>>>>> certificate authority are expired (or revoked) and everything
>>>>>>>>>>>>> ever
>>>>>>>>>>>> signed
>>>>>>>>>>> 
>>>>>>>>>>>> (ever) with that certificate's public key then becomes "no
>>>>>>>>>>>> longer
>>>>>>>>>>>> provable
>>>>>>>>>>>> 
>>>>>>>>>>>>> as ever being authentic".  That is where RFC3126 [1] and
>>>>>>>>>>>>> RFC3161
>>>>>>>>>>>>> [2]
>>>>>>>>>>>> come
>>>>>>>>>>> 
>>>>>>>>>>>> in (or worm drives [hardware], etc).
>>>>>>>>>>>>> 
>>>>>>>>>>>>> For either (or both) of these it is an operation of the
>>>>>>>>>>>>> encryptor
>>>>>>>>>>>>> to
>>>>>>>>>>>>> sign/hash the data (with or without third party trusted
>>>>>>>>>>>>> timestap of
>>>>>>>>>>>>> the
>>>>>>>>>>>>> signing event) and encrypt that with their own private key
>>>>>>>>>>>>> and
>>>>>>>>>>>> distribute
>>>>>>>>>>> 
>>>>>>>>>>>> the results (before and after encrypting if required) along
>>>>>>>>>>>> with
>>>>>>>>>>>> their
>>>>>>>>>>>>> public key. This structure is a bit more complex but
>>>>>>>>>>>>> feasible, it
>>>>>>>>>>>>> is a
>>>>>>>>>>>> map
>>>>>>>>>>>> 
>>>>>>>>>>>>> of digital signature formats and the chain of dig sig
>>>>>>>>>>>>> attestations.
>>>>>>>>>>>> The
>>>>>>>>>>> 
>>>>>>>>>>>> map's key being the method (i.e. CRC32, PKCS7 [3], XmlDigSig
>>>>>>>>>>>> [4])
>>>>>>>>>>>> and
>>>>>>>>>>>> then
>>>>>>>>>>>> 
>>>>>>>>>>>>> a list of map where that key is "purpose" of signature (what
>>>>>>>>>>>>> your
>>>>>>>>>>>> attesting
>>>>>>>>>>>> 
>>>>>>>>>>>>> too).  As a sibling field to the list another field for "the
>>>>>>>>>>>>> attester"
>>>>>>>>>>>> as
>>>>>>>>>>> 
>>>>>>>>>>>> bytes (e.g. their PKCS12 [5] for the map of PKCS7 signatures).
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 4) Authorization
>>>>>>>>>>>>> 
>>>>>>>>>>>>> We should have a policy of "404" for data, topics, partitions
>>>>>>>>>>>>> (etc) if
>>>>>>>>>>>>> authenticated connections do not have access.  In "secure
>>>>>>>>>>>>> mode" any
>>>>>>>>>>>>> non
>>>>>>>>>>>>> authenticated connections should get a "404" type message on
>>>>>>>>>>>> everything.
>>>>>>>>>>> 
>>>>>>>>>>>> Knowing "something is there" is a security risk in many uses
>>>>>>>>>>>> cases.
>>>>>>>>>>>> So
>>>>>>>>>>>> if
>>>>>>>>>>>> 
>>>>>>>>>>>>> you don't have access you don't even see it.  Baking "that"
>>>>>>>>>>>>> into
>>>>>>>>>>>>> Kafka
>>>>>>>>>>>>> along with some interface for entitlement (access management)
>>>>>>>>>>>>> systems
>>>>>>>>>>>>> (pretty standard) is all that I think needs to be done to the
>>>>>>>>>>>>> core
>>>>>>>>>>>> project.
>>>>>>>>>>>> 
>>>>>>>>>>>>> I want to tackle item later in the year after summer after
>>>>>>>>>>>>> the
>>>>>>>>>>>>> other
>>>>>>>>>>>> three
>>>>>>>>>>>> 
>>>>>>>>>>>>> are complete.
>>>>>>>>>>>>> 
>>>>>>>>>>>>> I look forward to thoughts on this and anyone else interested
>>>>>>>>>>>>> in
>>>>>>>>>>>> working
>>>>>>>>>>> 
>>>>>>>>>>>> with us on these items.
>>>>>>>>>>>>> 
>>>>>>>>>>>>> [0]
>>>>>>>>>>>> http://www.emc.com/emc-plus/rsa-labs/standards-
>>>>>>>>>>> initiatives/what-is-a-digital-envelope.htm
>>>>>>>>>>> 
>>>>>>>>>>>> [1] http://tools.ietf.org/html/rfc3126
>>>>>>>>>>>>> [2] http://tools.ietf.org/html/rfc3161
>>>>>>>>>>>>> [3]
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>http://www.emc.com/emc-plus/rsa-labs/standards-initiatives/pkcs
>>>>>>>>>>>>-7
>>>>>>>>>>>> -
>>>>>>>>>>> cryptographic-message-syntax-standar.htm
>>>>>>>>>>> 
>>>>>>>>>>>> [4] http://en.wikipedia.org/wiki/XML_Signature
>>>>>>>>>>>>> [5] http://en.wikipedia.org/wiki/PKCS_12
>>>>>>>>>>>>> 
>>>>>>>>>>>>> /*******************************************
>>>>>>>>>>>>> Joe Stein
>>>>>>>>>>>>> Founder, Principal Consultant
>>>>>>>>>>>>> Big Data Open Source Security LLC
>>>>>>>>>>>>> http://www.stealth.ly
>>>>>>>>>>>>> Twitter: @allthingshadoop
>>>>>>>>>>>>> <http://www.twitter.com/allthingshadoop
>>>>>>>>>>>>> ********************************************/
>> 


Re: [DISCUSS] Kafka Security Specific Features

Posted by Robert Withers <ro...@gmail.com>.
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 before the complete recipient list is known.

This seems to mean that the private key (cert or OTP) must be stored and interacted with.  My feeling is that key metadata are in a system encrypted Hbase store (session key store), for low latency reads, rather than a topic requiring scanning.  Store the private keys and then give client access (producers/consumers) with the hash of the OTP.  A new consumer comes along, create a new cert encoding the OTP hash.

On write, use the producer cert to send a topic hash with the msg which would allow the broker to reuse or generate an OTP, stored in the session key store.

On read (consumer), if we have a previously run reader, use the encrypted hash.  If new, create consumer cert and encrypt the hash for that session.

The reader/writer will pass a cert encrypted session hash.  The trick seems to be converting hash to PK to encrypt/decrypt.  Given Kafka resource distribution, we need system encryption for metadata and cert-based key exchange.  This seems to mean triple encryption:
1) client to/from broker
2) system key/hash  mgmt/translation
3) at-rest encryption

Thanks,
Rob

> On Jun 9, 2014, at 7:57 AM, Todd Palino <tp...@linkedin.com.INVALID> wrote:
> 
> 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 the encrypted copies of the session key are
> then included with the encrypted message. This way, you avoid having to
> encrypt the message multiple times (this assumes, of course, that the
> message itself is larger than the key).
> 
> In our case, we have some options available to us. We could do that, and
> put all the encrypted keys in the message metadata. Or we could treat it
> more like a session and have the encrypted session keys in a special topic
> (e.g. __session_keys), much like offsets are now. When the producer starts
> up, they create a session key and encrypt it for each consumer with the
> current consumer key. The producer publishes the bundle of encrypted keys
> into __session_keys as a single message. The producer then publishes
> messages to the normal topic encrypted with the session key. The metadata
> for each of those messages would contain something the offset into
> __session_keys to identify the bundle. This has the added benefit of not
> increasing the per-message data size too much.
> 
> Whenever a consumer key is invalidated, or however often the session key
> should be rotated, the producer would publish a new bundle. This maintains
> a history of session keys that can be used to decrypt any messages, so the
> retention on __session_keys must be at least as long as any topic which
> may potentially contain encrypted data. Past that point, it’s up to the
> consumer what they want to do with the data. A consumer like Hadoop might
> re-encrypt it for local storage, or store it in plaintext (depending on
> the security and requirements of that system).
> 
> -Todd
> 
>> On 6/8/14, 2:33 PM, "Rob Withers" <ro...@gmail.com> wrote:
>> 
>> 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 deeper for me, please?
>> The model I envision is internal OTP, with access to decryption
>> accessed by cert.  A double layer of security, with the internal at-
>> rest encryption being an unchanging OTP with ACL access to it as the
>> upper layer.  Are you saying it is possible to re-encrypt with new
>> keys or that there is a chain of keys over time?
>> 
>> Thanks,
>> Rob
>> 
>>> On Jun 8, 2014, at 3:06 PM, Todd Palino wrote:
>>> 
>>> 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, and other PII standards). Yes, it does add external overhead
>>> that
>>> must be managed, but it’s just the nature of the beast. We can’t
>>> solve all
>>> of the external infrastructure needed for this, but we can make it
>>> easier
>>> to use for consumers and producers by adding metadata.
>>> 
>>> There’s no need for unchanging encryption, and that’s specifically
>>> why I
>>> want to see a message envelope that will help consumers determine the
>>> encryption uses for a particular message.  You can definitely still
>>> expire
>>> keys, you just have to keep the expired keys around as long as the
>>> encrypted data stays around, and your endpoints need to know when
>>> they are
>>> decrypting data with an expired key (you might want to throw up a
>>> warning,
>>> or do something else to let the users know that it’s happening). And
>>> as
>>> someone else mentioned, there are solutions for encrypting data for
>>> multiple consumers. You can encrypt the data with an OTP, and then
>>> multiply encrypt the OTP once for each consumer and store those
>>> encrypted
>>> strings in the envelope.
>>> 
>>> -Todd
>>> 
>>>> On 6/7/14, 12:25 PM, "Rob Withers" <ro...@gmail.com> wrote:
>>>> 
>>>> 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 encryption for the data and the clients (producers/
>>>> consumers) will need to be able to decode frozen data.  If certs are
>>>> used they cannot expire.  Also, different clients would need to use
>>>> the same cert.
>>>> 
>>>> So, you statement that it should ABSOLUTELY not include internal
>>>> encryption rings seems misplaced.  There are some customers of kafka
>>>> that would opt to encrypt the on-disk data and key management is a
>>>> significant issue.  This is best handled internally, with key
>>>> management stored in either ZK or in a topic.  Truly, perhaps
>>>> annealing Hadoop/HBASE as a metadata store seems applicable.
>>>> 
>>>> Thanks, another 2 cents,
>>>> Rob
>>>> 
>>>>> On Jun 6, 2014, at 12:15 PM, Todd Palino wrote:
>>>>> 
>>>>> 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 server then has access to all the keys. You could say
>>>>> that
>>>>> this eliminates the need for TLS, but TLS is pretty low-hanging
>>>>> fruit, and
>>>>> there¹s definitely a need for encryption of the traffic across the
>>>>> network
>>>>> even if you don¹t need at-rest encryption as well.
>>>>> 
>>>>> And as you mentioned, something needs to be done about key
>>>>> management.
>>>>> Storing information with the message about which key(s) was used is
>>>>> a good
>>>>> idea, because it allows you to know when a producer has switched
>>>>> keys.
>>>>> There are definitely some alternative solutions to that as well. But
>>>>> storing the keys in the broker, Zookeeper, or other systems like
>>>>> that are
>>>>> not. There needs to be a system used where the keys are only
>>>>> available to
>>>>> the producers and consumers that need them, and they only get access
>>>>> to
>>>>> the appropriate part of the key pair.  Even as the guy running Kafka
>>>>> and
>>>>> Zookeeper, I should not have access to the keys being used, and if
>>>>> data is
>>>>> encrypted I should not be able to see the cleartext.
>>>>> 
>>>>> And even if we decide not to put anything about at-rest encryption
>>>>> in the
>>>>> consumer/producer clients directly, and leave it for an exercise
>>>>> above
>>>>> that level (you have to pass the ciphertext as the message to the
>>>>> client),
>>>>> I still think there is a good case for implementing a message
>>>>> envelope
>>>>> that can store the information about which key was used, and other
>>>>> pertinent metadata, and have the ability for special applications
>>>>> like
>>>>> mirror maker to be able to preserve it across clusters. This still
>>>>> helps
>>>>> to enable the use of encryption and other features (like auditing)
>>>>> even if
>>>>> we decide it¹s too large a scope to fully implement.
>>>>> 
>>>>> -Todd
>>>>> 
>>>>> On 6/6/14, 10:51 AM, "Pradeep Gollakota" <pr...@gmail.com>
>>>>> 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 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 better performance.
>>>>>> 
>>>>>> 
>>>>>> On Fri, Jun 6, 2014 at 10:48 AM, Rob Withers
>>>>>> <robert.w.withers@gmail.com
>>>>>> wrote:
>>>>>> 
>>>>>>> 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 key....certs won't work, unless
>>>>>>> you
>>>>>>> write
>>>>>>> a CertAuthority to build multiple certs with the same keys.
>>>>>>> Better
>>>>>>> seems
>>>>>>> to not use certs and wrap the encryption specification with an ACL
>>>>>>> capabilities for each group of access.
>>>>>>> 
>>>>>>> 
>>>>>>> On Jun 6, 2014, at 11:43 AM, Rob Withers wrote:
>>>>>>> 
>>>>>>> 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/WRITE/DELETE/LIST/SCAN).  Erights.org has the
>>>>>>>> quitenscential (??)
>>>>>>>> object capabilities model and capnproto is impleemting this for
>>>>>>>> C+
>>>>>>>> +.  I
>>>>>>>> have a java implementation at http://github.com/pauwau/pauwau but
>>>>>>>> the
>>>>>>>> master is broken.  0.2 works, basically.  B asically a TLS
>>>>>>>> connection
>>>>>>>> with
>>>>>>>> no certificate server, it is peer to peer.  It has some advanced
>>>>>>>> features,
>>>>>>>> but the lining of capabilities with authorization so that you can
>>>>>>>> only
>>>>>>>> invoke correct services is extended to the secure user.
>>>>>>>> 
>>>>>>>> Regarding non-repudiation, on disk, why not prepend a CRC?
>>>>>>>> 
>>>>>>>> Regarding on-disk encryption, multiple users/groups may need to
>>>>>>>> access,
>>>>>>>> with different capabilities.  Sounds like zookeeper needs to
>>>>>>>> store a
>>>>>>>> cert
>>>>>>>> for each class of access so that a group member can access the
>>>>>>>> decrypted
>>>>>>>> data from disk.  Use cert-based async decryption.  The only
>>>>>>>> isue is
>>>>>>>> storing
>>>>>>>> the private key in zookeeper.  Perhaps some hash magic could be
>>>>>>>> used.
>>>>>>>> 
>>>>>>>> Thanks for kafka,
>>>>>>>> Rob
>>>>>>>> 
>>>>>>>> On Jun 5, 2014, at 3:01 PM, Jay Kreps wrote:
>>>>>>>> 
>>>>>>>> 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 encryption scheme? Is this
>>>>>>>>> actually
>>>>>>>>> needed?
>>>>>>>>> Isn't a on-the-wire encryption when combined with mutual
>>>>>>>>> authentication
>>>>>>>>> and
>>>>>>>>> permissions sufficient for most uses?
>>>>>>>>> 
>>>>>>>>> On-disk encryption seems unnecessary because if an attacker can
>>>>>>>>> get
>>>>>>>>> root
>>>>>>>>> on
>>>>>>>>> the kafka boxes it can potentially modify Kafka to do anything
>>>>>>>>> he or
>>>>>>>>> she
>>>>>>>>> wants with data. So this seems to break any security model.
>>>>>>>>> 
>>>>>>>>> I understand the problem of a large organization not really
>>>>>>>>> having a
>>>>>>>>> trusted network and wanting to secure data transfer and limit
>>>>>>>>> and
>>>>>>>>> audit
>>>>>>>>> data access. The uses for these other things I don't totally
>>>>>>>>> understand.
>>>>>>>>> 
>>>>>>>>> Also it would be worth understanding the state of other
>>>>>>>>> messaging and
>>>>>>>>> storage systems (Hadoop, dbs, etc). What features do they
>>>>>>>>> support. I
>>>>>>>>> think
>>>>>>>>> there is a sense in which you don't have to run faster than the
>>>>>>>>> bear,
>>>>>>>>> but
>>>>>>>>> only faster then your friends. :-)
>>>>>>>>> 
>>>>>>>>> -Jay
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> On Wed, Jun 4, 2014 at 5:57 PM, Joe Stein <jo...@stealth.ly>
>>>>>>>>> wrote:
>>>>>>>>> 
>>>>>>>>> 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 <ja...@gmail.com>
>>>>>>>>>> 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
>>>>>>>>>>> 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 most the LinkedIn folks are working on the consumer
>>>>>>>>>>> right now
>>>>>>>>>>> I
>>>>>>>>>> think
>>>>>>>>>> 
>>>>>>>>>>> this would be a great project for any other interested
>>>>>>>>>>> people to
>>>>>>>>>>> take
>>>>>>>>>>> on.
>>>>>>>>>>> There are some challenges in doing these things distributed
>>>>>>>>>>> but it
>>>>>>>>>>> can
>>>>>>>>>> also
>>>>>>>>>> 
>>>>>>>>>>> be a lot of fun.
>>>>>>>>>>> 
>>>>>>>>>>> I think a good first step would be to get a written plan we
>>>>>>>>>>> can all
>>>>>>>>>>> agree
>>>>>>>>>>> on for how things should work. Then we can break things down
>>>>>>>>>>> into
>>>>>>>>>>> chunks
>>>>>>>>>>> that can be done independently while still aiming at a good
>>>>>>>>>>> end
>>>>>>>>>>> state.
>>>>>>>>>>> 
>>>>>>>>>>> I had tried to write up some notes that summarized at least
>>>>>>>>>>> the
>>>>>>>>>>> thoughts
>>>>>>>>>> I
>>>>>>>>>> 
>>>>>>>>>>> had had on security:
>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/Security
>>>>>>>>>>> 
>>>>>>>>>>> What do you think of that?
>>>>>>>>>>> 
>>>>>>>>>>> One assumption I had (which may be incorrect) is that although
>>>>>>>>>>> we
>>>>>>>>>>> want
>>>>>>>>>> all
>>>>>>>>>> 
>>>>>>>>>>> the things in your list, the two most pressing would be
>>>>>>>>>>> authentication
>>>>>>>>>> and
>>>>>>>>>> 
>>>>>>>>>>> authorization, and that was all that write up covered. You
>>>>>>>>>>> have more
>>>>>>>>>>> experience in this domain, so I wonder how you would
>>>>>>>>>>> prioritize?
>>>>>>>>>>> 
>>>>>>>>>>> Those notes are really sketchy, so I think the first goal I
>>>>>>>>>>> would
>>>>>>>>>>> have
>>>>>>>>>>> would be to get to a real spec we can all agree on and
>>>>>>>>>>> discuss. A
>>>>>>>>>>> lot
>>>>>>>>>>> of
>>>>>>>>>>> the security stuff has a high human interaction element and
>>>>>>>>>>> needs to
>>>>>>>>>>> work
>>>>>>>>>>> in pretty different domains and different companies so getting
>>>>>>>>>>> this
>>>>>>>>>>> kind
>>>>>>>>>> of
>>>>>>>>>> 
>>>>>>>>>>> review is important.
>>>>>>>>>>> 
>>>>>>>>>>> -Jay
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> On Tue, Jun 3, 2014 at 12:57 PM, Joe Stein
>>>>>>>>>>> <jo...@stealth.ly>
>>>>>>>>>>> wrote:
>>>>>>>>>>> 
>>>>>>>>>>> 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.  This comes from what folks are doing "in the
>>>>>>>>>>>> wild"
>>>>>>>>>>>> to
>>>>>>>>>>>> workaround and implement security with Kafka as it is today
>>>>>>>>>>>> and
>>>>>>>>>>>> also
>>>>>>>>>>> what I
>>>>>>>>>>> 
>>>>>>>>>>>> have discovered from organizations about their blockers. It
>>>>>>>>>>>> also
>>>>>>>>>>>> picks
>>>>>>>>>>> up
>>>>>>>>>> 
>>>>>>>>>>> from the wiki (which I should have time to update later in the
>>>>>>>>>>> week
>>>>>>>>>>> based
>>>>>>>>>> 
>>>>>>>>>>> on the below and feedback from the thread).
>>>>>>>>>>>> 
>>>>>>>>>>>> 1) Transport Layer Security (i.e. SSL)
>>>>>>>>>>>> 
>>>>>>>>>>>> This also includes client authentication in addition to in-
>>>>>>>>>>>> transit
>>>>>>>>>>> security
>>>>>>>>>>> 
>>>>>>>>>>>> layer.  This work has been picked up here
>>>>>>>>>>>> https://issues.apache.org/jira/browse/KAFKA-1477 and do
>>>>>>>>>>>> appreciate
>>>>>>>>>>>> any
>>>>>>>>>>>> thoughts, comments, feedback, tomatoes, whatever for this
>>>>>>>>>>>> patch.
>>>>>>>>>>>> It
>>>>>>>>>>> is a
>>>>>>>>>> 
>>>>>>>>>>> pickup from the fork of the work first done here
>>>>>>>>>>>> https://github.com/relango/kafka/tree/kafka_security.
>>>>>>>>>>>> 
>>>>>>>>>>>> 2) Data encryption at rest.
>>>>>>>>>>>> 
>>>>>>>>>>>> This is very important and something that can be facilitated
>>>>>>>>>>>> within
>>>>>>>>>>>> the
>>>>>>>>>>>> wire protocol. It requires an additional map data structure
>>>>>>>>>>>> for the
>>>>>>>>>>>> "encrypted [data encryption key]". With this map (either in
>>>>>>>>>>>> your
>>>>>>>>>>>> object
>>>>>>>>>>> or
>>>>>>>>>>> 
>>>>>>>>>>>> in the wire protocol) you can store the dynamically generated
>>>>>>>>>>>> symmetric
>>>>>>>>>>> key
>>>>>>>>>>> 
>>>>>>>>>>>> (for each message) and then encrypt the data using that
>>>>>>>>>>>> dynamically
>>>>>>>>>>>> generated key.  You then encrypt the encryption key using
>>>>>>>>>>>> each
>>>>>>>>>>>> public
>>>>>>>>>>> key
>>>>>>>>>> 
>>>>>>>>>>> for whom is expected to be able to decrypt the encryption
>>>>>>>>>>> key to
>>>>>>>>>>> then
>>>>>>>>>>>> decrypt the message.  For each public key encrypted symmetric
>>>>>>>>>>>> key
>>>>>>>>>>> (which
>>>>>>>>>> 
>>>>>>>>>>> is
>>>>>>>>>>> 
>>>>>>>>>>>> now the "encrypted [data encryption key]" along with which
>>>>>>>>>>>> public
>>>>>>>>>>>> key
>>>>>>>>>>> it
>>>>>>>>>> 
>>>>>>>>>>> was encrypted with for (so a map of [publicKey] =
>>>>>>>>>>>> encryptedDataEncryptionKey) as a chain.   Other patterns
>>>>>>>>>>>> can be
>>>>>>>>>>> implemented
>>>>>>>>>>> 
>>>>>>>>>>>> but this is a pretty standard digital enveloping [0] pattern
>>>>>>>>>>>> with
>>>>>>>>>>>> only
>>>>>>>>>>> 1
>>>>>>>>>> 
>>>>>>>>>>> field added. Other patterns should be able to use that field
>>>>>>>>>>> to-do
>>>>>>>>>>> their
>>>>>>>>>> 
>>>>>>>>>>> implementation too.
>>>>>>>>>>>> 
>>>>>>>>>>>> 3) Non-repudiation and long term non-repudiation.
>>>>>>>>>>>> 
>>>>>>>>>>>> Non-repudiation is proving data hasn't changed.  This is
>>>>>>>>>>>> often (if
>>>>>>>>>>>> not
>>>>>>>>>>>> always) done with x509 public certificates (chained to a
>>>>>>>>>>>> certificate
>>>>>>>>>>>> authority).
>>>>>>>>>>>> 
>>>>>>>>>>>> Long term non-repudiation is what happens when the
>>>>>>>>>>>> certificates of
>>>>>>>>>>>> the
>>>>>>>>>>>> certificate authority are expired (or revoked) and everything
>>>>>>>>>>>> ever
>>>>>>>>>>> signed
>>>>>>>>>> 
>>>>>>>>>>> (ever) with that certificate's public key then becomes "no
>>>>>>>>>>> longer
>>>>>>>>>>> provable
>>>>>>>>>>> 
>>>>>>>>>>>> as ever being authentic".  That is where RFC3126 [1] and
>>>>>>>>>>>> RFC3161
>>>>>>>>>>>> [2]
>>>>>>>>>>> come
>>>>>>>>>> 
>>>>>>>>>>> in (or worm drives [hardware], etc).
>>>>>>>>>>>> 
>>>>>>>>>>>> For either (or both) of these it is an operation of the
>>>>>>>>>>>> encryptor
>>>>>>>>>>>> to
>>>>>>>>>>>> sign/hash the data (with or without third party trusted
>>>>>>>>>>>> timestap of
>>>>>>>>>>>> the
>>>>>>>>>>>> signing event) and encrypt that with their own private key
>>>>>>>>>>>> and
>>>>>>>>>>> distribute
>>>>>>>>>> 
>>>>>>>>>>> the results (before and after encrypting if required) along
>>>>>>>>>>> with
>>>>>>>>>>> their
>>>>>>>>>>>> public key. This structure is a bit more complex but
>>>>>>>>>>>> feasible, it
>>>>>>>>>>>> is a
>>>>>>>>>>> map
>>>>>>>>>>> 
>>>>>>>>>>>> of digital signature formats and the chain of dig sig
>>>>>>>>>>>> attestations.
>>>>>>>>>>> The
>>>>>>>>>> 
>>>>>>>>>>> map's key being the method (i.e. CRC32, PKCS7 [3], XmlDigSig
>>>>>>>>>>> [4])
>>>>>>>>>>> and
>>>>>>>>>>> then
>>>>>>>>>>> 
>>>>>>>>>>>> a list of map where that key is "purpose" of signature (what
>>>>>>>>>>>> your
>>>>>>>>>>> attesting
>>>>>>>>>>> 
>>>>>>>>>>>> too).  As a sibling field to the list another field for "the
>>>>>>>>>>>> attester"
>>>>>>>>>>> as
>>>>>>>>>> 
>>>>>>>>>>> bytes (e.g. their PKCS12 [5] for the map of PKCS7 signatures).
>>>>>>>>>>>> 
>>>>>>>>>>>> 4) Authorization
>>>>>>>>>>>> 
>>>>>>>>>>>> We should have a policy of "404" for data, topics, partitions
>>>>>>>>>>>> (etc) if
>>>>>>>>>>>> authenticated connections do not have access.  In "secure
>>>>>>>>>>>> mode" any
>>>>>>>>>>>> non
>>>>>>>>>>>> authenticated connections should get a "404" type message on
>>>>>>>>>>> everything.
>>>>>>>>>> 
>>>>>>>>>>> Knowing "something is there" is a security risk in many uses
>>>>>>>>>>> cases.
>>>>>>>>>>> So
>>>>>>>>>>> if
>>>>>>>>>>> 
>>>>>>>>>>>> you don't have access you don't even see it.  Baking "that"
>>>>>>>>>>>> into
>>>>>>>>>>>> Kafka
>>>>>>>>>>>> along with some interface for entitlement (access management)
>>>>>>>>>>>> systems
>>>>>>>>>>>> (pretty standard) is all that I think needs to be done to the
>>>>>>>>>>>> core
>>>>>>>>>>> project.
>>>>>>>>>>> 
>>>>>>>>>>>> I want to tackle item later in the year after summer after
>>>>>>>>>>>> the
>>>>>>>>>>>> other
>>>>>>>>>>> three
>>>>>>>>>>> 
>>>>>>>>>>>> are complete.
>>>>>>>>>>>> 
>>>>>>>>>>>> I look forward to thoughts on this and anyone else interested
>>>>>>>>>>>> in
>>>>>>>>>>> working
>>>>>>>>>> 
>>>>>>>>>>> with us on these items.
>>>>>>>>>>>> 
>>>>>>>>>>>> [0]
>>>>>>>>>>> http://www.emc.com/emc-plus/rsa-labs/standards-
>>>>>>>>>> initiatives/what-is-a-digital-envelope.htm
>>>>>>>>>> 
>>>>>>>>>>> [1] http://tools.ietf.org/html/rfc3126
>>>>>>>>>>>> [2] http://tools.ietf.org/html/rfc3161
>>>>>>>>>>>> [3]
>>>>>>>>>>> 
>>>>>>>>>>> http://www.emc.com/emc-plus/rsa-labs/standards-initiatives/pkcs-7
>>>>>>>>>>> -
>>>>>>>>>> cryptographic-message-syntax-standar.htm
>>>>>>>>>> 
>>>>>>>>>>> [4] http://en.wikipedia.org/wiki/XML_Signature
>>>>>>>>>>>> [5] http://en.wikipedia.org/wiki/PKCS_12
>>>>>>>>>>>> 
>>>>>>>>>>>> /*******************************************
>>>>>>>>>>>> Joe Stein
>>>>>>>>>>>> Founder, Principal Consultant
>>>>>>>>>>>> Big Data Open Source Security LLC
>>>>>>>>>>>> http://www.stealth.ly
>>>>>>>>>>>> Twitter: @allthingshadoop
>>>>>>>>>>>> <http://www.twitter.com/allthingshadoop
>>>>>>>>>>>> ********************************************/
> 

Re: [DISCUSS] Kafka Security Specific Features

Posted by Todd Palino <tp...@linkedin.com.INVALID>.
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 the encrypted copies of the session key are
then included with the encrypted message. This way, you avoid having to
encrypt the message multiple times (this assumes, of course, that the
message itself is larger than the key).

In our case, we have some options available to us. We could do that, and
put all the encrypted keys in the message metadata. Or we could treat it
more like a session and have the encrypted session keys in a special topic
(e.g. __session_keys), much like offsets are now. When the producer starts
up, they create a session key and encrypt it for each consumer with the
current consumer key. The producer publishes the bundle of encrypted keys
into __session_keys as a single message. The producer then publishes
messages to the normal topic encrypted with the session key. The metadata
for each of those messages would contain something the offset into
__session_keys to identify the bundle. This has the added benefit of not
increasing the per-message data size too much.

Whenever a consumer key is invalidated, or however often the session key
should be rotated, the producer would publish a new bundle. This maintains
a history of session keys that can be used to decrypt any messages, so the
retention on __session_keys must be at least as long as any topic which
may potentially contain encrypted data. Past that point, it’s up to the
consumer what they want to do with the data. A consumer like Hadoop might
re-encrypt it for local storage, or store it in plaintext (depending on
the security and requirements of that system).

-Todd

On 6/8/14, 2:33 PM, "Rob Withers" <ro...@gmail.com> wrote:

>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 deeper for me, please?
>The model I envision is internal OTP, with access to decryption
>accessed by cert.  A double layer of security, with the internal at-
>rest encryption being an unchanging OTP with ACL access to it as the
>upper layer.  Are you saying it is possible to re-encrypt with new
>keys or that there is a chain of keys over time?
>
>Thanks,
>Rob
>
>On Jun 8, 2014, at 3:06 PM, Todd Palino wrote:
>
>> 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, and other PII standards). Yes, it does add external overhead
>> that
>> must be managed, but it’s just the nature of the beast. We can’t
>> solve all
>> of the external infrastructure needed for this, but we can make it
>> easier
>> to use for consumers and producers by adding metadata.
>>
>> There’s no need for unchanging encryption, and that’s specifically
>> why I
>> want to see a message envelope that will help consumers determine the
>> encryption uses for a particular message.  You can definitely still
>> expire
>> keys, you just have to keep the expired keys around as long as the
>> encrypted data stays around, and your endpoints need to know when
>> they are
>> decrypting data with an expired key (you might want to throw up a
>> warning,
>> or do something else to let the users know that it’s happening). And
>> as
>> someone else mentioned, there are solutions for encrypting data for
>> multiple consumers. You can encrypt the data with an OTP, and then
>> multiply encrypt the OTP once for each consumer and store those
>> encrypted
>> strings in the envelope.
>>
>> -Todd
>>
>> On 6/7/14, 12:25 PM, "Rob Withers" <ro...@gmail.com> wrote:
>>
>>> 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 encryption for the data and the clients (producers/
>>> consumers) will need to be able to decode frozen data.  If certs are
>>> used they cannot expire.  Also, different clients would need to use
>>> the same cert.
>>>
>>> So, you statement that it should ABSOLUTELY not include internal
>>> encryption rings seems misplaced.  There are some customers of kafka
>>> that would opt to encrypt the on-disk data and key management is a
>>> significant issue.  This is best handled internally, with key
>>> management stored in either ZK or in a topic.  Truly, perhaps
>>> annealing Hadoop/HBASE as a metadata store seems applicable.
>>>
>>> Thanks, another 2 cents,
>>> Rob
>>>
>>> On Jun 6, 2014, at 12:15 PM, Todd Palino wrote:
>>>
>>>> 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 server then has access to all the keys. You could say
>>>> that
>>>> this eliminates the need for TLS, but TLS is pretty low-hanging
>>>> fruit, and
>>>> there¹s definitely a need for encryption of the traffic across the
>>>> network
>>>> even if you don¹t need at-rest encryption as well.
>>>>
>>>> And as you mentioned, something needs to be done about key
>>>> management.
>>>> Storing information with the message about which key(s) was used is
>>>> a good
>>>> idea, because it allows you to know when a producer has switched
>>>> keys.
>>>> There are definitely some alternative solutions to that as well. But
>>>> storing the keys in the broker, Zookeeper, or other systems like
>>>> that are
>>>> not. There needs to be a system used where the keys are only
>>>> available to
>>>> the producers and consumers that need them, and they only get access
>>>> to
>>>> the appropriate part of the key pair.  Even as the guy running Kafka
>>>> and
>>>> Zookeeper, I should not have access to the keys being used, and if
>>>> data is
>>>> encrypted I should not be able to see the cleartext.
>>>>
>>>> And even if we decide not to put anything about at-rest encryption
>>>> in the
>>>> consumer/producer clients directly, and leave it for an exercise
>>>> above
>>>> that level (you have to pass the ciphertext as the message to the
>>>> client),
>>>> I still think there is a good case for implementing a message
>>>> envelope
>>>> that can store the information about which key was used, and other
>>>> pertinent metadata, and have the ability for special applications
>>>> like
>>>> mirror maker to be able to preserve it across clusters. This still
>>>> helps
>>>> to enable the use of encryption and other features (like auditing)
>>>> even if
>>>> we decide it¹s too large a scope to fully implement.
>>>>
>>>> -Todd
>>>>
>>>> On 6/6/14, 10:51 AM, "Pradeep Gollakota" <pr...@gmail.com>
>>>> 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 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 better performance.
>>>>>
>>>>>
>>>>> On Fri, Jun 6, 2014 at 10:48 AM, Rob Withers
>>>>> <robert.w.withers@gmail.com
>>>>>>
>>>>> wrote:
>>>>>
>>>>>> 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 key....certs won't work, unless
>>>>>> you
>>>>>> write
>>>>>> a CertAuthority to build multiple certs with the same keys.
>>>>>> Better
>>>>>> seems
>>>>>> to not use certs and wrap the encryption specification with an ACL
>>>>>> capabilities for each group of access.
>>>>>>
>>>>>>
>>>>>> On Jun 6, 2014, at 11:43 AM, Rob Withers wrote:
>>>>>>
>>>>>> 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/WRITE/DELETE/LIST/SCAN).  Erights.org has the
>>>>>>> quitenscential (??)
>>>>>>> object capabilities model and capnproto is impleemting this for
>>>>>>> C+
>>>>>>> +.  I
>>>>>>> have a java implementation at http://github.com/pauwau/pauwau but
>>>>>>> the
>>>>>>> master is broken.  0.2 works, basically.  B asically a TLS
>>>>>>> connection
>>>>>>> with
>>>>>>> no certificate server, it is peer to peer.  It has some advanced
>>>>>>> features,
>>>>>>> but the lining of capabilities with authorization so that you can
>>>>>>> only
>>>>>>> invoke correct services is extended to the secure user.
>>>>>>>
>>>>>>> Regarding non-repudiation, on disk, why not prepend a CRC?
>>>>>>>
>>>>>>> Regarding on-disk encryption, multiple users/groups may need to
>>>>>>> access,
>>>>>>> with different capabilities.  Sounds like zookeeper needs to
>>>>>>> store a
>>>>>>> cert
>>>>>>> for each class of access so that a group member can access the
>>>>>>> decrypted
>>>>>>> data from disk.  Use cert-based async decryption.  The only
>>>>>>> isue is
>>>>>>> storing
>>>>>>> the private key in zookeeper.  Perhaps some hash magic could be
>>>>>>> used.
>>>>>>>
>>>>>>> Thanks for kafka,
>>>>>>> Rob
>>>>>>>
>>>>>>> On Jun 5, 2014, at 3:01 PM, Jay Kreps wrote:
>>>>>>>
>>>>>>> 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 encryption scheme? Is this
>>>>>>>> actually
>>>>>>>> needed?
>>>>>>>> Isn't a on-the-wire encryption when combined with mutual
>>>>>>>> authentication
>>>>>>>> and
>>>>>>>> permissions sufficient for most uses?
>>>>>>>>
>>>>>>>> On-disk encryption seems unnecessary because if an attacker can
>>>>>>>> get
>>>>>>>> root
>>>>>>>> on
>>>>>>>> the kafka boxes it can potentially modify Kafka to do anything
>>>>>>>> he or
>>>>>>>> she
>>>>>>>> wants with data. So this seems to break any security model.
>>>>>>>>
>>>>>>>> I understand the problem of a large organization not really
>>>>>>>> having a
>>>>>>>> trusted network and wanting to secure data transfer and limit
>>>>>>>> and
>>>>>>>> audit
>>>>>>>> data access. The uses for these other things I don't totally
>>>>>>>> understand.
>>>>>>>>
>>>>>>>> Also it would be worth understanding the state of other
>>>>>>>> messaging and
>>>>>>>> storage systems (Hadoop, dbs, etc). What features do they
>>>>>>>> support. I
>>>>>>>> think
>>>>>>>> there is a sense in which you don't have to run faster than the
>>>>>>>> bear,
>>>>>>>> but
>>>>>>>> only faster then your friends. :-)
>>>>>>>>
>>>>>>>> -Jay
>>>>>>>>
>>>>>>>>
>>>>>>>> On Wed, Jun 4, 2014 at 5:57 PM, Joe Stein <jo...@stealth.ly>
>>>>>>>> wrote:
>>>>>>>>
>>>>>>>> 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 <ja...@gmail.com>
>>>>>>>>> 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
>>>>>>>>>> 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 most the LinkedIn folks are working on the consumer
>>>>>>>>>> right now
>>>>>>>>>> I
>>>>>>>>>>
>>>>>>>>> think
>>>>>>>>>
>>>>>>>>>> this would be a great project for any other interested
>>>>>>>>>> people to
>>>>>>>>>> take
>>>>>>>>>> on.
>>>>>>>>>> There are some challenges in doing these things distributed
>>>>>>>>>> but it
>>>>>>>>>> can
>>>>>>>>>>
>>>>>>>>> also
>>>>>>>>>
>>>>>>>>>> be a lot of fun.
>>>>>>>>>>
>>>>>>>>>> I think a good first step would be to get a written plan we
>>>>>>>>>> can all
>>>>>>>>>> agree
>>>>>>>>>> on for how things should work. Then we can break things down
>>>>>>>>>> into
>>>>>>>>>> chunks
>>>>>>>>>> that can be done independently while still aiming at a good
>>>>>>>>>> end
>>>>>>>>>> state.
>>>>>>>>>>
>>>>>>>>>> I had tried to write up some notes that summarized at least
>>>>>>>>>> the
>>>>>>>>>> thoughts
>>>>>>>>>>
>>>>>>>>> I
>>>>>>>>>
>>>>>>>>>> had had on security:
>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/Security
>>>>>>>>>>
>>>>>>>>>> What do you think of that?
>>>>>>>>>>
>>>>>>>>>> One assumption I had (which may be incorrect) is that although
>>>>>>>>>> we
>>>>>>>>>> want
>>>>>>>>>>
>>>>>>>>> all
>>>>>>>>>
>>>>>>>>>> the things in your list, the two most pressing would be
>>>>>>>>>> authentication
>>>>>>>>>>
>>>>>>>>> and
>>>>>>>>>
>>>>>>>>>> authorization, and that was all that write up covered. You
>>>>>>>>>> have more
>>>>>>>>>> experience in this domain, so I wonder how you would
>>>>>>>>>> prioritize?
>>>>>>>>>>
>>>>>>>>>> Those notes are really sketchy, so I think the first goal I
>>>>>>>>>> would
>>>>>>>>>> have
>>>>>>>>>> would be to get to a real spec we can all agree on and
>>>>>>>>>> discuss. A
>>>>>>>>>> lot
>>>>>>>>>> of
>>>>>>>>>> the security stuff has a high human interaction element and
>>>>>>>>>> needs to
>>>>>>>>>> work
>>>>>>>>>> in pretty different domains and different companies so getting
>>>>>>>>>> this
>>>>>>>>>> kind
>>>>>>>>>>
>>>>>>>>> of
>>>>>>>>>
>>>>>>>>>> review is important.
>>>>>>>>>>
>>>>>>>>>> -Jay
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> On Tue, Jun 3, 2014 at 12:57 PM, Joe Stein
>>>>>>>>>> <jo...@stealth.ly>
>>>>>>>>>> wrote:
>>>>>>>>>>
>>>>>>>>>> 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.  This comes from what folks are doing "in the
>>>>>>>>>>> wild"
>>>>>>>>>>> to
>>>>>>>>>>> workaround and implement security with Kafka as it is today
>>>>>>>>>>> and
>>>>>>>>>>> also
>>>>>>>>>>>
>>>>>>>>>> what I
>>>>>>>>>>
>>>>>>>>>>> have discovered from organizations about their blockers. It
>>>>>>>>>>> also
>>>>>>>>>>> picks
>>>>>>>>>>>
>>>>>>>>>> up
>>>>>>>>>
>>>>>>>>>> from the wiki (which I should have time to update later in the
>>>>>>>>>> week
>>>>>>>>>>>
>>>>>>>>>> based
>>>>>>>>>
>>>>>>>>>> on the below and feedback from the thread).
>>>>>>>>>>>
>>>>>>>>>>> 1) Transport Layer Security (i.e. SSL)
>>>>>>>>>>>
>>>>>>>>>>> This also includes client authentication in addition to in-
>>>>>>>>>>> transit
>>>>>>>>>>>
>>>>>>>>>> security
>>>>>>>>>>
>>>>>>>>>>> layer.  This work has been picked up here
>>>>>>>>>>> https://issues.apache.org/jira/browse/KAFKA-1477 and do
>>>>>>>>>>> appreciate
>>>>>>>>>>> any
>>>>>>>>>>> thoughts, comments, feedback, tomatoes, whatever for this
>>>>>>>>>>> patch.
>>>>>>>>>>> It
>>>>>>>>>>>
>>>>>>>>>> is a
>>>>>>>>>
>>>>>>>>>> pickup from the fork of the work first done here
>>>>>>>>>>> https://github.com/relango/kafka/tree/kafka_security.
>>>>>>>>>>>
>>>>>>>>>>> 2) Data encryption at rest.
>>>>>>>>>>>
>>>>>>>>>>> This is very important and something that can be facilitated
>>>>>>>>>>> within
>>>>>>>>>>> the
>>>>>>>>>>> wire protocol. It requires an additional map data structure
>>>>>>>>>>> for the
>>>>>>>>>>> "encrypted [data encryption key]". With this map (either in
>>>>>>>>>>> your
>>>>>>>>>>> object
>>>>>>>>>>>
>>>>>>>>>> or
>>>>>>>>>>
>>>>>>>>>>> in the wire protocol) you can store the dynamically generated
>>>>>>>>>>> symmetric
>>>>>>>>>>>
>>>>>>>>>> key
>>>>>>>>>>
>>>>>>>>>>> (for each message) and then encrypt the data using that
>>>>>>>>>>> dynamically
>>>>>>>>>>> generated key.  You then encrypt the encryption key using
>>>>>>>>>>> each
>>>>>>>>>>> public
>>>>>>>>>>>
>>>>>>>>>> key
>>>>>>>>>
>>>>>>>>>> for whom is expected to be able to decrypt the encryption
>>>>>>>>>> key to
>>>>>>>>>> then
>>>>>>>>>>> decrypt the message.  For each public key encrypted symmetric
>>>>>>>>>>> key
>>>>>>>>>>>
>>>>>>>>>> (which
>>>>>>>>>
>>>>>>>>>> is
>>>>>>>>>>
>>>>>>>>>>> now the "encrypted [data encryption key]" along with which
>>>>>>>>>>> public
>>>>>>>>>>> key
>>>>>>>>>>>
>>>>>>>>>> it
>>>>>>>>>
>>>>>>>>>> was encrypted with for (so a map of [publicKey] =
>>>>>>>>>>> encryptedDataEncryptionKey) as a chain.   Other patterns
>>>>>>>>>>> can be
>>>>>>>>>>>
>>>>>>>>>> implemented
>>>>>>>>>>
>>>>>>>>>>> but this is a pretty standard digital enveloping [0] pattern
>>>>>>>>>>> with
>>>>>>>>>>> only
>>>>>>>>>>>
>>>>>>>>>> 1
>>>>>>>>>
>>>>>>>>>> field added. Other patterns should be able to use that field
>>>>>>>>>> to-do
>>>>>>>>>>>
>>>>>>>>>> their
>>>>>>>>>
>>>>>>>>>> implementation too.
>>>>>>>>>>>
>>>>>>>>>>> 3) Non-repudiation and long term non-repudiation.
>>>>>>>>>>>
>>>>>>>>>>> Non-repudiation is proving data hasn't changed.  This is
>>>>>>>>>>> often (if
>>>>>>>>>>> not
>>>>>>>>>>> always) done with x509 public certificates (chained to a
>>>>>>>>>>> certificate
>>>>>>>>>>> authority).
>>>>>>>>>>>
>>>>>>>>>>> Long term non-repudiation is what happens when the
>>>>>>>>>>> certificates of
>>>>>>>>>>> the
>>>>>>>>>>> certificate authority are expired (or revoked) and everything
>>>>>>>>>>> ever
>>>>>>>>>>>
>>>>>>>>>> signed
>>>>>>>>>
>>>>>>>>>> (ever) with that certificate's public key then becomes "no
>>>>>>>>>> longer
>>>>>>>>>>>
>>>>>>>>>> provable
>>>>>>>>>>
>>>>>>>>>>> as ever being authentic".  That is where RFC3126 [1] and
>>>>>>>>>>> RFC3161
>>>>>>>>>>> [2]
>>>>>>>>>>>
>>>>>>>>>> come
>>>>>>>>>
>>>>>>>>>> in (or worm drives [hardware], etc).
>>>>>>>>>>>
>>>>>>>>>>> For either (or both) of these it is an operation of the
>>>>>>>>>>> encryptor
>>>>>>>>>>> to
>>>>>>>>>>> sign/hash the data (with or without third party trusted
>>>>>>>>>>> timestap of
>>>>>>>>>>> the
>>>>>>>>>>> signing event) and encrypt that with their own private key
>>>>>>>>>>> and
>>>>>>>>>>>
>>>>>>>>>> distribute
>>>>>>>>>
>>>>>>>>>> the results (before and after encrypting if required) along
>>>>>>>>>> with
>>>>>>>>>> their
>>>>>>>>>>> public key. This structure is a bit more complex but
>>>>>>>>>>> feasible, it
>>>>>>>>>>> is a
>>>>>>>>>>>
>>>>>>>>>> map
>>>>>>>>>>
>>>>>>>>>>> of digital signature formats and the chain of dig sig
>>>>>>>>>>> attestations.
>>>>>>>>>>>
>>>>>>>>>> The
>>>>>>>>>
>>>>>>>>>> map's key being the method (i.e. CRC32, PKCS7 [3], XmlDigSig
>>>>>>>>>> [4])
>>>>>>>>>> and
>>>>>>>>>>>
>>>>>>>>>> then
>>>>>>>>>>
>>>>>>>>>>> a list of map where that key is "purpose" of signature (what
>>>>>>>>>>> your
>>>>>>>>>>>
>>>>>>>>>> attesting
>>>>>>>>>>
>>>>>>>>>>> too).  As a sibling field to the list another field for "the
>>>>>>>>>>> attester"
>>>>>>>>>>>
>>>>>>>>>> as
>>>>>>>>>
>>>>>>>>>> bytes (e.g. their PKCS12 [5] for the map of PKCS7 signatures).
>>>>>>>>>>>
>>>>>>>>>>> 4) Authorization
>>>>>>>>>>>
>>>>>>>>>>> We should have a policy of "404" for data, topics, partitions
>>>>>>>>>>> (etc) if
>>>>>>>>>>> authenticated connections do not have access.  In "secure
>>>>>>>>>>> mode" any
>>>>>>>>>>> non
>>>>>>>>>>> authenticated connections should get a "404" type message on
>>>>>>>>>>>
>>>>>>>>>> everything.
>>>>>>>>>
>>>>>>>>>> Knowing "something is there" is a security risk in many uses
>>>>>>>>>> cases.
>>>>>>>>>> So
>>>>>>>>>>>
>>>>>>>>>> if
>>>>>>>>>>
>>>>>>>>>>> you don't have access you don't even see it.  Baking "that"
>>>>>>>>>>> into
>>>>>>>>>>> Kafka
>>>>>>>>>>> along with some interface for entitlement (access management)
>>>>>>>>>>> systems
>>>>>>>>>>> (pretty standard) is all that I think needs to be done to the
>>>>>>>>>>> core
>>>>>>>>>>>
>>>>>>>>>> project.
>>>>>>>>>>
>>>>>>>>>>> I want to tackle item later in the year after summer after
>>>>>>>>>>> the
>>>>>>>>>>> other
>>>>>>>>>>>
>>>>>>>>>> three
>>>>>>>>>>
>>>>>>>>>>> are complete.
>>>>>>>>>>>
>>>>>>>>>>> I look forward to thoughts on this and anyone else interested
>>>>>>>>>>> in
>>>>>>>>>>>
>>>>>>>>>> working
>>>>>>>>>
>>>>>>>>>> with us on these items.
>>>>>>>>>>>
>>>>>>>>>>> [0]
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>> http://www.emc.com/emc-plus/rsa-labs/standards-
>>>>>>>>> initiatives/what-is-a-digital-envelope.htm
>>>>>>>>>
>>>>>>>>>> [1] http://tools.ietf.org/html/rfc3126
>>>>>>>>>>> [2] http://tools.ietf.org/html/rfc3161
>>>>>>>>>>> [3]
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>> 
>>>>>>>>>>http://www.emc.com/emc-plus/rsa-labs/standards-initiatives/pkcs-7
>>>>>>>>>>-
>>>>>>>>> cryptographic-message-syntax-standar.htm
>>>>>>>>>
>>>>>>>>>> [4] http://en.wikipedia.org/wiki/XML_Signature
>>>>>>>>>>> [5] http://en.wikipedia.org/wiki/PKCS_12
>>>>>>>>>>>
>>>>>>>>>>> /*******************************************
>>>>>>>>>>> Joe Stein
>>>>>>>>>>> Founder, Principal Consultant
>>>>>>>>>>> Big Data Open Source Security LLC
>>>>>>>>>>> http://www.stealth.ly
>>>>>>>>>>> Twitter: @allthingshadoop
>>>>>>>>>>> <http://www.twitter.com/allthingshadoop
>>>>>>>>>>>>
>>>>>>>>>>> ********************************************/
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>
>>>>>>
>>>>
>>>
>>
>


Re: [DISCUSS] Kafka Security Specific Features

Posted by Rob Withers <ro...@gmail.com>.
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 deeper for me, please?   
The model I envision is internal OTP, with access to decryption  
accessed by cert.  A double layer of security, with the internal at- 
rest encryption being an unchanging OTP with ACL access to it as the  
upper layer.  Are you saying it is possible to re-encrypt with new  
keys or that there is a chain of keys over time?

Thanks,
Rob

On Jun 8, 2014, at 3:06 PM, Todd Palino wrote:

> 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, and other PII standards). Yes, it does add external overhead  
> that
> must be managed, but it’s just the nature of the beast. We can’t  
> solve all
> of the external infrastructure needed for this, but we can make it  
> easier
> to use for consumers and producers by adding metadata.
>
> There’s no need for unchanging encryption, and that’s specifically  
> why I
> want to see a message envelope that will help consumers determine the
> encryption uses for a particular message.  You can definitely still  
> expire
> keys, you just have to keep the expired keys around as long as the
> encrypted data stays around, and your endpoints need to know when  
> they are
> decrypting data with an expired key (you might want to throw up a  
> warning,
> or do something else to let the users know that it’s happening). And  
> as
> someone else mentioned, there are solutions for encrypting data for
> multiple consumers. You can encrypt the data with an OTP, and then
> multiply encrypt the OTP once for each consumer and store those  
> encrypted
> strings in the envelope.
>
> -Todd
>
> On 6/7/14, 12:25 PM, "Rob Withers" <ro...@gmail.com> wrote:
>
>> 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 encryption for the data and the clients (producers/
>> consumers) will need to be able to decode frozen data.  If certs are
>> used they cannot expire.  Also, different clients would need to use
>> the same cert.
>>
>> So, you statement that it should ABSOLUTELY not include internal
>> encryption rings seems misplaced.  There are some customers of kafka
>> that would opt to encrypt the on-disk data and key management is a
>> significant issue.  This is best handled internally, with key
>> management stored in either ZK or in a topic.  Truly, perhaps
>> annealing Hadoop/HBASE as a metadata store seems applicable.
>>
>> Thanks, another 2 cents,
>> Rob
>>
>> On Jun 6, 2014, at 12:15 PM, Todd Palino wrote:
>>
>>> 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 server then has access to all the keys. You could say  
>>> that
>>> this eliminates the need for TLS, but TLS is pretty low-hanging
>>> fruit, and
>>> there¹s definitely a need for encryption of the traffic across the
>>> network
>>> even if you don¹t need at-rest encryption as well.
>>>
>>> And as you mentioned, something needs to be done about key  
>>> management.
>>> Storing information with the message about which key(s) was used is
>>> a good
>>> idea, because it allows you to know when a producer has switched  
>>> keys.
>>> There are definitely some alternative solutions to that as well. But
>>> storing the keys in the broker, Zookeeper, or other systems like
>>> that are
>>> not. There needs to be a system used where the keys are only
>>> available to
>>> the producers and consumers that need them, and they only get access
>>> to
>>> the appropriate part of the key pair.  Even as the guy running Kafka
>>> and
>>> Zookeeper, I should not have access to the keys being used, and if
>>> data is
>>> encrypted I should not be able to see the cleartext.
>>>
>>> And even if we decide not to put anything about at-rest encryption
>>> in the
>>> consumer/producer clients directly, and leave it for an exercise  
>>> above
>>> that level (you have to pass the ciphertext as the message to the
>>> client),
>>> I still think there is a good case for implementing a message  
>>> envelope
>>> that can store the information about which key was used, and other
>>> pertinent metadata, and have the ability for special applications  
>>> like
>>> mirror maker to be able to preserve it across clusters. This still
>>> helps
>>> to enable the use of encryption and other features (like auditing)
>>> even if
>>> we decide it¹s too large a scope to fully implement.
>>>
>>> -Todd
>>>
>>> On 6/6/14, 10:51 AM, "Pradeep Gollakota" <pr...@gmail.com>  
>>> 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 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 better performance.
>>>>
>>>>
>>>> On Fri, Jun 6, 2014 at 10:48 AM, Rob Withers
>>>> <robert.w.withers@gmail.com
>>>>>
>>>> wrote:
>>>>
>>>>> 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 key....certs won't work, unless  
>>>>> you
>>>>> write
>>>>> a CertAuthority to build multiple certs with the same keys.   
>>>>> Better
>>>>> seems
>>>>> to not use certs and wrap the encryption specification with an ACL
>>>>> capabilities for each group of access.
>>>>>
>>>>>
>>>>> On Jun 6, 2014, at 11:43 AM, Rob Withers wrote:
>>>>>
>>>>> 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/WRITE/DELETE/LIST/SCAN).  Erights.org has the
>>>>>> quitenscential (??)
>>>>>> object capabilities model and capnproto is impleemting this for  
>>>>>> C+
>>>>>> +.  I
>>>>>> have a java implementation at http://github.com/pauwau/pauwau but
>>>>>> the
>>>>>> master is broken.  0.2 works, basically.  B asically a TLS
>>>>>> connection
>>>>>> with
>>>>>> no certificate server, it is peer to peer.  It has some advanced
>>>>>> features,
>>>>>> but the lining of capabilities with authorization so that you can
>>>>>> only
>>>>>> invoke correct services is extended to the secure user.
>>>>>>
>>>>>> Regarding non-repudiation, on disk, why not prepend a CRC?
>>>>>>
>>>>>> Regarding on-disk encryption, multiple users/groups may need to
>>>>>> access,
>>>>>> with different capabilities.  Sounds like zookeeper needs to
>>>>>> store a
>>>>>> cert
>>>>>> for each class of access so that a group member can access the
>>>>>> decrypted
>>>>>> data from disk.  Use cert-based async decryption.  The only  
>>>>>> isue is
>>>>>> storing
>>>>>> the private key in zookeeper.  Perhaps some hash magic could be
>>>>>> used.
>>>>>>
>>>>>> Thanks for kafka,
>>>>>> Rob
>>>>>>
>>>>>> On Jun 5, 2014, at 3:01 PM, Jay Kreps wrote:
>>>>>>
>>>>>> 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 encryption scheme? Is this
>>>>>>> actually
>>>>>>> needed?
>>>>>>> Isn't a on-the-wire encryption when combined with mutual
>>>>>>> authentication
>>>>>>> and
>>>>>>> permissions sufficient for most uses?
>>>>>>>
>>>>>>> On-disk encryption seems unnecessary because if an attacker can
>>>>>>> get
>>>>>>> root
>>>>>>> on
>>>>>>> the kafka boxes it can potentially modify Kafka to do anything
>>>>>>> he or
>>>>>>> she
>>>>>>> wants with data. So this seems to break any security model.
>>>>>>>
>>>>>>> I understand the problem of a large organization not really
>>>>>>> having a
>>>>>>> trusted network and wanting to secure data transfer and limit  
>>>>>>> and
>>>>>>> audit
>>>>>>> data access. The uses for these other things I don't totally
>>>>>>> understand.
>>>>>>>
>>>>>>> Also it would be worth understanding the state of other
>>>>>>> messaging and
>>>>>>> storage systems (Hadoop, dbs, etc). What features do they
>>>>>>> support. I
>>>>>>> think
>>>>>>> there is a sense in which you don't have to run faster than the
>>>>>>> bear,
>>>>>>> but
>>>>>>> only faster then your friends. :-)
>>>>>>>
>>>>>>> -Jay
>>>>>>>
>>>>>>>
>>>>>>> On Wed, Jun 4, 2014 at 5:57 PM, Joe Stein <jo...@stealth.ly>
>>>>>>> wrote:
>>>>>>>
>>>>>>> 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 <ja...@gmail.com>
>>>>>>>> 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
>>>>>>>>> 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 most the LinkedIn folks are working on the consumer
>>>>>>>>> right now
>>>>>>>>> I
>>>>>>>>>
>>>>>>>> think
>>>>>>>>
>>>>>>>>> this would be a great project for any other interested  
>>>>>>>>> people to
>>>>>>>>> take
>>>>>>>>> on.
>>>>>>>>> There are some challenges in doing these things distributed
>>>>>>>>> but it
>>>>>>>>> can
>>>>>>>>>
>>>>>>>> also
>>>>>>>>
>>>>>>>>> be a lot of fun.
>>>>>>>>>
>>>>>>>>> I think a good first step would be to get a written plan we
>>>>>>>>> can all
>>>>>>>>> agree
>>>>>>>>> on for how things should work. Then we can break things down
>>>>>>>>> into
>>>>>>>>> chunks
>>>>>>>>> that can be done independently while still aiming at a good  
>>>>>>>>> end
>>>>>>>>> state.
>>>>>>>>>
>>>>>>>>> I had tried to write up some notes that summarized at least  
>>>>>>>>> the
>>>>>>>>> thoughts
>>>>>>>>>
>>>>>>>> I
>>>>>>>>
>>>>>>>>> had had on security:
>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/Security
>>>>>>>>>
>>>>>>>>> What do you think of that?
>>>>>>>>>
>>>>>>>>> One assumption I had (which may be incorrect) is that although
>>>>>>>>> we
>>>>>>>>> want
>>>>>>>>>
>>>>>>>> all
>>>>>>>>
>>>>>>>>> the things in your list, the two most pressing would be
>>>>>>>>> authentication
>>>>>>>>>
>>>>>>>> and
>>>>>>>>
>>>>>>>>> authorization, and that was all that write up covered. You
>>>>>>>>> have more
>>>>>>>>> experience in this domain, so I wonder how you would  
>>>>>>>>> prioritize?
>>>>>>>>>
>>>>>>>>> Those notes are really sketchy, so I think the first goal I
>>>>>>>>> would
>>>>>>>>> have
>>>>>>>>> would be to get to a real spec we can all agree on and
>>>>>>>>> discuss. A
>>>>>>>>> lot
>>>>>>>>> of
>>>>>>>>> the security stuff has a high human interaction element and
>>>>>>>>> needs to
>>>>>>>>> work
>>>>>>>>> in pretty different domains and different companies so getting
>>>>>>>>> this
>>>>>>>>> kind
>>>>>>>>>
>>>>>>>> of
>>>>>>>>
>>>>>>>>> review is important.
>>>>>>>>>
>>>>>>>>> -Jay
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> On Tue, Jun 3, 2014 at 12:57 PM, Joe Stein
>>>>>>>>> <jo...@stealth.ly>
>>>>>>>>> wrote:
>>>>>>>>>
>>>>>>>>> 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.  This comes from what folks are doing "in the
>>>>>>>>>> wild"
>>>>>>>>>> to
>>>>>>>>>> workaround and implement security with Kafka as it is today  
>>>>>>>>>> and
>>>>>>>>>> also
>>>>>>>>>>
>>>>>>>>> what I
>>>>>>>>>
>>>>>>>>>> have discovered from organizations about their blockers. It
>>>>>>>>>> also
>>>>>>>>>> picks
>>>>>>>>>>
>>>>>>>>> up
>>>>>>>>
>>>>>>>>> from the wiki (which I should have time to update later in the
>>>>>>>>> week
>>>>>>>>>>
>>>>>>>>> based
>>>>>>>>
>>>>>>>>> on the below and feedback from the thread).
>>>>>>>>>>
>>>>>>>>>> 1) Transport Layer Security (i.e. SSL)
>>>>>>>>>>
>>>>>>>>>> This also includes client authentication in addition to in-
>>>>>>>>>> transit
>>>>>>>>>>
>>>>>>>>> security
>>>>>>>>>
>>>>>>>>>> layer.  This work has been picked up here
>>>>>>>>>> https://issues.apache.org/jira/browse/KAFKA-1477 and do
>>>>>>>>>> appreciate
>>>>>>>>>> any
>>>>>>>>>> thoughts, comments, feedback, tomatoes, whatever for this
>>>>>>>>>> patch.
>>>>>>>>>> It
>>>>>>>>>>
>>>>>>>>> is a
>>>>>>>>
>>>>>>>>> pickup from the fork of the work first done here
>>>>>>>>>> https://github.com/relango/kafka/tree/kafka_security.
>>>>>>>>>>
>>>>>>>>>> 2) Data encryption at rest.
>>>>>>>>>>
>>>>>>>>>> This is very important and something that can be facilitated
>>>>>>>>>> within
>>>>>>>>>> the
>>>>>>>>>> wire protocol. It requires an additional map data structure
>>>>>>>>>> for the
>>>>>>>>>> "encrypted [data encryption key]". With this map (either in
>>>>>>>>>> your
>>>>>>>>>> object
>>>>>>>>>>
>>>>>>>>> or
>>>>>>>>>
>>>>>>>>>> in the wire protocol) you can store the dynamically generated
>>>>>>>>>> symmetric
>>>>>>>>>>
>>>>>>>>> key
>>>>>>>>>
>>>>>>>>>> (for each message) and then encrypt the data using that
>>>>>>>>>> dynamically
>>>>>>>>>> generated key.  You then encrypt the encryption key using  
>>>>>>>>>> each
>>>>>>>>>> public
>>>>>>>>>>
>>>>>>>>> key
>>>>>>>>
>>>>>>>>> for whom is expected to be able to decrypt the encryption  
>>>>>>>>> key to
>>>>>>>>> then
>>>>>>>>>> decrypt the message.  For each public key encrypted symmetric
>>>>>>>>>> key
>>>>>>>>>>
>>>>>>>>> (which
>>>>>>>>
>>>>>>>>> is
>>>>>>>>>
>>>>>>>>>> now the "encrypted [data encryption key]" along with which
>>>>>>>>>> public
>>>>>>>>>> key
>>>>>>>>>>
>>>>>>>>> it
>>>>>>>>
>>>>>>>>> was encrypted with for (so a map of [publicKey] =
>>>>>>>>>> encryptedDataEncryptionKey) as a chain.   Other patterns  
>>>>>>>>>> can be
>>>>>>>>>>
>>>>>>>>> implemented
>>>>>>>>>
>>>>>>>>>> but this is a pretty standard digital enveloping [0] pattern
>>>>>>>>>> with
>>>>>>>>>> only
>>>>>>>>>>
>>>>>>>>> 1
>>>>>>>>
>>>>>>>>> field added. Other patterns should be able to use that field
>>>>>>>>> to-do
>>>>>>>>>>
>>>>>>>>> their
>>>>>>>>
>>>>>>>>> implementation too.
>>>>>>>>>>
>>>>>>>>>> 3) Non-repudiation and long term non-repudiation.
>>>>>>>>>>
>>>>>>>>>> Non-repudiation is proving data hasn't changed.  This is
>>>>>>>>>> often (if
>>>>>>>>>> not
>>>>>>>>>> always) done with x509 public certificates (chained to a
>>>>>>>>>> certificate
>>>>>>>>>> authority).
>>>>>>>>>>
>>>>>>>>>> Long term non-repudiation is what happens when the
>>>>>>>>>> certificates of
>>>>>>>>>> the
>>>>>>>>>> certificate authority are expired (or revoked) and everything
>>>>>>>>>> ever
>>>>>>>>>>
>>>>>>>>> signed
>>>>>>>>
>>>>>>>>> (ever) with that certificate's public key then becomes "no
>>>>>>>>> longer
>>>>>>>>>>
>>>>>>>>> provable
>>>>>>>>>
>>>>>>>>>> as ever being authentic".  That is where RFC3126 [1] and
>>>>>>>>>> RFC3161
>>>>>>>>>> [2]
>>>>>>>>>>
>>>>>>>>> come
>>>>>>>>
>>>>>>>>> in (or worm drives [hardware], etc).
>>>>>>>>>>
>>>>>>>>>> For either (or both) of these it is an operation of the
>>>>>>>>>> encryptor
>>>>>>>>>> to
>>>>>>>>>> sign/hash the data (with or without third party trusted
>>>>>>>>>> timestap of
>>>>>>>>>> the
>>>>>>>>>> signing event) and encrypt that with their own private key  
>>>>>>>>>> and
>>>>>>>>>>
>>>>>>>>> distribute
>>>>>>>>
>>>>>>>>> the results (before and after encrypting if required) along  
>>>>>>>>> with
>>>>>>>>> their
>>>>>>>>>> public key. This structure is a bit more complex but
>>>>>>>>>> feasible, it
>>>>>>>>>> is a
>>>>>>>>>>
>>>>>>>>> map
>>>>>>>>>
>>>>>>>>>> of digital signature formats and the chain of dig sig
>>>>>>>>>> attestations.
>>>>>>>>>>
>>>>>>>>> The
>>>>>>>>
>>>>>>>>> map's key being the method (i.e. CRC32, PKCS7 [3], XmlDigSig
>>>>>>>>> [4])
>>>>>>>>> and
>>>>>>>>>>
>>>>>>>>> then
>>>>>>>>>
>>>>>>>>>> a list of map where that key is "purpose" of signature (what
>>>>>>>>>> your
>>>>>>>>>>
>>>>>>>>> attesting
>>>>>>>>>
>>>>>>>>>> too).  As a sibling field to the list another field for "the
>>>>>>>>>> attester"
>>>>>>>>>>
>>>>>>>>> as
>>>>>>>>
>>>>>>>>> bytes (e.g. their PKCS12 [5] for the map of PKCS7 signatures).
>>>>>>>>>>
>>>>>>>>>> 4) Authorization
>>>>>>>>>>
>>>>>>>>>> We should have a policy of "404" for data, topics, partitions
>>>>>>>>>> (etc) if
>>>>>>>>>> authenticated connections do not have access.  In "secure
>>>>>>>>>> mode" any
>>>>>>>>>> non
>>>>>>>>>> authenticated connections should get a "404" type message on
>>>>>>>>>>
>>>>>>>>> everything.
>>>>>>>>
>>>>>>>>> Knowing "something is there" is a security risk in many uses
>>>>>>>>> cases.
>>>>>>>>> So
>>>>>>>>>>
>>>>>>>>> if
>>>>>>>>>
>>>>>>>>>> you don't have access you don't even see it.  Baking "that"
>>>>>>>>>> into
>>>>>>>>>> Kafka
>>>>>>>>>> along with some interface for entitlement (access management)
>>>>>>>>>> systems
>>>>>>>>>> (pretty standard) is all that I think needs to be done to the
>>>>>>>>>> core
>>>>>>>>>>
>>>>>>>>> project.
>>>>>>>>>
>>>>>>>>>> I want to tackle item later in the year after summer after  
>>>>>>>>>> the
>>>>>>>>>> other
>>>>>>>>>>
>>>>>>>>> three
>>>>>>>>>
>>>>>>>>>> are complete.
>>>>>>>>>>
>>>>>>>>>> I look forward to thoughts on this and anyone else interested
>>>>>>>>>> in
>>>>>>>>>>
>>>>>>>>> working
>>>>>>>>
>>>>>>>>> with us on these items.
>>>>>>>>>>
>>>>>>>>>> [0]
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>> http://www.emc.com/emc-plus/rsa-labs/standards-
>>>>>>>> initiatives/what-is-a-digital-envelope.htm
>>>>>>>>
>>>>>>>>> [1] http://tools.ietf.org/html/rfc3126
>>>>>>>>>> [2] http://tools.ietf.org/html/rfc3161
>>>>>>>>>> [3]
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>> http://www.emc.com/emc-plus/rsa-labs/standards-initiatives/pkcs-7-
>>>>>>>> cryptographic-message-syntax-standar.htm
>>>>>>>>
>>>>>>>>> [4] http://en.wikipedia.org/wiki/XML_Signature
>>>>>>>>>> [5] http://en.wikipedia.org/wiki/PKCS_12
>>>>>>>>>>
>>>>>>>>>> /*******************************************
>>>>>>>>>> Joe Stein
>>>>>>>>>> Founder, Principal Consultant
>>>>>>>>>> Big Data Open Source Security LLC
>>>>>>>>>> http://www.stealth.ly
>>>>>>>>>> Twitter: @allthingshadoop
>>>>>>>>>> <http://www.twitter.com/allthingshadoop
>>>>>>>>>>>
>>>>>>>>>> ********************************************/
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>
>>>>>
>>>
>>
>


Re: [DISCUSS] Kafka Security Specific Features

Posted by Todd Palino <tp...@linkedin.com.INVALID>.
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, and other PII standards). Yes, it does add external overhead that
must be managed, but it’s just the nature of the beast. We can’t solve all
of the external infrastructure needed for this, but we can make it easier
to use for consumers and producers by adding metadata.

There’s no need for unchanging encryption, and that’s specifically why I
want to see a message envelope that will help consumers determine the
encryption uses for a particular message.  You can definitely still expire
keys, you just have to keep the expired keys around as long as the
encrypted data stays around, and your endpoints need to know when they are
decrypting data with an expired key (you might want to throw up a warning,
or do something else to let the users know that it’s happening). And as
someone else mentioned, there are solutions for encrypting data for
multiple consumers. You can encrypt the data with an OTP, and then
multiply encrypt the OTP once for each consumer and store those encrypted
strings in the envelope.

-Todd

On 6/7/14, 12:25 PM, "Rob Withers" <ro...@gmail.com> wrote:

>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 encryption for the data and the clients (producers/
>consumers) will need to be able to decode frozen data.  If certs are
>used they cannot expire.  Also, different clients would need to use
>the same cert.
>
>So, you statement that it should ABSOLUTELY not include internal
>encryption rings seems misplaced.  There are some customers of kafka
>that would opt to encrypt the on-disk data and key management is a
>significant issue.  This is best handled internally, with key
>management stored in either ZK or in a topic.  Truly, perhaps
>annealing Hadoop/HBASE as a metadata store seems applicable.
>
>Thanks, another 2 cents,
>Rob
>
>On Jun 6, 2014, at 12:15 PM, Todd Palino wrote:
>
>> 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 server then has access to all the keys. You could say that
>> this eliminates the need for TLS, but TLS is pretty low-hanging
>> fruit, and
>> there¹s definitely a need for encryption of the traffic across the
>> network
>> even if you don¹t need at-rest encryption as well.
>>
>> And as you mentioned, something needs to be done about key management.
>> Storing information with the message about which key(s) was used is
>> a good
>> idea, because it allows you to know when a producer has switched keys.
>> There are definitely some alternative solutions to that as well. But
>> storing the keys in the broker, Zookeeper, or other systems like
>> that are
>> not. There needs to be a system used where the keys are only
>> available to
>> the producers and consumers that need them, and they only get access
>> to
>> the appropriate part of the key pair.  Even as the guy running Kafka
>> and
>> Zookeeper, I should not have access to the keys being used, and if
>> data is
>> encrypted I should not be able to see the cleartext.
>>
>> And even if we decide not to put anything about at-rest encryption
>> in the
>> consumer/producer clients directly, and leave it for an exercise above
>> that level (you have to pass the ciphertext as the message to the
>> client),
>> I still think there is a good case for implementing a message envelope
>> that can store the information about which key was used, and other
>> pertinent metadata, and have the ability for special applications like
>> mirror maker to be able to preserve it across clusters. This still
>> helps
>> to enable the use of encryption and other features (like auditing)
>> even if
>> we decide it¹s too large a scope to fully implement.
>>
>> -Todd
>>
>> On 6/6/14, 10:51 AM, "Pradeep Gollakota" <pr...@gmail.com> 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 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 better performance.
>>>
>>>
>>> On Fri, Jun 6, 2014 at 10:48 AM, Rob Withers
>>><robert.w.withers@gmail.com
>>> >
>>> wrote:
>>>
>>>> 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 key....certs won't work, unless you
>>>> write
>>>> a CertAuthority to build multiple certs with the same keys.  Better
>>>> seems
>>>> to not use certs and wrap the encryption specification with an ACL
>>>> capabilities for each group of access.
>>>>
>>>>
>>>> On Jun 6, 2014, at 11:43 AM, Rob Withers wrote:
>>>>
>>>> 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/WRITE/DELETE/LIST/SCAN).  Erights.org has the
>>>>> quitenscential (??)
>>>>> object capabilities model and capnproto is impleemting this for C+
>>>>> +.  I
>>>>> have a java implementation at http://github.com/pauwau/pauwau but
>>>>> the
>>>>> master is broken.  0.2 works, basically.  B asically a TLS
>>>>> connection
>>>>> with
>>>>> no certificate server, it is peer to peer.  It has some advanced
>>>>> features,
>>>>> but the lining of capabilities with authorization so that you can
>>>>> only
>>>>> invoke correct services is extended to the secure user.
>>>>>
>>>>> Regarding non-repudiation, on disk, why not prepend a CRC?
>>>>>
>>>>> Regarding on-disk encryption, multiple users/groups may need to
>>>>> access,
>>>>> with different capabilities.  Sounds like zookeeper needs to
>>>>> store a
>>>>> cert
>>>>> for each class of access so that a group member can access the
>>>>> decrypted
>>>>> data from disk.  Use cert-based async decryption.  The only isue is
>>>>> storing
>>>>> the private key in zookeeper.  Perhaps some hash magic could be
>>>>> used.
>>>>>
>>>>> Thanks for kafka,
>>>>> Rob
>>>>>
>>>>> On Jun 5, 2014, at 3:01 PM, Jay Kreps wrote:
>>>>>
>>>>> 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 encryption scheme? Is this
>>>>>> actually
>>>>>> needed?
>>>>>> Isn't a on-the-wire encryption when combined with mutual
>>>>>> authentication
>>>>>> and
>>>>>> permissions sufficient for most uses?
>>>>>>
>>>>>> On-disk encryption seems unnecessary because if an attacker can
>>>>>> get
>>>>>> root
>>>>>> on
>>>>>> the kafka boxes it can potentially modify Kafka to do anything
>>>>>> he or
>>>>>> she
>>>>>> wants with data. So this seems to break any security model.
>>>>>>
>>>>>> I understand the problem of a large organization not really
>>>>>> having a
>>>>>> trusted network and wanting to secure data transfer and limit and
>>>>>> audit
>>>>>> data access. The uses for these other things I don't totally
>>>>>> understand.
>>>>>>
>>>>>> Also it would be worth understanding the state of other
>>>>>> messaging and
>>>>>> storage systems (Hadoop, dbs, etc). What features do they
>>>>>> support. I
>>>>>> think
>>>>>> there is a sense in which you don't have to run faster than the
>>>>>> bear,
>>>>>> but
>>>>>> only faster then your friends. :-)
>>>>>>
>>>>>> -Jay
>>>>>>
>>>>>>
>>>>>> On Wed, Jun 4, 2014 at 5:57 PM, Joe Stein <jo...@stealth.ly>
>>>>>> wrote:
>>>>>>
>>>>>> 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 <ja...@gmail.com>
>>>>>>> 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
>>>>>>>> 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 most the LinkedIn folks are working on the consumer
>>>>>>>> right now
>>>>>>>> I
>>>>>>>>
>>>>>>> think
>>>>>>>
>>>>>>>> this would be a great project for any other interested people to
>>>>>>>> take
>>>>>>>> on.
>>>>>>>> There are some challenges in doing these things distributed
>>>>>>>> but it
>>>>>>>> can
>>>>>>>>
>>>>>>> also
>>>>>>>
>>>>>>>> be a lot of fun.
>>>>>>>>
>>>>>>>> I think a good first step would be to get a written plan we
>>>>>>>> can all
>>>>>>>> agree
>>>>>>>> on for how things should work. Then we can break things down
>>>>>>>> into
>>>>>>>> chunks
>>>>>>>> that can be done independently while still aiming at a good end
>>>>>>>> state.
>>>>>>>>
>>>>>>>> I had tried to write up some notes that summarized at least the
>>>>>>>> thoughts
>>>>>>>>
>>>>>>> I
>>>>>>>
>>>>>>>> had had on security:
>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/Security
>>>>>>>>
>>>>>>>> What do you think of that?
>>>>>>>>
>>>>>>>> One assumption I had (which may be incorrect) is that although
>>>>>>>> we
>>>>>>>> want
>>>>>>>>
>>>>>>> all
>>>>>>>
>>>>>>>> the things in your list, the two most pressing would be
>>>>>>>> authentication
>>>>>>>>
>>>>>>> and
>>>>>>>
>>>>>>>> authorization, and that was all that write up covered. You
>>>>>>>> have more
>>>>>>>> experience in this domain, so I wonder how you would prioritize?
>>>>>>>>
>>>>>>>> Those notes are really sketchy, so I think the first goal I
>>>>>>>> would
>>>>>>>> have
>>>>>>>> would be to get to a real spec we can all agree on and
>>>>>>>> discuss. A
>>>>>>>> lot
>>>>>>>> of
>>>>>>>> the security stuff has a high human interaction element and
>>>>>>>> needs to
>>>>>>>> work
>>>>>>>> in pretty different domains and different companies so getting
>>>>>>>> this
>>>>>>>> kind
>>>>>>>>
>>>>>>> of
>>>>>>>
>>>>>>>> review is important.
>>>>>>>>
>>>>>>>> -Jay
>>>>>>>>
>>>>>>>>
>>>>>>>> On Tue, Jun 3, 2014 at 12:57 PM, Joe Stein
>>>>>>>> <jo...@stealth.ly>
>>>>>>>> wrote:
>>>>>>>>
>>>>>>>> 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.  This comes from what folks are doing "in the
>>>>>>>>> wild"
>>>>>>>>> to
>>>>>>>>> workaround and implement security with Kafka as it is today and
>>>>>>>>> also
>>>>>>>>>
>>>>>>>> what I
>>>>>>>>
>>>>>>>>> have discovered from organizations about their blockers. It
>>>>>>>>> also
>>>>>>>>> picks
>>>>>>>>>
>>>>>>>> up
>>>>>>>
>>>>>>>> from the wiki (which I should have time to update later in the
>>>>>>>> week
>>>>>>>>>
>>>>>>>> based
>>>>>>>
>>>>>>>> on the below and feedback from the thread).
>>>>>>>>>
>>>>>>>>> 1) Transport Layer Security (i.e. SSL)
>>>>>>>>>
>>>>>>>>> This also includes client authentication in addition to in-
>>>>>>>>> transit
>>>>>>>>>
>>>>>>>> security
>>>>>>>>
>>>>>>>>> layer.  This work has been picked up here
>>>>>>>>> https://issues.apache.org/jira/browse/KAFKA-1477 and do
>>>>>>>>> appreciate
>>>>>>>>> any
>>>>>>>>> thoughts, comments, feedback, tomatoes, whatever for this
>>>>>>>>> patch.
>>>>>>>>> It
>>>>>>>>>
>>>>>>>> is a
>>>>>>>
>>>>>>>> pickup from the fork of the work first done here
>>>>>>>>> https://github.com/relango/kafka/tree/kafka_security.
>>>>>>>>>
>>>>>>>>> 2) Data encryption at rest.
>>>>>>>>>
>>>>>>>>> This is very important and something that can be facilitated
>>>>>>>>> within
>>>>>>>>> the
>>>>>>>>> wire protocol. It requires an additional map data structure
>>>>>>>>> for the
>>>>>>>>> "encrypted [data encryption key]". With this map (either in
>>>>>>>>> your
>>>>>>>>> object
>>>>>>>>>
>>>>>>>> or
>>>>>>>>
>>>>>>>>> in the wire protocol) you can store the dynamically generated
>>>>>>>>> symmetric
>>>>>>>>>
>>>>>>>> key
>>>>>>>>
>>>>>>>>> (for each message) and then encrypt the data using that
>>>>>>>>> dynamically
>>>>>>>>> generated key.  You then encrypt the encryption key using each
>>>>>>>>> public
>>>>>>>>>
>>>>>>>> key
>>>>>>>
>>>>>>>> for whom is expected to be able to decrypt the encryption key to
>>>>>>>> then
>>>>>>>>> decrypt the message.  For each public key encrypted symmetric
>>>>>>>>> key
>>>>>>>>>
>>>>>>>> (which
>>>>>>>
>>>>>>>> is
>>>>>>>>
>>>>>>>>> now the "encrypted [data encryption key]" along with which
>>>>>>>>> public
>>>>>>>>> key
>>>>>>>>>
>>>>>>>> it
>>>>>>>
>>>>>>>> was encrypted with for (so a map of [publicKey] =
>>>>>>>>> encryptedDataEncryptionKey) as a chain.   Other patterns can be
>>>>>>>>>
>>>>>>>> implemented
>>>>>>>>
>>>>>>>>> but this is a pretty standard digital enveloping [0] pattern
>>>>>>>>> with
>>>>>>>>> only
>>>>>>>>>
>>>>>>>> 1
>>>>>>>
>>>>>>>> field added. Other patterns should be able to use that field
>>>>>>>> to-do
>>>>>>>>>
>>>>>>>> their
>>>>>>>
>>>>>>>> implementation too.
>>>>>>>>>
>>>>>>>>> 3) Non-repudiation and long term non-repudiation.
>>>>>>>>>
>>>>>>>>> Non-repudiation is proving data hasn't changed.  This is
>>>>>>>>> often (if
>>>>>>>>> not
>>>>>>>>> always) done with x509 public certificates (chained to a
>>>>>>>>> certificate
>>>>>>>>> authority).
>>>>>>>>>
>>>>>>>>> Long term non-repudiation is what happens when the
>>>>>>>>> certificates of
>>>>>>>>> the
>>>>>>>>> certificate authority are expired (or revoked) and everything
>>>>>>>>> ever
>>>>>>>>>
>>>>>>>> signed
>>>>>>>
>>>>>>>> (ever) with that certificate's public key then becomes "no
>>>>>>>> longer
>>>>>>>>>
>>>>>>>> provable
>>>>>>>>
>>>>>>>>> as ever being authentic".  That is where RFC3126 [1] and
>>>>>>>>> RFC3161
>>>>>>>>> [2]
>>>>>>>>>
>>>>>>>> come
>>>>>>>
>>>>>>>> in (or worm drives [hardware], etc).
>>>>>>>>>
>>>>>>>>> For either (or both) of these it is an operation of the
>>>>>>>>> encryptor
>>>>>>>>> to
>>>>>>>>> sign/hash the data (with or without third party trusted
>>>>>>>>> timestap of
>>>>>>>>> the
>>>>>>>>> signing event) and encrypt that with their own private key and
>>>>>>>>>
>>>>>>>> distribute
>>>>>>>
>>>>>>>> the results (before and after encrypting if required) along with
>>>>>>>> their
>>>>>>>>> public key. This structure is a bit more complex but
>>>>>>>>> feasible, it
>>>>>>>>> is a
>>>>>>>>>
>>>>>>>> map
>>>>>>>>
>>>>>>>>> of digital signature formats and the chain of dig sig
>>>>>>>>> attestations.
>>>>>>>>>
>>>>>>>> The
>>>>>>>
>>>>>>>> map's key being the method (i.e. CRC32, PKCS7 [3], XmlDigSig
>>>>>>>> [4])
>>>>>>>> and
>>>>>>>>>
>>>>>>>> then
>>>>>>>>
>>>>>>>>> a list of map where that key is "purpose" of signature (what
>>>>>>>>> your
>>>>>>>>>
>>>>>>>> attesting
>>>>>>>>
>>>>>>>>> too).  As a sibling field to the list another field for "the
>>>>>>>>> attester"
>>>>>>>>>
>>>>>>>> as
>>>>>>>
>>>>>>>> bytes (e.g. their PKCS12 [5] for the map of PKCS7 signatures).
>>>>>>>>>
>>>>>>>>> 4) Authorization
>>>>>>>>>
>>>>>>>>> We should have a policy of "404" for data, topics, partitions
>>>>>>>>> (etc) if
>>>>>>>>> authenticated connections do not have access.  In "secure
>>>>>>>>> mode" any
>>>>>>>>> non
>>>>>>>>> authenticated connections should get a "404" type message on
>>>>>>>>>
>>>>>>>> everything.
>>>>>>>
>>>>>>>> Knowing "something is there" is a security risk in many uses
>>>>>>>> cases.
>>>>>>>> So
>>>>>>>>>
>>>>>>>> if
>>>>>>>>
>>>>>>>>> you don't have access you don't even see it.  Baking "that"
>>>>>>>>> into
>>>>>>>>> Kafka
>>>>>>>>> along with some interface for entitlement (access management)
>>>>>>>>> systems
>>>>>>>>> (pretty standard) is all that I think needs to be done to the
>>>>>>>>> core
>>>>>>>>>
>>>>>>>> project.
>>>>>>>>
>>>>>>>>> I want to tackle item later in the year after summer after the
>>>>>>>>> other
>>>>>>>>>
>>>>>>>> three
>>>>>>>>
>>>>>>>>> are complete.
>>>>>>>>>
>>>>>>>>> I look forward to thoughts on this and anyone else interested
>>>>>>>>> in
>>>>>>>>>
>>>>>>>> working
>>>>>>>
>>>>>>>> with us on these items.
>>>>>>>>>
>>>>>>>>> [0]
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>> http://www.emc.com/emc-plus/rsa-labs/standards-
>>>>>>> initiatives/what-is-a-digital-envelope.htm
>>>>>>>
>>>>>>>> [1] http://tools.ietf.org/html/rfc3126
>>>>>>>>> [2] http://tools.ietf.org/html/rfc3161
>>>>>>>>> [3]
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>> http://www.emc.com/emc-plus/rsa-labs/standards-initiatives/pkcs-7-
>>>>>>> cryptographic-message-syntax-standar.htm
>>>>>>>
>>>>>>>> [4] http://en.wikipedia.org/wiki/XML_Signature
>>>>>>>>> [5] http://en.wikipedia.org/wiki/PKCS_12
>>>>>>>>>
>>>>>>>>> /*******************************************
>>>>>>>>> Joe Stein
>>>>>>>>> Founder, Principal Consultant
>>>>>>>>> Big Data Open Source Security LLC
>>>>>>>>> http://www.stealth.ly
>>>>>>>>> Twitter: @allthingshadoop
>>>>>>>>><http://www.twitter.com/allthingshadoop
>>>>>>>>> >
>>>>>>>>> ********************************************/
>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>>
>>>>>
>>>>
>>
>


Re: [DISCUSS] Kafka Security Specific Features

Posted by Rob Withers <ro...@gmail.com>.
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 encryption for the data and the clients (producers/ 
consumers) will need to be able to decode frozen data.  If certs are  
used they cannot expire.  Also, different clients would need to use  
the same cert.

So, you statement that it should ABSOLUTELY not include internal  
encryption rings seems misplaced.  There are some customers of kafka  
that would opt to encrypt the on-disk data and key management is a  
significant issue.  This is best handled internally, with key  
management stored in either ZK or in a topic.  Truly, perhaps  
annealing Hadoop/HBASE as a metadata store seems applicable.

Thanks, another 2 cents,
Rob

On Jun 6, 2014, at 12:15 PM, Todd Palino wrote:

> 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 server then has access to all the keys. You could say that
> this eliminates the need for TLS, but TLS is pretty low-hanging  
> fruit, and
> there¹s definitely a need for encryption of the traffic across the  
> network
> even if you don¹t need at-rest encryption as well.
>
> And as you mentioned, something needs to be done about key management.
> Storing information with the message about which key(s) was used is  
> a good
> idea, because it allows you to know when a producer has switched keys.
> There are definitely some alternative solutions to that as well. But
> storing the keys in the broker, Zookeeper, or other systems like  
> that are
> not. There needs to be a system used where the keys are only  
> available to
> the producers and consumers that need them, and they only get access  
> to
> the appropriate part of the key pair.  Even as the guy running Kafka  
> and
> Zookeeper, I should not have access to the keys being used, and if  
> data is
> encrypted I should not be able to see the cleartext.
>
> And even if we decide not to put anything about at-rest encryption  
> in the
> consumer/producer clients directly, and leave it for an exercise above
> that level (you have to pass the ciphertext as the message to the  
> client),
> I still think there is a good case for implementing a message envelope
> that can store the information about which key was used, and other
> pertinent metadata, and have the ability for special applications like
> mirror maker to be able to preserve it across clusters. This still  
> helps
> to enable the use of encryption and other features (like auditing)  
> even if
> we decide it¹s too large a scope to fully implement.
>
> -Todd
>
> On 6/6/14, 10:51 AM, "Pradeep Gollakota" <pr...@gmail.com> 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 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 better performance.
>>
>>
>> On Fri, Jun 6, 2014 at 10:48 AM, Rob Withers <robert.w.withers@gmail.com 
>> >
>> wrote:
>>
>>> 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 key....certs won't work, unless you
>>> write
>>> a CertAuthority to build multiple certs with the same keys.  Better
>>> seems
>>> to not use certs and wrap the encryption specification with an ACL
>>> capabilities for each group of access.
>>>
>>>
>>> On Jun 6, 2014, at 11:43 AM, Rob Withers wrote:
>>>
>>> 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/WRITE/DELETE/LIST/SCAN).  Erights.org has the  
>>>> quitenscential (??)
>>>> object capabilities model and capnproto is impleemting this for C+ 
>>>> +.  I
>>>> have a java implementation at http://github.com/pauwau/pauwau but  
>>>> the
>>>> master is broken.  0.2 works, basically.  B asically a TLS  
>>>> connection
>>>> with
>>>> no certificate server, it is peer to peer.  It has some advanced
>>>> features,
>>>> but the lining of capabilities with authorization so that you can  
>>>> only
>>>> invoke correct services is extended to the secure user.
>>>>
>>>> Regarding non-repudiation, on disk, why not prepend a CRC?
>>>>
>>>> Regarding on-disk encryption, multiple users/groups may need to  
>>>> access,
>>>> with different capabilities.  Sounds like zookeeper needs to  
>>>> store a
>>>> cert
>>>> for each class of access so that a group member can access the
>>>> decrypted
>>>> data from disk.  Use cert-based async decryption.  The only isue is
>>>> storing
>>>> the private key in zookeeper.  Perhaps some hash magic could be  
>>>> used.
>>>>
>>>> Thanks for kafka,
>>>> Rob
>>>>
>>>> On Jun 5, 2014, at 3:01 PM, Jay Kreps wrote:
>>>>
>>>> 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 encryption scheme? Is this  
>>>>> actually
>>>>> needed?
>>>>> Isn't a on-the-wire encryption when combined with mutual
>>>>> authentication
>>>>> and
>>>>> permissions sufficient for most uses?
>>>>>
>>>>> On-disk encryption seems unnecessary because if an attacker can  
>>>>> get
>>>>> root
>>>>> on
>>>>> the kafka boxes it can potentially modify Kafka to do anything  
>>>>> he or
>>>>> she
>>>>> wants with data. So this seems to break any security model.
>>>>>
>>>>> I understand the problem of a large organization not really  
>>>>> having a
>>>>> trusted network and wanting to secure data transfer and limit and
>>>>> audit
>>>>> data access. The uses for these other things I don't totally
>>>>> understand.
>>>>>
>>>>> Also it would be worth understanding the state of other  
>>>>> messaging and
>>>>> storage systems (Hadoop, dbs, etc). What features do they  
>>>>> support. I
>>>>> think
>>>>> there is a sense in which you don't have to run faster than the  
>>>>> bear,
>>>>> but
>>>>> only faster then your friends. :-)
>>>>>
>>>>> -Jay
>>>>>
>>>>>
>>>>> On Wed, Jun 4, 2014 at 5:57 PM, Joe Stein <jo...@stealth.ly>
>>>>> wrote:
>>>>>
>>>>> 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 <ja...@gmail.com>
>>>>>> 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  
>>>>>>> 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 most the LinkedIn folks are working on the consumer  
>>>>>>> right now
>>>>>>> I
>>>>>>>
>>>>>> think
>>>>>>
>>>>>>> this would be a great project for any other interested people to
>>>>>>> take
>>>>>>> on.
>>>>>>> There are some challenges in doing these things distributed  
>>>>>>> but it
>>>>>>> can
>>>>>>>
>>>>>> also
>>>>>>
>>>>>>> be a lot of fun.
>>>>>>>
>>>>>>> I think a good first step would be to get a written plan we  
>>>>>>> can all
>>>>>>> agree
>>>>>>> on for how things should work. Then we can break things down  
>>>>>>> into
>>>>>>> chunks
>>>>>>> that can be done independently while still aiming at a good end
>>>>>>> state.
>>>>>>>
>>>>>>> I had tried to write up some notes that summarized at least the
>>>>>>> thoughts
>>>>>>>
>>>>>> I
>>>>>>
>>>>>>> had had on security:
>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/Security
>>>>>>>
>>>>>>> What do you think of that?
>>>>>>>
>>>>>>> One assumption I had (which may be incorrect) is that although  
>>>>>>> we
>>>>>>> want
>>>>>>>
>>>>>> all
>>>>>>
>>>>>>> the things in your list, the two most pressing would be
>>>>>>> authentication
>>>>>>>
>>>>>> and
>>>>>>
>>>>>>> authorization, and that was all that write up covered. You  
>>>>>>> have more
>>>>>>> experience in this domain, so I wonder how you would prioritize?
>>>>>>>
>>>>>>> Those notes are really sketchy, so I think the first goal I  
>>>>>>> would
>>>>>>> have
>>>>>>> would be to get to a real spec we can all agree on and  
>>>>>>> discuss. A
>>>>>>> lot
>>>>>>> of
>>>>>>> the security stuff has a high human interaction element and  
>>>>>>> needs to
>>>>>>> work
>>>>>>> in pretty different domains and different companies so getting  
>>>>>>> this
>>>>>>> kind
>>>>>>>
>>>>>> of
>>>>>>
>>>>>>> review is important.
>>>>>>>
>>>>>>> -Jay
>>>>>>>
>>>>>>>
>>>>>>> On Tue, Jun 3, 2014 at 12:57 PM, Joe Stein  
>>>>>>> <jo...@stealth.ly>
>>>>>>> wrote:
>>>>>>>
>>>>>>> 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.  This comes from what folks are doing "in the  
>>>>>>>> wild"
>>>>>>>> to
>>>>>>>> workaround and implement security with Kafka as it is today and
>>>>>>>> also
>>>>>>>>
>>>>>>> what I
>>>>>>>
>>>>>>>> have discovered from organizations about their blockers. It  
>>>>>>>> also
>>>>>>>> picks
>>>>>>>>
>>>>>>> up
>>>>>>
>>>>>>> from the wiki (which I should have time to update later in the  
>>>>>>> week
>>>>>>>>
>>>>>>> based
>>>>>>
>>>>>>> on the below and feedback from the thread).
>>>>>>>>
>>>>>>>> 1) Transport Layer Security (i.e. SSL)
>>>>>>>>
>>>>>>>> This also includes client authentication in addition to in- 
>>>>>>>> transit
>>>>>>>>
>>>>>>> security
>>>>>>>
>>>>>>>> layer.  This work has been picked up here
>>>>>>>> https://issues.apache.org/jira/browse/KAFKA-1477 and do  
>>>>>>>> appreciate
>>>>>>>> any
>>>>>>>> thoughts, comments, feedback, tomatoes, whatever for this  
>>>>>>>> patch.
>>>>>>>> It
>>>>>>>>
>>>>>>> is a
>>>>>>
>>>>>>> pickup from the fork of the work first done here
>>>>>>>> https://github.com/relango/kafka/tree/kafka_security.
>>>>>>>>
>>>>>>>> 2) Data encryption at rest.
>>>>>>>>
>>>>>>>> This is very important and something that can be facilitated  
>>>>>>>> within
>>>>>>>> the
>>>>>>>> wire protocol. It requires an additional map data structure  
>>>>>>>> for the
>>>>>>>> "encrypted [data encryption key]". With this map (either in  
>>>>>>>> your
>>>>>>>> object
>>>>>>>>
>>>>>>> or
>>>>>>>
>>>>>>>> in the wire protocol) you can store the dynamically generated
>>>>>>>> symmetric
>>>>>>>>
>>>>>>> key
>>>>>>>
>>>>>>>> (for each message) and then encrypt the data using that  
>>>>>>>> dynamically
>>>>>>>> generated key.  You then encrypt the encryption key using each
>>>>>>>> public
>>>>>>>>
>>>>>>> key
>>>>>>
>>>>>>> for whom is expected to be able to decrypt the encryption key to
>>>>>>> then
>>>>>>>> decrypt the message.  For each public key encrypted symmetric  
>>>>>>>> key
>>>>>>>>
>>>>>>> (which
>>>>>>
>>>>>>> is
>>>>>>>
>>>>>>>> now the "encrypted [data encryption key]" along with which  
>>>>>>>> public
>>>>>>>> key
>>>>>>>>
>>>>>>> it
>>>>>>
>>>>>>> was encrypted with for (so a map of [publicKey] =
>>>>>>>> encryptedDataEncryptionKey) as a chain.   Other patterns can be
>>>>>>>>
>>>>>>> implemented
>>>>>>>
>>>>>>>> but this is a pretty standard digital enveloping [0] pattern  
>>>>>>>> with
>>>>>>>> only
>>>>>>>>
>>>>>>> 1
>>>>>>
>>>>>>> field added. Other patterns should be able to use that field  
>>>>>>> to-do
>>>>>>>>
>>>>>>> their
>>>>>>
>>>>>>> implementation too.
>>>>>>>>
>>>>>>>> 3) Non-repudiation and long term non-repudiation.
>>>>>>>>
>>>>>>>> Non-repudiation is proving data hasn't changed.  This is  
>>>>>>>> often (if
>>>>>>>> not
>>>>>>>> always) done with x509 public certificates (chained to a
>>>>>>>> certificate
>>>>>>>> authority).
>>>>>>>>
>>>>>>>> Long term non-repudiation is what happens when the  
>>>>>>>> certificates of
>>>>>>>> the
>>>>>>>> certificate authority are expired (or revoked) and everything  
>>>>>>>> ever
>>>>>>>>
>>>>>>> signed
>>>>>>
>>>>>>> (ever) with that certificate's public key then becomes "no  
>>>>>>> longer
>>>>>>>>
>>>>>>> provable
>>>>>>>
>>>>>>>> as ever being authentic".  That is where RFC3126 [1] and  
>>>>>>>> RFC3161
>>>>>>>> [2]
>>>>>>>>
>>>>>>> come
>>>>>>
>>>>>>> in (or worm drives [hardware], etc).
>>>>>>>>
>>>>>>>> For either (or both) of these it is an operation of the  
>>>>>>>> encryptor
>>>>>>>> to
>>>>>>>> sign/hash the data (with or without third party trusted  
>>>>>>>> timestap of
>>>>>>>> the
>>>>>>>> signing event) and encrypt that with their own private key and
>>>>>>>>
>>>>>>> distribute
>>>>>>
>>>>>>> the results (before and after encrypting if required) along with
>>>>>>> their
>>>>>>>> public key. This structure is a bit more complex but  
>>>>>>>> feasible, it
>>>>>>>> is a
>>>>>>>>
>>>>>>> map
>>>>>>>
>>>>>>>> of digital signature formats and the chain of dig sig  
>>>>>>>> attestations.
>>>>>>>>
>>>>>>> The
>>>>>>
>>>>>>> map's key being the method (i.e. CRC32, PKCS7 [3], XmlDigSig  
>>>>>>> [4])
>>>>>>> and
>>>>>>>>
>>>>>>> then
>>>>>>>
>>>>>>>> a list of map where that key is "purpose" of signature (what  
>>>>>>>> your
>>>>>>>>
>>>>>>> attesting
>>>>>>>
>>>>>>>> too).  As a sibling field to the list another field for "the
>>>>>>>> attester"
>>>>>>>>
>>>>>>> as
>>>>>>
>>>>>>> bytes (e.g. their PKCS12 [5] for the map of PKCS7 signatures).
>>>>>>>>
>>>>>>>> 4) Authorization
>>>>>>>>
>>>>>>>> We should have a policy of "404" for data, topics, partitions
>>>>>>>> (etc) if
>>>>>>>> authenticated connections do not have access.  In "secure  
>>>>>>>> mode" any
>>>>>>>> non
>>>>>>>> authenticated connections should get a "404" type message on
>>>>>>>>
>>>>>>> everything.
>>>>>>
>>>>>>> Knowing "something is there" is a security risk in many uses  
>>>>>>> cases.
>>>>>>> So
>>>>>>>>
>>>>>>> if
>>>>>>>
>>>>>>>> you don't have access you don't even see it.  Baking "that"  
>>>>>>>> into
>>>>>>>> Kafka
>>>>>>>> along with some interface for entitlement (access management)
>>>>>>>> systems
>>>>>>>> (pretty standard) is all that I think needs to be done to the  
>>>>>>>> core
>>>>>>>>
>>>>>>> project.
>>>>>>>
>>>>>>>> I want to tackle item later in the year after summer after the
>>>>>>>> other
>>>>>>>>
>>>>>>> three
>>>>>>>
>>>>>>>> are complete.
>>>>>>>>
>>>>>>>> I look forward to thoughts on this and anyone else interested  
>>>>>>>> in
>>>>>>>>
>>>>>>> working
>>>>>>
>>>>>>> with us on these items.
>>>>>>>>
>>>>>>>> [0]
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>> http://www.emc.com/emc-plus/rsa-labs/standards-
>>>>>> initiatives/what-is-a-digital-envelope.htm
>>>>>>
>>>>>>> [1] http://tools.ietf.org/html/rfc3126
>>>>>>>> [2] http://tools.ietf.org/html/rfc3161
>>>>>>>> [3]
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>> http://www.emc.com/emc-plus/rsa-labs/standards-initiatives/pkcs-7-
>>>>>> cryptographic-message-syntax-standar.htm
>>>>>>
>>>>>>> [4] http://en.wikipedia.org/wiki/XML_Signature
>>>>>>>> [5] http://en.wikipedia.org/wiki/PKCS_12
>>>>>>>>
>>>>>>>> /*******************************************
>>>>>>>> Joe Stein
>>>>>>>> Founder, Principal Consultant
>>>>>>>> Big Data Open Source Security LLC
>>>>>>>> http://www.stealth.ly
>>>>>>>> Twitter: @allthingshadoop <http://www.twitter.com/allthingshadoop 
>>>>>>>> >
>>>>>>>> ********************************************/
>>>>>>>>
>>>>>>>>
>>>>>>>
>>>>>>
>>>>
>>>
>


Re: [DISCUSS] Kafka Security Specific Features

Posted by Steven Schlansker <ss...@opentable.com>.
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, even basic functionality like deleting topics or balancing partitions
does not work reliably / correctly, or is harder than it needs to be.

Kafka is a message broker and IMHO it would be nice if it fulfills that role well
over adding such complicated and time-consuming features, which inevitably will
introduce new bugs, new maintenance concerns, and split developer focus.

Not trying to rain on the parade, just voicing my 2¢,
Steven

On Jun 6, 2014, at 11:15 AM, Todd Palino <tp...@linkedin.com.INVALID> wrote:

> 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 server then has access to all the keys. You could say that
> this eliminates the need for TLS, but TLS is pretty low-hanging fruit, and
> there¹s definitely a need for encryption of the traffic across the network
> even if you don¹t need at-rest encryption as well.
> 
> And as you mentioned, something needs to be done about key management.
> Storing information with the message about which key(s) was used is a good
> idea, because it allows you to know when a producer has switched keys.
> There are definitely some alternative solutions to that as well. But
> storing the keys in the broker, Zookeeper, or other systems like that are
> not. There needs to be a system used where the keys are only available to
> the producers and consumers that need them, and they only get access to
> the appropriate part of the key pair.  Even as the guy running Kafka and
> Zookeeper, I should not have access to the keys being used, and if data is
> encrypted I should not be able to see the cleartext.
> 
> And even if we decide not to put anything about at-rest encryption in the
> consumer/producer clients directly, and leave it for an exercise above
> that level (you have to pass the ciphertext as the message to the client),
> I still think there is a good case for implementing a message envelope
> that can store the information about which key was used, and other
> pertinent metadata, and have the ability for special applications like
> mirror maker to be able to preserve it across clusters. This still helps
> to enable the use of encryption and other features (like auditing) even if
> we decide it¹s too large a scope to fully implement.
> 
> -Todd
> 
> On 6/6/14, 10:51 AM, "Pradeep Gollakota" <pr...@gmail.com> 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 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 better performance.
>> 
>> 
>> On Fri, Jun 6, 2014 at 10:48 AM, Rob Withers <ro...@gmail.com>
>> wrote:
>> 
>>> 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 key....certs won't work, unless you
>>> write
>>> a CertAuthority to build multiple certs with the same keys.  Better
>>> seems
>>> to not use certs and wrap the encryption specification with an ACL
>>> capabilities for each group of access.
>>> 
>>> 
>>> On Jun 6, 2014, at 11:43 AM, Rob Withers wrote:
>>> 
>>> 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/WRITE/DELETE/LIST/SCAN).  Erights.org has the quitenscential (??)
>>>> object capabilities model and capnproto is impleemting this for C++.  I
>>>> have a java implementation at http://github.com/pauwau/pauwau but the
>>>> master is broken.  0.2 works, basically.  B asically a TLS connection
>>>> with
>>>> no certificate server, it is peer to peer.  It has some advanced
>>>> features,
>>>> but the lining of capabilities with authorization so that you can only
>>>> invoke correct services is extended to the secure user.
>>>> 
>>>> Regarding non-repudiation, on disk, why not prepend a CRC?
>>>> 
>>>> Regarding on-disk encryption, multiple users/groups may need to access,
>>>> with different capabilities.  Sounds like zookeeper needs to store a
>>>> cert
>>>> for each class of access so that a group member can access the
>>>> decrypted
>>>> data from disk.  Use cert-based async decryption.  The only isue is
>>>> storing
>>>> the private key in zookeeper.  Perhaps some hash magic could be used.
>>>> 
>>>> Thanks for kafka,
>>>> Rob
>>>> 
>>>> On Jun 5, 2014, at 3:01 PM, Jay Kreps wrote:
>>>> 
>>>> 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 encryption scheme? Is this actually
>>>>> needed?
>>>>> Isn't a on-the-wire encryption when combined with mutual
>>>>> authentication
>>>>> and
>>>>> permissions sufficient for most uses?
>>>>> 
>>>>> On-disk encryption seems unnecessary because if an attacker can get
>>>>> root
>>>>> on
>>>>> the kafka boxes it can potentially modify Kafka to do anything he or
>>>>> she
>>>>> wants with data. So this seems to break any security model.
>>>>> 
>>>>> I understand the problem of a large organization not really having a
>>>>> trusted network and wanting to secure data transfer and limit and
>>>>> audit
>>>>> data access. The uses for these other things I don't totally
>>>>> understand.
>>>>> 
>>>>> Also it would be worth understanding the state of other messaging and
>>>>> storage systems (Hadoop, dbs, etc). What features do they support. I
>>>>> think
>>>>> there is a sense in which you don't have to run faster than the bear,
>>>>> but
>>>>> only faster then your friends. :-)
>>>>> 
>>>>> -Jay
>>>>> 
>>>>> 
>>>>> On Wed, Jun 4, 2014 at 5:57 PM, Joe Stein <jo...@stealth.ly>
>>>>> wrote:
>>>>> 
>>>>> 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 <ja...@gmail.com>
>>>>>> 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 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 most the LinkedIn folks are working on the consumer right now
>>>>>>> I
>>>>>>> 
>>>>>> think
>>>>>> 
>>>>>>> this would be a great project for any other interested people to
>>>>>>> take
>>>>>>> on.
>>>>>>> There are some challenges in doing these things distributed but it
>>>>>>> can
>>>>>>> 
>>>>>> also
>>>>>> 
>>>>>>> be a lot of fun.
>>>>>>> 
>>>>>>> I think a good first step would be to get a written plan we can all
>>>>>>> agree
>>>>>>> on for how things should work. Then we can break things down into
>>>>>>> chunks
>>>>>>> that can be done independently while still aiming at a good end
>>>>>>> state.
>>>>>>> 
>>>>>>> I had tried to write up some notes that summarized at least the
>>>>>>> thoughts
>>>>>>> 
>>>>>> I
>>>>>> 
>>>>>>> had had on security:
>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/Security
>>>>>>> 
>>>>>>> What do you think of that?
>>>>>>> 
>>>>>>> One assumption I had (which may be incorrect) is that although we
>>>>>>> want
>>>>>>> 
>>>>>> all
>>>>>> 
>>>>>>> the things in your list, the two most pressing would be
>>>>>>> authentication
>>>>>>> 
>>>>>> and
>>>>>> 
>>>>>>> authorization, and that was all that write up covered. You have more
>>>>>>> experience in this domain, so I wonder how you would prioritize?
>>>>>>> 
>>>>>>> Those notes are really sketchy, so I think the first goal I would
>>>>>>> have
>>>>>>> would be to get to a real spec we can all agree on and discuss. A
>>>>>>> lot
>>>>>>> of
>>>>>>> the security stuff has a high human interaction element and needs to
>>>>>>> work
>>>>>>> in pretty different domains and different companies so getting this
>>>>>>> kind
>>>>>>> 
>>>>>> of
>>>>>> 
>>>>>>> review is important.
>>>>>>> 
>>>>>>> -Jay
>>>>>>> 
>>>>>>> 
>>>>>>> On Tue, Jun 3, 2014 at 12:57 PM, Joe Stein <jo...@stealth.ly>
>>>>>>> wrote:
>>>>>>> 
>>>>>>> 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.  This comes from what folks are doing "in the wild"
>>>>>>>> to
>>>>>>>> workaround and implement security with Kafka as it is today and
>>>>>>>> also
>>>>>>>> 
>>>>>>> what I
>>>>>>> 
>>>>>>>> have discovered from organizations about their blockers. It also
>>>>>>>> picks
>>>>>>>> 
>>>>>>> up
>>>>>> 
>>>>>>> from the wiki (which I should have time to update later in the week
>>>>>>>> 
>>>>>>> based
>>>>>> 
>>>>>>> on the below and feedback from the thread).
>>>>>>>> 
>>>>>>>> 1) Transport Layer Security (i.e. SSL)
>>>>>>>> 
>>>>>>>> This also includes client authentication in addition to in-transit
>>>>>>>> 
>>>>>>> security
>>>>>>> 
>>>>>>>> layer.  This work has been picked up here
>>>>>>>> https://issues.apache.org/jira/browse/KAFKA-1477 and do appreciate
>>>>>>>> any
>>>>>>>> thoughts, comments, feedback, tomatoes, whatever for this patch.
>>>>>>>> It
>>>>>>>> 
>>>>>>> is a
>>>>>> 
>>>>>>> pickup from the fork of the work first done here
>>>>>>>> https://github.com/relango/kafka/tree/kafka_security.
>>>>>>>> 
>>>>>>>> 2) Data encryption at rest.
>>>>>>>> 
>>>>>>>> This is very important and something that can be facilitated within
>>>>>>>> the
>>>>>>>> wire protocol. It requires an additional map data structure for the
>>>>>>>> "encrypted [data encryption key]". With this map (either in your
>>>>>>>> object
>>>>>>>> 
>>>>>>> or
>>>>>>> 
>>>>>>>> in the wire protocol) you can store the dynamically generated
>>>>>>>> symmetric
>>>>>>>> 
>>>>>>> key
>>>>>>> 
>>>>>>>> (for each message) and then encrypt the data using that dynamically
>>>>>>>> generated key.  You then encrypt the encryption key using each
>>>>>>>> public
>>>>>>>> 
>>>>>>> key
>>>>>> 
>>>>>>> for whom is expected to be able to decrypt the encryption key to
>>>>>>> then
>>>>>>>> decrypt the message.  For each public key encrypted symmetric key
>>>>>>>> 
>>>>>>> (which
>>>>>> 
>>>>>>> is
>>>>>>> 
>>>>>>>> now the "encrypted [data encryption key]" along with which public
>>>>>>>> key
>>>>>>>> 
>>>>>>> it
>>>>>> 
>>>>>>> was encrypted with for (so a map of [publicKey] =
>>>>>>>> encryptedDataEncryptionKey) as a chain.   Other patterns can be
>>>>>>>> 
>>>>>>> implemented
>>>>>>> 
>>>>>>>> but this is a pretty standard digital enveloping [0] pattern with
>>>>>>>> only
>>>>>>>> 
>>>>>>> 1
>>>>>> 
>>>>>>> field added. Other patterns should be able to use that field to-do
>>>>>>>> 
>>>>>>> their
>>>>>> 
>>>>>>> implementation too.
>>>>>>>> 
>>>>>>>> 3) Non-repudiation and long term non-repudiation.
>>>>>>>> 
>>>>>>>> Non-repudiation is proving data hasn't changed.  This is often (if
>>>>>>>> not
>>>>>>>> always) done with x509 public certificates (chained to a
>>>>>>>> certificate
>>>>>>>> authority).
>>>>>>>> 
>>>>>>>> Long term non-repudiation is what happens when the certificates of
>>>>>>>> the
>>>>>>>> certificate authority are expired (or revoked) and everything ever
>>>>>>>> 
>>>>>>> signed
>>>>>> 
>>>>>>> (ever) with that certificate's public key then becomes "no longer
>>>>>>>> 
>>>>>>> provable
>>>>>>> 
>>>>>>>> as ever being authentic".  That is where RFC3126 [1] and RFC3161
>>>>>>>> [2]
>>>>>>>> 
>>>>>>> come
>>>>>> 
>>>>>>> in (or worm drives [hardware], etc).
>>>>>>>> 
>>>>>>>> For either (or both) of these it is an operation of the encryptor
>>>>>>>> to
>>>>>>>> sign/hash the data (with or without third party trusted timestap of
>>>>>>>> the
>>>>>>>> signing event) and encrypt that with their own private key and
>>>>>>>> 
>>>>>>> distribute
>>>>>> 
>>>>>>> the results (before and after encrypting if required) along with
>>>>>>> their
>>>>>>>> public key. This structure is a bit more complex but feasible, it
>>>>>>>> is a
>>>>>>>> 
>>>>>>> map
>>>>>>> 
>>>>>>>> of digital signature formats and the chain of dig sig attestations.
>>>>>>>> 
>>>>>>> The
>>>>>> 
>>>>>>> map's key being the method (i.e. CRC32, PKCS7 [3], XmlDigSig [4])
>>>>>>> and
>>>>>>>> 
>>>>>>> then
>>>>>>> 
>>>>>>>> a list of map where that key is "purpose" of signature (what your
>>>>>>>> 
>>>>>>> attesting
>>>>>>> 
>>>>>>>> too).  As a sibling field to the list another field for "the
>>>>>>>> attester"
>>>>>>>> 
>>>>>>> as
>>>>>> 
>>>>>>> bytes (e.g. their PKCS12 [5] for the map of PKCS7 signatures).
>>>>>>>> 
>>>>>>>> 4) Authorization
>>>>>>>> 
>>>>>>>> We should have a policy of "404" for data, topics, partitions
>>>>>>>> (etc) if
>>>>>>>> authenticated connections do not have access.  In "secure mode" any
>>>>>>>> non
>>>>>>>> authenticated connections should get a "404" type message on
>>>>>>>> 
>>>>>>> everything.
>>>>>> 
>>>>>>> Knowing "something is there" is a security risk in many uses cases.
>>>>>>> So
>>>>>>>> 
>>>>>>> if
>>>>>>> 
>>>>>>>> you don't have access you don't even see it.  Baking "that" into
>>>>>>>> Kafka
>>>>>>>> along with some interface for entitlement (access management)
>>>>>>>> systems
>>>>>>>> (pretty standard) is all that I think needs to be done to the core
>>>>>>>> 
>>>>>>> project.
>>>>>>> 
>>>>>>>> I want to tackle item later in the year after summer after the
>>>>>>>> other
>>>>>>>> 
>>>>>>> three
>>>>>>> 
>>>>>>>> are complete.
>>>>>>>> 
>>>>>>>> I look forward to thoughts on this and anyone else interested in
>>>>>>>> 
>>>>>>> working
>>>>>> 
>>>>>>> with us on these items.
>>>>>>>> 
>>>>>>>> [0]
>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> http://www.emc.com/emc-plus/rsa-labs/standards-
>>>>>> initiatives/what-is-a-digital-envelope.htm
>>>>>> 
>>>>>>> [1] http://tools.ietf.org/html/rfc3126
>>>>>>>> [2] http://tools.ietf.org/html/rfc3161
>>>>>>>> [3]
>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> http://www.emc.com/emc-plus/rsa-labs/standards-initiatives/pkcs-7-
>>>>>> cryptographic-message-syntax-standar.htm
>>>>>> 
>>>>>>> [4] http://en.wikipedia.org/wiki/XML_Signature
>>>>>>>> [5] http://en.wikipedia.org/wiki/PKCS_12
>>>>>>>> 
>>>>>>>> /*******************************************
>>>>>>>> Joe Stein
>>>>>>>> Founder, Principal Consultant
>>>>>>>> Big Data Open Source Security LLC
>>>>>>>> http://www.stealth.ly
>>>>>>>> Twitter: @allthingshadoop <http://www.twitter.com/allthingshadoop>
>>>>>>>> ********************************************/
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>> 
>>>> 
>>> 
> 


Re: [DISCUSS] Kafka Security Specific Features

Posted by Todd Palino <tp...@linkedin.com.INVALID>.
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 server then has access to all the keys. You could say that
this eliminates the need for TLS, but TLS is pretty low-hanging fruit, and
there¹s definitely a need for encryption of the traffic across the network
even if you don¹t need at-rest encryption as well.

And as you mentioned, something needs to be done about key management.
Storing information with the message about which key(s) was used is a good
idea, because it allows you to know when a producer has switched keys.
There are definitely some alternative solutions to that as well. But
storing the keys in the broker, Zookeeper, or other systems like that are
not. There needs to be a system used where the keys are only available to
the producers and consumers that need them, and they only get access to
the appropriate part of the key pair.  Even as the guy running Kafka and
Zookeeper, I should not have access to the keys being used, and if data is
encrypted I should not be able to see the cleartext.

And even if we decide not to put anything about at-rest encryption in the
consumer/producer clients directly, and leave it for an exercise above
that level (you have to pass the ciphertext as the message to the client),
I still think there is a good case for implementing a message envelope
that can store the information about which key was used, and other
pertinent metadata, and have the ability for special applications like
mirror maker to be able to preserve it across clusters. This still helps
to enable the use of encryption and other features (like auditing) even if
we decide it¹s too large a scope to fully implement.

-Todd

On 6/6/14, 10:51 AM, "Pradeep Gollakota" <pr...@gmail.com> 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 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 better performance.
>
>
>On Fri, Jun 6, 2014 at 10:48 AM, Rob Withers <ro...@gmail.com>
>wrote:
>
>> 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 key....certs won't work, unless you
>>write
>> a CertAuthority to build multiple certs with the same keys.  Better
>>seems
>> to not use certs and wrap the encryption specification with an ACL
>> capabilities for each group of access.
>>
>>
>> On Jun 6, 2014, at 11:43 AM, Rob Withers wrote:
>>
>>  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/WRITE/DELETE/LIST/SCAN).  Erights.org has the quitenscential (??)
>>> object capabilities model and capnproto is impleemting this for C++.  I
>>> have a java implementation at http://github.com/pauwau/pauwau but the
>>> master is broken.  0.2 works, basically.  B asically a TLS connection
>>>with
>>> no certificate server, it is peer to peer.  It has some advanced
>>>features,
>>> but the lining of capabilities with authorization so that you can only
>>> invoke correct services is extended to the secure user.
>>>
>>> Regarding non-repudiation, on disk, why not prepend a CRC?
>>>
>>> Regarding on-disk encryption, multiple users/groups may need to access,
>>> with different capabilities.  Sounds like zookeeper needs to store a
>>>cert
>>> for each class of access so that a group member can access the
>>>decrypted
>>> data from disk.  Use cert-based async decryption.  The only isue is
>>>storing
>>> the private key in zookeeper.  Perhaps some hash magic could be used.
>>>
>>> Thanks for kafka,
>>> Rob
>>>
>>> On Jun 5, 2014, at 3:01 PM, Jay Kreps wrote:
>>>
>>>  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 encryption scheme? Is this actually
>>>> needed?
>>>> Isn't a on-the-wire encryption when combined with mutual
>>>>authentication
>>>> and
>>>> permissions sufficient for most uses?
>>>>
>>>> On-disk encryption seems unnecessary because if an attacker can get
>>>>root
>>>> on
>>>> the kafka boxes it can potentially modify Kafka to do anything he or
>>>>she
>>>> wants with data. So this seems to break any security model.
>>>>
>>>> I understand the problem of a large organization not really having a
>>>> trusted network and wanting to secure data transfer and limit and
>>>>audit
>>>> data access. The uses for these other things I don't totally
>>>>understand.
>>>>
>>>> Also it would be worth understanding the state of other messaging and
>>>> storage systems (Hadoop, dbs, etc). What features do they support. I
>>>> think
>>>> there is a sense in which you don't have to run faster than the bear,
>>>>but
>>>> only faster then your friends. :-)
>>>>
>>>> -Jay
>>>>
>>>>
>>>> On Wed, Jun 4, 2014 at 5:57 PM, Joe Stein <jo...@stealth.ly>
>>>>wrote:
>>>>
>>>>  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 <ja...@gmail.com>
>>>>>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 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 most the LinkedIn folks are working on the consumer right now
>>>>>>I
>>>>>>
>>>>> think
>>>>>
>>>>>> this would be a great project for any other interested people to
>>>>>>take
>>>>>> on.
>>>>>> There are some challenges in doing these things distributed but it
>>>>>>can
>>>>>>
>>>>> also
>>>>>
>>>>>> be a lot of fun.
>>>>>>
>>>>>> I think a good first step would be to get a written plan we can all
>>>>>> agree
>>>>>> on for how things should work. Then we can break things down into
>>>>>> chunks
>>>>>> that can be done independently while still aiming at a good end
>>>>>>state.
>>>>>>
>>>>>> I had tried to write up some notes that summarized at least the
>>>>>> thoughts
>>>>>>
>>>>> I
>>>>>
>>>>>> had had on security:
>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/Security
>>>>>>
>>>>>> What do you think of that?
>>>>>>
>>>>>> One assumption I had (which may be incorrect) is that although we
>>>>>>want
>>>>>>
>>>>> all
>>>>>
>>>>>> the things in your list, the two most pressing would be
>>>>>>authentication
>>>>>>
>>>>> and
>>>>>
>>>>>> authorization, and that was all that write up covered. You have more
>>>>>> experience in this domain, so I wonder how you would prioritize?
>>>>>>
>>>>>> Those notes are really sketchy, so I think the first goal I would
>>>>>>have
>>>>>> would be to get to a real spec we can all agree on and discuss. A
>>>>>>lot
>>>>>> of
>>>>>> the security stuff has a high human interaction element and needs to
>>>>>> work
>>>>>> in pretty different domains and different companies so getting this
>>>>>> kind
>>>>>>
>>>>> of
>>>>>
>>>>>> review is important.
>>>>>>
>>>>>> -Jay
>>>>>>
>>>>>>
>>>>>> On Tue, Jun 3, 2014 at 12:57 PM, Joe Stein <jo...@stealth.ly>
>>>>>> wrote:
>>>>>>
>>>>>>  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.  This comes from what folks are doing "in the wild"
>>>>>>>to
>>>>>>> workaround and implement security with Kafka as it is today and
>>>>>>>also
>>>>>>>
>>>>>> what I
>>>>>>
>>>>>>> have discovered from organizations about their blockers. It also
>>>>>>>picks
>>>>>>>
>>>>>> up
>>>>>
>>>>>> from the wiki (which I should have time to update later in the week
>>>>>>>
>>>>>> based
>>>>>
>>>>>> on the below and feedback from the thread).
>>>>>>>
>>>>>>> 1) Transport Layer Security (i.e. SSL)
>>>>>>>
>>>>>>> This also includes client authentication in addition to in-transit
>>>>>>>
>>>>>> security
>>>>>>
>>>>>>> layer.  This work has been picked up here
>>>>>>> https://issues.apache.org/jira/browse/KAFKA-1477 and do appreciate
>>>>>>> any
>>>>>>> thoughts, comments, feedback, tomatoes, whatever for this patch.
>>>>>>>It
>>>>>>>
>>>>>> is a
>>>>>
>>>>>> pickup from the fork of the work first done here
>>>>>>> https://github.com/relango/kafka/tree/kafka_security.
>>>>>>>
>>>>>>> 2) Data encryption at rest.
>>>>>>>
>>>>>>> This is very important and something that can be facilitated within
>>>>>>> the
>>>>>>> wire protocol. It requires an additional map data structure for the
>>>>>>> "encrypted [data encryption key]". With this map (either in your
>>>>>>> object
>>>>>>>
>>>>>> or
>>>>>>
>>>>>>> in the wire protocol) you can store the dynamically generated
>>>>>>> symmetric
>>>>>>>
>>>>>> key
>>>>>>
>>>>>>> (for each message) and then encrypt the data using that dynamically
>>>>>>> generated key.  You then encrypt the encryption key using each
>>>>>>>public
>>>>>>>
>>>>>> key
>>>>>
>>>>>> for whom is expected to be able to decrypt the encryption key to
>>>>>>then
>>>>>>> decrypt the message.  For each public key encrypted symmetric key
>>>>>>>
>>>>>> (which
>>>>>
>>>>>> is
>>>>>>
>>>>>>> now the "encrypted [data encryption key]" along with which public
>>>>>>>key
>>>>>>>
>>>>>> it
>>>>>
>>>>>> was encrypted with for (so a map of [publicKey] =
>>>>>>> encryptedDataEncryptionKey) as a chain.   Other patterns can be
>>>>>>>
>>>>>> implemented
>>>>>>
>>>>>>> but this is a pretty standard digital enveloping [0] pattern with
>>>>>>>only
>>>>>>>
>>>>>> 1
>>>>>
>>>>>> field added. Other patterns should be able to use that field to-do
>>>>>>>
>>>>>> their
>>>>>
>>>>>> implementation too.
>>>>>>>
>>>>>>> 3) Non-repudiation and long term non-repudiation.
>>>>>>>
>>>>>>> Non-repudiation is proving data hasn't changed.  This is often (if
>>>>>>>not
>>>>>>> always) done with x509 public certificates (chained to a
>>>>>>>certificate
>>>>>>> authority).
>>>>>>>
>>>>>>> Long term non-repudiation is what happens when the certificates of
>>>>>>>the
>>>>>>> certificate authority are expired (or revoked) and everything ever
>>>>>>>
>>>>>> signed
>>>>>
>>>>>> (ever) with that certificate's public key then becomes "no longer
>>>>>>>
>>>>>> provable
>>>>>>
>>>>>>> as ever being authentic".  That is where RFC3126 [1] and RFC3161
>>>>>>>[2]
>>>>>>>
>>>>>> come
>>>>>
>>>>>> in (or worm drives [hardware], etc).
>>>>>>>
>>>>>>> For either (or both) of these it is an operation of the encryptor
>>>>>>>to
>>>>>>> sign/hash the data (with or without third party trusted timestap of
>>>>>>> the
>>>>>>> signing event) and encrypt that with their own private key and
>>>>>>>
>>>>>> distribute
>>>>>
>>>>>> the results (before and after encrypting if required) along with
>>>>>>their
>>>>>>> public key. This structure is a bit more complex but feasible, it
>>>>>>>is a
>>>>>>>
>>>>>> map
>>>>>>
>>>>>>> of digital signature formats and the chain of dig sig attestations.
>>>>>>>
>>>>>> The
>>>>>
>>>>>> map's key being the method (i.e. CRC32, PKCS7 [3], XmlDigSig [4])
>>>>>>and
>>>>>>>
>>>>>> then
>>>>>>
>>>>>>> a list of map where that key is "purpose" of signature (what your
>>>>>>>
>>>>>> attesting
>>>>>>
>>>>>>> too).  As a sibling field to the list another field for "the
>>>>>>>attester"
>>>>>>>
>>>>>> as
>>>>>
>>>>>> bytes (e.g. their PKCS12 [5] for the map of PKCS7 signatures).
>>>>>>>
>>>>>>> 4) Authorization
>>>>>>>
>>>>>>> We should have a policy of "404" for data, topics, partitions
>>>>>>>(etc) if
>>>>>>> authenticated connections do not have access.  In "secure mode" any
>>>>>>> non
>>>>>>> authenticated connections should get a "404" type message on
>>>>>>>
>>>>>> everything.
>>>>>
>>>>>> Knowing "something is there" is a security risk in many uses cases.
>>>>>> So
>>>>>>>
>>>>>> if
>>>>>>
>>>>>>> you don't have access you don't even see it.  Baking "that" into
>>>>>>>Kafka
>>>>>>> along with some interface for entitlement (access management)
>>>>>>>systems
>>>>>>> (pretty standard) is all that I think needs to be done to the core
>>>>>>>
>>>>>> project.
>>>>>>
>>>>>>> I want to tackle item later in the year after summer after the
>>>>>>>other
>>>>>>>
>>>>>> three
>>>>>>
>>>>>>> are complete.
>>>>>>>
>>>>>>> I look forward to thoughts on this and anyone else interested in
>>>>>>>
>>>>>> working
>>>>>
>>>>>> with us on these items.
>>>>>>>
>>>>>>> [0]
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>  http://www.emc.com/emc-plus/rsa-labs/standards-
>>>>> initiatives/what-is-a-digital-envelope.htm
>>>>>
>>>>>> [1] http://tools.ietf.org/html/rfc3126
>>>>>>> [2] http://tools.ietf.org/html/rfc3161
>>>>>>> [3]
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>  http://www.emc.com/emc-plus/rsa-labs/standards-initiatives/pkcs-7-
>>>>> cryptographic-message-syntax-standar.htm
>>>>>
>>>>>> [4] http://en.wikipedia.org/wiki/XML_Signature
>>>>>>> [5] http://en.wikipedia.org/wiki/PKCS_12
>>>>>>>
>>>>>>> /*******************************************
>>>>>>> Joe Stein
>>>>>>> Founder, Principal Consultant
>>>>>>> Big Data Open Source Security LLC
>>>>>>> http://www.stealth.ly
>>>>>>> Twitter: @allthingshadoop <http://www.twitter.com/allthingshadoop>
>>>>>>> ********************************************/
>>>>>>>
>>>>>>>
>>>>>>
>>>>>
>>>
>>


Re: [DISCUSS] Kafka Security Specific Features

Posted by Pradeep Gollakota <pr...@gmail.com>.
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 better performance.


On Fri, Jun 6, 2014 at 10:48 AM, Rob Withers <ro...@gmail.com>
wrote:

> 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 key....certs won't work, unless you write
> a CertAuthority to build multiple certs with the same keys.  Better seems
> to not use certs and wrap the encryption specification with an ACL
> capabilities for each group of access.
>
>
> On Jun 6, 2014, at 11:43 AM, Rob Withers wrote:
>
>  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/WRITE/DELETE/LIST/SCAN).  Erights.org has the quitenscential (??)
>> object capabilities model and capnproto is impleemting this for C++.  I
>> have a java implementation at http://github.com/pauwau/pauwau but the
>> master is broken.  0.2 works, basically.  B asically a TLS connection with
>> no certificate server, it is peer to peer.  It has some advanced features,
>> but the lining of capabilities with authorization so that you can only
>> invoke correct services is extended to the secure user.
>>
>> Regarding non-repudiation, on disk, why not prepend a CRC?
>>
>> Regarding on-disk encryption, multiple users/groups may need to access,
>> with different capabilities.  Sounds like zookeeper needs to store a cert
>> for each class of access so that a group member can access the decrypted
>> data from disk.  Use cert-based async decryption.  The only isue is storing
>> the private key in zookeeper.  Perhaps some hash magic could be used.
>>
>> Thanks for kafka,
>> Rob
>>
>> On Jun 5, 2014, at 3:01 PM, Jay Kreps wrote:
>>
>>  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 encryption scheme? Is this actually
>>> needed?
>>> Isn't a on-the-wire encryption when combined with mutual authentication
>>> and
>>> permissions sufficient for most uses?
>>>
>>> On-disk encryption seems unnecessary because if an attacker can get root
>>> on
>>> the kafka boxes it can potentially modify Kafka to do anything he or she
>>> wants with data. So this seems to break any security model.
>>>
>>> I understand the problem of a large organization not really having a
>>> trusted network and wanting to secure data transfer and limit and audit
>>> data access. The uses for these other things I don't totally understand.
>>>
>>> Also it would be worth understanding the state of other messaging and
>>> storage systems (Hadoop, dbs, etc). What features do they support. I
>>> think
>>> there is a sense in which you don't have to run faster than the bear, but
>>> only faster then your friends. :-)
>>>
>>> -Jay
>>>
>>>
>>> On Wed, Jun 4, 2014 at 5:57 PM, Joe Stein <jo...@stealth.ly> wrote:
>>>
>>>  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 <ja...@gmail.com> 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 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 most the LinkedIn folks are working on the consumer right now I
>>>>>
>>>> think
>>>>
>>>>> this would be a great project for any other interested people to take
>>>>> on.
>>>>> There are some challenges in doing these things distributed but it can
>>>>>
>>>> also
>>>>
>>>>> be a lot of fun.
>>>>>
>>>>> I think a good first step would be to get a written plan we can all
>>>>> agree
>>>>> on for how things should work. Then we can break things down into
>>>>> chunks
>>>>> that can be done independently while still aiming at a good end state.
>>>>>
>>>>> I had tried to write up some notes that summarized at least the
>>>>> thoughts
>>>>>
>>>> I
>>>>
>>>>> had had on security:
>>>>> https://cwiki.apache.org/confluence/display/KAFKA/Security
>>>>>
>>>>> What do you think of that?
>>>>>
>>>>> One assumption I had (which may be incorrect) is that although we want
>>>>>
>>>> all
>>>>
>>>>> the things in your list, the two most pressing would be authentication
>>>>>
>>>> and
>>>>
>>>>> authorization, and that was all that write up covered. You have more
>>>>> experience in this domain, so I wonder how you would prioritize?
>>>>>
>>>>> Those notes are really sketchy, so I think the first goal I would have
>>>>> would be to get to a real spec we can all agree on and discuss. A lot
>>>>> of
>>>>> the security stuff has a high human interaction element and needs to
>>>>> work
>>>>> in pretty different domains and different companies so getting this
>>>>> kind
>>>>>
>>>> of
>>>>
>>>>> review is important.
>>>>>
>>>>> -Jay
>>>>>
>>>>>
>>>>> On Tue, Jun 3, 2014 at 12:57 PM, Joe Stein <jo...@stealth.ly>
>>>>> wrote:
>>>>>
>>>>>  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.  This comes from what folks are doing "in the wild" to
>>>>>> workaround and implement security with Kafka as it is today and also
>>>>>>
>>>>> what I
>>>>>
>>>>>> have discovered from organizations about their blockers. It also picks
>>>>>>
>>>>> up
>>>>
>>>>> from the wiki (which I should have time to update later in the week
>>>>>>
>>>>> based
>>>>
>>>>> on the below and feedback from the thread).
>>>>>>
>>>>>> 1) Transport Layer Security (i.e. SSL)
>>>>>>
>>>>>> This also includes client authentication in addition to in-transit
>>>>>>
>>>>> security
>>>>>
>>>>>> layer.  This work has been picked up here
>>>>>> https://issues.apache.org/jira/browse/KAFKA-1477 and do appreciate
>>>>>> any
>>>>>> thoughts, comments, feedback, tomatoes, whatever for this patch.  It
>>>>>>
>>>>> is a
>>>>
>>>>> pickup from the fork of the work first done here
>>>>>> https://github.com/relango/kafka/tree/kafka_security.
>>>>>>
>>>>>> 2) Data encryption at rest.
>>>>>>
>>>>>> This is very important and something that can be facilitated within
>>>>>> the
>>>>>> wire protocol. It requires an additional map data structure for the
>>>>>> "encrypted [data encryption key]". With this map (either in your
>>>>>> object
>>>>>>
>>>>> or
>>>>>
>>>>>> in the wire protocol) you can store the dynamically generated
>>>>>> symmetric
>>>>>>
>>>>> key
>>>>>
>>>>>> (for each message) and then encrypt the data using that dynamically
>>>>>> generated key.  You then encrypt the encryption key using each public
>>>>>>
>>>>> key
>>>>
>>>>> for whom is expected to be able to decrypt the encryption key to then
>>>>>> decrypt the message.  For each public key encrypted symmetric key
>>>>>>
>>>>> (which
>>>>
>>>>> is
>>>>>
>>>>>> now the "encrypted [data encryption key]" along with which public key
>>>>>>
>>>>> it
>>>>
>>>>> was encrypted with for (so a map of [publicKey] =
>>>>>> encryptedDataEncryptionKey) as a chain.   Other patterns can be
>>>>>>
>>>>> implemented
>>>>>
>>>>>> but this is a pretty standard digital enveloping [0] pattern with only
>>>>>>
>>>>> 1
>>>>
>>>>> field added. Other patterns should be able to use that field to-do
>>>>>>
>>>>> their
>>>>
>>>>> implementation too.
>>>>>>
>>>>>> 3) Non-repudiation and long term non-repudiation.
>>>>>>
>>>>>> Non-repudiation is proving data hasn't changed.  This is often (if not
>>>>>> always) done with x509 public certificates (chained to a certificate
>>>>>> authority).
>>>>>>
>>>>>> Long term non-repudiation is what happens when the certificates of the
>>>>>> certificate authority are expired (or revoked) and everything ever
>>>>>>
>>>>> signed
>>>>
>>>>> (ever) with that certificate's public key then becomes "no longer
>>>>>>
>>>>> provable
>>>>>
>>>>>> as ever being authentic".  That is where RFC3126 [1] and RFC3161 [2]
>>>>>>
>>>>> come
>>>>
>>>>> in (or worm drives [hardware], etc).
>>>>>>
>>>>>> For either (or both) of these it is an operation of the encryptor to
>>>>>> sign/hash the data (with or without third party trusted timestap of
>>>>>> the
>>>>>> signing event) and encrypt that with their own private key and
>>>>>>
>>>>> distribute
>>>>
>>>>> the results (before and after encrypting if required) along with their
>>>>>> public key. This structure is a bit more complex but feasible, it is a
>>>>>>
>>>>> map
>>>>>
>>>>>> of digital signature formats and the chain of dig sig attestations.
>>>>>>
>>>>> The
>>>>
>>>>> map's key being the method (i.e. CRC32, PKCS7 [3], XmlDigSig [4]) and
>>>>>>
>>>>> then
>>>>>
>>>>>> a list of map where that key is "purpose" of signature (what your
>>>>>>
>>>>> attesting
>>>>>
>>>>>> too).  As a sibling field to the list another field for "the attester"
>>>>>>
>>>>> as
>>>>
>>>>> bytes (e.g. their PKCS12 [5] for the map of PKCS7 signatures).
>>>>>>
>>>>>> 4) Authorization
>>>>>>
>>>>>> We should have a policy of "404" for data, topics, partitions (etc) if
>>>>>> authenticated connections do not have access.  In "secure mode" any
>>>>>> non
>>>>>> authenticated connections should get a "404" type message on
>>>>>>
>>>>> everything.
>>>>
>>>>> Knowing "something is there" is a security risk in many uses cases.  So
>>>>>>
>>>>> if
>>>>>
>>>>>> you don't have access you don't even see it.  Baking "that" into Kafka
>>>>>> along with some interface for entitlement (access management) systems
>>>>>> (pretty standard) is all that I think needs to be done to the core
>>>>>>
>>>>> project.
>>>>>
>>>>>> I want to tackle item later in the year after summer after the other
>>>>>>
>>>>> three
>>>>>
>>>>>> are complete.
>>>>>>
>>>>>> I look forward to thoughts on this and anyone else interested in
>>>>>>
>>>>> working
>>>>
>>>>> with us on these items.
>>>>>>
>>>>>> [0]
>>>>>>
>>>>>>
>>>>>>
>>>>>  http://www.emc.com/emc-plus/rsa-labs/standards-
>>>> initiatives/what-is-a-digital-envelope.htm
>>>>
>>>>> [1] http://tools.ietf.org/html/rfc3126
>>>>>> [2] http://tools.ietf.org/html/rfc3161
>>>>>> [3]
>>>>>>
>>>>>>
>>>>>>
>>>>>  http://www.emc.com/emc-plus/rsa-labs/standards-initiatives/pkcs-7-
>>>> cryptographic-message-syntax-standar.htm
>>>>
>>>>> [4] http://en.wikipedia.org/wiki/XML_Signature
>>>>>> [5] http://en.wikipedia.org/wiki/PKCS_12
>>>>>>
>>>>>> /*******************************************
>>>>>> Joe Stein
>>>>>> Founder, Principal Consultant
>>>>>> Big Data Open Source Security LLC
>>>>>> http://www.stealth.ly
>>>>>> Twitter: @allthingshadoop <http://www.twitter.com/allthingshadoop>
>>>>>> ********************************************/
>>>>>>
>>>>>>
>>>>>
>>>>
>>
>

Re: [DISCUSS] Kafka Security Specific Features

Posted by Rob Withers <ro...@gmail.com>.
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 key....certs won't  
work, unless you write a CertAuthority to build multiple certs with  
the same keys.  Better seems to not use certs and wrap the encryption  
specification with an ACL capabilities for each group of access.

On Jun 6, 2014, at 11:43 AM, Rob Withers wrote:

> 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/WRITE/DELETE/LIST/SCAN).   
> Erights.org has the quitenscential (??) object capabilities model  
> and capnproto is impleemting this for C++.  I have a java  
> implementation at http://github.com/pauwau/pauwau but the master is  
> broken.  0.2 works, basically.  B asically a TLS connection with no  
> certificate server, it is peer to peer.  It has some advanced  
> features, but the lining of capabilities with authorization so that  
> you can only invoke correct services is extended to the secure user.
>
> Regarding non-repudiation, on disk, why not prepend a CRC?
>
> Regarding on-disk encryption, multiple users/groups may need to  
> access, with different capabilities.  Sounds like zookeeper needs to  
> store a cert for each class of access so that a group member can  
> access the decrypted data from disk.  Use cert-based async  
> decryption.  The only isue is storing the private key in zookeeper.   
> Perhaps some hash magic could be used.
>
> Thanks for kafka,
> Rob
>
> On Jun 5, 2014, at 3:01 PM, Jay Kreps wrote:
>
>> 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 encryption scheme? Is this actually  
>> needed?
>> Isn't a on-the-wire encryption when combined with mutual  
>> authentication and
>> permissions sufficient for most uses?
>>
>> On-disk encryption seems unnecessary because if an attacker can get  
>> root on
>> the kafka boxes it can potentially modify Kafka to do anything he  
>> or she
>> wants with data. So this seems to break any security model.
>>
>> I understand the problem of a large organization not really having a
>> trusted network and wanting to secure data transfer and limit and  
>> audit
>> data access. The uses for these other things I don't totally  
>> understand.
>>
>> Also it would be worth understanding the state of other messaging and
>> storage systems (Hadoop, dbs, etc). What features do they support.  
>> I think
>> there is a sense in which you don't have to run faster than the  
>> bear, but
>> only faster then your friends. :-)
>>
>> -Jay
>>
>>
>> On Wed, Jun 4, 2014 at 5:57 PM, Joe Stein <jo...@stealth.ly>  
>> wrote:
>>
>>> 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 <ja...@gmail.com>  
>>> 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  
>>>> 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 most the LinkedIn folks are working on the consumer right  
>>>> now I
>>> think
>>>> this would be a great project for any other interested people to  
>>>> take on.
>>>> There are some challenges in doing these things distributed but  
>>>> it can
>>> also
>>>> be a lot of fun.
>>>>
>>>> I think a good first step would be to get a written plan we can  
>>>> all agree
>>>> on for how things should work. Then we can break things down into  
>>>> chunks
>>>> that can be done independently while still aiming at a good end  
>>>> state.
>>>>
>>>> I had tried to write up some notes that summarized at least the  
>>>> thoughts
>>> I
>>>> had had on security:
>>>> https://cwiki.apache.org/confluence/display/KAFKA/Security
>>>>
>>>> What do you think of that?
>>>>
>>>> One assumption I had (which may be incorrect) is that although we  
>>>> want
>>> all
>>>> the things in your list, the two most pressing would be  
>>>> authentication
>>> and
>>>> authorization, and that was all that write up covered. You have  
>>>> more
>>>> experience in this domain, so I wonder how you would prioritize?
>>>>
>>>> Those notes are really sketchy, so I think the first goal I would  
>>>> have
>>>> would be to get to a real spec we can all agree on and discuss. A  
>>>> lot of
>>>> the security stuff has a high human interaction element and needs  
>>>> to work
>>>> in pretty different domains and different companies so getting  
>>>> this kind
>>> of
>>>> review is important.
>>>>
>>>> -Jay
>>>>
>>>>
>>>> On Tue, Jun 3, 2014 at 12:57 PM, Joe Stein <jo...@stealth.ly>  
>>>> wrote:
>>>>
>>>>> 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.  This comes from what folks are doing "in the  
>>>>> wild" to
>>>>> workaround and implement security with Kafka as it is today and  
>>>>> also
>>>> what I
>>>>> have discovered from organizations about their blockers. It also  
>>>>> picks
>>> up
>>>>> from the wiki (which I should have time to update later in the  
>>>>> week
>>> based
>>>>> on the below and feedback from the thread).
>>>>>
>>>>> 1) Transport Layer Security (i.e. SSL)
>>>>>
>>>>> This also includes client authentication in addition to in-transit
>>>> security
>>>>> layer.  This work has been picked up here
>>>>> https://issues.apache.org/jira/browse/KAFKA-1477 and do  
>>>>> appreciate any
>>>>> thoughts, comments, feedback, tomatoes, whatever for this  
>>>>> patch.  It
>>> is a
>>>>> pickup from the fork of the work first done here
>>>>> https://github.com/relango/kafka/tree/kafka_security.
>>>>>
>>>>> 2) Data encryption at rest.
>>>>>
>>>>> This is very important and something that can be facilitated  
>>>>> within the
>>>>> wire protocol. It requires an additional map data structure for  
>>>>> the
>>>>> "encrypted [data encryption key]". With this map (either in your  
>>>>> object
>>>> or
>>>>> in the wire protocol) you can store the dynamically generated  
>>>>> symmetric
>>>> key
>>>>> (for each message) and then encrypt the data using that  
>>>>> dynamically
>>>>> generated key.  You then encrypt the encryption key using each  
>>>>> public
>>> key
>>>>> for whom is expected to be able to decrypt the encryption key to  
>>>>> then
>>>>> decrypt the message.  For each public key encrypted symmetric key
>>> (which
>>>> is
>>>>> now the "encrypted [data encryption key]" along with which  
>>>>> public key
>>> it
>>>>> was encrypted with for (so a map of [publicKey] =
>>>>> encryptedDataEncryptionKey) as a chain.   Other patterns can be
>>>> implemented
>>>>> but this is a pretty standard digital enveloping [0] pattern  
>>>>> with only
>>> 1
>>>>> field added. Other patterns should be able to use that field to-do
>>> their
>>>>> implementation too.
>>>>>
>>>>> 3) Non-repudiation and long term non-repudiation.
>>>>>
>>>>> Non-repudiation is proving data hasn't changed.  This is often  
>>>>> (if not
>>>>> always) done with x509 public certificates (chained to a  
>>>>> certificate
>>>>> authority).
>>>>>
>>>>> Long term non-repudiation is what happens when the certificates  
>>>>> of the
>>>>> certificate authority are expired (or revoked) and everything ever
>>> signed
>>>>> (ever) with that certificate's public key then becomes "no longer
>>>> provable
>>>>> as ever being authentic".  That is where RFC3126 [1] and RFC3161  
>>>>> [2]
>>> come
>>>>> in (or worm drives [hardware], etc).
>>>>>
>>>>> For either (or both) of these it is an operation of the  
>>>>> encryptor to
>>>>> sign/hash the data (with or without third party trusted timestap  
>>>>> of the
>>>>> signing event) and encrypt that with their own private key and
>>> distribute
>>>>> the results (before and after encrypting if required) along with  
>>>>> their
>>>>> public key. This structure is a bit more complex but feasible,  
>>>>> it is a
>>>> map
>>>>> of digital signature formats and the chain of dig sig  
>>>>> attestations.
>>> The
>>>>> map's key being the method (i.e. CRC32, PKCS7 [3], XmlDigSig  
>>>>> [4]) and
>>>> then
>>>>> a list of map where that key is "purpose" of signature (what your
>>>> attesting
>>>>> too).  As a sibling field to the list another field for "the  
>>>>> attester"
>>> as
>>>>> bytes (e.g. their PKCS12 [5] for the map of PKCS7 signatures).
>>>>>
>>>>> 4) Authorization
>>>>>
>>>>> We should have a policy of "404" for data, topics, partitions  
>>>>> (etc) if
>>>>> authenticated connections do not have access.  In "secure mode"  
>>>>> any non
>>>>> authenticated connections should get a "404" type message on
>>> everything.
>>>>> Knowing "something is there" is a security risk in many uses  
>>>>> cases.  So
>>>> if
>>>>> you don't have access you don't even see it.  Baking "that" into  
>>>>> Kafka
>>>>> along with some interface for entitlement (access management)  
>>>>> systems
>>>>> (pretty standard) is all that I think needs to be done to the core
>>>> project.
>>>>> I want to tackle item later in the year after summer after the  
>>>>> other
>>>> three
>>>>> are complete.
>>>>>
>>>>> I look forward to thoughts on this and anyone else interested in
>>> working
>>>>> with us on these items.
>>>>>
>>>>> [0]
>>>>>
>>>>>
>>>>
>>> http://www.emc.com/emc-plus/rsa-labs/standards-initiatives/what-is-a-digital-envelope.htm
>>>>> [1] http://tools.ietf.org/html/rfc3126
>>>>> [2] http://tools.ietf.org/html/rfc3161
>>>>> [3]
>>>>>
>>>>>
>>>>
>>> http://www.emc.com/emc-plus/rsa-labs/standards-initiatives/pkcs-7-cryptographic-message-syntax-standar.htm
>>>>> [4] http://en.wikipedia.org/wiki/XML_Signature
>>>>> [5] http://en.wikipedia.org/wiki/PKCS_12
>>>>>
>>>>> /*******************************************
>>>>> Joe Stein
>>>>> Founder, Principal Consultant
>>>>> Big Data Open Source Security LLC
>>>>> http://www.stealth.ly
>>>>> Twitter: @allthingshadoop <http://www.twitter.com/allthingshadoop>
>>>>> ********************************************/
>>>>>
>>>>
>>>
>


Re: [DISCUSS] Kafka Security Specific Features

Posted by Rob Withers <ro...@gmail.com>.
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/WRITE/DELETE/LIST/SCAN).  Erights.org has the  
quitenscential (??) object capabilities model and capnproto is  
impleemting this for C++.  I have a java implementation at http://github.com/pauwau/pauwau 
  but the master is broken.  0.2 works, basically.  B asically a TLS  
connection with no certificate server, it is peer to peer.  It has  
some advanced features, but the lining of capabilities with  
authorization so that you can only invoke correct services is extended  
to the secure user.

Regarding non-repudiation, on disk, why not prepend a CRC?

Regarding on-disk encryption, multiple users/groups may need to  
access, with different capabilities.  Sounds like zookeeper needs to  
store a cert for each class of access so that a group member can  
access the decrypted data from disk.  Use cert-based async  
decryption.  The only isue is storing the private key in zookeeper.   
Perhaps some hash magic could be used.

Thanks for kafka,
Rob

On Jun 5, 2014, at 3:01 PM, Jay Kreps wrote:

> 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 encryption scheme? Is this actually  
> needed?
> Isn't a on-the-wire encryption when combined with mutual  
> authentication and
> permissions sufficient for most uses?
>
> On-disk encryption seems unnecessary because if an attacker can get  
> root on
> the kafka boxes it can potentially modify Kafka to do anything he or  
> she
> wants with data. So this seems to break any security model.
>
> I understand the problem of a large organization not really having a
> trusted network and wanting to secure data transfer and limit and  
> audit
> data access. The uses for these other things I don't totally  
> understand.
>
> Also it would be worth understanding the state of other messaging and
> storage systems (Hadoop, dbs, etc). What features do they support. I  
> think
> there is a sense in which you don't have to run faster than the  
> bear, but
> only faster then your friends. :-)
>
> -Jay
>
>
> On Wed, Jun 4, 2014 at 5:57 PM, Joe Stein <jo...@stealth.ly>  
> wrote:
>
>> 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 <ja...@gmail.com>  
>> 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  
>>> 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 most the LinkedIn folks are working on the consumer right  
>>> now I
>> think
>>> this would be a great project for any other interested people to  
>>> take on.
>>> There are some challenges in doing these things distributed but it  
>>> can
>> also
>>> be a lot of fun.
>>>
>>> I think a good first step would be to get a written plan we can  
>>> all agree
>>> on for how things should work. Then we can break things down into  
>>> chunks
>>> that can be done independently while still aiming at a good end  
>>> state.
>>>
>>> I had tried to write up some notes that summarized at least the  
>>> thoughts
>> I
>>> had had on security:
>>> https://cwiki.apache.org/confluence/display/KAFKA/Security
>>>
>>> What do you think of that?
>>>
>>> One assumption I had (which may be incorrect) is that although we  
>>> want
>> all
>>> the things in your list, the two most pressing would be  
>>> authentication
>> and
>>> authorization, and that was all that write up covered. You have more
>>> experience in this domain, so I wonder how you would prioritize?
>>>
>>> Those notes are really sketchy, so I think the first goal I would  
>>> have
>>> would be to get to a real spec we can all agree on and discuss. A  
>>> lot of
>>> the security stuff has a high human interaction element and needs  
>>> to work
>>> in pretty different domains and different companies so getting  
>>> this kind
>> of
>>> review is important.
>>>
>>> -Jay
>>>
>>>
>>> On Tue, Jun 3, 2014 at 12:57 PM, Joe Stein <jo...@stealth.ly>  
>>> wrote:
>>>
>>>> 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.  This comes from what folks are doing "in the wild"  
>>>> to
>>>> workaround and implement security with Kafka as it is today and  
>>>> also
>>> what I
>>>> have discovered from organizations about their blockers. It also  
>>>> picks
>> up
>>>> from the wiki (which I should have time to update later in the week
>> based
>>>> on the below and feedback from the thread).
>>>>
>>>> 1) Transport Layer Security (i.e. SSL)
>>>>
>>>> This also includes client authentication in addition to in-transit
>>> security
>>>> layer.  This work has been picked up here
>>>> https://issues.apache.org/jira/browse/KAFKA-1477 and do  
>>>> appreciate any
>>>> thoughts, comments, feedback, tomatoes, whatever for this patch.   
>>>> It
>> is a
>>>> pickup from the fork of the work first done here
>>>> https://github.com/relango/kafka/tree/kafka_security.
>>>>
>>>> 2) Data encryption at rest.
>>>>
>>>> This is very important and something that can be facilitated  
>>>> within the
>>>> wire protocol. It requires an additional map data structure for the
>>>> "encrypted [data encryption key]". With this map (either in your  
>>>> object
>>> or
>>>> in the wire protocol) you can store the dynamically generated  
>>>> symmetric
>>> key
>>>> (for each message) and then encrypt the data using that dynamically
>>>> generated key.  You then encrypt the encryption key using each  
>>>> public
>> key
>>>> for whom is expected to be able to decrypt the encryption key to  
>>>> then
>>>> decrypt the message.  For each public key encrypted symmetric key
>> (which
>>> is
>>>> now the "encrypted [data encryption key]" along with which public  
>>>> key
>> it
>>>> was encrypted with for (so a map of [publicKey] =
>>>> encryptedDataEncryptionKey) as a chain.   Other patterns can be
>>> implemented
>>>> but this is a pretty standard digital enveloping [0] pattern with  
>>>> only
>> 1
>>>> field added. Other patterns should be able to use that field to-do
>> their
>>>> implementation too.
>>>>
>>>> 3) Non-repudiation and long term non-repudiation.
>>>>
>>>> Non-repudiation is proving data hasn't changed.  This is often  
>>>> (if not
>>>> always) done with x509 public certificates (chained to a  
>>>> certificate
>>>> authority).
>>>>
>>>> Long term non-repudiation is what happens when the certificates  
>>>> of the
>>>> certificate authority are expired (or revoked) and everything ever
>> signed
>>>> (ever) with that certificate's public key then becomes "no longer
>>> provable
>>>> as ever being authentic".  That is where RFC3126 [1] and RFC3161  
>>>> [2]
>> come
>>>> in (or worm drives [hardware], etc).
>>>>
>>>> For either (or both) of these it is an operation of the encryptor  
>>>> to
>>>> sign/hash the data (with or without third party trusted timestap  
>>>> of the
>>>> signing event) and encrypt that with their own private key and
>> distribute
>>>> the results (before and after encrypting if required) along with  
>>>> their
>>>> public key. This structure is a bit more complex but feasible, it  
>>>> is a
>>> map
>>>> of digital signature formats and the chain of dig sig attestations.
>> The
>>>> map's key being the method (i.e. CRC32, PKCS7 [3], XmlDigSig [4])  
>>>> and
>>> then
>>>> a list of map where that key is "purpose" of signature (what your
>>> attesting
>>>> too).  As a sibling field to the list another field for "the  
>>>> attester"
>> as
>>>> bytes (e.g. their PKCS12 [5] for the map of PKCS7 signatures).
>>>>
>>>> 4) Authorization
>>>>
>>>> We should have a policy of "404" for data, topics, partitions  
>>>> (etc) if
>>>> authenticated connections do not have access.  In "secure mode"  
>>>> any non
>>>> authenticated connections should get a "404" type message on
>> everything.
>>>> Knowing "something is there" is a security risk in many uses  
>>>> cases.  So
>>> if
>>>> you don't have access you don't even see it.  Baking "that" into  
>>>> Kafka
>>>> along with some interface for entitlement (access management)  
>>>> systems
>>>> (pretty standard) is all that I think needs to be done to the core
>>> project.
>>>> I want to tackle item later in the year after summer after the  
>>>> other
>>> three
>>>> are complete.
>>>>
>>>> I look forward to thoughts on this and anyone else interested in
>> working
>>>> with us on these items.
>>>>
>>>> [0]
>>>>
>>>>
>>>
>> http://www.emc.com/emc-plus/rsa-labs/standards-initiatives/what-is-a-digital-envelope.htm
>>>> [1] http://tools.ietf.org/html/rfc3126
>>>> [2] http://tools.ietf.org/html/rfc3161
>>>> [3]
>>>>
>>>>
>>>
>> http://www.emc.com/emc-plus/rsa-labs/standards-initiatives/pkcs-7-cryptographic-message-syntax-standar.htm
>>>> [4] http://en.wikipedia.org/wiki/XML_Signature
>>>> [5] http://en.wikipedia.org/wiki/PKCS_12
>>>>
>>>> /*******************************************
>>>> Joe Stein
>>>> Founder, Principal Consultant
>>>> Big Data Open Source Security LLC
>>>> http://www.stealth.ly
>>>> Twitter: @allthingshadoop <http://www.twitter.com/allthingshadoop>
>>>> ********************************************/
>>>>
>>>
>>


Re: [DISCUSS] Kafka Security Specific Features

Posted by Todd Palino <tp...@linkedin.com.INVALID>.
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" <ja...@gmail.com> wrote:

>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 encryption scheme? Is this actually
>needed?
>Isn't a on-the-wire encryption when combined with mutual authentication
>and
>permissions sufficient for most uses?
>
>On-disk encryption seems unnecessary because if an attacker can get root
>on
>the kafka boxes it can potentially modify Kafka to do anything he or she
>wants with data. So this seems to break any security model.
>
>I understand the problem of a large organization not really having a
>trusted network and wanting to secure data transfer and limit and audit
>data access. The uses for these other things I don't totally understand.
>
>Also it would be worth understanding the state of other messaging and
>storage systems (Hadoop, dbs, etc). What features do they support. I think
>there is a sense in which you don't have to run faster than the bear, but
>only faster then your friends. :-)
>
>-Jay
>
>
>On Wed, Jun 4, 2014 at 5:57 PM, Joe Stein <jo...@stealth.ly> wrote:
>
>> 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 <ja...@gmail.com> 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 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 most the LinkedIn folks are working on the consumer right now I
>> think
>> > this would be a great project for any other interested people to take
>>on.
>> > There are some challenges in doing these things distributed but it can
>> also
>> > be a lot of fun.
>> >
>> > I think a good first step would be to get a written plan we can all
>>agree
>> > on for how things should work. Then we can break things down into
>>chunks
>> > that can be done independently while still aiming at a good end state.
>> >
>> > I had tried to write up some notes that summarized at least the
>>thoughts
>> I
>> > had had on security:
>> > https://cwiki.apache.org/confluence/display/KAFKA/Security
>> >
>> > What do you think of that?
>> >
>> > One assumption I had (which may be incorrect) is that although we want
>> all
>> > the things in your list, the two most pressing would be authentication
>> and
>> > authorization, and that was all that write up covered. You have more
>> > experience in this domain, so I wonder how you would prioritize?
>> >
>> > Those notes are really sketchy, so I think the first goal I would have
>> > would be to get to a real spec we can all agree on and discuss. A lot
>>of
>> > the security stuff has a high human interaction element and needs to
>>work
>> > in pretty different domains and different companies so getting this
>>kind
>> of
>> > review is important.
>> >
>> > -Jay
>> >
>> >
>> > On Tue, Jun 3, 2014 at 12:57 PM, Joe Stein <jo...@stealth.ly>
>>wrote:
>> >
>> > > 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.  This comes from what folks are doing "in the wild" to
>> > > workaround and implement security with Kafka as it is today and also
>> > what I
>> > > have discovered from organizations about their blockers. It also
>>picks
>> up
>> > > from the wiki (which I should have time to update later in the week
>> based
>> > > on the below and feedback from the thread).
>> > >
>> > > 1) Transport Layer Security (i.e. SSL)
>> > >
>> > > This also includes client authentication in addition to in-transit
>> > security
>> > > layer.  This work has been picked up here
>> > > https://issues.apache.org/jira/browse/KAFKA-1477 and do appreciate
>>any
>> > > thoughts, comments, feedback, tomatoes, whatever for this patch.  It
>> is a
>> > > pickup from the fork of the work first done here
>> > > https://github.com/relango/kafka/tree/kafka_security.
>> > >
>> > > 2) Data encryption at rest.
>> > >
>> > > This is very important and something that can be facilitated within
>>the
>> > > wire protocol. It requires an additional map data structure for the
>> > > "encrypted [data encryption key]". With this map (either in your
>>object
>> > or
>> > > in the wire protocol) you can store the dynamically generated
>>symmetric
>> > key
>> > > (for each message) and then encrypt the data using that dynamically
>> > > generated key.  You then encrypt the encryption key using each
>>public
>> key
>> > > for whom is expected to be able to decrypt the encryption key to
>>then
>> > > decrypt the message.  For each public key encrypted symmetric key
>> (which
>> > is
>> > > now the "encrypted [data encryption key]" along with which public
>>key
>> it
>> > > was encrypted with for (so a map of [publicKey] =
>> > > encryptedDataEncryptionKey) as a chain.   Other patterns can be
>> > implemented
>> > > but this is a pretty standard digital enveloping [0] pattern with
>>only
>> 1
>> > > field added. Other patterns should be able to use that field to-do
>> their
>> > > implementation too.
>> > >
>> > > 3) Non-repudiation and long term non-repudiation.
>> > >
>> > > Non-repudiation is proving data hasn't changed.  This is often (if
>>not
>> > > always) done with x509 public certificates (chained to a certificate
>> > > authority).
>> > >
>> > > Long term non-repudiation is what happens when the certificates of
>>the
>> > > certificate authority are expired (or revoked) and everything ever
>> signed
>> > > (ever) with that certificate's public key then becomes "no longer
>> > provable
>> > > as ever being authentic".  That is where RFC3126 [1] and RFC3161 [2]
>> come
>> > > in (or worm drives [hardware], etc).
>> > >
>> > > For either (or both) of these it is an operation of the encryptor to
>> > > sign/hash the data (with or without third party trusted timestap of
>>the
>> > > signing event) and encrypt that with their own private key and
>> distribute
>> > > the results (before and after encrypting if required) along with
>>their
>> > > public key. This structure is a bit more complex but feasible, it
>>is a
>> > map
>> > > of digital signature formats and the chain of dig sig attestations.
>>  The
>> > > map's key being the method (i.e. CRC32, PKCS7 [3], XmlDigSig [4])
>>and
>> > then
>> > > a list of map where that key is "purpose" of signature (what your
>> > attesting
>> > > too).  As a sibling field to the list another field for "the
>>attester"
>> as
>> > > bytes (e.g. their PKCS12 [5] for the map of PKCS7 signatures).
>> > >
>> > > 4) Authorization
>> > >
>> > > We should have a policy of "404" for data, topics, partitions (etc)
>>if
>> > > authenticated connections do not have access.  In "secure mode" any
>>non
>> > > authenticated connections should get a "404" type message on
>> everything.
>> > > Knowing "something is there" is a security risk in many uses cases.
>> So
>> > if
>> > > you don't have access you don't even see it.  Baking "that" into
>>Kafka
>> > > along with some interface for entitlement (access management)
>>systems
>> > > (pretty standard) is all that I think needs to be done to the core
>> > project.
>> > >  I want to tackle item later in the year after summer after the
>>other
>> > three
>> > > are complete.
>> > >
>> > > I look forward to thoughts on this and anyone else interested in
>> working
>> > > with us on these items.
>> > >
>> > > [0]
>> > >
>> > >
>> >
>> 
>>http://www.emc.com/emc-plus/rsa-labs/standards-initiatives/what-is-a-digi
>>tal-envelope.htm
>> > > [1] http://tools.ietf.org/html/rfc3126
>> > > [2] http://tools.ietf.org/html/rfc3161
>> > > [3]
>> > >
>> > >
>> >
>> 
>>http://www.emc.com/emc-plus/rsa-labs/standards-initiatives/pkcs-7-cryptog
>>raphic-message-syntax-standar.htm
>> > > [4] http://en.wikipedia.org/wiki/XML_Signature
>> > > [5] http://en.wikipedia.org/wiki/PKCS_12
>> > >
>> > > /*******************************************
>> > >  Joe Stein
>> > >  Founder, Principal Consultant
>> > >  Big Data Open Source Security LLC
>> > >  http://www.stealth.ly
>> > >  Twitter: @allthingshadoop <http://www.twitter.com/allthingshadoop>
>> > > ********************************************/
>> > >
>> >
>>


Re: [DISCUSS] Kafka Security Specific Features

Posted by Jay Kreps <ja...@gmail.com>.
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 encryption scheme? Is this actually needed?
Isn't a on-the-wire encryption when combined with mutual authentication and
permissions sufficient for most uses?

On-disk encryption seems unnecessary because if an attacker can get root on
the kafka boxes it can potentially modify Kafka to do anything he or she
wants with data. So this seems to break any security model.

I understand the problem of a large organization not really having a
trusted network and wanting to secure data transfer and limit and audit
data access. The uses for these other things I don't totally understand.

Also it would be worth understanding the state of other messaging and
storage systems (Hadoop, dbs, etc). What features do they support. I think
there is a sense in which you don't have to run faster than the bear, but
only faster then your friends. :-)

-Jay


On Wed, Jun 4, 2014 at 5:57 PM, Joe Stein <jo...@stealth.ly> wrote:

> 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 <ja...@gmail.com> 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 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 most the LinkedIn folks are working on the consumer right now I
> think
> > this would be a great project for any other interested people to take on.
> > There are some challenges in doing these things distributed but it can
> also
> > be a lot of fun.
> >
> > I think a good first step would be to get a written plan we can all agree
> > on for how things should work. Then we can break things down into chunks
> > that can be done independently while still aiming at a good end state.
> >
> > I had tried to write up some notes that summarized at least the thoughts
> I
> > had had on security:
> > https://cwiki.apache.org/confluence/display/KAFKA/Security
> >
> > What do you think of that?
> >
> > One assumption I had (which may be incorrect) is that although we want
> all
> > the things in your list, the two most pressing would be authentication
> and
> > authorization, and that was all that write up covered. You have more
> > experience in this domain, so I wonder how you would prioritize?
> >
> > Those notes are really sketchy, so I think the first goal I would have
> > would be to get to a real spec we can all agree on and discuss. A lot of
> > the security stuff has a high human interaction element and needs to work
> > in pretty different domains and different companies so getting this kind
> of
> > review is important.
> >
> > -Jay
> >
> >
> > On Tue, Jun 3, 2014 at 12:57 PM, Joe Stein <jo...@stealth.ly> wrote:
> >
> > > 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.  This comes from what folks are doing "in the wild" to
> > > workaround and implement security with Kafka as it is today and also
> > what I
> > > have discovered from organizations about their blockers. It also picks
> up
> > > from the wiki (which I should have time to update later in the week
> based
> > > on the below and feedback from the thread).
> > >
> > > 1) Transport Layer Security (i.e. SSL)
> > >
> > > This also includes client authentication in addition to in-transit
> > security
> > > layer.  This work has been picked up here
> > > https://issues.apache.org/jira/browse/KAFKA-1477 and do appreciate any
> > > thoughts, comments, feedback, tomatoes, whatever for this patch.  It
> is a
> > > pickup from the fork of the work first done here
> > > https://github.com/relango/kafka/tree/kafka_security.
> > >
> > > 2) Data encryption at rest.
> > >
> > > This is very important and something that can be facilitated within the
> > > wire protocol. It requires an additional map data structure for the
> > > "encrypted [data encryption key]". With this map (either in your object
> > or
> > > in the wire protocol) you can store the dynamically generated symmetric
> > key
> > > (for each message) and then encrypt the data using that dynamically
> > > generated key.  You then encrypt the encryption key using each public
> key
> > > for whom is expected to be able to decrypt the encryption key to then
> > > decrypt the message.  For each public key encrypted symmetric key
> (which
> > is
> > > now the "encrypted [data encryption key]" along with which public key
> it
> > > was encrypted with for (so a map of [publicKey] =
> > > encryptedDataEncryptionKey) as a chain.   Other patterns can be
> > implemented
> > > but this is a pretty standard digital enveloping [0] pattern with only
> 1
> > > field added. Other patterns should be able to use that field to-do
> their
> > > implementation too.
> > >
> > > 3) Non-repudiation and long term non-repudiation.
> > >
> > > Non-repudiation is proving data hasn't changed.  This is often (if not
> > > always) done with x509 public certificates (chained to a certificate
> > > authority).
> > >
> > > Long term non-repudiation is what happens when the certificates of the
> > > certificate authority are expired (or revoked) and everything ever
> signed
> > > (ever) with that certificate's public key then becomes "no longer
> > provable
> > > as ever being authentic".  That is where RFC3126 [1] and RFC3161 [2]
> come
> > > in (or worm drives [hardware], etc).
> > >
> > > For either (or both) of these it is an operation of the encryptor to
> > > sign/hash the data (with or without third party trusted timestap of the
> > > signing event) and encrypt that with their own private key and
> distribute
> > > the results (before and after encrypting if required) along with their
> > > public key. This structure is a bit more complex but feasible, it is a
> > map
> > > of digital signature formats and the chain of dig sig attestations.
>  The
> > > map's key being the method (i.e. CRC32, PKCS7 [3], XmlDigSig [4]) and
> > then
> > > a list of map where that key is "purpose" of signature (what your
> > attesting
> > > too).  As a sibling field to the list another field for "the attester"
> as
> > > bytes (e.g. their PKCS12 [5] for the map of PKCS7 signatures).
> > >
> > > 4) Authorization
> > >
> > > We should have a policy of "404" for data, topics, partitions (etc) if
> > > authenticated connections do not have access.  In "secure mode" any non
> > > authenticated connections should get a "404" type message on
> everything.
> > > Knowing "something is there" is a security risk in many uses cases.  So
> > if
> > > you don't have access you don't even see it.  Baking "that" into Kafka
> > > along with some interface for entitlement (access management) systems
> > > (pretty standard) is all that I think needs to be done to the core
> > project.
> > >  I want to tackle item later in the year after summer after the other
> > three
> > > are complete.
> > >
> > > I look forward to thoughts on this and anyone else interested in
> working
> > > with us on these items.
> > >
> > > [0]
> > >
> > >
> >
> http://www.emc.com/emc-plus/rsa-labs/standards-initiatives/what-is-a-digital-envelope.htm
> > > [1] http://tools.ietf.org/html/rfc3126
> > > [2] http://tools.ietf.org/html/rfc3161
> > > [3]
> > >
> > >
> >
> http://www.emc.com/emc-plus/rsa-labs/standards-initiatives/pkcs-7-cryptographic-message-syntax-standar.htm
> > > [4] http://en.wikipedia.org/wiki/XML_Signature
> > > [5] http://en.wikipedia.org/wiki/PKCS_12
> > >
> > > /*******************************************
> > >  Joe Stein
> > >  Founder, Principal Consultant
> > >  Big Data Open Source Security LLC
> > >  http://www.stealth.ly
> > >  Twitter: @allthingshadoop <http://www.twitter.com/allthingshadoop>
> > > ********************************************/
> > >
> >
>

Re: [DISCUSS] Kafka Security Specific Features

Posted by Joe Stein <jo...@stealth.ly>.
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 other edits I have for the page.

/*******************************************
 Joe Stein
 Founder, Principal Consultant
 Big Data Open Source Security LLC
 http://www.stealth.ly
 Twitter: @allthingshadoop <http://www.twitter.com/allthingshadoop>
********************************************/


On Thu, Jun 5, 2014 at 11:56 AM, Rajasekar Elango <re...@salesforce.com>
wrote:

> 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 have it in spec.
>
> I could not find a link to edit the page after login into wiki. Do I need
> any special permission to make edits?
>
> Thanks,
> Raja.
>
>
> On Wed, Jun 4, 2014 at 8:57 PM, Joe Stein <jo...@stealth.ly> wrote:
>
> > 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 <ja...@gmail.com> 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 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 most the LinkedIn folks are working on the consumer right now I
> > think
> > > this would be a great project for any other interested people to take
> on.
> > > There are some challenges in doing these things distributed but it can
> > also
> > > be a lot of fun.
> > >
> > > I think a good first step would be to get a written plan we can all
> agree
> > > on for how things should work. Then we can break things down into
> chunks
> > > that can be done independently while still aiming at a good end state.
> > >
> > > I had tried to write up some notes that summarized at least the
> thoughts
> > I
> > > had had on security:
> > > https://cwiki.apache.org/confluence/display/KAFKA/Security
> > >
> > > What do you think of that?
> > >
> > > One assumption I had (which may be incorrect) is that although we want
> > all
> > > the things in your list, the two most pressing would be authentication
> > and
> > > authorization, and that was all that write up covered. You have more
> > > experience in this domain, so I wonder how you would prioritize?
> > >
> > > Those notes are really sketchy, so I think the first goal I would have
> > > would be to get to a real spec we can all agree on and discuss. A lot
> of
> > > the security stuff has a high human interaction element and needs to
> work
> > > in pretty different domains and different companies so getting this
> kind
> > of
> > > review is important.
> > >
> > > -Jay
> > >
> > >
> > > On Tue, Jun 3, 2014 at 12:57 PM, Joe Stein <jo...@stealth.ly>
> wrote:
> > >
> > > > 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.  This comes from what folks are doing "in the wild" to
> > > > workaround and implement security with Kafka as it is today and also
> > > what I
> > > > have discovered from organizations about their blockers. It also
> picks
> > up
> > > > from the wiki (which I should have time to update later in the week
> > based
> > > > on the below and feedback from the thread).
> > > >
> > > > 1) Transport Layer Security (i.e. SSL)
> > > >
> > > > This also includes client authentication in addition to in-transit
> > > security
> > > > layer.  This work has been picked up here
> > > > https://issues.apache.org/jira/browse/KAFKA-1477 and do appreciate
> any
> > > > thoughts, comments, feedback, tomatoes, whatever for this patch.  It
> > is a
> > > > pickup from the fork of the work first done here
> > > > https://github.com/relango/kafka/tree/kafka_security.
> > > >
> > > > 2) Data encryption at rest.
> > > >
> > > > This is very important and something that can be facilitated within
> the
> > > > wire protocol. It requires an additional map data structure for the
> > > > "encrypted [data encryption key]". With this map (either in your
> object
> > > or
> > > > in the wire protocol) you can store the dynamically generated
> symmetric
> > > key
> > > > (for each message) and then encrypt the data using that dynamically
> > > > generated key.  You then encrypt the encryption key using each public
> > key
> > > > for whom is expected to be able to decrypt the encryption key to then
> > > > decrypt the message.  For each public key encrypted symmetric key
> > (which
> > > is
> > > > now the "encrypted [data encryption key]" along with which public key
> > it
> > > > was encrypted with for (so a map of [publicKey] =
> > > > encryptedDataEncryptionKey) as a chain.   Other patterns can be
> > > implemented
> > > > but this is a pretty standard digital enveloping [0] pattern with
> only
> > 1
> > > > field added. Other patterns should be able to use that field to-do
> > their
> > > > implementation too.
> > > >
> > > > 3) Non-repudiation and long term non-repudiation.
> > > >
> > > > Non-repudiation is proving data hasn't changed.  This is often (if
> not
> > > > always) done with x509 public certificates (chained to a certificate
> > > > authority).
> > > >
> > > > Long term non-repudiation is what happens when the certificates of
> the
> > > > certificate authority are expired (or revoked) and everything ever
> > signed
> > > > (ever) with that certificate's public key then becomes "no longer
> > > provable
> > > > as ever being authentic".  That is where RFC3126 [1] and RFC3161 [2]
> > come
> > > > in (or worm drives [hardware], etc).
> > > >
> > > > For either (or both) of these it is an operation of the encryptor to
> > > > sign/hash the data (with or without third party trusted timestap of
> the
> > > > signing event) and encrypt that with their own private key and
> > distribute
> > > > the results (before and after encrypting if required) along with
> their
> > > > public key. This structure is a bit more complex but feasible, it is
> a
> > > map
> > > > of digital signature formats and the chain of dig sig attestations.
> >  The
> > > > map's key being the method (i.e. CRC32, PKCS7 [3], XmlDigSig [4]) and
> > > then
> > > > a list of map where that key is "purpose" of signature (what your
> > > attesting
> > > > too).  As a sibling field to the list another field for "the
> attester"
> > as
> > > > bytes (e.g. their PKCS12 [5] for the map of PKCS7 signatures).
> > > >
> > > > 4) Authorization
> > > >
> > > > We should have a policy of "404" for data, topics, partitions (etc)
> if
> > > > authenticated connections do not have access.  In "secure mode" any
> non
> > > > authenticated connections should get a "404" type message on
> > everything.
> > > > Knowing "something is there" is a security risk in many uses cases.
>  So
> > > if
> > > > you don't have access you don't even see it.  Baking "that" into
> Kafka
> > > > along with some interface for entitlement (access management) systems
> > > > (pretty standard) is all that I think needs to be done to the core
> > > project.
> > > >  I want to tackle item later in the year after summer after the other
> > > three
> > > > are complete.
> > > >
> > > > I look forward to thoughts on this and anyone else interested in
> > working
> > > > with us on these items.
> > > >
> > > > [0]
> > > >
> > > >
> > >
> >
> http://www.emc.com/emc-plus/rsa-labs/standards-initiatives/what-is-a-digital-envelope.htm
> > > > [1] http://tools.ietf.org/html/rfc3126
> > > > [2] http://tools.ietf.org/html/rfc3161
> > > > [3]
> > > >
> > > >
> > >
> >
> http://www.emc.com/emc-plus/rsa-labs/standards-initiatives/pkcs-7-cryptographic-message-syntax-standar.htm
> > > > [4] http://en.wikipedia.org/wiki/XML_Signature
> > > > [5] http://en.wikipedia.org/wiki/PKCS_12
> > > >
> > > > /*******************************************
> > > >  Joe Stein
> > > >  Founder, Principal Consultant
> > > >  Big Data Open Source Security LLC
> > > >  http://www.stealth.ly
> > > >  Twitter: @allthingshadoop <http://www.twitter.com/allthingshadoop>
> > > > ********************************************/
> > > >
> > >
> >
>
>
>
> --
> Thanks,
> Raja.
>

Re: [DISCUSS] Kafka Security Specific Features

Posted by Joe Stein <jo...@stealth.ly>.
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 other edits I have for the page.

/*******************************************
 Joe Stein
 Founder, Principal Consultant
 Big Data Open Source Security LLC
 http://www.stealth.ly
 Twitter: @allthingshadoop <http://www.twitter.com/allthingshadoop>
********************************************/


On Thu, Jun 5, 2014 at 11:56 AM, Rajasekar Elango <re...@salesforce.com>
wrote:

> 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 have it in spec.
>
> I could not find a link to edit the page after login into wiki. Do I need
> any special permission to make edits?
>
> Thanks,
> Raja.
>
>
> On Wed, Jun 4, 2014 at 8:57 PM, Joe Stein <jo...@stealth.ly> wrote:
>
> > 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 <ja...@gmail.com> 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 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 most the LinkedIn folks are working on the consumer right now I
> > think
> > > this would be a great project for any other interested people to take
> on.
> > > There are some challenges in doing these things distributed but it can
> > also
> > > be a lot of fun.
> > >
> > > I think a good first step would be to get a written plan we can all
> agree
> > > on for how things should work. Then we can break things down into
> chunks
> > > that can be done independently while still aiming at a good end state.
> > >
> > > I had tried to write up some notes that summarized at least the
> thoughts
> > I
> > > had had on security:
> > > https://cwiki.apache.org/confluence/display/KAFKA/Security
> > >
> > > What do you think of that?
> > >
> > > One assumption I had (which may be incorrect) is that although we want
> > all
> > > the things in your list, the two most pressing would be authentication
> > and
> > > authorization, and that was all that write up covered. You have more
> > > experience in this domain, so I wonder how you would prioritize?
> > >
> > > Those notes are really sketchy, so I think the first goal I would have
> > > would be to get to a real spec we can all agree on and discuss. A lot
> of
> > > the security stuff has a high human interaction element and needs to
> work
> > > in pretty different domains and different companies so getting this
> kind
> > of
> > > review is important.
> > >
> > > -Jay
> > >
> > >
> > > On Tue, Jun 3, 2014 at 12:57 PM, Joe Stein <jo...@stealth.ly>
> wrote:
> > >
> > > > 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.  This comes from what folks are doing "in the wild" to
> > > > workaround and implement security with Kafka as it is today and also
> > > what I
> > > > have discovered from organizations about their blockers. It also
> picks
> > up
> > > > from the wiki (which I should have time to update later in the week
> > based
> > > > on the below and feedback from the thread).
> > > >
> > > > 1) Transport Layer Security (i.e. SSL)
> > > >
> > > > This also includes client authentication in addition to in-transit
> > > security
> > > > layer.  This work has been picked up here
> > > > https://issues.apache.org/jira/browse/KAFKA-1477 and do appreciate
> any
> > > > thoughts, comments, feedback, tomatoes, whatever for this patch.  It
> > is a
> > > > pickup from the fork of the work first done here
> > > > https://github.com/relango/kafka/tree/kafka_security.
> > > >
> > > > 2) Data encryption at rest.
> > > >
> > > > This is very important and something that can be facilitated within
> the
> > > > wire protocol. It requires an additional map data structure for the
> > > > "encrypted [data encryption key]". With this map (either in your
> object
> > > or
> > > > in the wire protocol) you can store the dynamically generated
> symmetric
> > > key
> > > > (for each message) and then encrypt the data using that dynamically
> > > > generated key.  You then encrypt the encryption key using each public
> > key
> > > > for whom is expected to be able to decrypt the encryption key to then
> > > > decrypt the message.  For each public key encrypted symmetric key
> > (which
> > > is
> > > > now the "encrypted [data encryption key]" along with which public key
> > it
> > > > was encrypted with for (so a map of [publicKey] =
> > > > encryptedDataEncryptionKey) as a chain.   Other patterns can be
> > > implemented
> > > > but this is a pretty standard digital enveloping [0] pattern with
> only
> > 1
> > > > field added. Other patterns should be able to use that field to-do
> > their
> > > > implementation too.
> > > >
> > > > 3) Non-repudiation and long term non-repudiation.
> > > >
> > > > Non-repudiation is proving data hasn't changed.  This is often (if
> not
> > > > always) done with x509 public certificates (chained to a certificate
> > > > authority).
> > > >
> > > > Long term non-repudiation is what happens when the certificates of
> the
> > > > certificate authority are expired (or revoked) and everything ever
> > signed
> > > > (ever) with that certificate's public key then becomes "no longer
> > > provable
> > > > as ever being authentic".  That is where RFC3126 [1] and RFC3161 [2]
> > come
> > > > in (or worm drives [hardware], etc).
> > > >
> > > > For either (or both) of these it is an operation of the encryptor to
> > > > sign/hash the data (with or without third party trusted timestap of
> the
> > > > signing event) and encrypt that with their own private key and
> > distribute
> > > > the results (before and after encrypting if required) along with
> their
> > > > public key. This structure is a bit more complex but feasible, it is
> a
> > > map
> > > > of digital signature formats and the chain of dig sig attestations.
> >  The
> > > > map's key being the method (i.e. CRC32, PKCS7 [3], XmlDigSig [4]) and
> > > then
> > > > a list of map where that key is "purpose" of signature (what your
> > > attesting
> > > > too).  As a sibling field to the list another field for "the
> attester"
> > as
> > > > bytes (e.g. their PKCS12 [5] for the map of PKCS7 signatures).
> > > >
> > > > 4) Authorization
> > > >
> > > > We should have a policy of "404" for data, topics, partitions (etc)
> if
> > > > authenticated connections do not have access.  In "secure mode" any
> non
> > > > authenticated connections should get a "404" type message on
> > everything.
> > > > Knowing "something is there" is a security risk in many uses cases.
>  So
> > > if
> > > > you don't have access you don't even see it.  Baking "that" into
> Kafka
> > > > along with some interface for entitlement (access management) systems
> > > > (pretty standard) is all that I think needs to be done to the core
> > > project.
> > > >  I want to tackle item later in the year after summer after the other
> > > three
> > > > are complete.
> > > >
> > > > I look forward to thoughts on this and anyone else interested in
> > working
> > > > with us on these items.
> > > >
> > > > [0]
> > > >
> > > >
> > >
> >
> http://www.emc.com/emc-plus/rsa-labs/standards-initiatives/what-is-a-digital-envelope.htm
> > > > [1] http://tools.ietf.org/html/rfc3126
> > > > [2] http://tools.ietf.org/html/rfc3161
> > > > [3]
> > > >
> > > >
> > >
> >
> http://www.emc.com/emc-plus/rsa-labs/standards-initiatives/pkcs-7-cryptographic-message-syntax-standar.htm
> > > > [4] http://en.wikipedia.org/wiki/XML_Signature
> > > > [5] http://en.wikipedia.org/wiki/PKCS_12
> > > >
> > > > /*******************************************
> > > >  Joe Stein
> > > >  Founder, Principal Consultant
> > > >  Big Data Open Source Security LLC
> > > >  http://www.stealth.ly
> > > >  Twitter: @allthingshadoop <http://www.twitter.com/allthingshadoop>
> > > > ********************************************/
> > > >
> > >
> >
>
>
>
> --
> Thanks,
> Raja.
>

Re: [DISCUSS] Kafka Security Specific Features

Posted by Rajasekar Elango <re...@salesforce.com>.
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 have it in spec.

I could not find a link to edit the page after login into wiki. Do I need
any special permission to make edits?

Thanks,
Raja.


On Wed, Jun 4, 2014 at 8:57 PM, Joe Stein <jo...@stealth.ly> wrote:

> 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 <ja...@gmail.com> 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 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 most the LinkedIn folks are working on the consumer right now I
> think
> > this would be a great project for any other interested people to take on.
> > There are some challenges in doing these things distributed but it can
> also
> > be a lot of fun.
> >
> > I think a good first step would be to get a written plan we can all agree
> > on for how things should work. Then we can break things down into chunks
> > that can be done independently while still aiming at a good end state.
> >
> > I had tried to write up some notes that summarized at least the thoughts
> I
> > had had on security:
> > https://cwiki.apache.org/confluence/display/KAFKA/Security
> >
> > What do you think of that?
> >
> > One assumption I had (which may be incorrect) is that although we want
> all
> > the things in your list, the two most pressing would be authentication
> and
> > authorization, and that was all that write up covered. You have more
> > experience in this domain, so I wonder how you would prioritize?
> >
> > Those notes are really sketchy, so I think the first goal I would have
> > would be to get to a real spec we can all agree on and discuss. A lot of
> > the security stuff has a high human interaction element and needs to work
> > in pretty different domains and different companies so getting this kind
> of
> > review is important.
> >
> > -Jay
> >
> >
> > On Tue, Jun 3, 2014 at 12:57 PM, Joe Stein <jo...@stealth.ly> wrote:
> >
> > > 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.  This comes from what folks are doing "in the wild" to
> > > workaround and implement security with Kafka as it is today and also
> > what I
> > > have discovered from organizations about their blockers. It also picks
> up
> > > from the wiki (which I should have time to update later in the week
> based
> > > on the below and feedback from the thread).
> > >
> > > 1) Transport Layer Security (i.e. SSL)
> > >
> > > This also includes client authentication in addition to in-transit
> > security
> > > layer.  This work has been picked up here
> > > https://issues.apache.org/jira/browse/KAFKA-1477 and do appreciate any
> > > thoughts, comments, feedback, tomatoes, whatever for this patch.  It
> is a
> > > pickup from the fork of the work first done here
> > > https://github.com/relango/kafka/tree/kafka_security.
> > >
> > > 2) Data encryption at rest.
> > >
> > > This is very important and something that can be facilitated within the
> > > wire protocol. It requires an additional map data structure for the
> > > "encrypted [data encryption key]". With this map (either in your object
> > or
> > > in the wire protocol) you can store the dynamically generated symmetric
> > key
> > > (for each message) and then encrypt the data using that dynamically
> > > generated key.  You then encrypt the encryption key using each public
> key
> > > for whom is expected to be able to decrypt the encryption key to then
> > > decrypt the message.  For each public key encrypted symmetric key
> (which
> > is
> > > now the "encrypted [data encryption key]" along with which public key
> it
> > > was encrypted with for (so a map of [publicKey] =
> > > encryptedDataEncryptionKey) as a chain.   Other patterns can be
> > implemented
> > > but this is a pretty standard digital enveloping [0] pattern with only
> 1
> > > field added. Other patterns should be able to use that field to-do
> their
> > > implementation too.
> > >
> > > 3) Non-repudiation and long term non-repudiation.
> > >
> > > Non-repudiation is proving data hasn't changed.  This is often (if not
> > > always) done with x509 public certificates (chained to a certificate
> > > authority).
> > >
> > > Long term non-repudiation is what happens when the certificates of the
> > > certificate authority are expired (or revoked) and everything ever
> signed
> > > (ever) with that certificate's public key then becomes "no longer
> > provable
> > > as ever being authentic".  That is where RFC3126 [1] and RFC3161 [2]
> come
> > > in (or worm drives [hardware], etc).
> > >
> > > For either (or both) of these it is an operation of the encryptor to
> > > sign/hash the data (with or without third party trusted timestap of the
> > > signing event) and encrypt that with their own private key and
> distribute
> > > the results (before and after encrypting if required) along with their
> > > public key. This structure is a bit more complex but feasible, it is a
> > map
> > > of digital signature formats and the chain of dig sig attestations.
>  The
> > > map's key being the method (i.e. CRC32, PKCS7 [3], XmlDigSig [4]) and
> > then
> > > a list of map where that key is "purpose" of signature (what your
> > attesting
> > > too).  As a sibling field to the list another field for "the attester"
> as
> > > bytes (e.g. their PKCS12 [5] for the map of PKCS7 signatures).
> > >
> > > 4) Authorization
> > >
> > > We should have a policy of "404" for data, topics, partitions (etc) if
> > > authenticated connections do not have access.  In "secure mode" any non
> > > authenticated connections should get a "404" type message on
> everything.
> > > Knowing "something is there" is a security risk in many uses cases.  So
> > if
> > > you don't have access you don't even see it.  Baking "that" into Kafka
> > > along with some interface for entitlement (access management) systems
> > > (pretty standard) is all that I think needs to be done to the core
> > project.
> > >  I want to tackle item later in the year after summer after the other
> > three
> > > are complete.
> > >
> > > I look forward to thoughts on this and anyone else interested in
> working
> > > with us on these items.
> > >
> > > [0]
> > >
> > >
> >
> http://www.emc.com/emc-plus/rsa-labs/standards-initiatives/what-is-a-digital-envelope.htm
> > > [1] http://tools.ietf.org/html/rfc3126
> > > [2] http://tools.ietf.org/html/rfc3161
> > > [3]
> > >
> > >
> >
> http://www.emc.com/emc-plus/rsa-labs/standards-initiatives/pkcs-7-cryptographic-message-syntax-standar.htm
> > > [4] http://en.wikipedia.org/wiki/XML_Signature
> > > [5] http://en.wikipedia.org/wiki/PKCS_12
> > >
> > > /*******************************************
> > >  Joe Stein
> > >  Founder, Principal Consultant
> > >  Big Data Open Source Security LLC
> > >  http://www.stealth.ly
> > >  Twitter: @allthingshadoop <http://www.twitter.com/allthingshadoop>
> > > ********************************************/
> > >
> >
>



-- 
Thanks,
Raja.

Re: [DISCUSS] Kafka Security Specific Features

Posted by Rajasekar Elango <re...@salesforce.com>.
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 have it in spec.

I could not find a link to edit the page after login into wiki. Do I need
any special permission to make edits?

Thanks,
Raja.


On Wed, Jun 4, 2014 at 8:57 PM, Joe Stein <jo...@stealth.ly> wrote:

> 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 <ja...@gmail.com> 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 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 most the LinkedIn folks are working on the consumer right now I
> think
> > this would be a great project for any other interested people to take on.
> > There are some challenges in doing these things distributed but it can
> also
> > be a lot of fun.
> >
> > I think a good first step would be to get a written plan we can all agree
> > on for how things should work. Then we can break things down into chunks
> > that can be done independently while still aiming at a good end state.
> >
> > I had tried to write up some notes that summarized at least the thoughts
> I
> > had had on security:
> > https://cwiki.apache.org/confluence/display/KAFKA/Security
> >
> > What do you think of that?
> >
> > One assumption I had (which may be incorrect) is that although we want
> all
> > the things in your list, the two most pressing would be authentication
> and
> > authorization, and that was all that write up covered. You have more
> > experience in this domain, so I wonder how you would prioritize?
> >
> > Those notes are really sketchy, so I think the first goal I would have
> > would be to get to a real spec we can all agree on and discuss. A lot of
> > the security stuff has a high human interaction element and needs to work
> > in pretty different domains and different companies so getting this kind
> of
> > review is important.
> >
> > -Jay
> >
> >
> > On Tue, Jun 3, 2014 at 12:57 PM, Joe Stein <jo...@stealth.ly> wrote:
> >
> > > 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.  This comes from what folks are doing "in the wild" to
> > > workaround and implement security with Kafka as it is today and also
> > what I
> > > have discovered from organizations about their blockers. It also picks
> up
> > > from the wiki (which I should have time to update later in the week
> based
> > > on the below and feedback from the thread).
> > >
> > > 1) Transport Layer Security (i.e. SSL)
> > >
> > > This also includes client authentication in addition to in-transit
> > security
> > > layer.  This work has been picked up here
> > > https://issues.apache.org/jira/browse/KAFKA-1477 and do appreciate any
> > > thoughts, comments, feedback, tomatoes, whatever for this patch.  It
> is a
> > > pickup from the fork of the work first done here
> > > https://github.com/relango/kafka/tree/kafka_security.
> > >
> > > 2) Data encryption at rest.
> > >
> > > This is very important and something that can be facilitated within the
> > > wire protocol. It requires an additional map data structure for the
> > > "encrypted [data encryption key]". With this map (either in your object
> > or
> > > in the wire protocol) you can store the dynamically generated symmetric
> > key
> > > (for each message) and then encrypt the data using that dynamically
> > > generated key.  You then encrypt the encryption key using each public
> key
> > > for whom is expected to be able to decrypt the encryption key to then
> > > decrypt the message.  For each public key encrypted symmetric key
> (which
> > is
> > > now the "encrypted [data encryption key]" along with which public key
> it
> > > was encrypted with for (so a map of [publicKey] =
> > > encryptedDataEncryptionKey) as a chain.   Other patterns can be
> > implemented
> > > but this is a pretty standard digital enveloping [0] pattern with only
> 1
> > > field added. Other patterns should be able to use that field to-do
> their
> > > implementation too.
> > >
> > > 3) Non-repudiation and long term non-repudiation.
> > >
> > > Non-repudiation is proving data hasn't changed.  This is often (if not
> > > always) done with x509 public certificates (chained to a certificate
> > > authority).
> > >
> > > Long term non-repudiation is what happens when the certificates of the
> > > certificate authority are expired (or revoked) and everything ever
> signed
> > > (ever) with that certificate's public key then becomes "no longer
> > provable
> > > as ever being authentic".  That is where RFC3126 [1] and RFC3161 [2]
> come
> > > in (or worm drives [hardware], etc).
> > >
> > > For either (or both) of these it is an operation of the encryptor to
> > > sign/hash the data (with or without third party trusted timestap of the
> > > signing event) and encrypt that with their own private key and
> distribute
> > > the results (before and after encrypting if required) along with their
> > > public key. This structure is a bit more complex but feasible, it is a
> > map
> > > of digital signature formats and the chain of dig sig attestations.
>  The
> > > map's key being the method (i.e. CRC32, PKCS7 [3], XmlDigSig [4]) and
> > then
> > > a list of map where that key is "purpose" of signature (what your
> > attesting
> > > too).  As a sibling field to the list another field for "the attester"
> as
> > > bytes (e.g. their PKCS12 [5] for the map of PKCS7 signatures).
> > >
> > > 4) Authorization
> > >
> > > We should have a policy of "404" for data, topics, partitions (etc) if
> > > authenticated connections do not have access.  In "secure mode" any non
> > > authenticated connections should get a "404" type message on
> everything.
> > > Knowing "something is there" is a security risk in many uses cases.  So
> > if
> > > you don't have access you don't even see it.  Baking "that" into Kafka
> > > along with some interface for entitlement (access management) systems
> > > (pretty standard) is all that I think needs to be done to the core
> > project.
> > >  I want to tackle item later in the year after summer after the other
> > three
> > > are complete.
> > >
> > > I look forward to thoughts on this and anyone else interested in
> working
> > > with us on these items.
> > >
> > > [0]
> > >
> > >
> >
> http://www.emc.com/emc-plus/rsa-labs/standards-initiatives/what-is-a-digital-envelope.htm
> > > [1] http://tools.ietf.org/html/rfc3126
> > > [2] http://tools.ietf.org/html/rfc3161
> > > [3]
> > >
> > >
> >
> http://www.emc.com/emc-plus/rsa-labs/standards-initiatives/pkcs-7-cryptographic-message-syntax-standar.htm
> > > [4] http://en.wikipedia.org/wiki/XML_Signature
> > > [5] http://en.wikipedia.org/wiki/PKCS_12
> > >
> > > /*******************************************
> > >  Joe Stein
> > >  Founder, Principal Consultant
> > >  Big Data Open Source Security LLC
> > >  http://www.stealth.ly
> > >  Twitter: @allthingshadoop <http://www.twitter.com/allthingshadoop>
> > > ********************************************/
> > >
> >
>



-- 
Thanks,
Raja.

Re: [DISCUSS] Kafka Security Specific Features

Posted by Jay Kreps <ja...@gmail.com>.
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 encryption scheme? Is this actually needed?
Isn't a on-the-wire encryption when combined with mutual authentication and
permissions sufficient for most uses?

On-disk encryption seems unnecessary because if an attacker can get root on
the kafka boxes it can potentially modify Kafka to do anything he or she
wants with data. So this seems to break any security model.

I understand the problem of a large organization not really having a
trusted network and wanting to secure data transfer and limit and audit
data access. The uses for these other things I don't totally understand.

Also it would be worth understanding the state of other messaging and
storage systems (Hadoop, dbs, etc). What features do they support. I think
there is a sense in which you don't have to run faster than the bear, but
only faster then your friends. :-)

-Jay


On Wed, Jun 4, 2014 at 5:57 PM, Joe Stein <jo...@stealth.ly> wrote:

> 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 <ja...@gmail.com> 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 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 most the LinkedIn folks are working on the consumer right now I
> think
> > this would be a great project for any other interested people to take on.
> > There are some challenges in doing these things distributed but it can
> also
> > be a lot of fun.
> >
> > I think a good first step would be to get a written plan we can all agree
> > on for how things should work. Then we can break things down into chunks
> > that can be done independently while still aiming at a good end state.
> >
> > I had tried to write up some notes that summarized at least the thoughts
> I
> > had had on security:
> > https://cwiki.apache.org/confluence/display/KAFKA/Security
> >
> > What do you think of that?
> >
> > One assumption I had (which may be incorrect) is that although we want
> all
> > the things in your list, the two most pressing would be authentication
> and
> > authorization, and that was all that write up covered. You have more
> > experience in this domain, so I wonder how you would prioritize?
> >
> > Those notes are really sketchy, so I think the first goal I would have
> > would be to get to a real spec we can all agree on and discuss. A lot of
> > the security stuff has a high human interaction element and needs to work
> > in pretty different domains and different companies so getting this kind
> of
> > review is important.
> >
> > -Jay
> >
> >
> > On Tue, Jun 3, 2014 at 12:57 PM, Joe Stein <jo...@stealth.ly> wrote:
> >
> > > 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.  This comes from what folks are doing "in the wild" to
> > > workaround and implement security with Kafka as it is today and also
> > what I
> > > have discovered from organizations about their blockers. It also picks
> up
> > > from the wiki (which I should have time to update later in the week
> based
> > > on the below and feedback from the thread).
> > >
> > > 1) Transport Layer Security (i.e. SSL)
> > >
> > > This also includes client authentication in addition to in-transit
> > security
> > > layer.  This work has been picked up here
> > > https://issues.apache.org/jira/browse/KAFKA-1477 and do appreciate any
> > > thoughts, comments, feedback, tomatoes, whatever for this patch.  It
> is a
> > > pickup from the fork of the work first done here
> > > https://github.com/relango/kafka/tree/kafka_security.
> > >
> > > 2) Data encryption at rest.
> > >
> > > This is very important and something that can be facilitated within the
> > > wire protocol. It requires an additional map data structure for the
> > > "encrypted [data encryption key]". With this map (either in your object
> > or
> > > in the wire protocol) you can store the dynamically generated symmetric
> > key
> > > (for each message) and then encrypt the data using that dynamically
> > > generated key.  You then encrypt the encryption key using each public
> key
> > > for whom is expected to be able to decrypt the encryption key to then
> > > decrypt the message.  For each public key encrypted symmetric key
> (which
> > is
> > > now the "encrypted [data encryption key]" along with which public key
> it
> > > was encrypted with for (so a map of [publicKey] =
> > > encryptedDataEncryptionKey) as a chain.   Other patterns can be
> > implemented
> > > but this is a pretty standard digital enveloping [0] pattern with only
> 1
> > > field added. Other patterns should be able to use that field to-do
> their
> > > implementation too.
> > >
> > > 3) Non-repudiation and long term non-repudiation.
> > >
> > > Non-repudiation is proving data hasn't changed.  This is often (if not
> > > always) done with x509 public certificates (chained to a certificate
> > > authority).
> > >
> > > Long term non-repudiation is what happens when the certificates of the
> > > certificate authority are expired (or revoked) and everything ever
> signed
> > > (ever) with that certificate's public key then becomes "no longer
> > provable
> > > as ever being authentic".  That is where RFC3126 [1] and RFC3161 [2]
> come
> > > in (or worm drives [hardware], etc).
> > >
> > > For either (or both) of these it is an operation of the encryptor to
> > > sign/hash the data (with or without third party trusted timestap of the
> > > signing event) and encrypt that with their own private key and
> distribute
> > > the results (before and after encrypting if required) along with their
> > > public key. This structure is a bit more complex but feasible, it is a
> > map
> > > of digital signature formats and the chain of dig sig attestations.
>  The
> > > map's key being the method (i.e. CRC32, PKCS7 [3], XmlDigSig [4]) and
> > then
> > > a list of map where that key is "purpose" of signature (what your
> > attesting
> > > too).  As a sibling field to the list another field for "the attester"
> as
> > > bytes (e.g. their PKCS12 [5] for the map of PKCS7 signatures).
> > >
> > > 4) Authorization
> > >
> > > We should have a policy of "404" for data, topics, partitions (etc) if
> > > authenticated connections do not have access.  In "secure mode" any non
> > > authenticated connections should get a "404" type message on
> everything.
> > > Knowing "something is there" is a security risk in many uses cases.  So
> > if
> > > you don't have access you don't even see it.  Baking "that" into Kafka
> > > along with some interface for entitlement (access management) systems
> > > (pretty standard) is all that I think needs to be done to the core
> > project.
> > >  I want to tackle item later in the year after summer after the other
> > three
> > > are complete.
> > >
> > > I look forward to thoughts on this and anyone else interested in
> working
> > > with us on these items.
> > >
> > > [0]
> > >
> > >
> >
> http://www.emc.com/emc-plus/rsa-labs/standards-initiatives/what-is-a-digital-envelope.htm
> > > [1] http://tools.ietf.org/html/rfc3126
> > > [2] http://tools.ietf.org/html/rfc3161
> > > [3]
> > >
> > >
> >
> http://www.emc.com/emc-plus/rsa-labs/standards-initiatives/pkcs-7-cryptographic-message-syntax-standar.htm
> > > [4] http://en.wikipedia.org/wiki/XML_Signature
> > > [5] http://en.wikipedia.org/wiki/PKCS_12
> > >
> > > /*******************************************
> > >  Joe Stein
> > >  Founder, Principal Consultant
> > >  Big Data Open Source Security LLC
> > >  http://www.stealth.ly
> > >  Twitter: @allthingshadoop <http://www.twitter.com/allthingshadoop>
> > > ********************************************/
> > >
> >
>

Re: [DISCUSS] Kafka Security Specific Features

Posted by Joe Stein <jo...@stealth.ly>.
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 <ja...@gmail.com> 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 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 most the LinkedIn folks are working on the consumer right now I think
> this would be a great project for any other interested people to take on.
> There are some challenges in doing these things distributed but it can also
> be a lot of fun.
>
> I think a good first step would be to get a written plan we can all agree
> on for how things should work. Then we can break things down into chunks
> that can be done independently while still aiming at a good end state.
>
> I had tried to write up some notes that summarized at least the thoughts I
> had had on security:
> https://cwiki.apache.org/confluence/display/KAFKA/Security
>
> What do you think of that?
>
> One assumption I had (which may be incorrect) is that although we want all
> the things in your list, the two most pressing would be authentication and
> authorization, and that was all that write up covered. You have more
> experience in this domain, so I wonder how you would prioritize?
>
> Those notes are really sketchy, so I think the first goal I would have
> would be to get to a real spec we can all agree on and discuss. A lot of
> the security stuff has a high human interaction element and needs to work
> in pretty different domains and different companies so getting this kind of
> review is important.
>
> -Jay
>
>
> On Tue, Jun 3, 2014 at 12:57 PM, Joe Stein <jo...@stealth.ly> wrote:
>
> > 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.  This comes from what folks are doing "in the wild" to
> > workaround and implement security with Kafka as it is today and also
> what I
> > have discovered from organizations about their blockers. It also picks up
> > from the wiki (which I should have time to update later in the week based
> > on the below and feedback from the thread).
> >
> > 1) Transport Layer Security (i.e. SSL)
> >
> > This also includes client authentication in addition to in-transit
> security
> > layer.  This work has been picked up here
> > https://issues.apache.org/jira/browse/KAFKA-1477 and do appreciate any
> > thoughts, comments, feedback, tomatoes, whatever for this patch.  It is a
> > pickup from the fork of the work first done here
> > https://github.com/relango/kafka/tree/kafka_security.
> >
> > 2) Data encryption at rest.
> >
> > This is very important and something that can be facilitated within the
> > wire protocol. It requires an additional map data structure for the
> > "encrypted [data encryption key]". With this map (either in your object
> or
> > in the wire protocol) you can store the dynamically generated symmetric
> key
> > (for each message) and then encrypt the data using that dynamically
> > generated key.  You then encrypt the encryption key using each public key
> > for whom is expected to be able to decrypt the encryption key to then
> > decrypt the message.  For each public key encrypted symmetric key (which
> is
> > now the "encrypted [data encryption key]" along with which public key it
> > was encrypted with for (so a map of [publicKey] =
> > encryptedDataEncryptionKey) as a chain.   Other patterns can be
> implemented
> > but this is a pretty standard digital enveloping [0] pattern with only 1
> > field added. Other patterns should be able to use that field to-do their
> > implementation too.
> >
> > 3) Non-repudiation and long term non-repudiation.
> >
> > Non-repudiation is proving data hasn't changed.  This is often (if not
> > always) done with x509 public certificates (chained to a certificate
> > authority).
> >
> > Long term non-repudiation is what happens when the certificates of the
> > certificate authority are expired (or revoked) and everything ever signed
> > (ever) with that certificate's public key then becomes "no longer
> provable
> > as ever being authentic".  That is where RFC3126 [1] and RFC3161 [2] come
> > in (or worm drives [hardware], etc).
> >
> > For either (or both) of these it is an operation of the encryptor to
> > sign/hash the data (with or without third party trusted timestap of the
> > signing event) and encrypt that with their own private key and distribute
> > the results (before and after encrypting if required) along with their
> > public key. This structure is a bit more complex but feasible, it is a
> map
> > of digital signature formats and the chain of dig sig attestations.  The
> > map's key being the method (i.e. CRC32, PKCS7 [3], XmlDigSig [4]) and
> then
> > a list of map where that key is "purpose" of signature (what your
> attesting
> > too).  As a sibling field to the list another field for "the attester" as
> > bytes (e.g. their PKCS12 [5] for the map of PKCS7 signatures).
> >
> > 4) Authorization
> >
> > We should have a policy of "404" for data, topics, partitions (etc) if
> > authenticated connections do not have access.  In "secure mode" any non
> > authenticated connections should get a "404" type message on everything.
> > Knowing "something is there" is a security risk in many uses cases.  So
> if
> > you don't have access you don't even see it.  Baking "that" into Kafka
> > along with some interface for entitlement (access management) systems
> > (pretty standard) is all that I think needs to be done to the core
> project.
> >  I want to tackle item later in the year after summer after the other
> three
> > are complete.
> >
> > I look forward to thoughts on this and anyone else interested in working
> > with us on these items.
> >
> > [0]
> >
> >
> http://www.emc.com/emc-plus/rsa-labs/standards-initiatives/what-is-a-digital-envelope.htm
> > [1] http://tools.ietf.org/html/rfc3126
> > [2] http://tools.ietf.org/html/rfc3161
> > [3]
> >
> >
> http://www.emc.com/emc-plus/rsa-labs/standards-initiatives/pkcs-7-cryptographic-message-syntax-standar.htm
> > [4] http://en.wikipedia.org/wiki/XML_Signature
> > [5] http://en.wikipedia.org/wiki/PKCS_12
> >
> > /*******************************************
> >  Joe Stein
> >  Founder, Principal Consultant
> >  Big Data Open Source Security LLC
> >  http://www.stealth.ly
> >  Twitter: @allthingshadoop <http://www.twitter.com/allthingshadoop>
> > ********************************************/
> >
>

Re: [DISCUSS] Kafka Security Specific Features

Posted by Joe Stein <jo...@stealth.ly>.
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 <ja...@gmail.com> 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 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 most the LinkedIn folks are working on the consumer right now I think
> this would be a great project for any other interested people to take on.
> There are some challenges in doing these things distributed but it can also
> be a lot of fun.
>
> I think a good first step would be to get a written plan we can all agree
> on for how things should work. Then we can break things down into chunks
> that can be done independently while still aiming at a good end state.
>
> I had tried to write up some notes that summarized at least the thoughts I
> had had on security:
> https://cwiki.apache.org/confluence/display/KAFKA/Security
>
> What do you think of that?
>
> One assumption I had (which may be incorrect) is that although we want all
> the things in your list, the two most pressing would be authentication and
> authorization, and that was all that write up covered. You have more
> experience in this domain, so I wonder how you would prioritize?
>
> Those notes are really sketchy, so I think the first goal I would have
> would be to get to a real spec we can all agree on and discuss. A lot of
> the security stuff has a high human interaction element and needs to work
> in pretty different domains and different companies so getting this kind of
> review is important.
>
> -Jay
>
>
> On Tue, Jun 3, 2014 at 12:57 PM, Joe Stein <jo...@stealth.ly> wrote:
>
> > 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.  This comes from what folks are doing "in the wild" to
> > workaround and implement security with Kafka as it is today and also
> what I
> > have discovered from organizations about their blockers. It also picks up
> > from the wiki (which I should have time to update later in the week based
> > on the below and feedback from the thread).
> >
> > 1) Transport Layer Security (i.e. SSL)
> >
> > This also includes client authentication in addition to in-transit
> security
> > layer.  This work has been picked up here
> > https://issues.apache.org/jira/browse/KAFKA-1477 and do appreciate any
> > thoughts, comments, feedback, tomatoes, whatever for this patch.  It is a
> > pickup from the fork of the work first done here
> > https://github.com/relango/kafka/tree/kafka_security.
> >
> > 2) Data encryption at rest.
> >
> > This is very important and something that can be facilitated within the
> > wire protocol. It requires an additional map data structure for the
> > "encrypted [data encryption key]". With this map (either in your object
> or
> > in the wire protocol) you can store the dynamically generated symmetric
> key
> > (for each message) and then encrypt the data using that dynamically
> > generated key.  You then encrypt the encryption key using each public key
> > for whom is expected to be able to decrypt the encryption key to then
> > decrypt the message.  For each public key encrypted symmetric key (which
> is
> > now the "encrypted [data encryption key]" along with which public key it
> > was encrypted with for (so a map of [publicKey] =
> > encryptedDataEncryptionKey) as a chain.   Other patterns can be
> implemented
> > but this is a pretty standard digital enveloping [0] pattern with only 1
> > field added. Other patterns should be able to use that field to-do their
> > implementation too.
> >
> > 3) Non-repudiation and long term non-repudiation.
> >
> > Non-repudiation is proving data hasn't changed.  This is often (if not
> > always) done with x509 public certificates (chained to a certificate
> > authority).
> >
> > Long term non-repudiation is what happens when the certificates of the
> > certificate authority are expired (or revoked) and everything ever signed
> > (ever) with that certificate's public key then becomes "no longer
> provable
> > as ever being authentic".  That is where RFC3126 [1] and RFC3161 [2] come
> > in (or worm drives [hardware], etc).
> >
> > For either (or both) of these it is an operation of the encryptor to
> > sign/hash the data (with or without third party trusted timestap of the
> > signing event) and encrypt that with their own private key and distribute
> > the results (before and after encrypting if required) along with their
> > public key. This structure is a bit more complex but feasible, it is a
> map
> > of digital signature formats and the chain of dig sig attestations.  The
> > map's key being the method (i.e. CRC32, PKCS7 [3], XmlDigSig [4]) and
> then
> > a list of map where that key is "purpose" of signature (what your
> attesting
> > too).  As a sibling field to the list another field for "the attester" as
> > bytes (e.g. their PKCS12 [5] for the map of PKCS7 signatures).
> >
> > 4) Authorization
> >
> > We should have a policy of "404" for data, topics, partitions (etc) if
> > authenticated connections do not have access.  In "secure mode" any non
> > authenticated connections should get a "404" type message on everything.
> > Knowing "something is there" is a security risk in many uses cases.  So
> if
> > you don't have access you don't even see it.  Baking "that" into Kafka
> > along with some interface for entitlement (access management) systems
> > (pretty standard) is all that I think needs to be done to the core
> project.
> >  I want to tackle item later in the year after summer after the other
> three
> > are complete.
> >
> > I look forward to thoughts on this and anyone else interested in working
> > with us on these items.
> >
> > [0]
> >
> >
> http://www.emc.com/emc-plus/rsa-labs/standards-initiatives/what-is-a-digital-envelope.htm
> > [1] http://tools.ietf.org/html/rfc3126
> > [2] http://tools.ietf.org/html/rfc3161
> > [3]
> >
> >
> http://www.emc.com/emc-plus/rsa-labs/standards-initiatives/pkcs-7-cryptographic-message-syntax-standar.htm
> > [4] http://en.wikipedia.org/wiki/XML_Signature
> > [5] http://en.wikipedia.org/wiki/PKCS_12
> >
> > /*******************************************
> >  Joe Stein
> >  Founder, Principal Consultant
> >  Big Data Open Source Security LLC
> >  http://www.stealth.ly
> >  Twitter: @allthingshadoop <http://www.twitter.com/allthingshadoop>
> > ********************************************/
> >
>

Re: [DISCUSS] Kafka Security Specific Features

Posted by Jay Kreps <ja...@gmail.com>.
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 most the LinkedIn folks are working on the consumer right now I think
this would be a great project for any other interested people to take on.
There are some challenges in doing these things distributed but it can also
be a lot of fun.

I think a good first step would be to get a written plan we can all agree
on for how things should work. Then we can break things down into chunks
that can be done independently while still aiming at a good end state.

I had tried to write up some notes that summarized at least the thoughts I
had had on security:
https://cwiki.apache.org/confluence/display/KAFKA/Security

What do you think of that?

One assumption I had (which may be incorrect) is that although we want all
the things in your list, the two most pressing would be authentication and
authorization, and that was all that write up covered. You have more
experience in this domain, so I wonder how you would prioritize?

Those notes are really sketchy, so I think the first goal I would have
would be to get to a real spec we can all agree on and discuss. A lot of
the security stuff has a high human interaction element and needs to work
in pretty different domains and different companies so getting this kind of
review is important.

-Jay


On Tue, Jun 3, 2014 at 12:57 PM, Joe Stein <jo...@stealth.ly> wrote:

> 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.  This comes from what folks are doing "in the wild" to
> workaround and implement security with Kafka as it is today and also what I
> have discovered from organizations about their blockers. It also picks up
> from the wiki (which I should have time to update later in the week based
> on the below and feedback from the thread).
>
> 1) Transport Layer Security (i.e. SSL)
>
> This also includes client authentication in addition to in-transit security
> layer.  This work has been picked up here
> https://issues.apache.org/jira/browse/KAFKA-1477 and do appreciate any
> thoughts, comments, feedback, tomatoes, whatever for this patch.  It is a
> pickup from the fork of the work first done here
> https://github.com/relango/kafka/tree/kafka_security.
>
> 2) Data encryption at rest.
>
> This is very important and something that can be facilitated within the
> wire protocol. It requires an additional map data structure for the
> "encrypted [data encryption key]". With this map (either in your object or
> in the wire protocol) you can store the dynamically generated symmetric key
> (for each message) and then encrypt the data using that dynamically
> generated key.  You then encrypt the encryption key using each public key
> for whom is expected to be able to decrypt the encryption key to then
> decrypt the message.  For each public key encrypted symmetric key (which is
> now the "encrypted [data encryption key]" along with which public key it
> was encrypted with for (so a map of [publicKey] =
> encryptedDataEncryptionKey) as a chain.   Other patterns can be implemented
> but this is a pretty standard digital enveloping [0] pattern with only 1
> field added. Other patterns should be able to use that field to-do their
> implementation too.
>
> 3) Non-repudiation and long term non-repudiation.
>
> Non-repudiation is proving data hasn't changed.  This is often (if not
> always) done with x509 public certificates (chained to a certificate
> authority).
>
> Long term non-repudiation is what happens when the certificates of the
> certificate authority are expired (or revoked) and everything ever signed
> (ever) with that certificate's public key then becomes "no longer provable
> as ever being authentic".  That is where RFC3126 [1] and RFC3161 [2] come
> in (or worm drives [hardware], etc).
>
> For either (or both) of these it is an operation of the encryptor to
> sign/hash the data (with or without third party trusted timestap of the
> signing event) and encrypt that with their own private key and distribute
> the results (before and after encrypting if required) along with their
> public key. This structure is a bit more complex but feasible, it is a map
> of digital signature formats and the chain of dig sig attestations.  The
> map's key being the method (i.e. CRC32, PKCS7 [3], XmlDigSig [4]) and then
> a list of map where that key is "purpose" of signature (what your attesting
> too).  As a sibling field to the list another field for "the attester" as
> bytes (e.g. their PKCS12 [5] for the map of PKCS7 signatures).
>
> 4) Authorization
>
> We should have a policy of "404" for data, topics, partitions (etc) if
> authenticated connections do not have access.  In "secure mode" any non
> authenticated connections should get a "404" type message on everything.
> Knowing "something is there" is a security risk in many uses cases.  So if
> you don't have access you don't even see it.  Baking "that" into Kafka
> along with some interface for entitlement (access management) systems
> (pretty standard) is all that I think needs to be done to the core project.
>  I want to tackle item later in the year after summer after the other three
> are complete.
>
> I look forward to thoughts on this and anyone else interested in working
> with us on these items.
>
> [0]
>
> http://www.emc.com/emc-plus/rsa-labs/standards-initiatives/what-is-a-digital-envelope.htm
> [1] http://tools.ietf.org/html/rfc3126
> [2] http://tools.ietf.org/html/rfc3161
> [3]
>
> http://www.emc.com/emc-plus/rsa-labs/standards-initiatives/pkcs-7-cryptographic-message-syntax-standar.htm
> [4] http://en.wikipedia.org/wiki/XML_Signature
> [5] http://en.wikipedia.org/wiki/PKCS_12
>
> /*******************************************
>  Joe Stein
>  Founder, Principal Consultant
>  Big Data Open Source Security LLC
>  http://www.stealth.ly
>  Twitter: @allthingshadoop <http://www.twitter.com/allthingshadoop>
> ********************************************/
>

Re: [DISCUSS] Kafka Security Specific Features

Posted by Joe Stein <jo...@stealth.ly>.
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 file or something with a list
entitlements (where name is the "label" and value is some class that gets
run with the ability to extend your own or integrate with some other
system).   e.g. you associate with a topic named "hasCreditCardDataInIt" a
label == "PCIDSS".  Your config would be "PCIDSS" = "CheckForPCIDSS" and
in CheckForPCIDSS code you could do functions like "verify the topic is
going over an encrypted channel return true else false", etc, whatever.

- Joe Stein


On Tue, Jun 3, 2014 at 6:12 PM, Todd Palino <tp...@linkedin.com.invalid>
wrote:

> 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, I may want radius, and someone else is going to
> want LDAP.
>
> Authorization is the other piece that¹s needed, and that could be
> internal. Since what you¹re authorizing (topic name, read or write, may
> rate limiting) is specific to the application, it may not make sense to
> modularize it.
>
> -Todd
>
> On 6/3/14, 1:03 PM, "Robert Rodgers" <rs...@gmail.com> wrote:
>
> >... 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 <jo...@stealth.ly> wrote:
> >
> >> 4) Authorization
> >>
> >> We should have a policy of "404" for data, topics, partitions (etc) if
> >> authenticated connections do not have access.  In "secure mode" any non
> >> authenticated connections should get a "404" type message on everything.
> >> Knowing "something is there" is a security risk in many uses cases.  So
> >>if
> >> you don't have access you don't even see it.  Baking "that" into Kafka
> >> along with some interface for entitlement (access management) systems
> >> (pretty standard) is all that I think needs to be done to the core
> >>project.
> >> I want to tackle item later in the year after summer after the other
> >>three
> >> are complete.
> >
>
>

Re: [DISCUSS] Kafka Security Specific Features

Posted by Joe Stein <jo...@stealth.ly>.
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 file or something with a list
entitlements (where name is the "label" and value is some class that gets
run with the ability to extend your own or integrate with some other
system).   e.g. you associate with a topic named "hasCreditCardDataInIt" a
label == "PCIDSS".  Your config would be "PCIDSS" = "CheckForPCIDSS" and
in CheckForPCIDSS code you could do functions like "verify the topic is
going over an encrypted channel return true else false", etc, whatever.

- Joe Stein


On Tue, Jun 3, 2014 at 6:12 PM, Todd Palino <tp...@linkedin.com.invalid>
wrote:

> 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, I may want radius, and someone else is going to
> want LDAP.
>
> Authorization is the other piece that¹s needed, and that could be
> internal. Since what you¹re authorizing (topic name, read or write, may
> rate limiting) is specific to the application, it may not make sense to
> modularize it.
>
> -Todd
>
> On 6/3/14, 1:03 PM, "Robert Rodgers" <rs...@gmail.com> wrote:
>
> >... 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 <jo...@stealth.ly> wrote:
> >
> >> 4) Authorization
> >>
> >> We should have a policy of "404" for data, topics, partitions (etc) if
> >> authenticated connections do not have access.  In "secure mode" any non
> >> authenticated connections should get a "404" type message on everything.
> >> Knowing "something is there" is a security risk in many uses cases.  So
> >>if
> >> you don't have access you don't even see it.  Baking "that" into Kafka
> >> along with some interface for entitlement (access management) systems
> >> (pretty standard) is all that I think needs to be done to the core
> >>project.
> >> I want to tackle item later in the year after summer after the other
> >>three
> >> are complete.
> >
>
>

Re: [DISCUSS] Kafka Security Specific Features

Posted by Todd Palino <tp...@linkedin.com.INVALID>.
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, I may want radius, and someone else is going to
want LDAP.

Authorization is the other piece that¹s needed, and that could be
internal. Since what you¹re authorizing (topic name, read or write, may
rate limiting) is specific to the application, it may not make sense to
modularize it.

-Todd

On 6/3/14, 1:03 PM, "Robert Rodgers" <rs...@gmail.com> wrote:

>... 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 <jo...@stealth.ly> wrote:
>
>> 4) Authorization
>> 
>> We should have a policy of "404" for data, topics, partitions (etc) if
>> authenticated connections do not have access.  In "secure mode" any non
>> authenticated connections should get a "404" type message on everything.
>> Knowing "something is there" is a security risk in many uses cases.  So
>>if
>> you don't have access you don't even see it.  Baking "that" into Kafka
>> along with some interface for entitlement (access management) systems
>> (pretty standard) is all that I think needs to be done to the core
>>project.
>> I want to tackle item later in the year after summer after the other
>>three
>> are complete.
>


Re: [DISCUSS] Kafka Security Specific Features

Posted by Todd Palino <tp...@linkedin.com.INVALID>.
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, I may want radius, and someone else is going to
want LDAP.

Authorization is the other piece that¹s needed, and that could be
internal. Since what you¹re authorizing (topic name, read or write, may
rate limiting) is specific to the application, it may not make sense to
modularize it.

-Todd

On 6/3/14, 1:03 PM, "Robert Rodgers" <rs...@gmail.com> wrote:

>... 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 <jo...@stealth.ly> wrote:
>
>> 4) Authorization
>> 
>> We should have a policy of "404" for data, topics, partitions (etc) if
>> authenticated connections do not have access.  In "secure mode" any non
>> authenticated connections should get a "404" type message on everything.
>> Knowing "something is there" is a security risk in many uses cases.  So
>>if
>> you don't have access you don't even see it.  Baking "that" into Kafka
>> along with some interface for entitlement (access management) systems
>> (pretty standard) is all that I think needs to be done to the core
>>project.
>> I want to tackle item later in the year after summer after the other
>>three
>> are complete.
>


Re: [DISCUSS] Kafka Security Specific Features

Posted by Robert Rodgers <rs...@gmail.com>.
... 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 <jo...@stealth.ly> wrote:

> 4) Authorization
> 
> We should have a policy of "404" for data, topics, partitions (etc) if
> authenticated connections do not have access.  In "secure mode" any non
> authenticated connections should get a "404" type message on everything.
> Knowing "something is there" is a security risk in many uses cases.  So if
> you don't have access you don't even see it.  Baking "that" into Kafka
> along with some interface for entitlement (access management) systems
> (pretty standard) is all that I think needs to be done to the core project.
> I want to tackle item later in the year after summer after the other three
> are complete.


Re: [DISCUSS] Kafka Security Specific Features

Posted by Robert Rodgers <rs...@gmail.com>.
... 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 <jo...@stealth.ly> wrote:

> 4) Authorization
> 
> We should have a policy of "404" for data, topics, partitions (etc) if
> authenticated connections do not have access.  In "secure mode" any non
> authenticated connections should get a "404" type message on everything.
> Knowing "something is there" is a security risk in many uses cases.  So if
> you don't have access you don't even see it.  Baking "that" into Kafka
> along with some interface for entitlement (access management) systems
> (pretty standard) is all that I think needs to be done to the core project.
> I want to tackle item later in the year after summer after the other three
> are complete.


Re: [DISCUSS] Kafka Security Specific Features

Posted by Jay Kreps <ja...@gmail.com>.
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 most the LinkedIn folks are working on the consumer right now I think
this would be a great project for any other interested people to take on.
There are some challenges in doing these things distributed but it can also
be a lot of fun.

I think a good first step would be to get a written plan we can all agree
on for how things should work. Then we can break things down into chunks
that can be done independently while still aiming at a good end state.

I had tried to write up some notes that summarized at least the thoughts I
had had on security:
https://cwiki.apache.org/confluence/display/KAFKA/Security

What do you think of that?

One assumption I had (which may be incorrect) is that although we want all
the things in your list, the two most pressing would be authentication and
authorization, and that was all that write up covered. You have more
experience in this domain, so I wonder how you would prioritize?

Those notes are really sketchy, so I think the first goal I would have
would be to get to a real spec we can all agree on and discuss. A lot of
the security stuff has a high human interaction element and needs to work
in pretty different domains and different companies so getting this kind of
review is important.

-Jay


On Tue, Jun 3, 2014 at 12:57 PM, Joe Stein <jo...@stealth.ly> wrote:

> 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.  This comes from what folks are doing "in the wild" to
> workaround and implement security with Kafka as it is today and also what I
> have discovered from organizations about their blockers. It also picks up
> from the wiki (which I should have time to update later in the week based
> on the below and feedback from the thread).
>
> 1) Transport Layer Security (i.e. SSL)
>
> This also includes client authentication in addition to in-transit security
> layer.  This work has been picked up here
> https://issues.apache.org/jira/browse/KAFKA-1477 and do appreciate any
> thoughts, comments, feedback, tomatoes, whatever for this patch.  It is a
> pickup from the fork of the work first done here
> https://github.com/relango/kafka/tree/kafka_security.
>
> 2) Data encryption at rest.
>
> This is very important and something that can be facilitated within the
> wire protocol. It requires an additional map data structure for the
> "encrypted [data encryption key]". With this map (either in your object or
> in the wire protocol) you can store the dynamically generated symmetric key
> (for each message) and then encrypt the data using that dynamically
> generated key.  You then encrypt the encryption key using each public key
> for whom is expected to be able to decrypt the encryption key to then
> decrypt the message.  For each public key encrypted symmetric key (which is
> now the "encrypted [data encryption key]" along with which public key it
> was encrypted with for (so a map of [publicKey] =
> encryptedDataEncryptionKey) as a chain.   Other patterns can be implemented
> but this is a pretty standard digital enveloping [0] pattern with only 1
> field added. Other patterns should be able to use that field to-do their
> implementation too.
>
> 3) Non-repudiation and long term non-repudiation.
>
> Non-repudiation is proving data hasn't changed.  This is often (if not
> always) done with x509 public certificates (chained to a certificate
> authority).
>
> Long term non-repudiation is what happens when the certificates of the
> certificate authority are expired (or revoked) and everything ever signed
> (ever) with that certificate's public key then becomes "no longer provable
> as ever being authentic".  That is where RFC3126 [1] and RFC3161 [2] come
> in (or worm drives [hardware], etc).
>
> For either (or both) of these it is an operation of the encryptor to
> sign/hash the data (with or without third party trusted timestap of the
> signing event) and encrypt that with their own private key and distribute
> the results (before and after encrypting if required) along with their
> public key. This structure is a bit more complex but feasible, it is a map
> of digital signature formats and the chain of dig sig attestations.  The
> map's key being the method (i.e. CRC32, PKCS7 [3], XmlDigSig [4]) and then
> a list of map where that key is "purpose" of signature (what your attesting
> too).  As a sibling field to the list another field for "the attester" as
> bytes (e.g. their PKCS12 [5] for the map of PKCS7 signatures).
>
> 4) Authorization
>
> We should have a policy of "404" for data, topics, partitions (etc) if
> authenticated connections do not have access.  In "secure mode" any non
> authenticated connections should get a "404" type message on everything.
> Knowing "something is there" is a security risk in many uses cases.  So if
> you don't have access you don't even see it.  Baking "that" into Kafka
> along with some interface for entitlement (access management) systems
> (pretty standard) is all that I think needs to be done to the core project.
>  I want to tackle item later in the year after summer after the other three
> are complete.
>
> I look forward to thoughts on this and anyone else interested in working
> with us on these items.
>
> [0]
>
> http://www.emc.com/emc-plus/rsa-labs/standards-initiatives/what-is-a-digital-envelope.htm
> [1] http://tools.ietf.org/html/rfc3126
> [2] http://tools.ietf.org/html/rfc3161
> [3]
>
> http://www.emc.com/emc-plus/rsa-labs/standards-initiatives/pkcs-7-cryptographic-message-syntax-standar.htm
> [4] http://en.wikipedia.org/wiki/XML_Signature
> [5] http://en.wikipedia.org/wiki/PKCS_12
>
> /*******************************************
>  Joe Stein
>  Founder, Principal Consultant
>  Big Data Open Source Security LLC
>  http://www.stealth.ly
>  Twitter: @allthingshadoop <http://www.twitter.com/allthingshadoop>
> ********************************************/
>

Re: [DISCUSS] Kafka Security Specific Features

Posted by Rajasekar Elango <re...@salesforce.com>.
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 <jo...@stealth.ly> 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 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 is better to spend the time to get it right for everyone.  We are going
> to have it in the 0.9 release (possibly also with authorization, encryption
> and more of the security features too) then.
>
> What we will do is keep the patch rebased against trunk and then then 0.8.2
> branch (once we get to that point) so that folks can apply it to the 0.8.2
> release and do a build from src.  When we get to that I can create a write
> or something if folks find problems doing it.
>
> /*******************************************
>  Joe Stein
>  Founder, Principal Consultant
>  Big Data Open Source Security LLC
>  http://www.stealth.ly
>  Twitter: @allthingshadoop <http://www.twitter.com/allthingshadoop>
> ********************************************/
>
>
> On Wed, Jul 30, 2014 at 7:10 PM, Calvin Lei <ck...@gmail.com> wrote:
>
> > 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 <
> relango@salesforce.com
> > >
> > 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://github.com/stealthly/kafka/tree/v0.8.2_KAFKA-1477
> .
> > >
> > > Thanks,
> > > Raja.
> > >
> > >
> > > On Tue, Jul 29, 2014 at 10:54 PM, Calvin Lei <ck...@gmail.com> wrote:
> > >
> > > > Raja,
> > > >    Which Kafka version is your security enhancement based on?
> > > >
> > > > thanks,
> > > > Cal
> > > >
> > > >
> > > > On Wed, Jul 23, 2014 at 5:01 PM, Chris Neal <cw...@gmail.com>
> 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
> > > > > console_producer.sh, it is slightly different, which is also
> slightly
> > > > > different than the scala code for the ConsoleProducer. :)
> > > > >
> > > > > When I changed this:
> > > > >
> > > > > bin/kafka-console-producer.sh --broker-list n5:9092:true --topic
> test
> > > > >
> > > > > to this:
> > > > >
> > > > > bin/kafka-console-producer.sh --broker-list n5:9092 --secure
> > > > > --client.security.file config/client.security.properties --topic
> test
> > > > >
> > > > > I was able to push messages to the topic, although I got a WARN
> about
> > > the
> > > > > property "topic" not being valid, even though it is required.
> > > > >
> > > > > Also, the Producer reported this warning to me:
> > > > >
> > > > > [2014-07-23 20:45:24,509] WARN Attempt to reinitialize auth context
> > > > > (kafka.network.security.SecureAuth$)
> > > > >
> > > > > and the broker gave me this:
> > > > > [2014-07-23 20:45:24,114] INFO begin ssl handshake for
> > > > > n5.example.com/192.168.1.144:48817//192.168.1.144:9092
> > > > > (kafka.network.security.SSLSocketChannel)
> > > > > [2014-07-23 20:45:24,374] INFO finished ssl handshake for
> > > > > n5.example.com/192.168.1.144:48817//192.168.1.144:9092
> > > > > (kafka.network.security.SSLSocketChannel)
> > > > > [2014-07-23 20:45:24,493] INFO Closing socket connection to
> > > > > n5.example.com/192.168.1.144. (kafka.network.Processor)
> > > > > [2014-07-23 20:45:24,555] INFO begin ssl handshake for
> > > > > n5.example.com/192.168.1.144:48818//192.168.1.144:9092
> > > > > (kafka.network.security.SSLSocketChannel)
> > > > > [2014-07-23 20:45:24,566] INFO finished ssl handshake for
> > > > > n5.example.com/192.168.1.144:48818//192.168.1.144:9092
> > > > > (kafka.network.security.SSLSocketChannel)
> > > > >
> > > > > It's like it did the SSL piece twice :)
> > > > >
> > > > > Subsequent puts to the topic did not exhibit this behavior though:
> > > > >
> > > > > root@n5[937]:~/kafka_2.10-0-8-2-0.1.0.0>
> > bin/kafka-console-producer.sh
> > > > > --broker-list n5:9092 --secure --client.security.file
> > > > > config/client.security.properties --topic test
> > > > > [2014-07-23 20:45:17,530] WARN Property topic is not valid
> > > > > (kafka.utils.VerifiableProperties)
> > > > > 1
> > > > > [2014-07-23 20:45:24,509] WARN Attempt to reinitialize auth context
> > > > > (kafka.network.security.SecureAuth$)
> > > > > 2
> > > > > 3
> > > > > 4
> > > > >
> > > > > Consuming worked with these options:
> > > > >
> > > > > root@n5[918]:~/kafka_2.10-0-8-2-0.1.0.0>
> > bin/kafka-console-consumer.sh
> > > > > --topic test --zookeeper n5:2181 --from-beginning
> > > --security.config.file
> > > > > config/client.security.properties
> > > > > 1
> > > > > 2
> > > > > 3
> > > > > 4
> > > > > ^CConsumed 5 messages
> > > > >
> > > > > I hope that helps!
> > > > > Chris
> > > > >
> > > > >
> > > > > On Tue, Jul 22, 2014 at 2:10 PM, Pramod Deshmukh <
> dpramodv@gmail.com
> > >
> > > > > wrote:
> > > > >
> > > > > > 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 not valid
> > > > > > (kafka.utils.VerifiableProperties)
> > > > > >
> > > > > > Hare Krishna
> > > > > >
> > > > > > [2014-07-18 13:12:45,256] WARN Fetching topic metadata with
> > > correlation
> > > > > id
> > > > > > 0 for topics [Set(secureTopic)] from broker
> > > > > > [id:0,host:localhost,port:9092,secure:true] failed
> > > > > > (kafka.client.ClientUtils$)
> > > > > >
> > > > > > java.io.EOFException: Received -1 when reading from channel,
> socket
> > > has
> > > > > > likely been closed.
> > > > > >
> > > > > > at kafka.utils.Utils$.read(Utils.scala:381)
> > > > > >
> > > > > > at
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.network.BoundedByteBufferReceive.readFrom(BoundedByteBufferReceive.scala:67)
> > > > > >
> > > > > > at
> > kafka.network.Receive$class.readCompletely(Transmission.scala:56)
> > > > > >
> > > > > > at
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.network.BoundedByteBufferReceive.readCompletely(BoundedByteBufferReceive.scala:29)
> > > > > >
> > > > > > at
> kafka.network.BlockingChannel.receive(BlockingChannel.scala:102)
> > > > > >
> > > > > > at
> kafka.producer.SyncProducer.liftedTree1$1(SyncProducer.scala:79)
> > > > > >
> > > > > > at
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.producer.SyncProducer.kafka$producer$SyncProducer$$doSend(SyncProducer.scala:76)
> > > > > >
> > > > > > at kafka.producer.SyncProducer.send(SyncProducer.scala:117)
> > > > > >
> > > > > > at
> > kafka.client.ClientUtils$.fetchTopicMetadata(ClientUtils.scala:58)
> > > > > >
> > > > > > at
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.producer.BrokerPartitionInfo.updateInfo(BrokerPartitionInfo.scala:82)
> > > > > >
> > > > > > at
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.producer.async.DefaultEventHandler$$anonfun$handle$1.apply$mcV$sp(DefaultEventHandler.scala:67)
> > > > > >
> > > > > > at kafka.utils.Utils$.swallow(Utils.scala:172)
> > > > > >
> > > > > > at kafka.utils.Logging$class.swallowError(Logging.scala:106)
> > > > > >
> > > > > > at kafka.utils.Utils$.swallowError(Utils.scala:45)
> > > > > >
> > > > > > at
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.producer.async.DefaultEventHandler.handle(DefaultEventHandler.scala:67)
> > > > > >
> > > > > > at
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.producer.async.ProducerSendThread.tryToHandle(ProducerSendThread.scala:104)
> > > > > >
> > > > > > at
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.producer.async.ProducerSendThread$$anonfun$processEvents$3.apply(ProducerSendThread.scala:87)
> > > > > >
> > > > > > at
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.producer.async.ProducerSendThread$$anonfun$processEvents$3.apply(ProducerSendThread.scala:67)
> > > > > >
> > > > > > at scala.collection.immutable.Stream.foreach(Stream.scala:526)
> > > > > >
> > > > > > at
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.producer.async.ProducerSendThread.processEvents(ProducerSendThread.scala:66)
> > > > > >
> > > > > > at
> > > > >
> > >
> kafka.producer.async.ProducerSendThread.run(ProducerSendThread.scala:44)
> > > > > >
> > > > > >
> > > > > > On Fri, Jul 18, 2014 at 1:20 PM, Pramod Deshmukh <
> > dpramodv@gmail.com
> > > >
> > > > > > wrote:
> > > > > >
> > > > > > > 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 topic "secureTopic".
> > > > > > >
> > > > > > > pdeshmukh$ bin/kafka-topics.sh --list --zookeeper
> localhost:2181
> > > > > > >
> > > > > > > secure.test
> > > > > > >
> > > > > > > secureTopic
> > > > > > >
> > > > > > > >> Run producer, tried both localhost:9092:true and
> > localhost:9092
> > > > > > >
> > > > > > > pdeshmukh$ bin/kafka-console-producer.sh --broker-list
> > > > > > localhost:9092:true
> > > > > > > --topic secureTopic
> > > > > > >
> > > > > > > [2014-07-18 13:12:29,817] WARN Property topic is not valid
> > > > > > > (kafka.utils.VerifiableProperties)
> > > > > > >
> > > > > > > Hare Krishna
> > > > > > >
> > > > > > > [2014-07-18 13:12:45,256] WARN Fetching topic metadata with
> > > > correlation
> > > > > > id
> > > > > > > 0 for topics [Set(secureTopic)] from broker
> > > > > > > [id:0,host:localhost,port:9092,secure:true] failed
> > > > > > > (kafka.client.ClientUtils$)
> > > > > > >
> > > > > > > java.io.EOFException: Received -1 when reading from channel,
> > socket
> > > > has
> > > > > > > likely been closed.
> > > > > > >
> > > > > > > at kafka.utils.Utils$.read(Utils.scala:381)
> > > > > > >
> > > > > > > at
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.network.BoundedByteBufferReceive.readFrom(BoundedByteBufferReceive.scala:67)
> > > > > > >
> > > > > > > at
> > > kafka.network.Receive$class.readCompletely(Transmission.scala:56)
> > > > > > >
> > > > > > > at
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.network.BoundedByteBufferReceive.readCompletely(BoundedByteBufferReceive.scala:29)
> > > > > > >
> > > > > > > at
> > kafka.network.BlockingChannel.receive(BlockingChannel.scala:102)
> > > > > > >
> > > > > > > at
> > kafka.producer.SyncProducer.liftedTree1$1(SyncProducer.scala:79)
> > > > > > >
> > > > > > > at
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.producer.SyncProducer.kafka$producer$SyncProducer$$doSend(SyncProducer.scala:76)
> > > > > > >
> > > > > > > at kafka.producer.SyncProducer.send(SyncProducer.scala:117)
> > > > > > >
> > > > > > > at
> > > kafka.client.ClientUtils$.fetchTopicMetadata(ClientUtils.scala:58)
> > > > > > >
> > > > > > > at
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.producer.BrokerPartitionInfo.updateInfo(BrokerPartitionInfo.scala:82)
> > > > > > >
> > > > > > > at
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.producer.async.DefaultEventHandler$$anonfun$handle$1.apply$mcV$sp(DefaultEventHandler.scala:67)
> > > > > > >
> > > > > > > at kafka.utils.Utils$.swallow(Utils.scala:172)
> > > > > > >
> > > > > > > at kafka.utils.Logging$class.swallowError(Logging.scala:106)
> > > > > > >
> > > > > > > at kafka.utils.Utils$.swallowError(Utils.scala:45)
> > > > > > >
> > > > > > > at
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.producer.async.DefaultEventHandler.handle(DefaultEventHandler.scala:67)
> > > > > > >
> > > > > > > at
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.producer.async.ProducerSendThread.tryToHandle(ProducerSendThread.scala:104)
> > > > > > >
> > > > > > > at
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.producer.async.ProducerSendThread$$anonfun$processEvents$3.apply(ProducerSendThread.scala:87)
> > > > > > >
> > > > > > > at
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.producer.async.ProducerSendThread$$anonfun$processEvents$3.apply(ProducerSendThread.scala:67)
> > > > > > >
> > > > > > > at scala.collection.immutable.Stream.foreach(Stream.scala:526)
> > > > > > >
> > > > > > > at
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.producer.async.ProducerSendThread.processEvents(ProducerSendThread.scala:66)
> > > > > > >
> > > > > > > at
> > > > > >
> > > >
> > kafka.producer.async.ProducerSendThread.run(ProducerSendThread.scala:44)
> > > > > > >
> > > > > > > [2014-07-18 13:12:45,258] ERROR fetching topic metadata for
> > topics
> > > > > > > [Set(secureTopic)] from broker
> > > > > > > [ArrayBuffer(id:0,host:localhost,port:9092,secure:true)] failed
> > > > > > > (kafka.utils.Utils$)
> > > > > > >
> > > > > > > kafka.common.KafkaException: fetching topic metadata for topics
> > > > > > > [Set(secureTopic)] from broker
> > > > > > > [ArrayBuffer(id:0,host:localhost,port:9092,secure:true)] failed
> > > > > > >
> > > > > > > at
> > > kafka.client.ClientUtils$.fetchTopicMetadata(ClientUtils.scala:72)
> > > > > > >
> > > > > > > at
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.producer.BrokerPartitionInfo.updateInfo(BrokerPartitionInfo.scala:82)
> > > > > > >
> > > > > > > at
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.producer.async.DefaultEventHandler$$anonfun$handle$1.apply$mcV$sp(DefaultEventHandler.scala:67)
> > > > > > >
> > > > > > > at kafka.utils.Utils$.swallow(Utils.scala:172)
> > > > > > >
> > > > > > > at kafka.utils.Logging$class.swallowError(Logging.scala:106)
> > > > > > >
> > > > > > > at kafka.utils.Utils$.swallowError(Utils.scala:45)
> > > > > > >
> > > > > > > at
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.producer.async.DefaultEventHandler.handle(DefaultEventHandler.scala:67)
> > > > > > >
> > > > > > > at
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.producer.async.ProducerSendThread.tryToHandle(ProducerSendThread.scala:104)
> > > > > > >
> > > > > > > at
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.producer.async.ProducerSendThread$$anonfun$processEvents$3.apply(ProducerSendThread.scala:87)
> > > > > > >
> > > > > > > at
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.producer.async.ProducerSendThread$$anonfun$processEvents$3.apply(ProducerSendThread.scala:67)
> > > > > > >
> > > > > > > at scala.collection.immutable.Stream.foreach(Stream.scala:526)
> > > > > > >
> > > > > > > at
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.producer.async.ProducerSendThread.processEvents(ProducerSendThread.scala:66)
> > > > > > >
> > > > > > > at
> > > > > >
> > > >
> > kafka.producer.async.ProducerSendThread.run(ProducerSendThread.scala:44)
> > > > > > >
> > > > > > > Caused by: java.io.EOFException: Received -1 when reading from
> > > > channel,
> > > > > > > socket has likely been closed.
> > > > > > >
> > > > > > > at kafka.utils.Utils$.read(Utils.scala:381)
> > > > > > >
> > > > > > > at
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.network.BoundedByteBufferReceive.readFrom(BoundedByteBufferReceive.scala:67)
> > > > > > >
> > > > > > > at
> > > kafka.network.Receive$class.readCompletely(Transmission.scala:56)
> > > > > > >
> > > > > > > at
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.network.BoundedByteBufferReceive.readCompletely(BoundedByteBufferReceive.scala:29)
> > > > > > >
> > > > > > > at
> > kafka.network.BlockingChannel.receive(BlockingChannel.scala:102)
> > > > > > >
> > > > > > > at
> > kafka.producer.SyncProducer.liftedTree1$1(SyncProducer.scala:79)
> > > > > > >
> > > > > > > at
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.producer.SyncProducer.kafka$producer$SyncProducer$$doSend(SyncProducer.scala:76)
> > > > > > >
> > > > > > > at kafka.producer.SyncProducer.send(SyncProducer.scala:117)
> > > > > > >
> > > > > > > at
> > > kafka.client.ClientUtils$.fetchTopicMetadata(ClientUtils.scala:58)
> > > > > > >
> > > > > > > ... 12 more
> > > > > > > [2014-07-18 13:12:45,337] WARN Fetching topic metadata with
> > > > correlation
> > > > > > id
> > > > > > > 1 for topics [Set(secureTopic)] from broker
> > > > > > > [id:0,host:localhost,port:9092,secure:true] failed
> > > > > > > (kafka.client.ClientUtils$)
> > > > > > >
> > > > > > > 2014-07-18 13:12:46,282] ERROR Failed to send requests for
> topics
> > > > > > > secureTopic with correlation ids in [0,8]
> > > > > > > (kafka.producer.async.DefaultEventHandler)
> > > > > > >
> > > > > > > [2014-07-18 13:12:46,283] ERROR Error in handling batch of 1
> > events
> > > > > > > (kafka.producer.async.ProducerSendThread)
> > > > > > >
> > > > > > > kafka.common.FailedToSendMessageException: Failed to send
> > messages
> > > > > after
> > > > > > 3
> > > > > > > tries.
> > > > > > >
> > > > > > > at
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.producer.async.DefaultEventHandler.handle(DefaultEventHandler.scala:90)
> > > > > > >
> > > > > > > at
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.producer.async.ProducerSendThread.tryToHandle(ProducerSendThread.scala:104)
> > > > > > >
> > > > > > > at
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.producer.async.ProducerSendThread$$anonfun$processEvents$3.apply(ProducerSendThread.scala:87)
> > > > > > >
> > > > > > > at
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.producer.async.ProducerSendThread$$anonfun$processEvents$3.apply(ProducerSendThread.scala:67)
> > > > > > >
> > > > > > > at scala.collection.immutable.Stream.foreach(Stream.scala:526)
> > > > > > >
> > > > > > > at
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.producer.async.ProducerSendThread.processEvents(ProducerSendThread.scala:66)
> > > > > > >
> > > > > > > at
> > > > > >
> > > >
> > kafka.producer.async.ProducerSendThread.run(ProducerSendThread.scala:44)
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > > On Fri, Jul 18, 2014 at 11:56 AM, Joe Stein <
> > joe.stein@stealth.ly>
> > > > > > wrote:
> > > > > > >
> > > > > > >> 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://www.stealth.ly
> > > > > > >>  Twitter: @allthingshadoop <
> > > http://www.twitter.com/allthingshadoop>
> > > > > > >> ********************************************/
> > > > > > >>
> > > > > > >>
> > > > > > >> On Fri, Jul 18, 2014 at 10:24 AM, Pramod Deshmukh <
> > > > dpramodv@gmail.com
> > > > > >
> > > > > > >> wrote:
> > > > > > >>
> > > > > > >> > 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.OutOfMemoryError: Java heap space
> > > > > > >> >
> > > > > > >> > *broker log*
> > > > > > >> >
> > > > > > >> > INFO begin ssl handshake for localhost/
> > > > > > 127.0.0.1:50199//127.0.0.1:9092
> > > > > > >> >
> > > > > > >> >
> > > > > > >> >
> > > > > > >> >
> > > > > > >> >
> > > > > > >> > On Thu, Jul 17, 2014 at 6:07 PM, Pramod Deshmukh <
> > > > > dpramodv@gmail.com>
> > > > > > >> > wrote:
> > > > > > >> >
> > > > > > >> > > 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 <
> > > > > > >> > relango@salesforce.com>
> > > > > > >> > > wrote:
> > > > > > >> > >
> > > > > > >> > >> 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 <
> > > > > > dpramodv@gmail.com
> > > > > > >> >
> > > > > > >> > >> 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.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 config/server.properties*
> > > > > > >> > >> >
> > > > > > >> > >> >
> > > > > > >> > >> >
> > > > > > >> > >> > *broker.log:*
> > > > > > >> > >> >
> > > > > > >> > >> > [2014-07-17 15:34:46,281] INFO zookeeper state changed
> > > > > > >> (SyncConnected)
> > > > > > >> > >> > (org.I0Itec.zkclient.ZkClient)
> > > > > > >> > >> >
> > > > > > >> > >> > [2014-07-17 15:34:46,523] INFO Loading log
> > 'secure.test-0'
> > > > > > >> > >> > (kafka.log.LogManager)
> > > > > > >> > >> >
> > > > > > >> > >> > [2014-07-17 15:34:46,558] INFO Recovering unflushed
> > > segment 0
> > > > > in
> > > > > > >> log
> > > > > > >> > >> > secure.test-0. (kafka.log.Log)
> > > > > > >> > >> >
> > > > > > >> > >> > [2014-07-17 15:34:46,571] INFO Completed load of log
> > > > > > secure.test-0
> > > > > > >> > with
> > > > > > >> > >> log
> > > > > > >> > >> > end offset 0 (kafka.log.Log)
> > > > > > >> > >> >
> > > > > > >> > >> > [2014-07-17 15:34:46,582] INFO Starting log cleanup
> with
> > a
> > > > > period
> > > > > > >> of
> > > > > > >> > >> 60000
> > > > > > >> > >> > ms. (kafka.log.LogManager)
> > > > > > >> > >> >
> > > > > > >> > >> > [2014-07-17 15:34:46,587] INFO Starting log flusher
> with
> > a
> > > > > > default
> > > > > > >> > >> period
> > > > > > >> > >> > of 9223372036854775807 ms. (kafka.log.LogManager)
> > > > > > >> > >> >
> > > > > > >> > >> > [2014-07-17 15:34:46,614] INFO Initializing secure
> > > > > authentication
> > > > > > >> > >> > (kafka.network.security.SecureAuth$)
> > > > > > >> > >> >
> > > > > > >> > >> > [2014-07-17 15:34:46,678] INFO Secure authentication
> > > > > > initialization
> > > > > > >> > has
> > > > > > >> > >> > been successfully completed
> > > > > (kafka.network.security.SecureAuth$)
> > > > > > >> > >> >
> > > > > > >> > >> > [2014-07-17 15:34:46,691] INFO Awaiting socket
> > connections
> > > on
> > > > > > >> > >> 0.0.0.0:9092
> > > > > > >> > >> > .
> > > > > > >> > >> > (kafka.network.Acceptor)
> > > > > > >> > >> >
> > > > > > >> > >> > [2014-07-17 15:34:46,692] INFO [Socket Server on Broker
> > 0],
> > > > > > Started
> > > > > > >> > >> > (kafka.network.SocketServer)
> > > > > > >> > >> >
> > > > > > >> > >> > [2014-07-17 15:34:46,794] INFO Will not load MX4J,
> > > > > mx4j-tools.jar
> > > > > > >> is
> > > > > > >> > >> not in
> > > > > > >> > >> > the classpath (kafka.utils.Mx4jLoader$)
> > > > > > >> > >> >
> > > > > > >> > >> > [2014-07-17 15:34:46,837] INFO 0 successfully elected
> as
> > > > leader
> > > > > > >> > >> > (kafka.server.ZookeeperLeaderElector)
> > > > > > >> > >> >
> > > > > > >> > >> > [2014-07-17 15:34:47,057] INFO Registered broker 0 at
> > path
> > > > > > >> > >> /brokers/ids/0
> > > > > > >> > >> > with address 10.1.100.130:9092. (kafka.utils.ZkUtils$)
> > > > > > >> > >> >
> > > > > > >> > >> > [2014-07-17 15:34:47,059] INFO New leader is 0
> > > > > > >> > >> >
> > (kafka.server.ZookeeperLeaderElector$LeaderChangeListener)
> > > > > > >> > >> >
> > > > > > >> > >> > *[2014-07-17 15:34:47,068] INFO [Kafka Server 0],
> started
> > > > > > >> > >> > (kafka.server.KafkaServer)*
> > > > > > >> > >> >
> > > > > > >> > >> > *[2014-07-17 15:34:47,383] INFO begin ssl handshake for
> > > > > > >> > >> > /10.1.100.130:9092//10.1.100.130:51685
> > > > > > >> > >> > <http://10.1.100.130:9092//10.1.100.130:51685>
> > > > > > >> > >> > (kafka.network.security.SSLSocketChannel)*
> > > > > > >> > >> >
> > > > > > >> > >> > *[2014-07-17 15:34:47,392] INFO begin ssl handshake for
> > > > > > >> > >> > 10.1.100.130/10.1.100.130:51685//10.1.100.130:9092
> > > > > > >> > >> > <
> > http://10.1.100.130/10.1.100.130:51685//10.1.100.130:9092
> > > >
> > > > > > >> > >> > (kafka.network.security.SSLSocketChannel)*
> > > > > > >> > >> >
> > > > > > >> > >> > *[2014-07-17 15:34:47,465] INFO finished ssl handshake
> > for
> > > > > > >> > >> > 10.1.100.130/10.1.100.130:51685//10.1.100.130:9092
> > > > > > >> > >> > <
> > http://10.1.100.130/10.1.100.130:51685//10.1.100.130:9092
> > > >
> > > > > > >> > >> > (kafka.network.security.SSLSocketChannel)*
> > > > > > >> > >> >
> > > > > > >> > >> > *[2014-07-17 15:34:47,465] INFO finished ssl handshake
> > for
> > > > > > >> > >> > /10.1.100.130:9092//10.1.100.130:51685
> > > > > > >> > >> > <http://10.1.100.130:9092//10.1.100.130:51685>
> > > > > > >> > >> > (kafka.network.security.SSLSocketChannel)*
> > > > > > >> > >> >
> > > > > > >> > >> > *[2014-07-17 15:34:47,617] INFO [ReplicaFetcherManager
> on
> > > > > broker
> > > > > > 0]
> > > > > > >> > >> Removed
> > > > > > >> > >> > fetcher for partitions
> > >  (kafka.server.ReplicaFetcherManager)*
> > > > > > >> > >> >
> > > > > > >> > >> > *[2014-07-17 15:34:47,627] INFO [ReplicaFetcherManager
> on
> > > > > broker
> > > > > > 0]
> > > > > > >> > >> Added
> > > > > > >> > >> > fetcher for partitions List()
> > > > > > (kafka.server.ReplicaFetcherManager)*
> > > > > > >> > >> >
> > > > > > >> > >> > *[2014-07-17 15:34:47,656] INFO [ReplicaFetcherManager
> on
> > > > > broker
> > > > > > 0]
> > > > > > >> > >> Removed
> > > > > > >> > >> > fetcher for partitions [secure.test,0]
> > > > > > >> > >> > (kafka.server.ReplicaFetcherManager)*
> > > > > > >> > >> >
> > > > > > >> > >> > [2014-07-17 15:37:15,970] INFO begin ssl handshake for
> > > > > > >> > >> > 10.1.100.130/10.1.100.130:51689//10.1.100.130:9092
> > > > > > >> > >> > (kafka.network.security.SSLSocketChannel)
> > > > > > >> > >> >
> > > > > > >> > >> > [2014-07-17 15:37:16,075] INFO begin ssl handshake for
> > > > > > >> > >> > 10.1.100.130/10.1.100.130:51690//10.1.100.130:9092
> > > > > > >> > >> > (kafka.network.security.SSLSocketChannel)
> > > > > > >> > >> >
> > > > > > >> > >> > [2014-07-17 15:37:16,434] INFO begin ssl handshake for
> > > > > > >> > >> > 10.1.100.130/10.1.100.130:51691//10.1.100.130:9092
> > > > > > >> > >> > (kafka.network.security.SSLSocketChannel)
> > > > > > >> > >> >
> > > > > > >> > >> > [2014-07-17 15:37:16,530] INFO begin ssl handshake for
> > > > > > >> > >> > 10.1.100.130/10.1.100.130:51692//10.1.100.130:9092
> > > > > > >> > >> > (kafka.network.security.SSLSocketChannel)
> > > > > > >> > >> >
> > > > > > >> > >> > [2014-07-17 15:37:16,743] INFO begin ssl handshake for
> > > > > > >> > >> > 10.1.100.130/10.1.100.130:51693//10.1.100.130:9092
> > > > > > >> > >> > (kafka.network.security.SSLSocketChannel)
> > > > > > >> > >> >
> > > > > > >> > >> > [2014-07-17 15:37:16,834] INFO begin ssl handshake for
> > > > > > >> > >> > 10.1.100.130/10.1.100.130:51694//10.1.100.130:9092
> > > > > > >> > >> > (kafka.network.security.SSLSocketChannel)
> > > > > > >> > >> >
> > > > > > >> > >> > [2014-07-17 15:37:17,043] INFO begin ssl handshake for
> > > > > > >> > >> > 10.1.100.130/10.1.100.130:51695//10.1.100.130:9092
> > > > > > >> > >> > (kafka.network.security.SSLSocketChannel)
> > > > > > >> > >> >
> > > > > > >> > >> > [2014-07-17 15:37:17,137] INFO begin ssl handshake for
> > > > > > >> > >> > 10.1.100.130/10.1.100.130:51696//10.1.100.130:9092
> > > > > > >> > >> > (kafka.network.security.SSLSocketChannel)
> > > > > > >> > >> >
> > > > > > >> > >> > [2014-07-17 15:37:17,342] INFO begin ssl handshake for
> > > > > > >> > >> > 10.1.100.130/10.1.100.130:51697//10.1.100.130:9092
> > > > > > >> > >> > (kafka.network.security.SSLSocketChannel)
> > > > > > >> > >> >
> > > > > > >> > >> >
> > > > > > >> > >> > *Start producer*
> > > > > > >> > >> >
> > > > > > >> > >> > *bin/kafka-console-producer.sh --broker-list
> > > > 10.1.100.130:9092
> > > > > > >> :true
> > > > > > >> > >> > --topic
> > > > > > >> > >> > secure.test*
> > > > > > >> > >> >
> > > > > > >> > >> >
> > > > > > >> > >> > *producer.log:*
> > > > > > >> > >> >
> > > > > > >> > >> > bin/kafka-console-producer.sh --broker-list
> > > > 10.1.100.130:9092
> > > > > > :true
> > > > > > >> > >> --topic
> > > > > > >> > >> > secure.test
> > > > > > >> > >> >
> > > > > > >> > >> > [2014-07-17 15:37:46,889] WARN Property topic is not
> > valid
> > > > > > >> > >> > (kafka.utils.VerifiableProperties)
> > > > > > >> > >> >
> > > > > > >> > >> > Hello Secure Kafka
> > > > > > >> > >> >
> > > > > > >> > >> > *[2014-07-17 15:38:14,186] ERROR OOME with size
> 352518400
> > > > > > >> > >> > (kafka.network.BoundedByteBufferReceive)*
> > > > > > >> > >> >
> > > > > > >> > >> > *java.lang.OutOfMemoryError: Java heap space*
> > > > > > >> > >> >
> > > > > > >> > >> > at
> java.nio.HeapByteBuffer.<init>(HeapByteBuffer.java:57)
> > > > > > >> > >> >
> > > > > > >> > >> > at java.nio.ByteBuffer.allocate(ByteBuffer.java:331)
> > > > > > >> > >> >
> > > > > > >> > >> > at
> > > > > > >> > >> >
> > > > > > >> > >> >
> > > > > > >> > >>
> > > > > > >> >
> > > > > > >>
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.network.BoundedByteBufferReceive.byteBufferAllocate(BoundedByteBufferReceive.scala:80)
> > > > > > >> > >> >
> > > > > > >> > >> > at
> > > > > > >> > >> >
> > > > > > >> > >> >
> > > > > > >> > >>
> > > > > > >> >
> > > > > > >>
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.network.BoundedByteBufferReceive.readFrom(BoundedByteBufferReceive.scala:63)
> > > > > > >> > >> >
> > > > > > >> > >> > at
> > > > > > >>
> > kafka.network.Receive$class.readCompletely(Transmission.scala:56)
> > > > > > >> > >> >
> > > > > > >> > >> > at
> > > > > > >> > >> >
> > > > > > >> > >> >
> > > > > > >> > >>
> > > > > > >> >
> > > > > > >>
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.network.BoundedByteBufferReceive.readCompletely(BoundedByteBufferReceive.scala:29)
> > > > > > >> > >> >
> > > > > > >> > >> > at
> > > > > > kafka.network.BlockingChannel.receive(BlockingChannel.scala:102)
> > > > > > >> > >> >
> > > > > > >> > >> > at
> > > > > > kafka.producer.SyncProducer.liftedTree1$1(SyncProducer.scala:79)
> > > > > > >> > >> >
> > > > > > >> > >> > at
> > > > > > >> > >> >
> > > > > > >> > >> >
> > > > > > >> > >>
> > > > > > >> >
> > > > > > >>
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.producer.SyncProducer.kafka$producer$SyncProducer$$doSend(SyncProducer.scala:76)
> > > > > > >> > >> >
> > > > > > >> > >> > at
> > kafka.producer.SyncProducer.send(SyncProducer.scala:117)
> > > > > > >> > >> >
> > > > > > >> > >> > at
> > > > > > >>
> > kafka.client.ClientUtils$.fetchTopicMetadata(ClientUtils.scala:58)
> > > > > > >> > >> >
> > > > > > >> > >> > at
> > > > > > >> > >> >
> > > > > > >> > >>
> > > > > > >> >
> > > > > > >>
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.producer.BrokerPartitionInfo.updateInfo(BrokerPartitionInfo.scala:82)
> > > > > > >> > >> >
> > > > > > >> > >> > at
> > > > > > >> > >> >
> > > > > > >> > >> >
> > > > > > >> > >>
> > > > > > >> >
> > > > > > >>
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.producer.async.DefaultEventHandler$$anonfun$handle$1.apply$mcV$sp(DefaultEventHandler.scala:67)
> > > > > > >> > >> >
> > > > > > >> > >> > at kafka.utils.Utils$.swallow(Utils.scala:172)
> > > > > > >> > >> >
> > > > > > >> > >> > at
> > > kafka.utils.Logging$class.swallowError(Logging.scala:106)
> > > > > > >> > >> >
> > > > > > >> > >> > at kafka.utils.Utils$.swallowError(Utils.scala:45)
> > > > > > >> > >> >
> > > > > > >> > >> > at
> > > > > > >> > >> >
> > > > > > >> > >> >
> > > > > > >> > >>
> > > > > > >> >
> > > > > > >>
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.producer.async.DefaultEventHandler.handle(DefaultEventHandler.scala:67)
> > > > > > >> > >> >
> > > > > > >> > >> > at
> > > > > > >> > >> >
> > > > > > >> > >> >
> > > > > > >> > >>
> > > > > > >> >
> > > > > > >>
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.producer.async.ProducerSendThread.tryToHandle(ProducerSendThread.scala:104)
> > > > > > >> > >> >
> > > > > > >> > >> > at
> > > > > > >> > >> >
> > > > > > >> > >> >
> > > > > > >> > >>
> > > > > > >> >
> > > > > > >>
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.producer.async.ProducerSendThread$$anonfun$processEvents$3.apply(ProducerSendThread.scala:87)
> > > > > > >> > >> >
> > > > > > >> > >> > at
> > > > > > >> > >> >
> > > > > > >> > >> >
> > > > > > >> > >>
> > > > > > >> >
> > > > > > >>
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.producer.async.ProducerSendThread$$anonfun$processEvents$3.apply(ProducerSendThread.scala:67)
> > > > > > >> > >> >
> > > > > > >> > >> > at
> > > > scala.collection.immutable.Stream.foreach(Stream.scala:526)
> > > > > > >> > >> >
> > > > > > >> > >> > at
> > > > > > >> > >> >
> > > > > > >> > >> >
> > > > > > >> > >>
> > > > > > >> >
> > > > > > >>
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.producer.async.ProducerSendThread.processEvents(ProducerSendThread.scala:66)
> > > > > > >> > >> >
> > > > > > >> > >> > at
> > > > > > >> > >>
> > > > > > >>
> > > > >
> > >
> kafka.producer.async.ProducerSendThread.run(ProducerSendThread.scala:44)
> > > > > > >> > >> >
> > > > > > >> > >> >
> > > > > > >> > >> >
> > > > > > >> > >> > On Wed, Jul 16, 2014 at 6:07 PM, Rajasekar Elango <
> > > > > > >> > >> relango@salesforce.com>
> > > > > > >> > >> > wrote:
> > > > > > >> > >> >
> > > > > > >> > >> > > Pramod,
> > > > > > >> > >> > >
> > > > > > >> > >> > >
> > > > > > >> > >> > > I presented secure kafka configuration and usage at
> > last
> > > > meet
> > > > > > >> up. So
> > > > > > >> > >> hope
> > > > > > >> > >> > > this
> > > > > > >> > >> > > video recording <
> > http://www.ustream.tv/recorded/48396701
> > > > > >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 <
> > > > > > >> > dpramodv@gmail.com>
> > > > > > >> > >> > > wrote:
> > > > > > >> > >> > >
> > > > > > >> > >> > > > 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 <
> > > > > > >> > >> dpramodv@gmail.com>
> > > > > > >> > >> > > > 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, Jul 16, 2014 at 3:57 PM, Joe Stein <
> > > > > > >> > joe.stein@stealth.ly>
> > > > > > >> > >> > > 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
> > > > > > >> > >> (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 Stein
> > > > > > >> > >> > > > >>  Founder, Principal Consultant
> > > > > > >> > >> > > > >>  Big Data Open Source Security LLC
> > > > > > >> > >> > > > >>  http://www.stealth.ly
> > > > > > >> > >> > > > >>  Twitter: @allthingshadoop <
> > > > > > >> > >> http://www.twitter.com/allthingshadoop>
> > > > > > >> > >> > > > >> ********************************************/
> > > > > > >> > >> > > > >>
> > > > > > >> > >> > > > >>
> > > > > > >> > >> > > > >> On Wed, Jul 16, 2014 at 3:49 PM, Pramod
> Deshmukh <
> > > > > > >> > >> > dpramodv@gmail.com>
> > > > > > >> > >> > > > >> wrote:
> > > > > > >> > >> > > > >>
> > > > > > >> > >> > > > >> > 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 installed sbt and scala and followed
> your
> > > > > > >> instructions
> > > > > > >> > >> for
> > > > > > >> > >> > > > >> different
> > > > > > >> > >> > > > >> > scala versions. I was able to bring zookeeper
> up
> > > but
> > > > > > >> brokers
> > > > > > >> > >> fail
> > > > > > >> > >> > to
> > > > > > >> > >> > > > >> start
> > > > > > >> > >> > > > >> > with error
> > > > > > >> > >> > > > >> >
> > > > > > >> > >> > > > >> > Error: Could not find or load main class
> > > kafka.Kafka
> > > > > > >> > >> > > > >> >
> > > > > > >> > >> > > > >> > I think I am doing something wrong. Can you
> > please
> > > > > help
> > > > > > >> me?
> > > > > > >> > >> > > > >> >
> > > > > > >> > >> > > > >> > Our current production setup is with 2.8.0 and
> > > want
> > > > to
> > > > > > >> stick
> > > > > > >> > to
> > > > > > >> > >> > it.
> > > > > > >> > >> > > > >> >
> > > > > > >> > >> > > > >> > Thanks,
> > > > > > >> > >> > > > >> >
> > > > > > >> > >> > > > >> > Pramod
> > > > > > >> > >> > > > >> >
> > > > > > >> > >> > > > >> >
> > > > > > >> > >> > > > >> > On Tue, Jun 3, 2014 at 3:57 PM, Joe Stein <
> > > > > > >> > >> joe.stein@stealth.ly>
> > > > > > >> > >> > > > wrote:
> > > > > > >> > >> > > > >> >
> > > > > > >> > >> > > > >> > > 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.  This comes from what folks
> are
> > > > doing
> > > > > > "in
> > > > > > >> the
> > > > > > >> > >> > wild"
> > > > > > >> > >> > > to
> > > > > > >> > >> > > > >> > > workaround and implement security with Kafka
> > as
> > > it
> > > > > is
> > > > > > >> today
> > > > > > >> > >> and
> > > > > > >> > >> > > also
> > > > > > >> > >> > > > >> > what I
> > > > > > >> > >> > > > >> > > have discovered from organizations about
> their
> > > > > > >> blockers. It
> > > > > > >> > >> also
> > > > > > >> > >> > > > >> picks up
> > > > > > >> > >> > > > >> > > from the wiki (which I should have time to
> > > update
> > > > > > later
> > > > > > >> in
> > > > > > >> > >> the
> > > > > > >> > >> > > week
> > > > > > >> > >> > > > >> based
> > > > > > >> > >> > > > >> > > on the below and feedback from the thread).
> > > > > > >> > >> > > > >> > >
> > > > > > >> > >> > > > >> > > 1) Transport Layer Security (i.e. SSL)
> > > > > > >> > >> > > > >> > >
> > > > > > >> > >> > > > >> > > This also includes client authentication in
> > > > addition
> > > > > > to
> > > > > > >> > >> > in-transit
> > > > > > >> > >> > > > >> > security
> > > > > > >> > >> > > > >> > > layer.  This work has been picked up here
> > > > > > >> > >> > > > >> > >
> > > https://issues.apache.org/jira/browse/KAFKA-1477
> > > > > and
> > > > > > do
> > > > > > >> > >> > > appreciate
> > > > > > >> > >> > > > >> any
> > > > > > >> > >> > > > >> > > thoughts, comments, feedback, tomatoes,
> > whatever
> > > > for
> > > > > > >> this
> > > > > > >> > >> patch.
> > > > > > >> > >> > >  It
> > > > > > >> > >> > > > >> is a
> > > > > > >> > >> > > > >> > > pickup from the fork of the work first done
> > here
> > > > > > >> > >> > > > >> > >
> > > > > https://github.com/relango/kafka/tree/kafka_security.
> > > > > > >> > >> > > > >> > >
> > > > > > >> > >> > > > >> > > 2) Data encryption at rest.
> > > > > > >> > >> > > > >> > >
> > > > > > >> > >> > > > >> > > This is very important and something that
> can
> > be
> > > > > > >> > facilitated
> > > > > > >> > >> > > within
> > > > > > >> > >> > > > >> the
> > > > > > >> > >> > > > >> > > wire protocol. It requires an additional map
> > > data
> > > > > > >> structure
> > > > > > >> > >> for
> > > > > > >> > >> > > the
> > > > > > >> > >> > > > >> > > "encrypted [data encryption key]". With this
> > map
> > > > > > >> (either in
> > > > > > >> > >> your
> > > > > > >> > >> > > > >> object
> > > > > > >> > >> > > > >> > or
> > > > > > >> > >> > > > >> > > in the wire protocol) you can store the
> > > > dynamically
> > > > > > >> > generated
> > > > > > >> > >> > > > >> symmetric
> > > > > > >> > >> > > > >> > key
> > > > > > >> > >> > > > >> > > (for each message) and then encrypt the data
> > > using
> > > > > > that
> > > > > > >> > >> > > dynamically
> > > > > > >> > >> > > > >> > > generated key.  You then encrypt the
> > encryption
> > > > key
> > > > > > >> using
> > > > > > >> > >> each
> > > > > > >> > >> > > > public
> > > > > > >> > >> > > > >> key
> > > > > > >> > >> > > > >> > > for whom is expected to be able to decrypt
> the
> > > > > > >> encryption
> > > > > > >> > >> key to
> > > > > > >> > >> > > > then
> > > > > > >> > >> > > > >> > > decrypt the message.  For each public key
> > > > encrypted
> > > > > > >> > symmetric
> > > > > > >> > >> > key
> > > > > > >> > >> > > > >> (which
> > > > > > >> > >> > > > >> > is
> > > > > > >> > >> > > > >> > > now the "encrypted [data encryption key]"
> > along
> > > > with
> > > > > > >> which
> > > > > > >> > >> > public
> > > > > > >> > >> > > > key
> > > > > > >> > >> > > > >> it
> > > > > > >> > >> > > > >> > > was encrypted with for (so a map of
> > [publicKey]
> > > =
> > > > > > >> > >> > > > >> > > encryptedDataEncryptionKey) as a chain.
> > Other
> > > > > > patterns
> > > > > > >> > can
> > > > > > >> > >> be
> > > > > > >> > >> > > > >> > implemented
> > > > > > >> > >> > > > >> > > but this is a pretty standard digital
> > enveloping
> > > > [0]
> > > > > > >> > pattern
> > > > > > >> > >> > with
> > > > > > >> > >> > > > >> only 1
> > > > > > >> > >> > > > >> > > field added. Other patterns should be able
> to
> > > use
> > > > > that
> > > > > > >> > field
> > > > > > >> > >> > to-do
> > > > > > >> > >> > > > >> their
> > > > > > >> > >> > > > >> > > implementation too.
> > > > > > >> > >> > > > >> > >
> > > > > > >> > >> > > > >> > > 3) Non-repudiation and long term
> > > non-repudiation.
> > > > > > >> > >> > > > >> > >
> > > > > > >> > >> > > > >> > > Non-repudiation is proving data hasn't
> > changed.
> > > > >  This
> > > > > > is
> > > > > > >> > >> often
> > > > > > >> > >> > (if
> > > > > > >> > >> > > > not
> > > > > > >> > >> > > > >> > > always) done with x509 public certificates
> > > > (chained
> > > > > > to a
> > > > > > >> > >> > > certificate
> > > > > > >> > >> > > > >> > > authority).
> > > > > > >> > >> > > > >> > >
> > > > > > >> > >> > > > >> > > Long term non-repudiation is what happens
> when
> > > the
> > > > > > >> > >> certificates
> > > > > > >> > >> > of
> > > > > > >> > >> > > > the
> > > > > > >> > >> > > > >> > > certificate authority are expired (or
> revoked)
> > > and
> > > > > > >> > everything
> > > > > > >> > >> > ever
> > > > > > >> > >> > > > >> signed
> > > > > > >> > >> > > > >> > > (ever) with that certificate's public key
> then
> > > > > becomes
> > > > > > >> "no
> > > > > > >> > >> > longer
> > > > > > >> > >> > > > >> > provable
> > > > > > >> > >> > > > >> > > as ever being authentic".  That is where
> > RFC3126
> > > > [1]
> > > > > > and
> > > > > > >> > >> RFC3161
> > > > > > >> > >> > > [2]
> > > > > > >> > >> > > > >> come
> > > > > > >> > >> > > > >> > > in (or worm drives [hardware], etc).
> > > > > > >> > >> > > > >> > >
> > > > > > >> > >> > > > >> > > For either (or both) of these it is an
> > operation
> > > > of
> > > > > > the
> > > > > > >> > >> > encryptor
> > > > > > >> > >> > > to
> > > > > > >> > >> > > > >> > > sign/hash the data (with or without third
> > party
> > > > > > trusted
> > > > > > >> > >> timestap
> > > > > > >> > >> > > of
> > > > > > >> > >> > > > >> the
> > > > > > >> > >> > > > >> > > signing event) and encrypt that with their
> own
> > > > > private
> > > > > > >> key
> > > > > > >> > >> and
> > > > > > >> > >> > > > >> distribute
> > > > > > >> > >> > > > >> > > the results (before and after encrypting if
> > > > > required)
> > > > > > >> along
> > > > > > >> > >> with
> > > > > > >> > >> > > > their
> > > > > > >> > >> > > > >> > > public key. This structure is a bit more
> > complex
> > > > but
> > > > > > >> > >> feasible,
> > > > > > >> > >> > it
> > > > > > >> > >> > > > is a
> > > > > > >> > >> > > > >> > map
> > > > > > >> > >> > > > >> > > of digital signature formats and the chain
> of
> > > dig
> > > > > sig
> > > > > > >> > >> > > attestations.
> > > > > > >> > >> > > > >>  The
> > > > > > >> > >> > > > >> > > map's key being the method (i.e. CRC32,
> PKCS7
> > > [3],
> > > > > > >> > XmlDigSig
> > > > > > >> > >> > [4])
> > > > > > >> > >> > > > and
> > > > > > >> > >> > > > >> > then
> > > > > > >> > >> > > > >> > > a list of map where that key is "purpose" of
> > > > > signature
> > > > > > >> > (what
> > > > > > >> > >> > your
> > > > > > >> > >> > > > >> > attesting
> > > > > > >> > >> > > > >> > > too).  As a sibling field to the list
> another
> > > > field
> > > > > > for
> > > > > > >> > "the
> > > > > > >> > >> > > > >> attester" as
> > > > > > >> > >> > > > >> > > bytes (e.g. their PKCS12 [5] for the map of
> > > PKCS7
> > > > > > >> > >> signatures).
> > > > > > >> > >> > > > >> > >
> > > > > > >> > >> > > > >> > > 4) Authorization
> > > > > > >> > >> > > > >> > >
> > > > > > >> > >> > > > >> > > We should have a policy of "404" for data,
> > > topics,
> > > > > > >> > partitions
> > > > > > >> > >> > > (etc)
> > > > > > >> > >> > > > if
> > > > > > >> > >> > > > >> > > authenticated connections do not have
> access.
> > >  In
> > > > > > >> "secure
> > > > > > >> > >> mode"
> > > > > > >> > >> > > any
> > > > > > >> > >> > > > >> non
> > > > > > >> > >> > > > >> > > authenticated connections should get a "404"
> > > type
> > > > > > >> message
> > > > > > >> > on
> > > > > > >> > >> > > > >> everything.
> > > > > > >> > >> > > > >> > > Knowing "something is there" is a security
> > risk
> > > in
> > > > > > many
> > > > > > >> > uses
> > > > > > >> > >> > > cases.
> > > > > > >> > >> > > > >>  So
> > > > > > >> > >> > > > >> > if
> > > > > > >> > >> > > > >> > > you don't have access you don't even see it.
> > > >  Baking
> > > > > > >> "that"
> > > > > > >> > >> into
> > > > > > >> > >> > > > Kafka
> > > > > > >> > >> > > > >> > > along with some interface for entitlement
> > > (access
> > > > > > >> > management)
> > > > > > >> > >> > > > systems
> > > > > > >> > >> > > > >> > > (pretty standard) is all that I think needs
> to
> > > be
> > > > > done
> > > > > > >> to
> > > > > > >> > the
> > > > > > >> > >> > core
> > > > > > >> > >> > > > >> > project.
> > > > > > >> > >> > > > >> > >  I want to tackle item later in the year
> after
> > > > > summer
> > > > > > >> after
> > > > > > >> > >> the
> > > > > > >> > >> > > > other
> > > > > > >> > >> > > > >> > three
> > > > > > >> > >> > > > >> > > are complete.
> > > > > > >> > >> > > > >> > >
> > > > > > >> > >> > > > >> > > I look forward to thoughts on this and
> anyone
> > > else
> > > > > > >> > >> interested in
> > > > > > >> > >> > > > >> working
> > > > > > >> > >> > > > >> > > with us on these items.
> > > > > > >> > >> > > > >> > >
> > > > > > >> > >> > > > >> > > [0]
> > > > > > >> > >> > > > >> > >
> > > > > > >> > >> > > > >> > >
> > > > > > >> > >> > > > >> >
> > > > > > >> > >> > > > >>
> > > > > > >> > >> > > >
> > > > > > >> > >> > >
> > > > > > >> > >> >
> > > > > > >> > >>
> > > > > > >> >
> > > > > > >>
> > > > > >
> > > > >
> > > >
> > >
> >
> http://www.emc.com/emc-plus/rsa-labs/standards-initiatives/what-is-a-digital-envelope.htm
> > > > > > >> > >> > > > >> > > [1] http://tools.ietf.org/html/rfc3126
> > > > > > >> > >> > > > >> > > [2] http://tools.ietf.org/html/rfc3161
> > > > > > >> > >> > > > >> > > [3]
> > > > > > >> > >> > > > >> > >
> > > > > > >> > >> > > > >> > >
> > > > > > >> > >> > > > >> >
> > > > > > >> > >> > > > >>
> > > > > > >> > >> > > >
> > > > > > >> > >> > >
> > > > > > >> > >> >
> > > > > > >> > >>
> > > > > > >> >
> > > > > > >>
> > > > > >
> > > > >
> > > >
> > >
> >
> http://www.emc.com/emc-plus/rsa-labs/standards-initiatives/pkcs-7-cryptographic-message-syntax-standar.htm
> > > > > > >> > >> > > > >> > > [4]
> > http://en.wikipedia.org/wiki/XML_Signature
> > > > > > >> > >> > > > >> > > [5] http://en.wikipedia.org/wiki/PKCS_12
> > > > > > >> > >> > > > >> > >
> > > > > > >> > >> > > > >> > > /*******************************************
> > > > > > >> > >> > > > >> > >  Joe Stein
> > > > > > >> > >> > > > >> > >  Founder, Principal Consultant
> > > > > > >> > >> > > > >> > >  Big Data Open Source Security LLC
> > > > > > >> > >> > > > >> > >  http://www.stealth.ly
> > > > > > >> > >> > > > >> > >  Twitter: @allthingshadoop <
> > > > > > >> > >> > > http://www.twitter.com/allthingshadoop>
> > > > > > >> > >> > > > >> > >
> ********************************************/
> > > > > > >> > >> > > > >> > >
> > > > > > >> > >> > > > >> >
> > > > > > >> > >> > > > >>
> > > > > > >> > >> > > > >
> > > > > > >> > >> > > > >
> > > > > > >> > >> > > >
> > > > > > >> > >> > >
> > > > > > >> > >> > >
> > > > > > >> > >> > >
> > > > > > >> > >> > > --
> > > > > > >> > >> > > Thanks,
> > > > > > >> > >> > > Raja.
> > > > > > >> > >> > >
> > > > > > >> > >> >
> > > > > > >> > >>
> > > > > > >> > >>
> > > > > > >> > >>
> > > > > > >> > >> --
> > > > > > >> > >> Thanks,
> > > > > > >> > >> Raja.
> > > > > > >> > >>
> > > > > > >> > >
> > > > > > >> > >
> > > > > > >> >
> > > > > > >>
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> > >
> > >
> > > --
> > > Thanks,
> > > Raja.
> > >
> >
>



-- 
Thanks,
Raja.

Re: [DISCUSS] Kafka Security Specific Features

Posted by Joe Stein <jo...@stealth.ly>.
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 is better to spend the time to get it right for everyone.  We are going
to have it in the 0.9 release (possibly also with authorization, encryption
and more of the security features too) then.

What we will do is keep the patch rebased against trunk and then then 0.8.2
branch (once we get to that point) so that folks can apply it to the 0.8.2
release and do a build from src.  When we get to that I can create a write
or something if folks find problems doing it.

/*******************************************
 Joe Stein
 Founder, Principal Consultant
 Big Data Open Source Security LLC
 http://www.stealth.ly
 Twitter: @allthingshadoop <http://www.twitter.com/allthingshadoop>
********************************************/


On Wed, Jul 30, 2014 at 7:10 PM, Calvin Lei <ck...@gmail.com> wrote:

> 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 <relango@salesforce.com
> >
> 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://github.com/stealthly/kafka/tree/v0.8.2_KAFKA-1477.
> >
> > Thanks,
> > Raja.
> >
> >
> > On Tue, Jul 29, 2014 at 10:54 PM, Calvin Lei <ck...@gmail.com> wrote:
> >
> > > Raja,
> > >    Which Kafka version is your security enhancement based on?
> > >
> > > thanks,
> > > Cal
> > >
> > >
> > > On Wed, Jul 23, 2014 at 5:01 PM, Chris Neal <cw...@gmail.com> 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
> > > > console_producer.sh, it is slightly different, which is also slightly
> > > > different than the scala code for the ConsoleProducer. :)
> > > >
> > > > When I changed this:
> > > >
> > > > bin/kafka-console-producer.sh --broker-list n5:9092:true --topic test
> > > >
> > > > to this:
> > > >
> > > > bin/kafka-console-producer.sh --broker-list n5:9092 --secure
> > > > --client.security.file config/client.security.properties --topic test
> > > >
> > > > I was able to push messages to the topic, although I got a WARN about
> > the
> > > > property "topic" not being valid, even though it is required.
> > > >
> > > > Also, the Producer reported this warning to me:
> > > >
> > > > [2014-07-23 20:45:24,509] WARN Attempt to reinitialize auth context
> > > > (kafka.network.security.SecureAuth$)
> > > >
> > > > and the broker gave me this:
> > > > [2014-07-23 20:45:24,114] INFO begin ssl handshake for
> > > > n5.example.com/192.168.1.144:48817//192.168.1.144:9092
> > > > (kafka.network.security.SSLSocketChannel)
> > > > [2014-07-23 20:45:24,374] INFO finished ssl handshake for
> > > > n5.example.com/192.168.1.144:48817//192.168.1.144:9092
> > > > (kafka.network.security.SSLSocketChannel)
> > > > [2014-07-23 20:45:24,493] INFO Closing socket connection to
> > > > n5.example.com/192.168.1.144. (kafka.network.Processor)
> > > > [2014-07-23 20:45:24,555] INFO begin ssl handshake for
> > > > n5.example.com/192.168.1.144:48818//192.168.1.144:9092
> > > > (kafka.network.security.SSLSocketChannel)
> > > > [2014-07-23 20:45:24,566] INFO finished ssl handshake for
> > > > n5.example.com/192.168.1.144:48818//192.168.1.144:9092
> > > > (kafka.network.security.SSLSocketChannel)
> > > >
> > > > It's like it did the SSL piece twice :)
> > > >
> > > > Subsequent puts to the topic did not exhibit this behavior though:
> > > >
> > > > root@n5[937]:~/kafka_2.10-0-8-2-0.1.0.0>
> bin/kafka-console-producer.sh
> > > > --broker-list n5:9092 --secure --client.security.file
> > > > config/client.security.properties --topic test
> > > > [2014-07-23 20:45:17,530] WARN Property topic is not valid
> > > > (kafka.utils.VerifiableProperties)
> > > > 1
> > > > [2014-07-23 20:45:24,509] WARN Attempt to reinitialize auth context
> > > > (kafka.network.security.SecureAuth$)
> > > > 2
> > > > 3
> > > > 4
> > > >
> > > > Consuming worked with these options:
> > > >
> > > > root@n5[918]:~/kafka_2.10-0-8-2-0.1.0.0>
> bin/kafka-console-consumer.sh
> > > > --topic test --zookeeper n5:2181 --from-beginning
> > --security.config.file
> > > > config/client.security.properties
> > > > 1
> > > > 2
> > > > 3
> > > > 4
> > > > ^CConsumed 5 messages
> > > >
> > > > I hope that helps!
> > > > Chris
> > > >
> > > >
> > > > On Tue, Jul 22, 2014 at 2:10 PM, Pramod Deshmukh <dpramodv@gmail.com
> >
> > > > wrote:
> > > >
> > > > > 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 not valid
> > > > > (kafka.utils.VerifiableProperties)
> > > > >
> > > > > Hare Krishna
> > > > >
> > > > > [2014-07-18 13:12:45,256] WARN Fetching topic metadata with
> > correlation
> > > > id
> > > > > 0 for topics [Set(secureTopic)] from broker
> > > > > [id:0,host:localhost,port:9092,secure:true] failed
> > > > > (kafka.client.ClientUtils$)
> > > > >
> > > > > java.io.EOFException: Received -1 when reading from channel, socket
> > has
> > > > > likely been closed.
> > > > >
> > > > > at kafka.utils.Utils$.read(Utils.scala:381)
> > > > >
> > > > > at
> > > > >
> > > > >
> > > >
> > >
> >
> kafka.network.BoundedByteBufferReceive.readFrom(BoundedByteBufferReceive.scala:67)
> > > > >
> > > > > at
> kafka.network.Receive$class.readCompletely(Transmission.scala:56)
> > > > >
> > > > > at
> > > > >
> > > > >
> > > >
> > >
> >
> kafka.network.BoundedByteBufferReceive.readCompletely(BoundedByteBufferReceive.scala:29)
> > > > >
> > > > > at kafka.network.BlockingChannel.receive(BlockingChannel.scala:102)
> > > > >
> > > > > at kafka.producer.SyncProducer.liftedTree1$1(SyncProducer.scala:79)
> > > > >
> > > > > at
> > > > >
> > > > >
> > > >
> > >
> >
> kafka.producer.SyncProducer.kafka$producer$SyncProducer$$doSend(SyncProducer.scala:76)
> > > > >
> > > > > at kafka.producer.SyncProducer.send(SyncProducer.scala:117)
> > > > >
> > > > > at
> kafka.client.ClientUtils$.fetchTopicMetadata(ClientUtils.scala:58)
> > > > >
> > > > > at
> > > > >
> > > >
> > >
> >
> kafka.producer.BrokerPartitionInfo.updateInfo(BrokerPartitionInfo.scala:82)
> > > > >
> > > > > at
> > > > >
> > > > >
> > > >
> > >
> >
> kafka.producer.async.DefaultEventHandler$$anonfun$handle$1.apply$mcV$sp(DefaultEventHandler.scala:67)
> > > > >
> > > > > at kafka.utils.Utils$.swallow(Utils.scala:172)
> > > > >
> > > > > at kafka.utils.Logging$class.swallowError(Logging.scala:106)
> > > > >
> > > > > at kafka.utils.Utils$.swallowError(Utils.scala:45)
> > > > >
> > > > > at
> > > > >
> > > > >
> > > >
> > >
> >
> kafka.producer.async.DefaultEventHandler.handle(DefaultEventHandler.scala:67)
> > > > >
> > > > > at
> > > > >
> > > > >
> > > >
> > >
> >
> kafka.producer.async.ProducerSendThread.tryToHandle(ProducerSendThread.scala:104)
> > > > >
> > > > > at
> > > > >
> > > > >
> > > >
> > >
> >
> kafka.producer.async.ProducerSendThread$$anonfun$processEvents$3.apply(ProducerSendThread.scala:87)
> > > > >
> > > > > at
> > > > >
> > > > >
> > > >
> > >
> >
> kafka.producer.async.ProducerSendThread$$anonfun$processEvents$3.apply(ProducerSendThread.scala:67)
> > > > >
> > > > > at scala.collection.immutable.Stream.foreach(Stream.scala:526)
> > > > >
> > > > > at
> > > > >
> > > > >
> > > >
> > >
> >
> kafka.producer.async.ProducerSendThread.processEvents(ProducerSendThread.scala:66)
> > > > >
> > > > > at
> > > >
> > kafka.producer.async.ProducerSendThread.run(ProducerSendThread.scala:44)
> > > > >
> > > > >
> > > > > On Fri, Jul 18, 2014 at 1:20 PM, Pramod Deshmukh <
> dpramodv@gmail.com
> > >
> > > > > wrote:
> > > > >
> > > > > > 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 topic "secureTopic".
> > > > > >
> > > > > > pdeshmukh$ bin/kafka-topics.sh --list --zookeeper localhost:2181
> > > > > >
> > > > > > secure.test
> > > > > >
> > > > > > secureTopic
> > > > > >
> > > > > > >> Run producer, tried both localhost:9092:true and
> localhost:9092
> > > > > >
> > > > > > pdeshmukh$ bin/kafka-console-producer.sh --broker-list
> > > > > localhost:9092:true
> > > > > > --topic secureTopic
> > > > > >
> > > > > > [2014-07-18 13:12:29,817] WARN Property topic is not valid
> > > > > > (kafka.utils.VerifiableProperties)
> > > > > >
> > > > > > Hare Krishna
> > > > > >
> > > > > > [2014-07-18 13:12:45,256] WARN Fetching topic metadata with
> > > correlation
> > > > > id
> > > > > > 0 for topics [Set(secureTopic)] from broker
> > > > > > [id:0,host:localhost,port:9092,secure:true] failed
> > > > > > (kafka.client.ClientUtils$)
> > > > > >
> > > > > > java.io.EOFException: Received -1 when reading from channel,
> socket
> > > has
> > > > > > likely been closed.
> > > > > >
> > > > > > at kafka.utils.Utils$.read(Utils.scala:381)
> > > > > >
> > > > > > at
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.network.BoundedByteBufferReceive.readFrom(BoundedByteBufferReceive.scala:67)
> > > > > >
> > > > > > at
> > kafka.network.Receive$class.readCompletely(Transmission.scala:56)
> > > > > >
> > > > > > at
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.network.BoundedByteBufferReceive.readCompletely(BoundedByteBufferReceive.scala:29)
> > > > > >
> > > > > > at
> kafka.network.BlockingChannel.receive(BlockingChannel.scala:102)
> > > > > >
> > > > > > at
> kafka.producer.SyncProducer.liftedTree1$1(SyncProducer.scala:79)
> > > > > >
> > > > > > at
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.producer.SyncProducer.kafka$producer$SyncProducer$$doSend(SyncProducer.scala:76)
> > > > > >
> > > > > > at kafka.producer.SyncProducer.send(SyncProducer.scala:117)
> > > > > >
> > > > > > at
> > kafka.client.ClientUtils$.fetchTopicMetadata(ClientUtils.scala:58)
> > > > > >
> > > > > > at
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.producer.BrokerPartitionInfo.updateInfo(BrokerPartitionInfo.scala:82)
> > > > > >
> > > > > > at
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.producer.async.DefaultEventHandler$$anonfun$handle$1.apply$mcV$sp(DefaultEventHandler.scala:67)
> > > > > >
> > > > > > at kafka.utils.Utils$.swallow(Utils.scala:172)
> > > > > >
> > > > > > at kafka.utils.Logging$class.swallowError(Logging.scala:106)
> > > > > >
> > > > > > at kafka.utils.Utils$.swallowError(Utils.scala:45)
> > > > > >
> > > > > > at
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.producer.async.DefaultEventHandler.handle(DefaultEventHandler.scala:67)
> > > > > >
> > > > > > at
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.producer.async.ProducerSendThread.tryToHandle(ProducerSendThread.scala:104)
> > > > > >
> > > > > > at
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.producer.async.ProducerSendThread$$anonfun$processEvents$3.apply(ProducerSendThread.scala:87)
> > > > > >
> > > > > > at
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.producer.async.ProducerSendThread$$anonfun$processEvents$3.apply(ProducerSendThread.scala:67)
> > > > > >
> > > > > > at scala.collection.immutable.Stream.foreach(Stream.scala:526)
> > > > > >
> > > > > > at
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.producer.async.ProducerSendThread.processEvents(ProducerSendThread.scala:66)
> > > > > >
> > > > > > at
> > > > >
> > >
> kafka.producer.async.ProducerSendThread.run(ProducerSendThread.scala:44)
> > > > > >
> > > > > > [2014-07-18 13:12:45,258] ERROR fetching topic metadata for
> topics
> > > > > > [Set(secureTopic)] from broker
> > > > > > [ArrayBuffer(id:0,host:localhost,port:9092,secure:true)] failed
> > > > > > (kafka.utils.Utils$)
> > > > > >
> > > > > > kafka.common.KafkaException: fetching topic metadata for topics
> > > > > > [Set(secureTopic)] from broker
> > > > > > [ArrayBuffer(id:0,host:localhost,port:9092,secure:true)] failed
> > > > > >
> > > > > > at
> > kafka.client.ClientUtils$.fetchTopicMetadata(ClientUtils.scala:72)
> > > > > >
> > > > > > at
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.producer.BrokerPartitionInfo.updateInfo(BrokerPartitionInfo.scala:82)
> > > > > >
> > > > > > at
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.producer.async.DefaultEventHandler$$anonfun$handle$1.apply$mcV$sp(DefaultEventHandler.scala:67)
> > > > > >
> > > > > > at kafka.utils.Utils$.swallow(Utils.scala:172)
> > > > > >
> > > > > > at kafka.utils.Logging$class.swallowError(Logging.scala:106)
> > > > > >
> > > > > > at kafka.utils.Utils$.swallowError(Utils.scala:45)
> > > > > >
> > > > > > at
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.producer.async.DefaultEventHandler.handle(DefaultEventHandler.scala:67)
> > > > > >
> > > > > > at
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.producer.async.ProducerSendThread.tryToHandle(ProducerSendThread.scala:104)
> > > > > >
> > > > > > at
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.producer.async.ProducerSendThread$$anonfun$processEvents$3.apply(ProducerSendThread.scala:87)
> > > > > >
> > > > > > at
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.producer.async.ProducerSendThread$$anonfun$processEvents$3.apply(ProducerSendThread.scala:67)
> > > > > >
> > > > > > at scala.collection.immutable.Stream.foreach(Stream.scala:526)
> > > > > >
> > > > > > at
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.producer.async.ProducerSendThread.processEvents(ProducerSendThread.scala:66)
> > > > > >
> > > > > > at
> > > > >
> > >
> kafka.producer.async.ProducerSendThread.run(ProducerSendThread.scala:44)
> > > > > >
> > > > > > Caused by: java.io.EOFException: Received -1 when reading from
> > > channel,
> > > > > > socket has likely been closed.
> > > > > >
> > > > > > at kafka.utils.Utils$.read(Utils.scala:381)
> > > > > >
> > > > > > at
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.network.BoundedByteBufferReceive.readFrom(BoundedByteBufferReceive.scala:67)
> > > > > >
> > > > > > at
> > kafka.network.Receive$class.readCompletely(Transmission.scala:56)
> > > > > >
> > > > > > at
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.network.BoundedByteBufferReceive.readCompletely(BoundedByteBufferReceive.scala:29)
> > > > > >
> > > > > > at
> kafka.network.BlockingChannel.receive(BlockingChannel.scala:102)
> > > > > >
> > > > > > at
> kafka.producer.SyncProducer.liftedTree1$1(SyncProducer.scala:79)
> > > > > >
> > > > > > at
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.producer.SyncProducer.kafka$producer$SyncProducer$$doSend(SyncProducer.scala:76)
> > > > > >
> > > > > > at kafka.producer.SyncProducer.send(SyncProducer.scala:117)
> > > > > >
> > > > > > at
> > kafka.client.ClientUtils$.fetchTopicMetadata(ClientUtils.scala:58)
> > > > > >
> > > > > > ... 12 more
> > > > > > [2014-07-18 13:12:45,337] WARN Fetching topic metadata with
> > > correlation
> > > > > id
> > > > > > 1 for topics [Set(secureTopic)] from broker
> > > > > > [id:0,host:localhost,port:9092,secure:true] failed
> > > > > > (kafka.client.ClientUtils$)
> > > > > >
> > > > > > 2014-07-18 13:12:46,282] ERROR Failed to send requests for topics
> > > > > > secureTopic with correlation ids in [0,8]
> > > > > > (kafka.producer.async.DefaultEventHandler)
> > > > > >
> > > > > > [2014-07-18 13:12:46,283] ERROR Error in handling batch of 1
> events
> > > > > > (kafka.producer.async.ProducerSendThread)
> > > > > >
> > > > > > kafka.common.FailedToSendMessageException: Failed to send
> messages
> > > > after
> > > > > 3
> > > > > > tries.
> > > > > >
> > > > > > at
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.producer.async.DefaultEventHandler.handle(DefaultEventHandler.scala:90)
> > > > > >
> > > > > > at
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.producer.async.ProducerSendThread.tryToHandle(ProducerSendThread.scala:104)
> > > > > >
> > > > > > at
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.producer.async.ProducerSendThread$$anonfun$processEvents$3.apply(ProducerSendThread.scala:87)
> > > > > >
> > > > > > at
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.producer.async.ProducerSendThread$$anonfun$processEvents$3.apply(ProducerSendThread.scala:67)
> > > > > >
> > > > > > at scala.collection.immutable.Stream.foreach(Stream.scala:526)
> > > > > >
> > > > > > at
> > > > > >
> > > > >
> > > >
> > >
> >
> kafka.producer.async.ProducerSendThread.processEvents(ProducerSendThread.scala:66)
> > > > > >
> > > > > > at
> > > > >
> > >
> kafka.producer.async.ProducerSendThread.run(ProducerSendThread.scala:44)
> > > > > >
> > > > > >
> > > > > >
> > > > > > On Fri, Jul 18, 2014 at 11:56 AM, Joe Stein <
> joe.stein@stealth.ly>
> > > > > wrote:
> > > > > >
> > > > > >> 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://www.stealth.ly
> > > > > >>  Twitter: @allthingshadoop <
> > http://www.twitter.com/allthingshadoop>
> > > > > >> ********************************************/
> > > > > >>
> > > > > >>
> > > > > >> On Fri, Jul 18, 2014 at 10:24 AM, Pramod Deshmukh <
> > > dpramodv@gmail.com
> > > > >
> > > > > >> wrote:
> > > > > >>
> > > > > >> > 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.OutOfMemoryError: Java heap space
> > > > > >> >
> > > > > >> > *broker log*
> > > > > >> >
> > > > > >> > INFO begin ssl handshake for localhost/
> > > > > 127.0.0.1:50199//127.0.0.1:9092
> > > > > >> >
> > > > > >> >
> > > > > >> >
> > > > > >> >
> > > > > >> >
> > > > > >> > On Thu, Jul 17, 2014 at 6:07 PM, Pramod Deshmukh <
> > > > dpramodv@gmail.com>
> > > > > >> > wrote:
> > > > > >> >
> > > > > >> > > 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 <
> > > > > >> > relango@salesforce.com>
> > > > > >> > > wrote:
> > > > > >> > >
> > > > > >> > >> 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 <
> > > > > dpramodv@gmail.com
> > > > > >> >
> > > > > >> > >> 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.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 config/server.properties*
> > > > > >> > >> >
> > > > > >> > >> >
> > > > > >> > >> >
> > > > > >> > >> > *broker.log:*
> > > > > >> > >> >
> > > > > >> > >> > [2014-07-17 15:34:46,281] INFO zookeeper state changed
> > > > > >> (SyncConnected)
> > > > > >> > >> > (org.I0Itec.zkclient.ZkClient)
> > > > > >> > >> >
> > > > > >> > >> > [2014-07-17 15:34:46,523] INFO Loading log
> 'secure.test-0'
> > > > > >> > >> > (kafka.log.LogManager)
> > > > > >> > >> >
> > > > > >> > >> > [2014-07-17 15:34:46,558] INFO Recovering unflushed
> > segment 0
> > > > in
> > > > > >> log
> > > > > >> > >> > secure.test-0. (kafka.log.Log)
> > > > > >> > >> >
> > > > > >> > >> > [2014-07-17 15:34:46,571] INFO Completed load of log
> > > > > secure.test-0
> > > > > >> > with
> > > > > >> > >> log
> > > > > >> > >> > end offset 0 (kafka.log.Log)
> > > > > >> > >> >
> > > > > >> > >> > [2014-07-17 15:34:46,582] INFO Starting log cleanup with
> a
> > > > period
> > > > > >> of
> > > > > >> > >> 60000
> > > > > >> > >> > ms. (kafka.log.LogManager)
> > > > > >> > >> >
> > > > > >> > >> > [2014-07-17 15:34:46,587] INFO Starting log flusher with
> a
> > > > > default
> > > > > >> > >> period
> > > > > >> > >> > of 9223372036854775807 ms. (kafka.log.LogManager)
> > > > > >> > >> >
> > > > > >> > >> > [2014-07-17 15:34:46,614] INFO Initializing secure
> > > > authentication
> > > > > >> > >> > (kafka.network.security.SecureAuth$)
> > > > > >> > >> >
> > > > > >> > >> > [2014-07-17 15:34:46,678] INFO Secure authentication
> > > > > initialization
> > > > > >> > has
> > > > > >> > >> > been successfully completed
> > > > (kafka.network.security.SecureAuth$)
> > > > > >> > >> >
> > > > > >> > >> > [2014-07-17 15:34:46,691] INFO Awaiting socket
> connections
> > on
> > > > > >> > >> 0.0.0.0:9092
> > > > > >> > >> > .
> > > > > >> > >> > (kafka.network.Acceptor)
> > > > > >> > >> >
> > > > > >> > >> > [2014-07-17 15:34:46,692] INFO [Socket Server on Broker
> 0],
> > > > > Started
> > > > > >> > >> > (kafka.network.SocketServer)
> > > > > >> > >> >
> > > > > >> > >> > [2014-07-17 15:34:46,794] INFO Will not load MX4J,
> > > > mx4j-tools.jar
> > > > > >> is
> > > > > >> > >> not in
> > > > > >> > >> > the classpath (kafka.utils.Mx4jLoader$)
> > > > > >> > >> >
> > > > > >> > >> > [2014-07-17 15:34:46,837] INFO 0 successfully elected as
> > > leader
> > > > > >> > >> > (kafka.server.ZookeeperLeaderElector)
> > > > > >> > >> >
> > > > > >> > >> > [2014-07-17 15:34:47,057] INFO Registered broker 0 at
> path
> > > > > >> > >> /brokers/ids/0
> > > > > >> > >> > with address 10.1.100.130:9092. (kafka.utils.ZkUtils$)
> > > > > >> > >> >
> > > > > >> > >> > [2014-07-17 15:34:47,059] INFO New leader is 0
> > > > > >> > >> >
> (kafka.server.ZookeeperLeaderElector$LeaderChangeListener)
> > > > > >> > >> >
> > > > > >> > >> > *[2014-07-17 15:34:47,068] INFO [Kafka Server 0], started
> > > > > >> > >> > (kafka.server.KafkaServer)*
> > > > > >> > >> >
> > > > > >> > >> > *[2014-07-17 15:34:47,383] INFO begin ssl handshake for
> > > > > >> > >> > /10.1.100.130:9092//10.1.100.130:51685
> > > > > >> > >> > <http://10.1.100.130:9092//10.1.100.130:51685>
> > > > > >> > >> > (kafka.network.security.SSLSocketChannel)*
> > > > > >> > >> >
> > > > > >> > >> > *[2014-07-17 15:34:47,392] INFO begin ssl handshake for
> > > > > >> > >> > 10.1.100.130/10.1.100.130:51685//10.1.100.130:9092
> > > > > >> > >> > <
> http://10.1.100.130/10.1.100.130:51685//10.1.100.130:9092
> > >
> > > > > >> > >> > (kafka.network.security.SSLSocketChannel)*
> > > > > >> > >> >
> > > > > >> > >> > *[2014-07-17 15:34:47,465] INFO finished ssl handshake
> for
> > > > > >> > >> > 10.1.100.130/10.1.100.130:51685//10.1.100.130:9092
> > > > > >> > >> > <
> http://10.1.100.130/10.1.100.130:51685//10.1.100.130:9092
> > >
> > > > > >> > >> > (kafka.network.security.SSLSocketChannel)*
> > > > > >> > >> >
> > > > > >> > >> > *[2014-07-17 15:34:47,465] INFO finished ssl handshake
> for
> > > > > >> > >> > /10.1.100.130:9092//10.1.100.130:51685
> > > > > >> > >> > <http://10.1.100.130:9092//10.1.100.130:51685>
> > > > > >> > >> > (kafka.network.security.SSLSocketChannel)*
> > > > > >> > >> >
> > > > > >> > >> > *[2014-07-17 15:34:47,617] INFO [ReplicaFetcherManager on
> > > > broker
> > > > > 0]
> > > > > >> > >> Removed
> > > > > >> > >> > fetcher for partitions
> >  (kafka.server.ReplicaFetcherManager)*
> > > > > >> > >> >
> > > > > >> > >> > *[2014-07-17 15:34:47,627] INFO [ReplicaFetcherManager on
> > > > broker
> > > > > 0]
> > > > > >> > >> Added
> > > > > >> > >> > fetcher for partitions List()
> > > > > (kafka.server.ReplicaFetcherManager)*
> > > > > >> > >> >
> > > > > >> > >> > *[2014-07-17 15:34:47,656] INFO [ReplicaFetcherManager on
> > > > broker
> > > > > 0]
> > > > > >> > >> Removed
> > > > > >> > >> > fetcher for partitions [secure.test,0]
> > > > > >> > >> > (kafka.server.ReplicaFetcherManager)*
> > > > > >> > >> >
> > > > > >> > >> > [2014-07-17 15:37:15,970] INFO begin ssl handshake for
> > > > > >> > >> > 10.1.100.130/10.1.100.130:51689//10.1.100.130:9092
> > > > > >> > >> > (kafka.network.security.SSLSocketChannel)
> > > > > >> > >> >
> > > > > >> > >> > [2014-07-17 15:37:16,075] INFO begin ssl handshake for
> > > > > >> > >> > 10.1.100.130/10.1.100.130:51690//10.1.100.130:9092
> > > > > >> > >> > (kafka.network.security.SSLSocketChannel)
> > > > > >> > >> >
> > > > > >> > >> > [2014-07-17 15:37:16,434] INFO begin ssl handshake for
> > > > > >> > >> > 10.1.100.130/10.1.100.130:51691//10.1.100.130:9092
> > > > > >> > >> > (kafka.network.security.SSLSocketChannel)
> > > > > >> > >> >
> > > > > >> > >> > [2014-07-17 15:37:16,530] INFO begin ssl handshake for
> > > > > >> > >> > 10.1.100.130/10.1.100.130:51692//10.1.100.130:9092
> > > > > >> > >> > (kafka.network.security.SSLSocketChannel)
> > > > > >> > >> >
> > > > > >> > >> > [2014-07-17 15:37:16,743] INFO begin ssl handshake for
> > > > > >> > >> > 10.1.100.130/10.1.100.130:51693//10.1.100.130:9092
> > > > > >> > >> > (kafka.network.security.SSLSocketChannel)
> > > > > >> > >> >
> > > > > >> > >> > [2014-07-17 15:37:16,834] INFO begin ssl handshake for
> > > > > >> > >> > 10.1.100.130/10.1.100.130:51694//10.1.100.130:9092
> > > > > >> > >> > (kafka.network.security.SSLSocketChannel)
> > > > > >> > >> >
> > > > > >> > >> > [2014-07-17 15:37:17,043] INFO begin ssl handshake for
> > > > > >> > >> > 10.1.100.130/10.1.100.130:51695//10.1.100.130:9092
> > > > > >> > >> > (kafka.network.security.SSLSocketChannel)
> > > > > >> > >> >
> > > > > >> > >> > [2014-07-17 15:37:17,137] INFO begin ssl handshake for
> > > > > >> > >> > 10.1.100.130/10.1.100.130:51696//10.1.100.130:9092
> > > > > >> > >> > (kafka.network.security.SSLSocketChannel)
> > > > > >> > >> >
> > > > > >> > >> > [2014-07-17 15:37:17,342] INFO begin ssl handshake for
> > > > > >> > >> > 10.1.100.130/10.1.100.130:51697//10.1.100.130:9092
> > > > > >> > >> > (kafka.network.security.SSLSocketChannel)
> > > > > >> > >> >
> > > > > >> > >> >
> > > > > >> > >> > *Start producer*
> > > > > >> > >> >
> > > > > >> > >> > *bin/kafka-console-producer.sh --broker-list
> > > 10.1.100.130:9092
> > > > > >> :true
> > > > > >> > >> > --topic
> > > > > >> > >> > secure.test*
> > > > > >> > >> >
> > > > > >> > >> >
> > > > > >> > >> > *producer.log:*
> > > > > >> > >> >
> > > > > >> > >> > bin/kafka-console-producer.sh --broker-list
> > > 10.1.100.130:9092
> > > > > :true
> > > > > >> > >> --topic
> > > > > >> > >> > secure.test
> > > > > >> > >> >
> > > > > >> > >> > [2014-07-17 15:37:46,889] WARN Property topic is not
> valid
> > > > > >> > >> > (kafka.utils.VerifiableProperties)
> > > > > >> > >> >
> > > > > >> > >> > Hello Secure Kafka
> > > > > >> > >> >
> > > > > >> > >> > *[2014-07-17 15:38:14,186] ERROR OOME with size 352518400
> > > > > >> > >> > (kafka.network.BoundedByteBufferReceive)*
> > > > > >> > >> >
> > > > > >> > >> > *java.lang.OutOfMemoryError: Java heap space*
> > > > > >> > >> >
> > > > > >> > >> > at java.nio.HeapByteBuffer.<init>(HeapByteBuffer.java:57)
> > > > > >> > >> >
> > > > > >> > >> > at java.nio.ByteBuffer.allocate(ByteBuffer.java:331)
> > > > > >> > >> >
> > > > > >> > >> > at
> > > > > >> > >> >
> > > > > >> > >> >
> > > > > >> > >>
> > > > > >> >
> > > > > >>
> > > > >
> > > >
> > >
> >
> kafka.network.BoundedByteBufferReceive.byteBufferAllocate(BoundedByteBufferReceive.scala:80)
> > > > > >> > >> >
> > > > > >> > >> > at
> > > > > >> > >> >
> > > > > >> > >> >
> > > > > >> > >>
> > > > > >> >
> > > > > >>
> > > > >
> > > >
> > >
> >
> kafka.network.BoundedByteBufferReceive.readFrom(BoundedByteBufferReceive.scala:63)
> > > > > >> > >> >
> > > > > >> > >> > at
> > > > > >>
> kafka.network.Receive$class.readCompletely(Transmission.scala:56)
> > > > > >> > >> >
> > > > > >> > >> > at
> > > > > >> > >> >
> > > > > >> > >> >
> > > > > >> > >>
> > > > > >> >
> > > > > >>
> > > > >
> > > >
> > >
> >
> kafka.network.BoundedByteBufferReceive.readCompletely(BoundedByteBufferReceive.scala:29)
> > > > > >> > >> >
> > > > > >> > >> > at
> > > > > kafka.network.BlockingChannel.receive(BlockingChannel.scala:102)
> > > > > >> > >> >
> > > > > >> > >> > at
> > > > > kafka.producer.SyncProducer.liftedTree1$1(SyncProducer.scala:79)
> > > > > >> > >> >
> > > > > >> > >> > at
> > > > > >> > >> >
> > > > > >> > >> >
> > > > > >> > >>
> > > > > >> >
> > > > > >>
> > > > >
> > > >
> > >
> >
> kafka.producer.SyncProducer.kafka$producer$SyncProducer$$doSend(SyncProducer.scala:76)
> > > > > >> > >> >
> > > > > >> > >> > at
> kafka.producer.SyncProducer.send(SyncProducer.scala:117)
> > > > > >> > >> >
> > > > > >> > >> > at
> > > > > >>
> kafka.client.ClientUtils$.fetchTopicMetadata(ClientUtils.scala:58)
> > > > > >> > >> >
> > > > > >> > >> > at
> > > > > >> > >> >
> > > > > >> > >>
> > > > > >> >
> > > > > >>
> > > > >
> > > >
> > >
> >
> kafka.producer.BrokerPartitionInfo.updateInfo(BrokerPartitionInfo.scala:82)
> > > > > >> > >> >
> > > > > >> > >> > at
> > > > > >> > >> >
> > > > > >> > >> >
> > > > > >> > >>
> > > > > >> >
> > > > > >>
> > > > >
> > > >
> > >
> >
> kafka.producer.async.DefaultEventHandler$$anonfun$handle$1.apply$mcV$sp(DefaultEventHandler.scala:67)
> > > > > >> > >> >
> > > > > >> > >> > at kafka.utils.Utils$.swallow(Utils.scala:172)
> > > > > >> > >> >
> > > > > >> > >> > at
> > kafka.utils.Logging$class.swallowError(Logging.scala:106)
> > > > > >> > >> >
> > > > > >> > >> > at kafka.utils.Utils$.swallowError(Utils.scala:45)
> > > > > >> > >> >
> > > > > >> > >> > at
> > > > > >> > >> >
> > > > > >> > >> >
> > > > > >> > >>
> > > > > >> >
> > > > > >>
> > > > >
> > > >
> > >
> >
> kafka.producer.async.DefaultEventHandler.handle(DefaultEventHandler.scala:67)
> > > > > >> > >> >
> > > > > >> > >> > at
> > > > > >> > >> >
> > > > > >> > >> >
> > > > > >> > >>
> > > > > >> >
> > > > > >>
> > > > >
> > > >
> > >
> >
> kafka.producer.async.ProducerSendThread.tryToHandle(ProducerSendThread.scala:104)
> > > > > >> > >> >
> > > > > >> > >> > at
> > > > > >> > >> >
> > > > > >> > >> >
> > > > > >> > >>
> > > > > >> >
> > > > > >>
> > > > >
> > > >
> > >
> >
> kafka.producer.async.ProducerSendThread$$anonfun$processEvents$3.apply(ProducerSendThread.scala:87)
> > > > > >> > >> >
> > > > > >> > >> > at
> > > > > >> > >> >
> > > > > >> > >> >
> > > > > >> > >>
> > > > > >> >
> > > > > >>
> > > > >
> > > >
> > >
> >
> kafka.producer.async.ProducerSendThread$$anonfun$processEvents$3.apply(ProducerSendThread.scala:67)
> > > > > >> > >> >
> > > > > >> > >> > at
> > > scala.collection.immutable.Stream.foreach(Stream.scala:526)
> > > > > >> > >> >
> > > > > >> > >> > at
> > > > > >> > >> >
> > > > > >> > >> >
> > > > > >> > >>
> > > > > >> >
> > > > > >>
> > > > >
> > > >
> > >
> >
> kafka.producer.async.ProducerSendThread.processEvents(ProducerSendThread.scala:66)
> > > > > >> > >> >
> > > > > >> > >> > at
> > > > > >> > >>
> > > > > >>
> > > >
> > kafka.producer.async.ProducerSendThread.run(ProducerSendThread.scala:44)
> > > > > >> > >> >
> > > > > >> > >> >
> > > > > >> > >> >
> > > > > >> > >> > On Wed, Jul 16, 2014 at 6:07 PM, Rajasekar Elango <
> > > > > >> > >> relango@salesforce.com>
> > > > > >> > >> > wrote:
> > > > > >> > >> >
> > > > > >> > >> > > Pramod,
> > > > > >> > >> > >
> > > > > >> > >> > >
> > > > > >> > >> > > I presented secure kafka configuration and usage at
> last
> > > meet
> > > > > >> up. So
> > > > > >> > >> hope
> > > > > >> > >> > > this
> > > > > >> > >> > > video recording <
> http://www.ustream.tv/recorded/48396701
> > > > >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 <
> > > > > >> > dpramodv@gmail.com>
> > > > > >> > >> > > wrote:
> > > > > >> > >> > >
> > > > > >> > >> > > > 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 <
> > > > > >> > >> dpramodv@gmail.com>
> > > > > >> > >> > > > 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, Jul 16, 2014 at 3:57 PM, Joe Stein <
> > > > > >> > joe.stein@stealth.ly>
> > > > > >> > >> > > 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
> > > > > >> > >> (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 Stein
> > > > > >> > >> > > > >>  Founder, Principal Consultant
> > > > > >> > >> > > > >>  Big Data Open Source Security LLC
> > > > > >> > >> > > > >>  http://www.stealth.ly
> > > > > >> > >> > > > >>  Twitter: @allthingshadoop <
> > > > > >> > >> http://www.twitter.com/allthingshadoop>
> > > > > >> > >> > > > >> ********************************************/
> > > > > >> > >> > > > >>
> > > > > >> > >> > > > >>
> > > > > >> > >> > > > >> On Wed, Jul 16, 2014 at 3:49 PM, Pramod Deshmukh <
> > > > > >> > >> > dpramodv@gmail.com>
> > > > > >> > >> > > > >> wrote:
> > > > > >> > >> > > > >>
> > > > > >> > >> > > > >> > 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 installed sbt and scala and followed your
> > > > > >> instructions
> > > > > >> > >> for
> > > > > >> > >> > > > >> different
> > > > > >> > >> > > > >> > scala versions. I was able to bring zookeeper up
> > but
> > > > > >> brokers
> > > > > >> > >> fail
> > > > > >> > >> > to
> > > > > >> > >> > > > >> start
> > > > > >> > >> > > > >> > with error
> > > > > >> > >> > > > >> >
> > > > > >> > >> > > > >> > Error: Could not find or load main class
> > kafka.Kafka
> > > > > >> > >> > > > >> >
> > > > > >> > >> > > > >> > I think I am doing something wrong. Can you
> please
> > > > help
> > > > > >> me?
> > > > > >> > >> > > > >> >
> > > > > >> > >> > > > >> > Our current production setup is with 2.8.0 and
> > want
> > > to
> > > > > >> stick
> > > > > >> > to
> > > > > >> > >> > it.
> > > > > >> > >> > > > >> >
> > > > > >> > >> > > > >> > Thanks,
> > > > > >> > >> > > > >> >
> > > > > >> > >> > > > >> > Pramod
> > > > > >> > >> > > > >> >
> > > > > >> > >> > > > >> >
> > > > > >> > >> > > > >> > On Tue, Jun 3, 2014 at 3:57 PM, Joe Stein <
> > > > > >> > >> joe.stein@stealth.ly>
> > > > > >> > >> > > > wrote:
> > > > > >> > >> > > > >> >
> > > > > >> > >> > > > >> > > 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.  This comes from what folks are
> > > doing
> > > > > "in
> > > > > >> the
> > > > > >> > >> > wild"
> > > > > >> > >> > > to
> > > > > >> > >> > > > >> > > workaround and implement security with Kafka
> as
> > it
> > > > is
> > > > > >> today
> > > > > >> > >> and
> > > > > >> > >> > > also
> > > > > >> > >> > > > >> > what I
> > > > > >> > >> > > > >> > > have discovered from organizations about their
> > > > > >> blockers. It
> > > > > >> > >> also
> > > > > >> > >> > > > >> picks up
> > > > > >> > >> > > > >> > > from the wiki (which I should have time to
> > update
> > > > > later
> > > > > >> in
> > > > > >> > >> the
> > > > > >> > >> > > week
> > > > > >> > >> > > > >> based
> > > > > >> > >> > > > >> > > on the below and feedback from the thread).
> > > > > >> > >> > > > >> > >
> > > > > >> > >> > > > >> > > 1) Transport Layer Security (i.e. SSL)
> > > > > >> > >> > > > >> > >
> > > > > >> > >> > > > >> > > This also includes client authentication in
> > > addition
> > > > > to
> > > > > >> > >> > in-transit
> > > > > >> > >> > > > >> > security
> > > > > >> > >> > > > >> > > layer.  This work has been picked up here
> > > > > >> > >> > > > >> > >
> > https://issues.apache.org/jira/browse/KAFKA-1477
> > > > and
> > > > > do
> > > > > >> > >> > > appreciate
> > > > > >> > >> > > > >> any
> > > > > >> > >> > > > >> > > thoughts, comments, feedback, tomatoes,
> whatever
> > > for
> > > > > >> this
> > > > > >> > >> patch.
> > > > > >> > >> > >  It
> > > > > >> > >> > > > >> is a
> > > > > >> > >> > > > >> > > pickup from the fork of the work first done
> here
> > > > > >> > >> > > > >> > >
> > > > https://github.com/relango/kafka/tree/kafka_security.
> > > > > >> > >> > > > >> > >
> > > > > >> > >> > > > >> > > 2) Data encryption at rest.
> > > > > >> > >> > > > >> > >
> > > > > >> > >> > > > >> > > This is very important and something that can
> be
> > > > > >> > facilitated
> > > > > >> > >> > > within
> > > > > >> > >> > > > >> the
> > > > > >> > >> > > > >> > > wire protocol. It requires an additional map
> > data
> > > > > >> structure
> > > > > >> > >> for
> > > > > >> > >> > > the
> > > > > >> > >> > > > >> > > "encrypted [data encryption key]". With this
> map
> > > > > >> (either in
> > > > > >> > >> your
> > > > > >> > >> > > > >> object
> > > > > >> > >> > > > >> > or
> > > > > >> > >> > > > >> > > in the wire protocol) you can store the
> > > dynamically
> > > > > >> > generated
> > > > > >> > >> > > > >> symmetric
> > > > > >> > >> > > > >> > key
> > > > > >> > >> > > > >> > > (for each message) and then encrypt the data
> > using
> > > > > that
> > > > > >> > >> > > dynamically
> > > > > >> > >> > > > >> > > generated key.  You then encrypt the
> encryption
> > > key
> > > > > >> using
> > > > > >> > >> each
> > > > > >> > >> > > > public
> > > > > >> > >> > > > >> key
> > > > > >> > >> > > > >> > > for whom is expected to be able to decrypt the
> > > > > >> encryption
> > > > > >> > >> key to
> > > > > >> > >> > > > then
> > > > > >> > >> > > > >> > > decrypt the message.  For each public key
> > > encrypted
> > > > > >> > symmetric
> > > > > >> > >> > key
> > > > > >> > >> > > > >> (which
> > > > > >> > >> > > > >> > is
> > > > > >> > >> > > > >> > > now the "encrypted [data encryption key]"
> along
> > > with
> > > > > >> which
> > > > > >> > >> > public
> > > > > >> > >> > > > key
> > > > > >> > >> > > > >> it
> > > > > >> > >> > > > >> > > was encrypted with for (so a map of
> [publicKey]
> > =
> > > > > >> > >> > > > >> > > encryptedDataEncryptionKey) as a chain.
> Other
> > > > > patterns
> > > > > >> > can
> > > > > >> > >> be
> > > > > >> > >> > > > >> > implemented
> > > > > >> > >> > > > >> > > but this is a pretty standard digital
> enveloping
> > > [0]
> > > > > >> > pattern
> > > > > >> > >> > with
> > > > > >> > >> > > > >> only 1
> > > > > >> > >> > > > >> > > field added. Other patterns should be able to
> > use
> > > > that
> > > > > >> > field
> > > > > >> > >> > to-do
> > > > > >> > >> > > > >> their
> > > > > >> > >> > > > >> > > implementation too.
> > > > > >> > >> > > > >> > >
> > > > > >> > >> > > > >> > > 3) Non-repudiation and long term
> > non-repudiation.
> > > > > >> > >> > > > >> > >
> > > > > >> > >> > > > >> > > Non-repudiation is proving data hasn't
> changed.
> > > >  This
> > > > > is
> > > > > >> > >> often
> > > > > >> > >> > (if
> > > > > >> > >> > > > not
> > > > > >> > >> > > > >> > > always) done with x509 public certificates
> > > (chained
> > > > > to a
> > > > > >> > >> > > certificate
> > > > > >> > >> > > > >> > > authority).
> > > > > >> > >> > > > >> > >
> > > > > >> > >> > > > >> > > Long term non-repudiation is what happens when
> > the
> > > > > >> > >> certificates
> > > > > >> > >> > of
> > > > > >> > >> > > > the
> > > > > >> > >> > > > >> > > certificate authority are expired (or revoked)
> > and
> > > > > >> > everything
> > > > > >> > >> > ever
> > > > > >> > >> > > > >> signed
> > > > > >> > >> > > > >> > > (ever) with that certificate's public key then
> > > > becomes
> > > > > >> "no
> > > > > >> > >> > longer
> > > > > >> > >> > > > >> > provable
> > > > > >> > >> > > > >> > > as ever being authentic".  That is where
> RFC3126
> > > [1]
> > > > > and
> > > > > >> > >> RFC3161
> > > > > >> > >> > > [2]
> > > > > >> > >> > > > >> come
> > > > > >> > >> > > > >> > > in (or worm drives [hardware], etc).
> > > > > >> > >> > > > >> > >
> > > > > >> > >> > > > >> > > For either (or both) of these it is an
> operation
> > > of
> > > > > the
> > > > > >> > >> > encryptor
> > > > > >> > >> > > to
> > > > > >> > >> > > > >> > > sign/hash the data (with or without third
> party
> > > > > trusted
> > > > > >> > >> timestap
> > > > > >> > >> > > of
> > > > > >> > >> > > > >> the
> > > > > >> > >> > > > >> > > signing event) and encrypt that with their own
> > > > private
> > > > > >> key
> > > > > >> > >> and
> > > > > >> > >> > > > >> distribute
> > > > > >> > >> > > > >> > > the results (before and after encrypting if
> > > > required)
> > > > > >> along
> > > > > >> > >> with
> > > > > >> > >> > > > their
> > > > > >> > >> > > > >> > > public key. This structure is a bit more
> complex
> > > but
> > > > > >> > >> feasible,
> > > > > >> > >> > it
> > > > > >> > >> > > > is a
> > > > > >> > >> > > > >> > map
> > > > > >> > >> > > > >> > > of digital signature formats and the chain of
> > dig
> > > > sig
> > > > > >> > >> > > attestations.
> > > > > >> > >> > > > >>  The
> > > > > >> > >> > > > >> > > map's key being the method (i.e. CRC32, PKCS7
> > [3],
> > > > > >> > XmlDigSig
> > > > > >> > >> > [4])
> > > > > >> > >> > > > and
> > > > > >> > >> > > > >> > then
> > > > > >> > >> > > > >> > > a list of map where that key is "purpose" of
> > > > signature
> > > > > >> > (what
> > > > > >> > >> > your
> > > > > >> > >> > > > >> > attesting
> > > > > >> > >> > > > >> > > too).  As a sibling field to the list another
> > > field
> > > > > for
> > > > > >> > "the
> > > > > >> > >> > > > >> attester" as
> > > > > >> > >> > > > >> > > bytes (e.g. their PKCS12 [5] for the map of
> > PKCS7
> > > > > >> > >> signatures).
> > > > > >> > >> > > > >> > >
> > > > > >> > >> > > > >> > > 4) Authorization
> > > > > >> > >> > > > >> > >
> > > > > >> > >> > > > >> > > We should have a policy of "404" for data,
> > topics,
> > > > > >> > partitions
> > > > > >> > >> > > (etc)
> > > > > >> > >> > > > if
> > > > > >> > >> > > > >> > > authenticated connections do not have access.
> >  In
> > > > > >> "secure
> > > > > >> > >> mode"
> > > > > >> > >> > > any
> > > > > >> > >> > > > >> non
> > > > > >> > >> > > > >> > > authenticated connections should get a "404"
> > type
> > > > > >> message
> > > > > >> > on
> > > > > >> > >> > > > >> everything.
> > > > > >> > >> > > > >> > > Knowing "something is there" is a security
> risk
> > in
> > > > > many
> > > > > >> > uses
> > > > > >> > >> > > cases.
> > > > > >> > >> > > > >>  So
> > > > > >> > >> > > > >> > if
> > > > > >> > >> > > > >> > > you don't have access you don't even see it.
> > >  Baking
> > > > > >> "that"
> > > > > >> > >> into
> > > > > >> > >> > > > Kafka
> > > > > >> > >> > > > >> > > along with some interface for entitlement
> > (access
> > > > > >> > management)
> > > > > >> > >> > > > systems
> > > > > >> > >> > > > >> > > (pretty standard) is all that I think needs to
> > be
> > > > done
> > > > > >> to
> > > > > >> > the
> > > > > >> > >> > core
> > > > > >> > >> > > > >> > project.
> > > > > >> > >> > > > >> > >  I want to tackle item later in the year after
> > > > summer
> > > > > >> after
> > > > > >> > >> the
> > > > > >> > >> > > > other
> > > > > >> > >> > > > >> > three
> > > > > >> > >> > > > >> > > are complete.
> > > > > >> > >> > > > >> > >
> > > > > >> > >> > > > >> > > I look forward to thoughts on this and anyone
> > else
> > > > > >> > >> interested in
> > > > > >> > >> > > > >> working
> > > > > >> > >> > > > >> > > with us on these items.
> > > > > >> > >> > > > >> > >
> > > > > >> > >> > > > >> > > [0]
> > > > > >> > >> > > > >> > >
> > > > > >> > >> > > > >> > >
> > > > > >> > >> > > > >> >
> > > > > >> > >> > > > >>
> > > > > >> > >> > > >
> > > > > >> > >> > >
> > > > > >> > >> >
> > > > > >> > >>
> > > > > >> >
> > > > > >>
> > > > >
> > > >
> > >
> >
> http://www.emc.com/emc-plus/rsa-labs/standards-initiatives/what-is-a-digital-envelope.htm
> > > > > >> > >> > > > >> > > [1] http://tools.ietf.org/html/rfc3126
> > > > > >> > >> > > > >> > > [2] http://tools.ietf.org/html/rfc3161
> > > > > >> > >> > > > >> > > [3]
> > > > > >> > >> > > > >> > >
> > > > > >> > >> > > > >> > >
> > > > > >> > >> > > > >> >
> > > > > >> > >> > > > >>
> > > > > >> > >> > > >
> > > > > >> > >> > >
> > > > > >> > >> >
> > > > > >> > >>
> > > > > >> >
> > > > > >>
> > > > >
> > > >
> > >
> >
> http://www.emc.com/emc-plus/rsa-labs/standards-initiatives/pkcs-7-cryptographic-message-syntax-standar.htm
> > > > > >> > >> > > > >> > > [4]
> http://en.wikipedia.org/wiki/XML_Signature
> > > > > >> > >> > > > >> > > [5] http://en.wikipedia.org/wiki/PKCS_12
> > > > > >> > >> > > > >> > >
> > > > > >> > >> > > > >> > > /*******************************************
> > > > > >> > >> > > > >> > >  Joe Stein
> > > > > >> > >> > > > >> > >  Founder, Principal Consultant
> > > > > >> > >> > > > >> > >  Big Data Open Source Security LLC
> > > > > >> > >> > > > >> > >  http://www.stealth.ly
> > > > > >> > >> > > > >> > >  Twitter: @allthingshadoop <
> > > > > >> > >> > > http://www.twitter.com/allthingshadoop>
> > > > > >> > >> > > > >> > > ********************************************/
> > > > > >> > >> > > > >> > >
> > > > > >> > >> > > > >> >
> > > > > >> > >> > > > >>
> > > > > >> > >> > > > >
> > > > > >> > >> > > > >
> > > > > >> > >> > > >
> > > > > >> > >> > >
> > > > > >> > >> > >
> > > > > >> > >> > >
> > > > > >> > >> > > --
> > > > > >> > >> > > Thanks,
> > > > > >> > >> > > Raja.
> > > > > >> > >> > >
> > > > > >> > >> >
> > > > > >> > >>
> > > > > >> > >>
> > > > > >> > >>
> > > > > >> > >> --
> > > > > >> > >> Thanks,
> > > > > >> > >> Raja.
> > > > > >> > >>
> > > > > >> > >
> > > > > >> > >
> > > > > >> >
> > > > > >>
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> >
> >
> > --
> > Thanks,
> > Raja.
> >
>

Re: [DISCUSS] Kafka Security Specific Features

Posted by Calvin Lei <ck...@gmail.com>.
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 <re...@salesforce.com>
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://github.com/stealthly/kafka/tree/v0.8.2_KAFKA-1477.
>
> Thanks,
> Raja.
>
>
> On Tue, Jul 29, 2014 at 10:54 PM, Calvin Lei <ck...@gmail.com> wrote:
>
> > Raja,
> >    Which Kafka version is your security enhancement based on?
> >
> > thanks,
> > Cal
> >
> >
> > On Wed, Jul 23, 2014 at 5:01 PM, Chris Neal <cw...@gmail.com> 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
> > > console_producer.sh, it is slightly different, which is also slightly
> > > different than the scala code for the ConsoleProducer. :)
> > >
> > > When I changed this:
> > >
> > > bin/kafka-console-producer.sh --broker-list n5:9092:true --topic test
> > >
> > > to this:
> > >
> > > bin/kafka-console-producer.sh --broker-list n5:9092 --secure
> > > --client.security.file config/client.security.properties --topic test
> > >
> > > I was able to push messages to the topic, although I got a WARN about
> the
> > > property "topic" not being valid, even though it is required.
> > >
> > > Also, the Producer reported this warning to me:
> > >
> > > [2014-07-23 20:45:24,509] WARN Attempt to reinitialize auth context
> > > (kafka.network.security.SecureAuth$)
> > >
> > > and the broker gave me this:
> > > [2014-07-23 20:45:24,114] INFO begin ssl handshake for
> > > n5.example.com/192.168.1.144:48817//192.168.1.144:9092
> > > (kafka.network.security.SSLSocketChannel)
> > > [2014-07-23 20:45:24,374] INFO finished ssl handshake for
> > > n5.example.com/192.168.1.144:48817//192.168.1.144:9092
> > > (kafka.network.security.SSLSocketChannel)
> > > [2014-07-23 20:45:24,493] INFO Closing socket connection to
> > > n5.example.com/192.168.1.144. (kafka.network.Processor)
> > > [2014-07-23 20:45:24,555] INFO begin ssl handshake for
> > > n5.example.com/192.168.1.144:48818//192.168.1.144:9092
> > > (kafka.network.security.SSLSocketChannel)
> > > [2014-07-23 20:45:24,566] INFO finished ssl handshake for
> > > n5.example.com/192.168.1.144:48818//192.168.1.144:9092
> > > (kafka.network.security.SSLSocketChannel)
> > >
> > > It's like it did the SSL piece twice :)
> > >
> > > Subsequent puts to the topic did not exhibit this behavior though:
> > >
> > > root@n5[937]:~/kafka_2.10-0-8-2-0.1.0.0> bin/kafka-console-producer.sh
> > > --broker-list n5:9092 --secure --client.security.file
> > > config/client.security.properties --topic test
> > > [2014-07-23 20:45:17,530] WARN Property topic is not valid
> > > (kafka.utils.VerifiableProperties)
> > > 1
> > > [2014-07-23 20:45:24,509] WARN Attempt to reinitialize auth context
> > > (kafka.network.security.SecureAuth$)
> > > 2
> > > 3
> > > 4
> > >
> > > Consuming worked with these options:
> > >
> > > root@n5[918]:~/kafka_2.10-0-8-2-0.1.0.0> bin/kafka-console-consumer.sh
> > > --topic test --zookeeper n5:2181 --from-beginning
> --security.config.file
> > > config/client.security.properties
> > > 1
> > > 2
> > > 3
> > > 4
> > > ^CConsumed 5 messages
> > >
> > > I hope that helps!
> > > Chris
> > >
> > >
> > > On Tue, Jul 22, 2014 at 2:10 PM, Pramod Deshmukh <dp...@gmail.com>
> > > wrote:
> > >
> > > > 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 not valid
> > > > (kafka.utils.VerifiableProperties)
> > > >
> > > > Hare Krishna
> > > >
> > > > [2014-07-18 13:12:45,256] WARN Fetching topic metadata with
> correlation
> > > id
> > > > 0 for topics [Set(secureTopic)] from broker
> > > > [id:0,host:localhost,port:9092,secure:true] failed
> > > > (kafka.client.ClientUtils$)
> > > >
> > > > java.io.EOFException: Received -1 when reading from channel, socket
> has
> > > > likely been closed.
> > > >
> > > > at kafka.utils.Utils$.read(Utils.scala:381)
> > > >
> > > > at
> > > >
> > > >
> > >
> >
> kafka.network.BoundedByteBufferReceive.readFrom(BoundedByteBufferReceive.scala:67)
> > > >
> > > > at kafka.network.Receive$class.readCompletely(Transmission.scala:56)
> > > >
> > > > at
> > > >
> > > >
> > >
> >
> kafka.network.BoundedByteBufferReceive.readCompletely(BoundedByteBufferReceive.scala:29)
> > > >
> > > > at kafka.network.BlockingChannel.receive(BlockingChannel.scala:102)
> > > >
> > > > at kafka.producer.SyncProducer.liftedTree1$1(SyncProducer.scala:79)
> > > >
> > > > at
> > > >
> > > >
> > >
> >
> kafka.producer.SyncProducer.kafka$producer$SyncProducer$$doSend(SyncProducer.scala:76)
> > > >
> > > > at kafka.producer.SyncProducer.send(SyncProducer.scala:117)
> > > >
> > > > at kafka.client.ClientUtils$.fetchTopicMetadata(ClientUtils.scala:58)
> > > >
> > > > at
> > > >
> > >
> >
> kafka.producer.BrokerPartitionInfo.updateInfo(BrokerPartitionInfo.scala:82)
> > > >
> > > > at
> > > >
> > > >
> > >
> >
> kafka.producer.async.DefaultEventHandler$$anonfun$handle$1.apply$mcV$sp(DefaultEventHandler.scala:67)
> > > >
> > > > at kafka.utils.Utils$.swallow(Utils.scala:172)
> > > >
> > > > at kafka.utils.Logging$class.swallowError(Logging.scala:106)
> > > >
> > > > at kafka.utils.Utils$.swallowError(Utils.scala:45)
> > > >
> > > > at
> > > >
> > > >
> > >
> >
> kafka.producer.async.DefaultEventHandler.handle(DefaultEventHandler.scala:67)
> > > >
> > > > at
> > > >
> > > >
> > >
> >
> kafka.producer.async.ProducerSendThread.tryToHandle(ProducerSendThread.scala:104)
> > > >
> > > > at
> > > >
> > > >
> > >
> >
> kafka.producer.async.ProducerSendThread$$anonfun$processEvents$3.apply(ProducerSendThread.scala:87)
> > > >
> > > > at
> > > >
> > > >
> > >
> >
> kafka.producer.async.ProducerSendThread$$anonfun$processEvents$3.apply(ProducerSendThread.scala:67)
> > > >
> > > > at scala.collection.immutable.Stream.foreach(Stream.scala:526)
> > > >
> > > > at
> > > >
> > > >
> > >
> >
> kafka.producer.async.ProducerSendThread.processEvents(ProducerSendThread.scala:66)
> > > >
> > > > at
> > >
> kafka.producer.async.ProducerSendThread.run(ProducerSendThread.scala:44)
> > > >
> > > >
> > > > On Fri, Jul 18, 2014 at 1:20 PM, Pramod Deshmukh <dpramodv@gmail.com
> >
> > > > wrote:
> > > >
> > > > > 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 topic "secureTopic".
> > > > >
> > > > > pdeshmukh$ bin/kafka-topics.sh --list --zookeeper localhost:2181
> > > > >
> > > > > secure.test
> > > > >
> > > > > secureTopic
> > > > >
> > > > > >> Run producer, tried both localhost:9092:true and localhost:9092
> > > > >
> > > > > pdeshmukh$ bin/kafka-console-producer.sh --broker-list
> > > > localhost:9092:true
> > > > > --topic secureTopic
> > > > >
> > > > > [2014-07-18 13:12:29,817] WARN Property topic is not valid
> > > > > (kafka.utils.VerifiableProperties)
> > > > >
> > > > > Hare Krishna
> > > > >
> > > > > [2014-07-18 13:12:45,256] WARN Fetching topic metadata with
> > correlation
> > > > id
> > > > > 0 for topics [Set(secureTopic)] from broker
> > > > > [id:0,host:localhost,port:9092,secure:true] failed
> > > > > (kafka.client.ClientUtils$)
> > > > >
> > > > > java.io.EOFException: Received -1 when reading from channel, socket
> > has
> > > > > likely been closed.
> > > > >
> > > > > at kafka.utils.Utils$.read(Utils.scala:381)
> > > > >
> > > > > at
> > > > >
> > > >
> > >
> >
> kafka.network.BoundedByteBufferReceive.readFrom(BoundedByteBufferReceive.scala:67)
> > > > >
> > > > > at
> kafka.network.Receive$class.readCompletely(Transmission.scala:56)
> > > > >
> > > > > at
> > > > >
> > > >
> > >
> >
> kafka.network.BoundedByteBufferReceive.readCompletely(BoundedByteBufferReceive.scala:29)
> > > > >
> > > > > at kafka.network.BlockingChannel.receive(BlockingChannel.scala:102)
> > > > >
> > > > > at kafka.producer.SyncProducer.liftedTree1$1(SyncProducer.scala:79)
> > > > >
> > > > > at
> > > > >
> > > >
> > >
> >
> kafka.producer.SyncProducer.kafka$producer$SyncProducer$$doSend(SyncProducer.scala:76)
> > > > >
> > > > > at kafka.producer.SyncProducer.send(SyncProducer.scala:117)
> > > > >
> > > > > at
> kafka.client.ClientUtils$.fetchTopicMetadata(ClientUtils.scala:58)
> > > > >
> > > > > at
> > > > >
> > > >
> > >
> >
> kafka.producer.BrokerPartitionInfo.updateInfo(BrokerPartitionInfo.scala:82)
> > > > >
> > > > > at
> > > > >
> > > >
> > >
> >
> kafka.producer.async.DefaultEventHandler$$anonfun$handle$1.apply$mcV$sp(DefaultEventHandler.scala:67)
> > > > >
> > > > > at kafka.utils.Utils$.swallow(Utils.scala:172)
> > > > >
> > > > > at kafka.utils.Logging$class.swallowError(Logging.scala:106)
> > > > >
> > > > > at kafka.utils.Utils$.swallowError(Utils.scala:45)
> > > > >
> > > > > at
> > > > >
> > > >
> > >
> >
> kafka.producer.async.DefaultEventHandler.handle(DefaultEventHandler.scala:67)
> > > > >
> > > > > at
> > > > >
> > > >
> > >
> >
> kafka.producer.async.ProducerSendThread.tryToHandle(ProducerSendThread.scala:104)
> > > > >
> > > > > at
> > > > >
> > > >
> > >
> >
> kafka.producer.async.ProducerSendThread$$anonfun$processEvents$3.apply(ProducerSendThread.scala:87)
> > > > >
> > > > > at
> > > > >
> > > >
> > >
> >
> kafka.producer.async.ProducerSendThread$$anonfun$processEvents$3.apply(ProducerSendThread.scala:67)
> > > > >
> > > > > at scala.collection.immutable.Stream.foreach(Stream.scala:526)
> > > > >
> > > > > at
> > > > >
> > > >
> > >
> >
> kafka.producer.async.ProducerSendThread.processEvents(ProducerSendThread.scala:66)
> > > > >
> > > > > at
> > > >
> > kafka.producer.async.ProducerSendThread.run(ProducerSendThread.scala:44)
> > > > >
> > > > > [2014-07-18 13:12:45,258] ERROR fetching topic metadata for topics
> > > > > [Set(secureTopic)] from broker
> > > > > [ArrayBuffer(id:0,host:localhost,port:9092,secure:true)] failed
> > > > > (kafka.utils.Utils$)
> > > > >
> > > > > kafka.common.KafkaException: fetching topic metadata for topics
> > > > > [Set(secureTopic)] from broker
> > > > > [ArrayBuffer(id:0,host:localhost,port:9092,secure:true)] failed
> > > > >
> > > > > at
> kafka.client.ClientUtils$.fetchTopicMetadata(ClientUtils.scala:72)
> > > > >
> > > > > at
> > > > >
> > > >
> > >
> >
> kafka.producer.BrokerPartitionInfo.updateInfo(BrokerPartitionInfo.scala:82)
> > > > >
> > > > > at
> > > > >
> > > >
> > >
> >
> kafka.producer.async.DefaultEventHandler$$anonfun$handle$1.apply$mcV$sp(DefaultEventHandler.scala:67)
> > > > >
> > > > > at kafka.utils.Utils$.swallow(Utils.scala:172)
> > > > >
> > > > > at kafka.utils.Logging$class.swallowError(Logging.scala:106)
> > > > >
> > > > > at kafka.utils.Utils$.swallowError(Utils.scala:45)
> > > > >
> > > > > at
> > > > >
> > > >
> > >
> >
> kafka.producer.async.DefaultEventHandler.handle(DefaultEventHandler.scala:67)
> > > > >
> > > > > at
> > > > >
> > > >
> > >
> >
> kafka.producer.async.ProducerSendThread.tryToHandle(ProducerSendThread.scala:104)
> > > > >
> > > > > at
> > > > >
> > > >
> > >
> >
> kafka.producer.async.ProducerSendThread$$anonfun$processEvents$3.apply(ProducerSendThread.scala:87)
> > > > >
> > > > > at
> > > > >
> > > >
> > >
> >
> kafka.producer.async.ProducerSendThread$$anonfun$processEvents$3.apply(ProducerSendThread.scala:67)
> > > > >
> > > > > at scala.collection.immutable.Stream.foreach(Stream.scala:526)
> > > > >
> > > > > at
> > > > >
> > > >
> > >
> >
> kafka.producer.async.ProducerSendThread.processEvents(ProducerSendThread.scala:66)
> > > > >
> > > > > at
> > > >
> > kafka.producer.async.ProducerSendThread.run(ProducerSendThread.scala:44)
> > > > >
> > > > > Caused by: java.io.EOFException: Received -1 when reading from
> > channel,
> > > > > socket has likely been closed.
> > > > >
> > > > > at kafka.utils.Utils$.read(Utils.scala:381)
> > > > >
> > > > > at
> > > > >
> > > >
> > >
> >
> kafka.network.BoundedByteBufferReceive.readFrom(BoundedByteBufferReceive.scala:67)
> > > > >
> > > > > at
> kafka.network.Receive$class.readCompletely(Transmission.scala:56)
> > > > >
> > > > > at
> > > > >
> > > >
> > >
> >
> kafka.network.BoundedByteBufferReceive.readCompletely(BoundedByteBufferReceive.scala:29)
> > > > >
> > > > > at kafka.network.BlockingChannel.receive(BlockingChannel.scala:102)
> > > > >
> > > > > at kafka.producer.SyncProducer.liftedTree1$1(SyncProducer.scala:79)
> > > > >
> > > > > at
> > > > >
> > > >
> > >
> >
> kafka.producer.SyncProducer.kafka$producer$SyncProducer$$doSend(SyncProducer.scala:76)
> > > > >
> > > > > at kafka.producer.SyncProducer.send(SyncProducer.scala:117)
> > > > >
> > > > > at
> kafka.client.ClientUtils$.fetchTopicMetadata(ClientUtils.scala:58)
> > > > >
> > > > > ... 12 more
> > > > > [2014-07-18 13:12:45,337] WARN Fetching topic metadata with
> > correlation
> > > > id
> > > > > 1 for topics [Set(secureTopic)] from broker
> > > > > [id:0,host:localhost,port:9092,secure:true] failed
> > > > > (kafka.client.ClientUtils$)
> > > > >
> > > > > 2014-07-18 13:12:46,282] ERROR Failed to send requests for topics
> > > > > secureTopic with correlation ids in [0,8]
> > > > > (kafka.producer.async.DefaultEventHandler)
> > > > >
> > > > > [2014-07-18 13:12:46,283] ERROR Error in handling batch of 1 events
> > > > > (kafka.producer.async.ProducerSendThread)
> > > > >
> > > > > kafka.common.FailedToSendMessageException: Failed to send messages
> > > after
> > > > 3
> > > > > tries.
> > > > >
> > > > > at
> > > > >
> > > >
> > >
> >
> kafka.producer.async.DefaultEventHandler.handle(DefaultEventHandler.scala:90)
> > > > >
> > > > > at
> > > > >
> > > >
> > >
> >
> kafka.producer.async.ProducerSendThread.tryToHandle(ProducerSendThread.scala:104)
> > > > >
> > > > > at
> > > > >
> > > >
> > >
> >
> kafka.producer.async.ProducerSendThread$$anonfun$processEvents$3.apply(ProducerSendThread.scala:87)
> > > > >
> > > > > at
> > > > >
> > > >
> > >
> >
> kafka.producer.async.ProducerSendThread$$anonfun$processEvents$3.apply(ProducerSendThread.scala:67)
> > > > >
> > > > > at scala.collection.immutable.Stream.foreach(Stream.scala:526)
> > > > >
> > > > > at
> > > > >
> > > >
> > >
> >
> kafka.producer.async.ProducerSendThread.processEvents(ProducerSendThread.scala:66)
> > > > >
> > > > > at
> > > >
> > kafka.producer.async.ProducerSendThread.run(ProducerSendThread.scala:44)
> > > > >
> > > > >
> > > > >
> > > > > On Fri, Jul 18, 2014 at 11:56 AM, Joe Stein <jo...@stealth.ly>
> > > > wrote:
> > > > >
> > > > >> 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://www.stealth.ly
> > > > >>  Twitter: @allthingshadoop <
> http://www.twitter.com/allthingshadoop>
> > > > >> ********************************************/
> > > > >>
> > > > >>
> > > > >> On Fri, Jul 18, 2014 at 10:24 AM, Pramod Deshmukh <
> > dpramodv@gmail.com
> > > >
> > > > >> wrote:
> > > > >>
> > > > >> > 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.OutOfMemoryError: Java heap space
> > > > >> >
> > > > >> > *broker log*
> > > > >> >
> > > > >> > INFO begin ssl handshake for localhost/
> > > > 127.0.0.1:50199//127.0.0.1:9092
> > > > >> >
> > > > >> >
> > > > >> >
> > > > >> >
> > > > >> >
> > > > >> > On Thu, Jul 17, 2014 at 6:07 PM, Pramod Deshmukh <
> > > dpramodv@gmail.com>
> > > > >> > wrote:
> > > > >> >
> > > > >> > > 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 <
> > > > >> > relango@salesforce.com>
> > > > >> > > wrote:
> > > > >> > >
> > > > >> > >> 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 <
> > > > dpramodv@gmail.com
> > > > >> >
> > > > >> > >> 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.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 config/server.properties*
> > > > >> > >> >
> > > > >> > >> >
> > > > >> > >> >
> > > > >> > >> > *broker.log:*
> > > > >> > >> >
> > > > >> > >> > [2014-07-17 15:34:46,281] INFO zookeeper state changed
> > > > >> (SyncConnected)
> > > > >> > >> > (org.I0Itec.zkclient.ZkClient)
> > > > >> > >> >
> > > > >> > >> > [2014-07-17 15:34:46,523] INFO Loading log 'secure.test-0'
> > > > >> > >> > (kafka.log.LogManager)
> > > > >> > >> >
> > > > >> > >> > [2014-07-17 15:34:46,558] INFO Recovering unflushed
> segment 0
> > > in
> > > > >> log
> > > > >> > >> > secure.test-0. (kafka.log.Log)
> > > > >> > >> >
> > > > >> > >> > [2014-07-17 15:34:46,571] INFO Completed load of log
> > > > secure.test-0
> > > > >> > with
> > > > >> > >> log
> > > > >> > >> > end offset 0 (kafka.log.Log)
> > > > >> > >> >
> > > > >> > >> > [2014-07-17 15:34:46,582] INFO Starting log cleanup with a
> > > period
> > > > >> of
> > > > >> > >> 60000
> > > > >> > >> > ms. (kafka.log.LogManager)
> > > > >> > >> >
> > > > >> > >> > [2014-07-17 15:34:46,587] INFO Starting log flusher with a
> > > > default
> > > > >> > >> period
> > > > >> > >> > of 9223372036854775807 ms. (kafka.log.LogManager)
> > > > >> > >> >
> > > > >> > >> > [2014-07-17 15:34:46,614] INFO Initializing secure
> > > authentication
> > > > >> > >> > (kafka.network.security.SecureAuth$)
> > > > >> > >> >
> > > > >> > >> > [2014-07-17 15:34:46,678] INFO Secure authentication
> > > > initialization
> > > > >> > has
> > > > >> > >> > been successfully completed
> > > (kafka.network.security.SecureAuth$)
> > > > >> > >> >
> > > > >> > >> > [2014-07-17 15:34:46,691] INFO Awaiting socket connections
> on
> > > > >> > >> 0.0.0.0:9092
> > > > >> > >> > .
> > > > >> > >> > (kafka.network.Acceptor)
> > > > >> > >> >
> > > > >> > >> > [2014-07-17 15:34:46,692] INFO [Socket Server on Broker 0],
> > > > Started
> > > > >> > >> > (kafka.network.SocketServer)
> > > > >> > >> >
> > > > >> > >> > [2014-07-17 15:34:46,794] INFO Will not load MX4J,
> > > mx4j-tools.jar
> > > > >> is
> > > > >> > >> not in
> > > > >> > >> > the classpath (kafka.utils.Mx4jLoader$)
> > > > >> > >> >
> > > > >> > >> > [2014-07-17 15:34:46,837] INFO 0 successfully elected as
> > leader
> > > > >> > >> > (kafka.server.ZookeeperLeaderElector)
> > > > >> > >> >
> > > > >> > >> > [2014-07-17 15:34:47,057] INFO Registered broker 0 at path
> > > > >> > >> /brokers/ids/0
> > > > >> > >> > with address 10.1.100.130:9092. (kafka.utils.ZkUtils$)
> > > > >> > >> >
> > > > >> > >> > [2014-07-17 15:34:47,059] INFO New leader is 0
> > > > >> > >> > (kafka.server.ZookeeperLeaderElector$LeaderChangeListener)
> > > > >> > >> >
> > > > >> > >> > *[2014-07-17 15:34:47,068] INFO [Kafka Server 0], started
> > > > >> > >> > (kafka.server.KafkaServer)*
> > > > >> > >> >
> > > > >> > >> > *[2014-07-17 15:34:47,383] INFO begin ssl handshake for
> > > > >> > >> > /10.1.100.130:9092//10.1.100.130:51685
> > > > >> > >> > <http://10.1.100.130:9092//10.1.100.130:51685>
> > > > >> > >> > (kafka.network.security.SSLSocketChannel)*
> > > > >> > >> >
> > > > >> > >> > *[2014-07-17 15:34:47,392] INFO begin ssl handshake for
> > > > >> > >> > 10.1.100.130/10.1.100.130:51685//10.1.100.130:9092
> > > > >> > >> > <http://10.1.100.130/10.1.100.130:51685//10.1.100.130:9092
> >
> > > > >> > >> > (kafka.network.security.SSLSocketChannel)*
> > > > >> > >> >
> > > > >> > >> > *[2014-07-17 15:34:47,465] INFO finished ssl handshake for
> > > > >> > >> > 10.1.100.130/10.1.100.130:51685//10.1.100.130:9092
> > > > >> > >> > <http://10.1.100.130/10.1.100.130:51685//10.1.100.130:9092
> >
> > > > >> > >> > (kafka.network.security.SSLSocketChannel)*
> > > > >> > >> >
> > > > >> > >> > *[2014-07-17 15:34:47,465] INFO finished ssl handshake for
> > > > >> > >> > /10.1.100.130:9092//10.1.100.130:51685
> > > > >> > >> > <http://10.1.100.130:9092//10.1.100.130:51685>
> > > > >> > >> > (kafka.network.security.SSLSocketChannel)*
> > > > >> > >> >
> > > > >> > >> > *[2014-07-17 15:34:47,617] INFO [ReplicaFetcherManager on
> > > broker
> > > > 0]
> > > > >> > >> Removed
> > > > >> > >> > fetcher for partitions
>  (kafka.server.ReplicaFetcherManager)*
> > > > >> > >> >
> > > > >> > >> > *[2014-07-17 15:34:47,627] INFO [ReplicaFetcherManager on
> > > broker
> > > > 0]
> > > > >> > >> Added
> > > > >> > >> > fetcher for partitions List()
> > > > (kafka.server.ReplicaFetcherManager)*
> > > > >> > >> >
> > > > >> > >> > *[2014-07-17 15:34:47,656] INFO [ReplicaFetcherManager on
> > > broker
> > > > 0]
> > > > >> > >> Removed
> > > > >> > >> > fetcher for partitions [secure.test,0]
> > > > >> > >> > (kafka.server.ReplicaFetcherManager)*
> > > > >> > >> >
> > > > >> > >> > [2014-07-17 15:37:15,970] INFO begin ssl handshake for
> > > > >> > >> > 10.1.100.130/10.1.100.130:51689//10.1.100.130:9092
> > > > >> > >> > (kafka.network.security.SSLSocketChannel)
> > > > >> > >> >
> > > > >> > >> > [2014-07-17 15:37:16,075] INFO begin ssl handshake for
> > > > >> > >> > 10.1.100.130/10.1.100.130:51690//10.1.100.130:9092
> > > > >> > >> > (kafka.network.security.SSLSocketChannel)
> > > > >> > >> >
> > > > >> > >> > [2014-07-17 15:37:16,434] INFO begin ssl handshake for
> > > > >> > >> > 10.1.100.130/10.1.100.130:51691//10.1.100.130:9092
> > > > >> > >> > (kafka.network.security.SSLSocketChannel)
> > > > >> > >> >
> > > > >> > >> > [2014-07-17 15:37:16,530] INFO begin ssl handshake for
> > > > >> > >> > 10.1.100.130/10.1.100.130:51692//10.1.100.130:9092
> > > > >> > >> > (kafka.network.security.SSLSocketChannel)
> > > > >> > >> >
> > > > >> > >> > [2014-07-17 15:37:16,743] INFO begin ssl handshake for
> > > > >> > >> > 10.1.100.130/10.1.100.130:51693//10.1.100.130:9092
> > > > >> > >> > (kafka.network.security.SSLSocketChannel)
> > > > >> > >> >
> > > > >> > >> > [2014-07-17 15:37:16,834] INFO begin ssl handshake for
> > > > >> > >> > 10.1.100.130/10.1.100.130:51694//10.1.100.130:9092
> > > > >> > >> > (kafka.network.security.SSLSocketChannel)
> > > > >> > >> >
> > > > >> > >> > [2014-07-17 15:37:17,043] INFO begin ssl handshake for
> > > > >> > >> > 10.1.100.130/10.1.100.130:51695//10.1.100.130:9092
> > > > >> > >> > (kafka.network.security.SSLSocketChannel)
> > > > >> > >> >
> > > > >> > >> > [2014-07-17 15:37:17,137] INFO begin ssl handshake for
> > > > >> > >> > 10.1.100.130/10.1.100.130:51696//10.1.100.130:9092
> > > > >> > >> > (kafka.network.security.SSLSocketChannel)
> > > > >> > >> >
> > > > >> > >> > [2014-07-17 15:37:17,342] INFO begin ssl handshake for
> > > > >> > >> > 10.1.100.130/10.1.100.130:51697//10.1.100.130:9092
> > > > >> > >> > (kafka.network.security.SSLSocketChannel)
> > > > >> > >> >
> > > > >> > >> >
> > > > >> > >> > *Start producer*
> > > > >> > >> >
> > > > >> > >> > *bin/kafka-console-producer.sh --broker-list
> > 10.1.100.130:9092
> > > > >> :true
> > > > >> > >> > --topic
> > > > >> > >> > secure.test*
> > > > >> > >> >
> > > > >> > >> >
> > > > >> > >> > *producer.log:*
> > > > >> > >> >
> > > > >> > >> > bin/kafka-console-producer.sh --broker-list
> > 10.1.100.130:9092
> > > > :true
> > > > >> > >> --topic
> > > > >> > >> > secure.test
> > > > >> > >> >
> > > > >> > >> > [2014-07-17 15:37:46,889] WARN Property topic is not valid
> > > > >> > >> > (kafka.utils.VerifiableProperties)
> > > > >> > >> >
> > > > >> > >> > Hello Secure Kafka
> > > > >> > >> >
> > > > >> > >> > *[2014-07-17 15:38:14,186] ERROR OOME with size 352518400
> > > > >> > >> > (kafka.network.BoundedByteBufferReceive)*
> > > > >> > >> >
> > > > >> > >> > *java.lang.OutOfMemoryError: Java heap space*
> > > > >> > >> >
> > > > >> > >> > at java.nio.HeapByteBuffer.<init>(HeapByteBuffer.java:57)
> > > > >> > >> >
> > > > >> > >> > at java.nio.ByteBuffer.allocate(ByteBuffer.java:331)
> > > > >> > >> >
> > > > >> > >> > at
> > > > >> > >> >
> > > > >> > >> >
> > > > >> > >>
> > > > >> >
> > > > >>
> > > >
> > >
> >
> kafka.network.BoundedByteBufferReceive.byteBufferAllocate(BoundedByteBufferReceive.scala:80)
> > > > >> > >> >
> > > > >> > >> > at
> > > > >> > >> >
> > > > >> > >> >
> > > > >> > >>
> > > > >> >
> > > > >>
> > > >
> > >
> >
> kafka.network.BoundedByteBufferReceive.readFrom(BoundedByteBufferReceive.scala:63)
> > > > >> > >> >
> > > > >> > >> > at
> > > > >> kafka.network.Receive$class.readCompletely(Transmission.scala:56)
> > > > >> > >> >
> > > > >> > >> > at
> > > > >> > >> >
> > > > >> > >> >
> > > > >> > >>
> > > > >> >
> > > > >>
> > > >
> > >
> >
> kafka.network.BoundedByteBufferReceive.readCompletely(BoundedByteBufferReceive.scala:29)
> > > > >> > >> >
> > > > >> > >> > at
> > > > kafka.network.BlockingChannel.receive(BlockingChannel.scala:102)
> > > > >> > >> >
> > > > >> > >> > at
> > > > kafka.producer.SyncProducer.liftedTree1$1(SyncProducer.scala:79)
> > > > >> > >> >
> > > > >> > >> > at
> > > > >> > >> >
> > > > >> > >> >
> > > > >> > >>
> > > > >> >
> > > > >>
> > > >
> > >
> >
> kafka.producer.SyncProducer.kafka$producer$SyncProducer$$doSend(SyncProducer.scala:76)
> > > > >> > >> >
> > > > >> > >> > at kafka.producer.SyncProducer.send(SyncProducer.scala:117)
> > > > >> > >> >
> > > > >> > >> > at
> > > > >> kafka.client.ClientUtils$.fetchTopicMetadata(ClientUtils.scala:58)
> > > > >> > >> >
> > > > >> > >> > at
> > > > >> > >> >
> > > > >> > >>
> > > > >> >
> > > > >>
> > > >
> > >
> >
> kafka.producer.BrokerPartitionInfo.updateInfo(BrokerPartitionInfo.scala:82)
> > > > >> > >> >
> > > > >> > >> > at
> > > > >> > >> >
> > > > >> > >> >
> > > > >> > >>
> > > > >> >
> > > > >>
> > > >
> > >
> >
> kafka.producer.async.DefaultEventHandler$$anonfun$handle$1.apply$mcV$sp(DefaultEventHandler.scala:67)
> > > > >> > >> >
> > > > >> > >> > at kafka.utils.Utils$.swallow(Utils.scala:172)
> > > > >> > >> >
> > > > >> > >> > at
> kafka.utils.Logging$class.swallowError(Logging.scala:106)
> > > > >> > >> >
> > > > >> > >> > at kafka.utils.Utils$.swallowError(Utils.scala:45)
> > > > >> > >> >
> > > > >> > >> > at
> > > > >> > >> >
> > > > >> > >> >
> > > > >> > >>
> > > > >> >
> > > > >>
> > > >
> > >
> >
> kafka.producer.async.DefaultEventHandler.handle(DefaultEventHandler.scala:67)
> > > > >> > >> >
> > > > >> > >> > at
> > > > >> > >> >
> > > > >> > >> >
> > > > >> > >>
> > > > >> >
> > > > >>
> > > >
> > >
> >
> kafka.producer.async.ProducerSendThread.tryToHandle(ProducerSendThread.scala:104)
> > > > >> > >> >
> > > > >> > >> > at
> > > > >> > >> >
> > > > >> > >> >
> > > > >> > >>
> > > > >> >
> > > > >>
> > > >
> > >
> >
> kafka.producer.async.ProducerSendThread$$anonfun$processEvents$3.apply(ProducerSendThread.scala:87)
> > > > >> > >> >
> > > > >> > >> > at
> > > > >> > >> >
> > > > >> > >> >
> > > > >> > >>
> > > > >> >
> > > > >>
> > > >
> > >
> >
> kafka.producer.async.ProducerSendThread$$anonfun$processEvents$3.apply(ProducerSendThread.scala:67)
> > > > >> > >> >
> > > > >> > >> > at
> > scala.collection.immutable.Stream.foreach(Stream.scala:526)
> > > > >> > >> >
> > > > >> > >> > at
> > > > >> > >> >
> > > > >> > >> >
> > > > >> > >>
> > > > >> >
> > > > >>
> > > >
> > >
> >
> kafka.producer.async.ProducerSendThread.processEvents(ProducerSendThread.scala:66)
> > > > >> > >> >
> > > > >> > >> > at
> > > > >> > >>
> > > > >>
> > >
> kafka.producer.async.ProducerSendThread.run(ProducerSendThread.scala:44)
> > > > >> > >> >
> > > > >> > >> >
> > > > >> > >> >
> > > > >> > >> > On Wed, Jul 16, 2014 at 6:07 PM, Rajasekar Elango <
> > > > >> > >> relango@salesforce.com>
> > > > >> > >> > wrote:
> > > > >> > >> >
> > > > >> > >> > > Pramod,
> > > > >> > >> > >
> > > > >> > >> > >
> > > > >> > >> > > I presented secure kafka configuration and usage at last
> > meet
> > > > >> up. So
> > > > >> > >> hope
> > > > >> > >> > > this
> > > > >> > >> > > video recording <http://www.ustream.tv/recorded/48396701
> > > >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 <
> > > > >> > dpramodv@gmail.com>
> > > > >> > >> > > wrote:
> > > > >> > >> > >
> > > > >> > >> > > > 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 <
> > > > >> > >> dpramodv@gmail.com>
> > > > >> > >> > > > 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, Jul 16, 2014 at 3:57 PM, Joe Stein <
> > > > >> > joe.stein@stealth.ly>
> > > > >> > >> > > 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
> > > > >> > >> (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 Stein
> > > > >> > >> > > > >>  Founder, Principal Consultant
> > > > >> > >> > > > >>  Big Data Open Source Security LLC
> > > > >> > >> > > > >>  http://www.stealth.ly
> > > > >> > >> > > > >>  Twitter: @allthingshadoop <
> > > > >> > >> http://www.twitter.com/allthingshadoop>
> > > > >> > >> > > > >> ********************************************/
> > > > >> > >> > > > >>
> > > > >> > >> > > > >>
> > > > >> > >> > > > >> On Wed, Jul 16, 2014 at 3:49 PM, Pramod Deshmukh <
> > > > >> > >> > dpramodv@gmail.com>
> > > > >> > >> > > > >> wrote:
> > > > >> > >> > > > >>
> > > > >> > >> > > > >> > 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 installed sbt and scala and followed your
> > > > >> instructions
> > > > >> > >> for
> > > > >> > >> > > > >> different
> > > > >> > >> > > > >> > scala versions. I was able to bring zookeeper up
> but
> > > > >> brokers
> > > > >> > >> fail
> > > > >> > >> > to
> > > > >> > >> > > > >> start
> > > > >> > >> > > > >> > with error
> > > > >> > >> > > > >> >
> > > > >> > >> > > > >> > Error: Could not find or load main class
> kafka.Kafka
> > > > >> > >> > > > >> >
> > > > >> > >> > > > >> > I think I am doing something wrong. Can you please
> > > help
> > > > >> me?
> > > > >> > >> > > > >> >
> > > > >> > >> > > > >> > Our current production setup is with 2.8.0 and
> want
> > to
> > > > >> stick
> > > > >> > to
> > > > >> > >> > it.
> > > > >> > >> > > > >> >
> > > > >> > >> > > > >> > Thanks,
> > > > >> > >> > > > >> >
> > > > >> > >> > > > >> > Pramod
> > > > >> > >> > > > >> >
> > > > >> > >> > > > >> >
> > > > >> > >> > > > >> > On Tue, Jun 3, 2014 at 3:57 PM, Joe Stein <
> > > > >> > >> joe.stein@stealth.ly>
> > > > >> > >> > > > wrote:
> > > > >> > >> > > > >> >
> > > > >> > >> > > > >> > > 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.  This comes from what folks are
> > doing
> > > > "in
> > > > >> the
> > > > >> > >> > wild"
> > > > >> > >> > > to
> > > > >> > >> > > > >> > > workaround and implement security with Kafka as
> it
> > > is
> > > > >> today
> > > > >> > >> and
> > > > >> > >> > > also
> > > > >> > >> > > > >> > what I
> > > > >> > >> > > > >> > > have discovered from organizations about their
> > > > >> blockers. It
> > > > >> > >> also
> > > > >> > >> > > > >> picks up
> > > > >> > >> > > > >> > > from the wiki (which I should have time to
> update
> > > > later
> > > > >> in
> > > > >> > >> the
> > > > >> > >> > > week
> > > > >> > >> > > > >> based
> > > > >> > >> > > > >> > > on the below and feedback from the thread).
> > > > >> > >> > > > >> > >
> > > > >> > >> > > > >> > > 1) Transport Layer Security (i.e. SSL)
> > > > >> > >> > > > >> > >
> > > > >> > >> > > > >> > > This also includes client authentication in
> > addition
> > > > to
> > > > >> > >> > in-transit
> > > > >> > >> > > > >> > security
> > > > >> > >> > > > >> > > layer.  This work has been picked up here
> > > > >> > >> > > > >> > >
> https://issues.apache.org/jira/browse/KAFKA-1477
> > > and
> > > > do
> > > > >> > >> > > appreciate
> > > > >> > >> > > > >> any
> > > > >> > >> > > > >> > > thoughts, comments, feedback, tomatoes, whatever
> > for
> > > > >> this
> > > > >> > >> patch.
> > > > >> > >> > >  It
> > > > >> > >> > > > >> is a
> > > > >> > >> > > > >> > > pickup from the fork of the work first done here
> > > > >> > >> > > > >> > >
> > > https://github.com/relango/kafka/tree/kafka_security.
> > > > >> > >> > > > >> > >
> > > > >> > >> > > > >> > > 2) Data encryption at rest.
> > > > >> > >> > > > >> > >
> > > > >> > >> > > > >> > > This is very important and something that can be
> > > > >> > facilitated
> > > > >> > >> > > within
> > > > >> > >> > > > >> the
> > > > >> > >> > > > >> > > wire protocol. It requires an additional map
> data
> > > > >> structure
> > > > >> > >> for
> > > > >> > >> > > the
> > > > >> > >> > > > >> > > "encrypted [data encryption key]". With this map
> > > > >> (either in
> > > > >> > >> your
> > > > >> > >> > > > >> object
> > > > >> > >> > > > >> > or
> > > > >> > >> > > > >> > > in the wire protocol) you can store the
> > dynamically
> > > > >> > generated
> > > > >> > >> > > > >> symmetric
> > > > >> > >> > > > >> > key
> > > > >> > >> > > > >> > > (for each message) and then encrypt the data
> using
> > > > that
> > > > >> > >> > > dynamically
> > > > >> > >> > > > >> > > generated key.  You then encrypt the encryption
> > key
> > > > >> using
> > > > >> > >> each
> > > > >> > >> > > > public
> > > > >> > >> > > > >> key
> > > > >> > >> > > > >> > > for whom is expected to be able to decrypt the
> > > > >> encryption
> > > > >> > >> key to
> > > > >> > >> > > > then
> > > > >> > >> > > > >> > > decrypt the message.  For each public key
> > encrypted
> > > > >> > symmetric
> > > > >> > >> > key
> > > > >> > >> > > > >> (which
> > > > >> > >> > > > >> > is
> > > > >> > >> > > > >> > > now the "encrypted [data encryption key]" along
> > with
> > > > >> which
> > > > >> > >> > public
> > > > >> > >> > > > key
> > > > >> > >> > > > >> it
> > > > >> > >> > > > >> > > was encrypted with for (so a map of [publicKey]
> =
> > > > >> > >> > > > >> > > encryptedDataEncryptionKey) as a chain.   Other
> > > > patterns
> > > > >> > can
> > > > >> > >> be
> > > > >> > >> > > > >> > implemented
> > > > >> > >> > > > >> > > but this is a pretty standard digital enveloping
> > [0]
> > > > >> > pattern
> > > > >> > >> > with
> > > > >> > >> > > > >> only 1
> > > > >> > >> > > > >> > > field added. Other patterns should be able to
> use
> > > that
> > > > >> > field
> > > > >> > >> > to-do
> > > > >> > >> > > > >> their
> > > > >> > >> > > > >> > > implementation too.
> > > > >> > >> > > > >> > >
> > > > >> > >> > > > >> > > 3) Non-repudiation and long term
> non-repudiation.
> > > > >> > >> > > > >> > >
> > > > >> > >> > > > >> > > Non-repudiation is proving data hasn't changed.
> > >  This
> > > > is
> > > > >> > >> often
> > > > >> > >> > (if
> > > > >> > >> > > > not
> > > > >> > >> > > > >> > > always) done with x509 public certificates
> > (chained
> > > > to a
> > > > >> > >> > > certificate
> > > > >> > >> > > > >> > > authority).
> > > > >> > >> > > > >> > >
> > > > >> > >> > > > >> > > Long term non-repudiation is what happens when
> the
> > > > >> > >> certificates
> > > > >> > >> > of
> > > > >> > >> > > > the
> > > > >> > >> > > > >> > > certificate authority are expired (or revoked)
> and
> > > > >> > everything
> > > > >> > >> > ever
> > > > >> > >> > > > >> signed
> > > > >> > >> > > > >> > > (ever) with that certificate's public key then
> > > becomes
> > > > >> "no
> > > > >> > >> > longer
> > > > >> > >> > > > >> > provable
> > > > >> > >> > > > >> > > as ever being authentic".  That is where RFC3126
> > [1]
> > > > and
> > > > >> > >> RFC3161
> > > > >> > >> > > [2]
> > > > >> > >> > > > >> come
> > > > >> > >> > > > >> > > in (or worm drives [hardware], etc).
> > > > >> > >> > > > >> > >
> > > > >> > >> > > > >> > > For either (or both) of these it is an operation
> > of
> > > > the
> > > > >> > >> > encryptor
> > > > >> > >> > > to
> > > > >> > >> > > > >> > > sign/hash the data (with or without third party
> > > > trusted
> > > > >> > >> timestap
> > > > >> > >> > > of
> > > > >> > >> > > > >> the
> > > > >> > >> > > > >> > > signing event) and encrypt that with their own
> > > private
> > > > >> key
> > > > >> > >> and
> > > > >> > >> > > > >> distribute
> > > > >> > >> > > > >> > > the results (before and after encrypting if
> > > required)
> > > > >> along
> > > > >> > >> with
> > > > >> > >> > > > their
> > > > >> > >> > > > >> > > public key. This structure is a bit more complex
> > but
> > > > >> > >> feasible,
> > > > >> > >> > it
> > > > >> > >> > > > is a
> > > > >> > >> > > > >> > map
> > > > >> > >> > > > >> > > of digital signature formats and the chain of
> dig
> > > sig
> > > > >> > >> > > attestations.
> > > > >> > >> > > > >>  The
> > > > >> > >> > > > >> > > map's key being the method (i.e. CRC32, PKCS7
> [3],
> > > > >> > XmlDigSig
> > > > >> > >> > [4])
> > > > >> > >> > > > and
> > > > >> > >> > > > >> > then
> > > > >> > >> > > > >> > > a list of map where that key is "purpose" of
> > > signature
> > > > >> > (what
> > > > >> > >> > your
> > > > >> > >> > > > >> > attesting
> > > > >> > >> > > > >> > > too).  As a sibling field to the list another
> > field
> > > > for
> > > > >> > "the
> > > > >> > >> > > > >> attester" as
> > > > >> > >> > > > >> > > bytes (e.g. their PKCS12 [5] for the map of
> PKCS7
> > > > >> > >> signatures).
> > > > >> > >> > > > >> > >
> > > > >> > >> > > > >> > > 4) Authorization
> > > > >> > >> > > > >> > >
> > > > >> > >> > > > >> > > We should have a policy of "404" for data,
> topics,
> > > > >> > partitions
> > > > >> > >> > > (etc)
> > > > >> > >> > > > if
> > > > >> > >> > > > >> > > authenticated connections do not have access.
>  In
> > > > >> "secure
> > > > >> > >> mode"
> > > > >> > >> > > any
> > > > >> > >> > > > >> non
> > > > >> > >> > > > >> > > authenticated connections should get a "404"
> type
> > > > >> message
> > > > >> > on
> > > > >> > >> > > > >> everything.
> > > > >> > >> > > > >> > > Knowing "something is there" is a security risk
> in
> > > > many
> > > > >> > uses
> > > > >> > >> > > cases.
> > > > >> > >> > > > >>  So
> > > > >> > >> > > > >> > if
> > > > >> > >> > > > >> > > you don't have access you don't even see it.
> >  Baking
> > > > >> "that"
> > > > >> > >> into
> > > > >> > >> > > > Kafka
> > > > >> > >> > > > >> > > along with some interface for entitlement
> (access
> > > > >> > management)
> > > > >> > >> > > > systems
> > > > >> > >> > > > >> > > (pretty standard) is all that I think needs to
> be
> > > done
> > > > >> to
> > > > >> > the
> > > > >> > >> > core
> > > > >> > >> > > > >> > project.
> > > > >> > >> > > > >> > >  I want to tackle item later in the year after
> > > summer
> > > > >> after
> > > > >> > >> the
> > > > >> > >> > > > other
> > > > >> > >> > > > >> > three
> > > > >> > >> > > > >> > > are complete.
> > > > >> > >> > > > >> > >
> > > > >> > >> > > > >> > > I look forward to thoughts on this and anyone
> else
> > > > >> > >> interested in
> > > > >> > >> > > > >> working
> > > > >> > >> > > > >> > > with us on these items.
> > > > >> > >> > > > >> > >
> > > > >> > >> > > > >> > > [0]
> > > > >> > >> > > > >> > >
> > > > >> > >> > > > >> > >
> > > > >> > >> > > > >> >
> > > > >> > >> > > > >>
> > > > >> > >> > > >
> > > > >> > >> > >
> > > > >> > >> >
> > > > >> > >>
> > > > >> >
> > > > >>
> > > >
> > >
> >
> http://www.emc.com/emc-plus/rsa-labs/standards-initiatives/what-is-a-digital-envelope.htm
> > > > >> > >> > > > >> > > [1] http://tools.ietf.org/html/rfc3126
> > > > >> > >> > > > >> > > [2] http://tools.ietf.org/html/rfc3161
> > > > >> > >> > > > >> > > [3]
> > > > >> > >> > > > >> > >
> > > > >> > >> > > > >> > >
> > > > >> > >> > > > >> >
> > > > >> > >> > > > >>
> > > > >> > >> > > >
> > > > >> > >> > >
> > > > >> > >> >
> > > > >> > >>
> > > > >> >
> > > > >>
> > > >
> > >
> >
> http://www.emc.com/emc-plus/rsa-labs/standards-initiatives/pkcs-7-cryptographic-message-syntax-standar.htm
> > > > >> > >> > > > >> > > [4] http://en.wikipedia.org/wiki/XML_Signature
> > > > >> > >> > > > >> > > [5] http://en.wikipedia.org/wiki/PKCS_12
> > > > >> > >> > > > >> > >
> > > > >> > >> > > > >> > > /*******************************************
> > > > >> > >> > > > >> > >  Joe Stein
> > > > >> > >> > > > >> > >  Founder, Principal Consultant
> > > > >> > >> > > > >> > >  Big Data Open Source Security LLC
> > > > >> > >> > > > >> > >  http://www.stealth.ly
> > > > >> > >> > > > >> > >  Twitter: @allthingshadoop <
> > > > >> > >> > > http://www.twitter.com/allthingshadoop>
> > > > >> > >> > > > >> > > ********************************************/
> > > > >> > >> > > > >> > >
> > > > >> > >> > > > >> >
> > > > >> > >> > > > >>
> > > > >> > >> > > > >
> > > > >> > >> > > > >
> > > > >> > >> > > >
> > > > >> > >> > >
> > > > >> > >> > >
> > > > >> > >> > >
> > > > >> > >> > > --
> > > > >> > >> > > Thanks,
> > > > >> > >> > > Raja.
> > > > >> > >> > >
> > > > >> > >> >
> > > > >> > >>
> > > > >> > >>
> > > > >> > >>
> > > > >> > >> --
> > > > >> > >> Thanks,
> > > > >> > >> Raja.
> > > > >> > >>
> > > > >> > >
> > > > >> > >
> > > > >> >
> > > > >>
> > > > >
> > > > >
> > > >
> > >
> >
>
>
>
> --
> Thanks,
> Raja.
>

Re: [DISCUSS] Kafka Security Specific Features

Posted by Rajasekar Elango <re...@salesforce.com>.
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 <ck...@gmail.com> wrote:

> Raja,
>    Which Kafka version is your security enhancement based on?
>
> thanks,
> Cal
>
>
> On Wed, Jul 23, 2014 at 5:01 PM, Chris Neal <cw...@gmail.com> 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
> > console_producer.sh, it is slightly different, which is also slightly
> > different than the scala code for the ConsoleProducer. :)
> >
> > When I changed this:
> >
> > bin/kafka-console-producer.sh --broker-list n5:9092:true --topic test
> >
> > to this:
> >
> > bin/kafka-console-producer.sh --broker-list n5:9092 --secure
> > --client.security.file config/client.security.properties --topic test
> >
> > I was able to push messages to the topic, although I got a WARN about the
> > property "topic" not being valid, even though it is required.
> >
> > Also, the Producer reported this warning to me:
> >
> > [2014-07-23 20:45:24,509] WARN Attempt to reinitialize auth context
> > (kafka.network.security.SecureAuth$)
> >
> > and the broker gave me this:
> > [2014-07-23 20:45:24,114] INFO begin ssl handshake for
> > n5.example.com/192.168.1.144:48817//192.168.1.144:9092
> > (kafka.network.security.SSLSocketChannel)
> > [2014-07-23 20:45:24,374] INFO finished ssl handshake for
> > n5.example.com/192.168.1.144:48817//192.168.1.144:9092
> > (kafka.network.security.SSLSocketChannel)
> > [2014-07-23 20:45:24,493] INFO Closing socket connection to
> > n5.example.com/192.168.1.144. (kafka.network.Processor)
> > [2014-07-23 20:45:24,555] INFO begin ssl handshake for
> > n5.example.com/192.168.1.144:48818//192.168.1.144:9092
> > (kafka.network.security.SSLSocketChannel)
> > [2014-07-23 20:45:24,566] INFO finished ssl handshake for
> > n5.example.com/192.168.1.144:48818//192.168.1.144:9092
> > (kafka.network.security.SSLSocketChannel)
> >
> > It's like it did the SSL piece twice :)
> >
> > Subsequent puts to the topic did not exhibit this behavior though:
> >
> > root@n5[937]:~/kafka_2.10-0-8-2-0.1.0.0> bin/kafka-console-producer.sh
> > --broker-list n5:9092 --secure --client.security.file
> > config/client.security.properties --topic test
> > [2014-07-23 20:45:17,530] WARN Property topic is not valid
> > (kafka.utils.VerifiableProperties)
> > 1
> > [2014-07-23 20:45:24,509] WARN Attempt to reinitialize auth context
> > (kafka.network.security.SecureAuth$)
> > 2
> > 3
> > 4
> >
> > Consuming worked with these options:
> >
> > root@n5[918]:~/kafka_2.10-0-8-2-0.1.0.0> bin/kafka-console-consumer.sh
> > --topic test --zookeeper n5:2181 --from-beginning --security.config.file
> > config/client.security.properties
> > 1
> > 2
> > 3
> > 4
> > ^CConsumed 5 messages
> >
> > I hope that helps!
> > Chris
> >
> >
> > On Tue, Jul 22, 2014 at 2:10 PM, Pramod Deshmukh <dp...@gmail.com>
> > wrote:
> >
> > > 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 not valid
> > > (kafka.utils.VerifiableProperties)
> > >
> > > Hare Krishna
> > >
> > > [2014-07-18 13:12:45,256] WARN Fetching topic metadata with correlation
> > id
> > > 0 for topics [Set(secureTopic)] from broker
> > > [id:0,host:localhost,port:9092,secure:true] failed
> > > (kafka.client.ClientUtils$)
> > >
> > > java.io.EOFException: Received -1 when reading from channel, socket has
> > > likely been closed.
> > >
> > > at kafka.utils.Utils$.read(Utils.scala:381)
> > >
> > > at
> > >
> > >
> >
> kafka.network.BoundedByteBufferReceive.readFrom(BoundedByteBufferReceive.scala:67)
> > >
> > > at kafka.network.Receive$class.readCompletely(Transmission.scala:56)
> > >
> > > at
> > >
> > >
> >
> kafka.network.BoundedByteBufferReceive.readCompletely(BoundedByteBufferReceive.scala:29)
> > >
> > > at kafka.network.BlockingChannel.receive(BlockingChannel.scala:102)
> > >
> > > at kafka.producer.SyncProducer.liftedTree1$1(SyncProducer.scala:79)
> > >
> > > at
> > >
> > >
> >
> kafka.producer.SyncProducer.kafka$producer$SyncProducer$$doSend(SyncProducer.scala:76)
> > >
> > > at kafka.producer.SyncProducer.send(SyncProducer.scala:117)
> > >
> > > at kafka.client.ClientUtils$.fetchTopicMetadata(ClientUtils.scala:58)
> > >
> > > at
> > >
> >
> kafka.producer.BrokerPartitionInfo.updateInfo(BrokerPartitionInfo.scala:82)
> > >
> > > at
> > >
> > >
> >
> kafka.producer.async.DefaultEventHandler$$anonfun$handle$1.apply$mcV$sp(DefaultEventHandler.scala:67)
> > >
> > > at kafka.utils.Utils$.swallow(Utils.scala:172)
> > >
> > > at kafka.utils.Logging$class.swallowError(Logging.scala:106)
> > >
> > > at kafka.utils.Utils$.swallowError(Utils.scala:45)
> > >
> > > at
> > >
> > >
> >
> kafka.producer.async.DefaultEventHandler.handle(DefaultEventHandler.scala:67)
> > >
> > > at
> > >
> > >
> >
> kafka.producer.async.ProducerSendThread.tryToHandle(ProducerSendThread.scala:104)
> > >
> > > at
> > >
> > >
> >
> kafka.producer.async.ProducerSendThread$$anonfun$processEvents$3.apply(ProducerSendThread.scala:87)
> > >
> > > at
> > >
> > >
> >
> kafka.producer.async.ProducerSendThread$$anonfun$processEvents$3.apply(ProducerSendThread.scala:67)
> > >
> > > at scala.collection.immutable.Stream.foreach(Stream.scala:526)
> > >
> > > at
> > >
> > >
> >
> kafka.producer.async.ProducerSendThread.processEvents(ProducerSendThread.scala:66)
> > >
> > > at
> > kafka.producer.async.ProducerSendThread.run(ProducerSendThread.scala:44)
> > >
> > >
> > > On Fri, Jul 18, 2014 at 1:20 PM, Pramod Deshmukh <dp...@gmail.com>
> > > wrote:
> > >
> > > > 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 topic "secureTopic".
> > > >
> > > > pdeshmukh$ bin/kafka-topics.sh --list --zookeeper localhost:2181
> > > >
> > > > secure.test
> > > >
> > > > secureTopic
> > > >
> > > > >> Run producer, tried both localhost:9092:true and localhost:9092
> > > >
> > > > pdeshmukh$ bin/kafka-console-producer.sh --broker-list
> > > localhost:9092:true
> > > > --topic secureTopic
> > > >
> > > > [2014-07-18 13:12:29,817] WARN Property topic is not valid
> > > > (kafka.utils.VerifiableProperties)
> > > >
> > > > Hare Krishna
> > > >
> > > > [2014-07-18 13:12:45,256] WARN Fetching topic metadata with
> correlation
> > > id
> > > > 0 for topics [Set(secureTopic)] from broker
> > > > [id:0,host:localhost,port:9092,secure:true] failed
> > > > (kafka.client.ClientUtils$)
> > > >
> > > > java.io.EOFException: Received -1 when reading from channel, socket
> has
> > > > likely been closed.
> > > >
> > > > at kafka.utils.Utils$.read(Utils.scala:381)
> > > >
> > > > at
> > > >
> > >
> >
> kafka.network.BoundedByteBufferReceive.readFrom(BoundedByteBufferReceive.scala:67)
> > > >
> > > > at kafka.network.Receive$class.readCompletely(Transmission.scala:56)
> > > >
> > > > at
> > > >
> > >
> >
> kafka.network.BoundedByteBufferReceive.readCompletely(BoundedByteBufferReceive.scala:29)
> > > >
> > > > at kafka.network.BlockingChannel.receive(BlockingChannel.scala:102)
> > > >
> > > > at kafka.producer.SyncProducer.liftedTree1$1(SyncProducer.scala:79)
> > > >
> > > > at
> > > >
> > >
> >
> kafka.producer.SyncProducer.kafka$producer$SyncProducer$$doSend(SyncProducer.scala:76)
> > > >
> > > > at kafka.producer.SyncProducer.send(SyncProducer.scala:117)
> > > >
> > > > at kafka.client.ClientUtils$.fetchTopicMetadata(ClientUtils.scala:58)
> > > >
> > > > at
> > > >
> > >
> >
> kafka.producer.BrokerPartitionInfo.updateInfo(BrokerPartitionInfo.scala:82)
> > > >
> > > > at
> > > >
> > >
> >
> kafka.producer.async.DefaultEventHandler$$anonfun$handle$1.apply$mcV$sp(DefaultEventHandler.scala:67)
> > > >
> > > > at kafka.utils.Utils$.swallow(Utils.scala:172)
> > > >
> > > > at kafka.utils.Logging$class.swallowError(Logging.scala:106)
> > > >
> > > > at kafka.utils.Utils$.swallowError(Utils.scala:45)
> > > >
> > > > at
> > > >
> > >
> >
> kafka.producer.async.DefaultEventHandler.handle(DefaultEventHandler.scala:67)
> > > >
> > > > at
> > > >
> > >
> >
> kafka.producer.async.ProducerSendThread.tryToHandle(ProducerSendThread.scala:104)
> > > >
> > > > at
> > > >
> > >
> >
> kafka.producer.async.ProducerSendThread$$anonfun$processEvents$3.apply(ProducerSendThread.scala:87)
> > > >
> > > > at
> > > >
> > >
> >
> kafka.producer.async.ProducerSendThread$$anonfun$processEvents$3.apply(ProducerSendThread.scala:67)
> > > >
> > > > at scala.collection.immutable.Stream.foreach(Stream.scala:526)
> > > >
> > > > at
> > > >
> > >
> >
> kafka.producer.async.ProducerSendThread.processEvents(ProducerSendThread.scala:66)
> > > >
> > > > at
> > >
> kafka.producer.async.ProducerSendThread.run(ProducerSendThread.scala:44)
> > > >
> > > > [2014-07-18 13:12:45,258] ERROR fetching topic metadata for topics
> > > > [Set(secureTopic)] from broker
> > > > [ArrayBuffer(id:0,host:localhost,port:9092,secure:true)] failed
> > > > (kafka.utils.Utils$)
> > > >
> > > > kafka.common.KafkaException: fetching topic metadata for topics
> > > > [Set(secureTopic)] from broker
> > > > [ArrayBuffer(id:0,host:localhost,port:9092,secure:true)] failed
> > > >
> > > > at kafka.client.ClientUtils$.fetchTopicMetadata(ClientUtils.scala:72)
> > > >
> > > > at
> > > >
> > >
> >
> kafka.producer.BrokerPartitionInfo.updateInfo(BrokerPartitionInfo.scala:82)
> > > >
> > > > at
> > > >
> > >
> >
> kafka.producer.async.DefaultEventHandler$$anonfun$handle$1.apply$mcV$sp(DefaultEventHandler.scala:67)
> > > >
> > > > at kafka.utils.Utils$.swallow(Utils.scala:172)
> > > >
> > > > at kafka.utils.Logging$class.swallowError(Logging.scala:106)
> > > >
> > > > at kafka.utils.Utils$.swallowError(Utils.scala:45)
> > > >
> > > > at
> > > >
> > >
> >
> kafka.producer.async.DefaultEventHandler.handle(DefaultEventHandler.scala:67)
> > > >
> > > > at
> > > >
> > >
> >
> kafka.producer.async.ProducerSendThread.tryToHandle(ProducerSendThread.scala:104)
> > > >
> > > > at
> > > >
> > >
> >
> kafka.producer.async.ProducerSendThread$$anonfun$processEvents$3.apply(ProducerSendThread.scala:87)
> > > >
> > > > at
> > > >
> > >
> >
> kafka.producer.async.ProducerSendThread$$anonfun$processEvents$3.apply(ProducerSendThread.scala:67)
> > > >
> > > > at scala.collection.immutable.Stream.foreach(Stream.scala:526)
> > > >
> > > > at
> > > >
> > >
> >
> kafka.producer.async.ProducerSendThread.processEvents(ProducerSendThread.scala:66)
> > > >
> > > > at
> > >
> kafka.producer.async.ProducerSendThread.run(ProducerSendThread.scala:44)
> > > >
> > > > Caused by: java.io.EOFException: Received -1 when reading from
> channel,
> > > > socket has likely been closed.
> > > >
> > > > at kafka.utils.Utils$.read(Utils.scala:381)
> > > >
> > > > at
> > > >
> > >
> >
> kafka.network.BoundedByteBufferReceive.readFrom(BoundedByteBufferReceive.scala:67)
> > > >
> > > > at kafka.network.Receive$class.readCompletely(Transmission.scala:56)
> > > >
> > > > at
> > > >
> > >
> >
> kafka.network.BoundedByteBufferReceive.readCompletely(BoundedByteBufferReceive.scala:29)
> > > >
> > > > at kafka.network.BlockingChannel.receive(BlockingChannel.scala:102)
> > > >
> > > > at kafka.producer.SyncProducer.liftedTree1$1(SyncProducer.scala:79)
> > > >
> > > > at
> > > >
> > >
> >
> kafka.producer.SyncProducer.kafka$producer$SyncProducer$$doSend(SyncProducer.scala:76)
> > > >
> > > > at kafka.producer.SyncProducer.send(SyncProducer.scala:117)
> > > >
> > > > at kafka.client.ClientUtils$.fetchTopicMetadata(ClientUtils.scala:58)
> > > >
> > > > ... 12 more
> > > > [2014-07-18 13:12:45,337] WARN Fetching topic metadata with
> correlation
> > > id
> > > > 1 for topics [Set(secureTopic)] from broker
> > > > [id:0,host:localhost,port:9092,secure:true] failed
> > > > (kafka.client.ClientUtils$)
> > > >
> > > > 2014-07-18 13:12:46,282] ERROR Failed to send requests for topics
> > > > secureTopic with correlation ids in [0,8]
> > > > (kafka.producer.async.DefaultEventHandler)
> > > >
> > > > [2014-07-18 13:12:46,283] ERROR Error in handling batch of 1 events
> > > > (kafka.producer.async.ProducerSendThread)
> > > >
> > > > kafka.common.FailedToSendMessageException: Failed to send messages
> > after
> > > 3
> > > > tries.
> > > >
> > > > at
> > > >
> > >
> >
> kafka.producer.async.DefaultEventHandler.handle(DefaultEventHandler.scala:90)
> > > >
> > > > at
> > > >
> > >
> >
> kafka.producer.async.ProducerSendThread.tryToHandle(ProducerSendThread.scala:104)
> > > >
> > > > at
> > > >
> > >
> >
> kafka.producer.async.ProducerSendThread$$anonfun$processEvents$3.apply(ProducerSendThread.scala:87)
> > > >
> > > > at
> > > >
> > >
> >
> kafka.producer.async.ProducerSendThread$$anonfun$processEvents$3.apply(ProducerSendThread.scala:67)
> > > >
> > > > at scala.collection.immutable.Stream.foreach(Stream.scala:526)
> > > >
> > > > at
> > > >
> > >
> >
> kafka.producer.async.ProducerSendThread.processEvents(ProducerSendThread.scala:66)
> > > >
> > > > at
> > >
> kafka.producer.async.ProducerSendThread.run(ProducerSendThread.scala:44)
> > > >
> > > >
> > > >
> > > > On Fri, Jul 18, 2014 at 11:56 AM, Joe Stein <jo...@stealth.ly>
> > > wrote:
> > > >
> > > >> 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://www.stealth.ly
> > > >>  Twitter: @allthingshadoop <http://www.twitter.com/allthingshadoop>
> > > >> ********************************************/
> > > >>
> > > >>
> > > >> On Fri, Jul 18, 2014 at 10:24 AM, Pramod Deshmukh <
> dpramodv@gmail.com
> > >
> > > >> wrote:
> > > >>
> > > >> > 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.OutOfMemoryError: Java heap space
> > > >> >
> > > >> > *broker log*
> > > >> >
> > > >> > INFO begin ssl handshake for localhost/
> > > 127.0.0.1:50199//127.0.0.1:9092
> > > >> >
> > > >> >
> > > >> >
> > > >> >
> > > >> >
> > > >> > On Thu, Jul 17, 2014 at 6:07 PM, Pramod Deshmukh <
> > dpramodv@gmail.com>
> > > >> > wrote:
> > > >> >
> > > >> > > 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 <
> > > >> > relango@salesforce.com>
> > > >> > > wrote:
> > > >> > >
> > > >> > >> 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 <
> > > dpramodv@gmail.com
> > > >> >
> > > >> > >> 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.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 config/server.properties*
> > > >> > >> >
> > > >> > >> >
> > > >> > >> >
> > > >> > >> > *broker.log:*
> > > >> > >> >
> > > >> > >> > [2014-07-17 15:34:46,281] INFO zookeeper state changed
> > > >> (SyncConnected)
> > > >> > >> > (org.I0Itec.zkclient.ZkClient)
> > > >> > >> >
> > > >> > >> > [2014-07-17 15:34:46,523] INFO Loading log 'secure.test-0'
> > > >> > >> > (kafka.log.LogManager)
> > > >> > >> >
> > > >> > >> > [2014-07-17 15:34:46,558] INFO Recovering unflushed segment 0
> > in
> > > >> log
> > > >> > >> > secure.test-0. (kafka.log.Log)
> > > >> > >> >
> > > >> > >> > [2014-07-17 15:34:46,571] INFO Completed load of log
> > > secure.test-0
> > > >> > with
> > > >> > >> log
> > > >> > >> > end offset 0 (kafka.log.Log)
> > > >> > >> >
> > > >> > >> > [2014-07-17 15:34:46,582] INFO Starting log cleanup with a
> > period
> > > >> of
> > > >> > >> 60000
> > > >> > >> > ms. (kafka.log.LogManager)
> > > >> > >> >
> > > >> > >> > [2014-07-17 15:34:46,587] INFO Starting log flusher with a
> > > default
> > > >> > >> period
> > > >> > >> > of 9223372036854775807 ms. (kafka.log.LogManager)
> > > >> > >> >
> > > >> > >> > [2014-07-17 15:34:46,614] INFO Initializing secure
> > authentication
> > > >> > >> > (kafka.network.security.SecureAuth$)
> > > >> > >> >
> > > >> > >> > [2014-07-17 15:34:46,678] INFO Secure authentication
> > > initialization
> > > >> > has
> > > >> > >> > been successfully completed
> > (kafka.network.security.SecureAuth$)
> > > >> > >> >
> > > >> > >> > [2014-07-17 15:34:46,691] INFO Awaiting socket connections on
> > > >> > >> 0.0.0.0:9092
> > > >> > >> > .
> > > >> > >> > (kafka.network.Acceptor)
> > > >> > >> >
> > > >> > >> > [2014-07-17 15:34:46,692] INFO [Socket Server on Broker 0],
> > > Started
> > > >> > >> > (kafka.network.SocketServer)
> > > >> > >> >
> > > >> > >> > [2014-07-17 15:34:46,794] INFO Will not load MX4J,
> > mx4j-tools.jar
> > > >> is
> > > >> > >> not in
> > > >> > >> > the classpath (kafka.utils.Mx4jLoader$)
> > > >> > >> >
> > > >> > >> > [2014-07-17 15:34:46,837] INFO 0 successfully elected as
> leader
> > > >> > >> > (kafka.server.ZookeeperLeaderElector)
> > > >> > >> >
> > > >> > >> > [2014-07-17 15:34:47,057] INFO Registered broker 0 at path
> > > >> > >> /brokers/ids/0
> > > >> > >> > with address 10.1.100.130:9092. (kafka.utils.ZkUtils$)
> > > >> > >> >
> > > >> > >> > [2014-07-17 15:34:47,059] INFO New leader is 0
> > > >> > >> > (kafka.server.ZookeeperLeaderElector$LeaderChangeListener)
> > > >> > >> >
> > > >> > >> > *[2014-07-17 15:34:47,068] INFO [Kafka Server 0], started
> > > >> > >> > (kafka.server.KafkaServer)*
> > > >> > >> >
> > > >> > >> > *[2014-07-17 15:34:47,383] INFO begin ssl handshake for
> > > >> > >> > /10.1.100.130:9092//10.1.100.130:51685
> > > >> > >> > <http://10.1.100.130:9092//10.1.100.130:51685>
> > > >> > >> > (kafka.network.security.SSLSocketChannel)*
> > > >> > >> >
> > > >> > >> > *[2014-07-17 15:34:47,392] INFO begin ssl handshake for
> > > >> > >> > 10.1.100.130/10.1.100.130:51685//10.1.100.130:9092
> > > >> > >> > <http://10.1.100.130/10.1.100.130:51685//10.1.100.130:9092>
> > > >> > >> > (kafka.network.security.SSLSocketChannel)*
> > > >> > >> >
> > > >> > >> > *[2014-07-17 15:34:47,465] INFO finished ssl handshake for
> > > >> > >> > 10.1.100.130/10.1.100.130:51685//10.1.100.130:9092
> > > >> > >> > <http://10.1.100.130/10.1.100.130:51685//10.1.100.130:9092>
> > > >> > >> > (kafka.network.security.SSLSocketChannel)*
> > > >> > >> >
> > > >> > >> > *[2014-07-17 15:34:47,465] INFO finished ssl handshake for
> > > >> > >> > /10.1.100.130:9092//10.1.100.130:51685
> > > >> > >> > <http://10.1.100.130:9092//10.1.100.130:51685>
> > > >> > >> > (kafka.network.security.SSLSocketChannel)*
> > > >> > >> >
> > > >> > >> > *[2014-07-17 15:34:47,617] INFO [ReplicaFetcherManager on
> > broker
> > > 0]
> > > >> > >> Removed
> > > >> > >> > fetcher for partitions  (kafka.server.ReplicaFetcherManager)*
> > > >> > >> >
> > > >> > >> > *[2014-07-17 15:34:47,627] INFO [ReplicaFetcherManager on
> > broker
> > > 0]
> > > >> > >> Added
> > > >> > >> > fetcher for partitions List()
> > > (kafka.server.ReplicaFetcherManager)*
> > > >> > >> >
> > > >> > >> > *[2014-07-17 15:34:47,656] INFO [ReplicaFetcherManager on
> > broker
> > > 0]
> > > >> > >> Removed
> > > >> > >> > fetcher for partitions [secure.test,0]
> > > >> > >> > (kafka.server.ReplicaFetcherManager)*
> > > >> > >> >
> > > >> > >> > [2014-07-17 15:37:15,970] INFO begin ssl handshake for
> > > >> > >> > 10.1.100.130/10.1.100.130:51689//10.1.100.130:9092
> > > >> > >> > (kafka.network.security.SSLSocketChannel)
> > > >> > >> >
> > > >> > >> > [2014-07-17 15:37:16,075] INFO begin ssl handshake for
> > > >> > >> > 10.1.100.130/10.1.100.130:51690//10.1.100.130:9092
> > > >> > >> > (kafka.network.security.SSLSocketChannel)
> > > >> > >> >
> > > >> > >> > [2014-07-17 15:37:16,434] INFO begin ssl handshake for
> > > >> > >> > 10.1.100.130/10.1.100.130:51691//10.1.100.130:9092
> > > >> > >> > (kafka.network.security.SSLSocketChannel)
> > > >> > >> >
> > > >> > >> > [2014-07-17 15:37:16,530] INFO begin ssl handshake for
> > > >> > >> > 10.1.100.130/10.1.100.130:51692//10.1.100.130:9092
> > > >> > >> > (kafka.network.security.SSLSocketChannel)
> > > >> > >> >
> > > >> > >> > [2014-07-17 15:37:16,743] INFO begin ssl handshake for
> > > >> > >> > 10.1.100.130/10.1.100.130:51693//10.1.100.130:9092
> > > >> > >> > (kafka.network.security.SSLSocketChannel)
> > > >> > >> >
> > > >> > >> > [2014-07-17 15:37:16,834] INFO begin ssl handshake for
> > > >> > >> > 10.1.100.130/10.1.100.130:51694//10.1.100.130:9092
> > > >> > >> > (kafka.network.security.SSLSocketChannel)
> > > >> > >> >
> > > >> > >> > [2014-07-17 15:37:17,043] INFO begin ssl handshake for
> > > >> > >> > 10.1.100.130/10.1.100.130:51695//10.1.100.130:9092
> > > >> > >> > (kafka.network.security.SSLSocketChannel)
> > > >> > >> >
> > > >> > >> > [2014-07-17 15:37:17,137] INFO begin ssl handshake for
> > > >> > >> > 10.1.100.130/10.1.100.130:51696//10.1.100.130:9092
> > > >> > >> > (kafka.network.security.SSLSocketChannel)
> > > >> > >> >
> > > >> > >> > [2014-07-17 15:37:17,342] INFO begin ssl handshake for
> > > >> > >> > 10.1.100.130/10.1.100.130:51697//10.1.100.130:9092
> > > >> > >> > (kafka.network.security.SSLSocketChannel)
> > > >> > >> >
> > > >> > >> >
> > > >> > >> > *Start producer*
> > > >> > >> >
> > > >> > >> > *bin/kafka-console-producer.sh --broker-list
> 10.1.100.130:9092
> > > >> :true
> > > >> > >> > --topic
> > > >> > >> > secure.test*
> > > >> > >> >
> > > >> > >> >
> > > >> > >> > *producer.log:*
> > > >> > >> >
> > > >> > >> > bin/kafka-console-producer.sh --broker-list
> 10.1.100.130:9092
> > > :true
> > > >> > >> --topic
> > > >> > >> > secure.test
> > > >> > >> >
> > > >> > >> > [2014-07-17 15:37:46,889] WARN Property topic is not valid
> > > >> > >> > (kafka.utils.VerifiableProperties)
> > > >> > >> >
> > > >> > >> > Hello Secure Kafka
> > > >> > >> >
> > > >> > >> > *[2014-07-17 15:38:14,186] ERROR OOME with size 352518400
> > > >> > >> > (kafka.network.BoundedByteBufferReceive)*
> > > >> > >> >
> > > >> > >> > *java.lang.OutOfMemoryError: Java heap space*
> > > >> > >> >
> > > >> > >> > at java.nio.HeapByteBuffer.<init>(HeapByteBuffer.java:57)
> > > >> > >> >
> > > >> > >> > at java.nio.ByteBuffer.allocate(ByteBuffer.java:331)
> > > >> > >> >
> > > >> > >> > at
> > > >> > >> >
> > > >> > >> >
> > > >> > >>
> > > >> >
> > > >>
> > >
> >
> kafka.network.BoundedByteBufferReceive.byteBufferAllocate(BoundedByteBufferReceive.scala:80)
> > > >> > >> >
> > > >> > >> > at
> > > >> > >> >
> > > >> > >> >
> > > >> > >>
> > > >> >
> > > >>
> > >
> >
> kafka.network.BoundedByteBufferReceive.readFrom(BoundedByteBufferReceive.scala:63)
> > > >> > >> >
> > > >> > >> > at
> > > >> kafka.network.Receive$class.readCompletely(Transmission.scala:56)
> > > >> > >> >
> > > >> > >> > at
> > > >> > >> >
> > > >> > >> >
> > > >> > >>
> > > >> >
> > > >>
> > >
> >
> kafka.network.BoundedByteBufferReceive.readCompletely(BoundedByteBufferReceive.scala:29)
> > > >> > >> >
> > > >> > >> > at
> > > kafka.network.BlockingChannel.receive(BlockingChannel.scala:102)
> > > >> > >> >
> > > >> > >> > at
> > > kafka.producer.SyncProducer.liftedTree1$1(SyncProducer.scala:79)
> > > >> > >> >
> > > >> > >> > at
> > > >> > >> >
> > > >> > >> >
> > > >> > >>
> > > >> >
> > > >>
> > >
> >
> kafka.producer.SyncProducer.kafka$producer$SyncProducer$$doSend(SyncProducer.scala:76)
> > > >> > >> >
> > > >> > >> > at kafka.producer.SyncProducer.send(SyncProducer.scala:117)
> > > >> > >> >
> > > >> > >> > at
> > > >> kafka.client.ClientUtils$.fetchTopicMetadata(ClientUtils.scala:58)
> > > >> > >> >
> > > >> > >> > at
> > > >> > >> >
> > > >> > >>
> > > >> >
> > > >>
> > >
> >
> kafka.producer.BrokerPartitionInfo.updateInfo(BrokerPartitionInfo.scala:82)
> > > >> > >> >
> > > >> > >> > at
> > > >> > >> >
> > > >> > >> >
> > > >> > >>
> > > >> >
> > > >>
> > >
> >
> kafka.producer.async.DefaultEventHandler$$anonfun$handle$1.apply$mcV$sp(DefaultEventHandler.scala:67)
> > > >> > >> >
> > > >> > >> > at kafka.utils.Utils$.swallow(Utils.scala:172)
> > > >> > >> >
> > > >> > >> > at kafka.utils.Logging$class.swallowError(Logging.scala:106)
> > > >> > >> >
> > > >> > >> > at kafka.utils.Utils$.swallowError(Utils.scala:45)
> > > >> > >> >
> > > >> > >> > at
> > > >> > >> >
> > > >> > >> >
> > > >> > >>
> > > >> >
> > > >>
> > >
> >
> kafka.producer.async.DefaultEventHandler.handle(DefaultEventHandler.scala:67)
> > > >> > >> >
> > > >> > >> > at
> > > >> > >> >
> > > >> > >> >
> > > >> > >>
> > > >> >
> > > >>
> > >
> >
> kafka.producer.async.ProducerSendThread.tryToHandle(ProducerSendThread.scala:104)
> > > >> > >> >
> > > >> > >> > at
> > > >> > >> >
> > > >> > >> >
> > > >> > >>
> > > >> >
> > > >>
> > >
> >
> kafka.producer.async.ProducerSendThread$$anonfun$processEvents$3.apply(ProducerSendThread.scala:87)
> > > >> > >> >
> > > >> > >> > at
> > > >> > >> >
> > > >> > >> >
> > > >> > >>
> > > >> >
> > > >>
> > >
> >
> kafka.producer.async.ProducerSendThread$$anonfun$processEvents$3.apply(ProducerSendThread.scala:67)
> > > >> > >> >
> > > >> > >> > at
> scala.collection.immutable.Stream.foreach(Stream.scala:526)
> > > >> > >> >
> > > >> > >> > at
> > > >> > >> >
> > > >> > >> >
> > > >> > >>
> > > >> >
> > > >>
> > >
> >
> kafka.producer.async.ProducerSendThread.processEvents(ProducerSendThread.scala:66)
> > > >> > >> >
> > > >> > >> > at
> > > >> > >>
> > > >>
> > kafka.producer.async.ProducerSendThread.run(ProducerSendThread.scala:44)
> > > >> > >> >
> > > >> > >> >
> > > >> > >> >
> > > >> > >> > On Wed, Jul 16, 2014 at 6:07 PM, Rajasekar Elango <
> > > >> > >> relango@salesforce.com>
> > > >> > >> > wrote:
> > > >> > >> >
> > > >> > >> > > Pramod,
> > > >> > >> > >
> > > >> > >> > >
> > > >> > >> > > I presented secure kafka configuration and usage at last
> meet
> > > >> up. So
> > > >> > >> hope
> > > >> > >> > > this
> > > >> > >> > > video recording <http://www.ustream.tv/recorded/48396701
> > >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 <
> > > >> > dpramodv@gmail.com>
> > > >> > >> > > wrote:
> > > >> > >> > >
> > > >> > >> > > > 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 <
> > > >> > >> dpramodv@gmail.com>
> > > >> > >> > > > 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, Jul 16, 2014 at 3:57 PM, Joe Stein <
> > > >> > joe.stein@stealth.ly>
> > > >> > >> > > 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
> > > >> > >> (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 Stein
> > > >> > >> > > > >>  Founder, Principal Consultant
> > > >> > >> > > > >>  Big Data Open Source Security LLC
> > > >> > >> > > > >>  http://www.stealth.ly
> > > >> > >> > > > >>  Twitter: @allthingshadoop <
> > > >> > >> http://www.twitter.com/allthingshadoop>
> > > >> > >> > > > >> ********************************************/
> > > >> > >> > > > >>
> > > >> > >> > > > >>
> > > >> > >> > > > >> On Wed, Jul 16, 2014 at 3:49 PM, Pramod Deshmukh <
> > > >> > >> > dpramodv@gmail.com>
> > > >> > >> > > > >> wrote:
> > > >> > >> > > > >>
> > > >> > >> > > > >> > 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 installed sbt and scala and followed your
> > > >> instructions
> > > >> > >> for
> > > >> > >> > > > >> different
> > > >> > >> > > > >> > scala versions. I was able to bring zookeeper up but
> > > >> brokers
> > > >> > >> fail
> > > >> > >> > to
> > > >> > >> > > > >> start
> > > >> > >> > > > >> > with error
> > > >> > >> > > > >> >
> > > >> > >> > > > >> > Error: Could not find or load main class kafka.Kafka
> > > >> > >> > > > >> >
> > > >> > >> > > > >> > I think I am doing something wrong. Can you please
> > help
> > > >> me?
> > > >> > >> > > > >> >
> > > >> > >> > > > >> > Our current production setup is with 2.8.0 and want
> to
> > > >> stick
> > > >> > to
> > > >> > >> > it.
> > > >> > >> > > > >> >
> > > >> > >> > > > >> > Thanks,
> > > >> > >> > > > >> >
> > > >> > >> > > > >> > Pramod
> > > >> > >> > > > >> >
> > > >> > >> > > > >> >
> > > >> > >> > > > >> > On Tue, Jun 3, 2014 at 3:57 PM, Joe Stein <
> > > >> > >> joe.stein@stealth.ly>
> > > >> > >> > > > wrote:
> > > >> > >> > > > >> >
> > > >> > >> > > > >> > > 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.  This comes from what folks are
> doing
> > > "in
> > > >> the
> > > >> > >> > wild"
> > > >> > >> > > to
> > > >> > >> > > > >> > > workaround and implement security with Kafka as it
> > is
> > > >> today
> > > >> > >> and
> > > >> > >> > > also
> > > >> > >> > > > >> > what I
> > > >> > >> > > > >> > > have discovered from organizations about their
> > > >> blockers. It
> > > >> > >> also
> > > >> > >> > > > >> picks up
> > > >> > >> > > > >> > > from the wiki (which I should have time to update
> > > later
> > > >> in
> > > >> > >> the
> > > >> > >> > > week
> > > >> > >> > > > >> based
> > > >> > >> > > > >> > > on the below and feedback from the thread).
> > > >> > >> > > > >> > >
> > > >> > >> > > > >> > > 1) Transport Layer Security (i.e. SSL)
> > > >> > >> > > > >> > >
> > > >> > >> > > > >> > > This also includes client authentication in
> addition
> > > to
> > > >> > >> > in-transit
> > > >> > >> > > > >> > security
> > > >> > >> > > > >> > > layer.  This work has been picked up here
> > > >> > >> > > > >> > > https://issues.apache.org/jira/browse/KAFKA-1477
> > and
> > > do
> > > >> > >> > > appreciate
> > > >> > >> > > > >> any
> > > >> > >> > > > >> > > thoughts, comments, feedback, tomatoes, whatever
> for
> > > >> this
> > > >> > >> patch.
> > > >> > >> > >  It
> > > >> > >> > > > >> is a
> > > >> > >> > > > >> > > pickup from the fork of the work first done here
> > > >> > >> > > > >> > >
> > https://github.com/relango/kafka/tree/kafka_security.
> > > >> > >> > > > >> > >
> > > >> > >> > > > >> > > 2) Data encryption at rest.
> > > >> > >> > > > >> > >
> > > >> > >> > > > >> > > This is very important and something that can be
> > > >> > facilitated
> > > >> > >> > > within
> > > >> > >> > > > >> the
> > > >> > >> > > > >> > > wire protocol. It requires an additional map data
> > > >> structure
> > > >> > >> for
> > > >> > >> > > the
> > > >> > >> > > > >> > > "encrypted [data encryption key]". With this map
> > > >> (either in
> > > >> > >> your
> > > >> > >> > > > >> object
> > > >> > >> > > > >> > or
> > > >> > >> > > > >> > > in the wire protocol) you can store the
> dynamically
> > > >> > generated
> > > >> > >> > > > >> symmetric
> > > >> > >> > > > >> > key
> > > >> > >> > > > >> > > (for each message) and then encrypt the data using
> > > that
> > > >> > >> > > dynamically
> > > >> > >> > > > >> > > generated key.  You then encrypt the encryption
> key
> > > >> using
> > > >> > >> each
> > > >> > >> > > > public
> > > >> > >> > > > >> key
> > > >> > >> > > > >> > > for whom is expected to be able to decrypt the
> > > >> encryption
> > > >> > >> key to
> > > >> > >> > > > then
> > > >> > >> > > > >> > > decrypt the message.  For each public key
> encrypted
> > > >> > symmetric
> > > >> > >> > key
> > > >> > >> > > > >> (which
> > > >> > >> > > > >> > is
> > > >> > >> > > > >> > > now the "encrypted [data encryption key]" along
> with
> > > >> which
> > > >> > >> > public
> > > >> > >> > > > key
> > > >> > >> > > > >> it
> > > >> > >> > > > >> > > was encrypted with for (so a map of [publicKey] =
> > > >> > >> > > > >> > > encryptedDataEncryptionKey) as a chain.   Other
> > > patterns
> > > >> > can
> > > >> > >> be
> > > >> > >> > > > >> > implemented
> > > >> > >> > > > >> > > but this is a pretty standard digital enveloping
> [0]
> > > >> > pattern
> > > >> > >> > with
> > > >> > >> > > > >> only 1
> > > >> > >> > > > >> > > field added. Other patterns should be able to use
> > that
> > > >> > field
> > > >> > >> > to-do
> > > >> > >> > > > >> their
> > > >> > >> > > > >> > > implementation too.
> > > >> > >> > > > >> > >
> > > >> > >> > > > >> > > 3) Non-repudiation and long term non-repudiation.
> > > >> > >> > > > >> > >
> > > >> > >> > > > >> > > Non-repudiation is proving data hasn't changed.
> >  This
> > > is
> > > >> > >> often
> > > >> > >> > (if
> > > >> > >> > > > not
> > > >> > >> > > > >> > > always) done with x509 public certificates
> (chained
> > > to a
> > > >> > >> > > certificate
> > > >> > >> > > > >> > > authority).
> > > >> > >> > > > >> > >
> > > >> > >> > > > >> > > Long term non-repudiation is what happens when the
> > > >> > >> certificates
> > > >> > >> > of
> > > >> > >> > > > the
> > > >> > >> > > > >> > > certificate authority are expired (or revoked) and
> > > >> > everything
> > > >> > >> > ever
> > > >> > >> > > > >> signed
> > > >> > >> > > > >> > > (ever) with that certificate's public key then
> > becomes
> > > >> "no
> > > >> > >> > longer
> > > >> > >> > > > >> > provable
> > > >> > >> > > > >> > > as ever being authentic".  That is where RFC3126
> [1]
> > > and
> > > >> > >> RFC3161
> > > >> > >> > > [2]
> > > >> > >> > > > >> come
> > > >> > >> > > > >> > > in (or worm drives [hardware], etc).
> > > >> > >> > > > >> > >
> > > >> > >> > > > >> > > For either (or both) of these it is an operation
> of
> > > the
> > > >> > >> > encryptor
> > > >> > >> > > to
> > > >> > >> > > > >> > > sign/hash the data (with or without third party
> > > trusted
> > > >> > >> timestap
> > > >> > >> > > of
> > > >> > >> > > > >> the
> > > >> > >> > > > >> > > signing event) and encrypt that with their own
> > private
> > > >> key
> > > >> > >> and
> > > >> > >> > > > >> distribute
> > > >> > >> > > > >> > > the results (before and after encrypting if
> > required)
> > > >> along
> > > >> > >> with
> > > >> > >> > > > their
> > > >> > >> > > > >> > > public key. This structure is a bit more complex
> but
> > > >> > >> feasible,
> > > >> > >> > it
> > > >> > >> > > > is a
> > > >> > >> > > > >> > map
> > > >> > >> > > > >> > > of digital signature formats and the chain of dig
> > sig
> > > >> > >> > > attestations.
> > > >> > >> > > > >>  The
> > > >> > >> > > > >> > > map's key being the method (i.e. CRC32, PKCS7 [3],
> > > >> > XmlDigSig
> > > >> > >> > [4])
> > > >> > >> > > > and
> > > >> > >> > > > >> > then
> > > >> > >> > > > >> > > a list of map where that key is "purpose" of
> > signature
> > > >> > (what
> > > >> > >> > your
> > > >> > >> > > > >> > attesting
> > > >> > >> > > > >> > > too).  As a sibling field to the list another
> field
> > > for
> > > >> > "the
> > > >> > >> > > > >> attester" as
> > > >> > >> > > > >> > > bytes (e.g. their PKCS12 [5] for the map of PKCS7
> > > >> > >> signatures).
> > > >> > >> > > > >> > >
> > > >> > >> > > > >> > > 4) Authorization
> > > >> > >> > > > >> > >
> > > >> > >> > > > >> > > We should have a policy of "404" for data, topics,
> > > >> > partitions
> > > >> > >> > > (etc)
> > > >> > >> > > > if
> > > >> > >> > > > >> > > authenticated connections do not have access.  In
> > > >> "secure
> > > >> > >> mode"
> > > >> > >> > > any
> > > >> > >> > > > >> non
> > > >> > >> > > > >> > > authenticated connections should get a "404" type
> > > >> message
> > > >> > on
> > > >> > >> > > > >> everything.
> > > >> > >> > > > >> > > Knowing "something is there" is a security risk in
> > > many
> > > >> > uses
> > > >> > >> > > cases.
> > > >> > >> > > > >>  So
> > > >> > >> > > > >> > if
> > > >> > >> > > > >> > > you don't have access you don't even see it.
>  Baking
> > > >> "that"
> > > >> > >> into
> > > >> > >> > > > Kafka
> > > >> > >> > > > >> > > along with some interface for entitlement (access
> > > >> > management)
> > > >> > >> > > > systems
> > > >> > >> > > > >> > > (pretty standard) is all that I think needs to be
> > done
> > > >> to
> > > >> > the
> > > >> > >> > core
> > > >> > >> > > > >> > project.
> > > >> > >> > > > >> > >  I want to tackle item later in the year after
> > summer
> > > >> after
> > > >> > >> the
> > > >> > >> > > > other
> > > >> > >> > > > >> > three
> > > >> > >> > > > >> > > are complete.
> > > >> > >> > > > >> > >
> > > >> > >> > > > >> > > I look forward to thoughts on this and anyone else
> > > >> > >> interested in
> > > >> > >> > > > >> working
> > > >> > >> > > > >> > > with us on these items.
> > > >> > >> > > > >> > >
> > > >> > >> > > > >> > > [0]
> > > >> > >> > > > >> > >
> > > >> > >> > > > >> > >
> > > >> > >> > > > >> >
> > > >> > >> > > > >>
> > > >> > >> > > >
> > > >> > >> > >
> > > >> > >> >
> > > >> > >>
> > > >> >
> > > >>
> > >
> >
> http://www.emc.com/emc-plus/rsa-labs/standards-initiatives/what-is-a-digital-envelope.htm
> > > >> > >> > > > >> > > [1] http://tools.ietf.org/html/rfc3126
> > > >> > >> > > > >> > > [2] http://tools.ietf.org/html/rfc3161
> > > >> > >> > > > >> > > [3]
> > > >> > >> > > > >> > >
> > > >> > >> > > > >> > >
> > > >> > >> > > > >> >
> > > >> > >> > > > >>
> > > >> > >> > > >
> > > >> > >> > >
> > > >> > >> >
> > > >> > >>
> > > >> >
> > > >>
> > >
> >
> http://www.emc.com/emc-plus/rsa-labs/standards-initiatives/pkcs-7-cryptographic-message-syntax-standar.htm
> > > >> > >> > > > >> > > [4] http://en.wikipedia.org/wiki/XML_Signature
> > > >> > >> > > > >> > > [5] http://en.wikipedia.org/wiki/PKCS_12
> > > >> > >> > > > >> > >
> > > >> > >> > > > >> > > /*******************************************
> > > >> > >> > > > >> > >  Joe Stein
> > > >> > >> > > > >> > >  Founder, Principal Consultant
> > > >> > >> > > > >> > >  Big Data Open Source Security LLC
> > > >> > >> > > > >> > >  http://www.stealth.ly
> > > >> > >> > > > >> > >  Twitter: @allthingshadoop <
> > > >> > >> > > http://www.twitter.com/allthingshadoop>
> > > >> > >> > > > >> > > ********************************************/
> > > >> > >> > > > >> > >
> > > >> > >> > > > >> >
> > > >> > >> > > > >>
> > > >> > >> > > > >
> > > >> > >> > > > >
> > > >> > >> > > >
> > > >> > >> > >
> > > >> > >> > >
> > > >> > >> > >
> > > >> > >> > > --
> > > >> > >> > > Thanks,
> > > >> > >> > > Raja.
> > > >> > >> > >
> > > >> > >> >
> > > >> > >>
> > > >> > >>
> > > >> > >>
> > > >> > >> --
> > > >> > >> Thanks,
> > > >> > >> Raja.
> > > >> > >>
> > > >> > >
> > > >> > >
> > > >> >
> > > >>
> > > >
> > > >
> > >
> >
>



-- 
Thanks,
Raja.

Re: [DISCUSS] Kafka Security Specific Features

Posted by Calvin Lei <ck...@gmail.com>.
Raja,
   Which Kafka version is your security enhancement based on?

thanks,
Cal


On Wed, Jul 23, 2014 at 5:01 PM, Chris Neal <cw...@gmail.com> 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
> console_producer.sh, it is slightly different, which is also slightly
> different than the scala code for the ConsoleProducer. :)
>
> When I changed this:
>
> bin/kafka-console-producer.sh --broker-list n5:9092:true --topic test
>
> to this:
>
> bin/kafka-console-producer.sh --broker-list n5:9092 --secure
> --client.security.file config/client.security.properties --topic test
>
> I was able to push messages to the topic, although I got a WARN about the
> property "topic" not being valid, even though it is required.
>
> Also, the Producer reported this warning to me:
>
> [2014-07-23 20:45:24,509] WARN Attempt to reinitialize auth context
> (kafka.network.security.SecureAuth$)
>
> and the broker gave me this:
> [2014-07-23 20:45:24,114] INFO begin ssl handshake for
> n5.example.com/192.168.1.144:48817//192.168.1.144:9092
> (kafka.network.security.SSLSocketChannel)
> [2014-07-23 20:45:24,374] INFO finished ssl handshake for
> n5.example.com/192.168.1.144:48817//192.168.1.144:9092
> (kafka.network.security.SSLSocketChannel)
> [2014-07-23 20:45:24,493] INFO Closing socket connection to
> n5.example.com/192.168.1.144. (kafka.network.Processor)
> [2014-07-23 20:45:24,555] INFO begin ssl handshake for
> n5.example.com/192.168.1.144:48818//192.168.1.144:9092
> (kafka.network.security.SSLSocketChannel)
> [2014-07-23 20:45:24,566] INFO finished ssl handshake for
> n5.example.com/192.168.1.144:48818//192.168.1.144:9092
> (kafka.network.security.SSLSocketChannel)
>
> It's like it did the SSL piece twice :)
>
> Subsequent puts to the topic did not exhibit this behavior though:
>
> root@n5[937]:~/kafka_2.10-0-8-2-0.1.0.0> bin/kafka-console-producer.sh
> --broker-list n5:9092 --secure --client.security.file
> config/client.security.properties --topic test
> [2014-07-23 20:45:17,530] WARN Property topic is not valid
> (kafka.utils.VerifiableProperties)
> 1
> [2014-07-23 20:45:24,509] WARN Attempt to reinitialize auth context
> (kafka.network.security.SecureAuth$)
> 2
> 3
> 4
>
> Consuming worked with these options:
>
> root@n5[918]:~/kafka_2.10-0-8-2-0.1.0.0> bin/kafka-console-consumer.sh
> --topic test --zookeeper n5:2181 --from-beginning --security.config.file
> config/client.security.properties
> 1
> 2
> 3
> 4
> ^CConsumed 5 messages
>
> I hope that helps!
> Chris
>
>
> On Tue, Jul 22, 2014 at 2:10 PM, Pramod Deshmukh <dp...@gmail.com>
> wrote:
>
> > 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 not valid
> > (kafka.utils.VerifiableProperties)
> >
> > Hare Krishna
> >
> > [2014-07-18 13:12:45,256] WARN Fetching topic metadata with correlation
> id
> > 0 for topics [Set(secureTopic)] from broker
> > [id:0,host:localhost,port:9092,secure:true] failed
> > (kafka.client.ClientUtils$)
> >
> > java.io.EOFException: Received -1 when reading from channel, socket has
> > likely been closed.
> >
> > at kafka.utils.Utils$.read(Utils.scala:381)
> >
> > at
> >
> >
> kafka.network.BoundedByteBufferReceive.readFrom(BoundedByteBufferReceive.scala:67)
> >
> > at kafka.network.Receive$class.readCompletely(Transmission.scala:56)
> >
> > at
> >
> >
> kafka.network.BoundedByteBufferReceive.readCompletely(BoundedByteBufferReceive.scala:29)
> >
> > at kafka.network.BlockingChannel.receive(BlockingChannel.scala:102)
> >
> > at kafka.producer.SyncProducer.liftedTree1$1(SyncProducer.scala:79)
> >
> > at
> >
> >
> kafka.producer.SyncProducer.kafka$producer$SyncProducer$$doSend(SyncProducer.scala:76)
> >
> > at kafka.producer.SyncProducer.send(SyncProducer.scala:117)
> >
> > at kafka.client.ClientUtils$.fetchTopicMetadata(ClientUtils.scala:58)
> >
> > at
> >
> kafka.producer.BrokerPartitionInfo.updateInfo(BrokerPartitionInfo.scala:82)
> >
> > at
> >
> >
> kafka.producer.async.DefaultEventHandler$$anonfun$handle$1.apply$mcV$sp(DefaultEventHandler.scala:67)
> >
> > at kafka.utils.Utils$.swallow(Utils.scala:172)
> >
> > at kafka.utils.Logging$class.swallowError(Logging.scala:106)
> >
> > at kafka.utils.Utils$.swallowError(Utils.scala:45)
> >
> > at
> >
> >
> kafka.producer.async.DefaultEventHandler.handle(DefaultEventHandler.scala:67)
> >
> > at
> >
> >
> kafka.producer.async.ProducerSendThread.tryToHandle(ProducerSendThread.scala:104)
> >
> > at
> >
> >
> kafka.producer.async.ProducerSendThread$$anonfun$processEvents$3.apply(ProducerSendThread.scala:87)
> >
> > at
> >
> >
> kafka.producer.async.ProducerSendThread$$anonfun$processEvents$3.apply(ProducerSendThread.scala:67)
> >
> > at scala.collection.immutable.Stream.foreach(Stream.scala:526)
> >
> > at
> >
> >
> kafka.producer.async.ProducerSendThread.processEvents(ProducerSendThread.scala:66)
> >
> > at
> kafka.producer.async.ProducerSendThread.run(ProducerSendThread.scala:44)
> >
> >
> > On Fri, Jul 18, 2014 at 1:20 PM, Pramod Deshmukh <dp...@gmail.com>
> > wrote:
> >
> > > 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 topic "secureTopic".
> > >
> > > pdeshmukh$ bin/kafka-topics.sh --list --zookeeper localhost:2181
> > >
> > > secure.test
> > >
> > > secureTopic
> > >
> > > >> Run producer, tried both localhost:9092:true and localhost:9092
> > >
> > > pdeshmukh$ bin/kafka-console-producer.sh --broker-list
> > localhost:9092:true
> > > --topic secureTopic
> > >
> > > [2014-07-18 13:12:29,817] WARN Property topic is not valid
> > > (kafka.utils.VerifiableProperties)
> > >
> > > Hare Krishna
> > >
> > > [2014-07-18 13:12:45,256] WARN Fetching topic metadata with correlation
> > id
> > > 0 for topics [Set(secureTopic)] from broker
> > > [id:0,host:localhost,port:9092,secure:true] failed
> > > (kafka.client.ClientUtils$)
> > >
> > > java.io.EOFException: Received -1 when reading from channel, socket has
> > > likely been closed.
> > >
> > > at kafka.utils.Utils$.read(Utils.scala:381)
> > >
> > > at
> > >
> >
> kafka.network.BoundedByteBufferReceive.readFrom(BoundedByteBufferReceive.scala:67)
> > >
> > > at kafka.network.Receive$class.readCompletely(Transmission.scala:56)
> > >
> > > at
> > >
> >
> kafka.network.BoundedByteBufferReceive.readCompletely(BoundedByteBufferReceive.scala:29)
> > >
> > > at kafka.network.BlockingChannel.receive(BlockingChannel.scala:102)
> > >
> > > at kafka.producer.SyncProducer.liftedTree1$1(SyncProducer.scala:79)
> > >
> > > at
> > >
> >
> kafka.producer.SyncProducer.kafka$producer$SyncProducer$$doSend(SyncProducer.scala:76)
> > >
> > > at kafka.producer.SyncProducer.send(SyncProducer.scala:117)
> > >
> > > at kafka.client.ClientUtils$.fetchTopicMetadata(ClientUtils.scala:58)
> > >
> > > at
> > >
> >
> kafka.producer.BrokerPartitionInfo.updateInfo(BrokerPartitionInfo.scala:82)
> > >
> > > at
> > >
> >
> kafka.producer.async.DefaultEventHandler$$anonfun$handle$1.apply$mcV$sp(DefaultEventHandler.scala:67)
> > >
> > > at kafka.utils.Utils$.swallow(Utils.scala:172)
> > >
> > > at kafka.utils.Logging$class.swallowError(Logging.scala:106)
> > >
> > > at kafka.utils.Utils$.swallowError(Utils.scala:45)
> > >
> > > at
> > >
> >
> kafka.producer.async.DefaultEventHandler.handle(DefaultEventHandler.scala:67)
> > >
> > > at
> > >
> >
> kafka.producer.async.ProducerSendThread.tryToHandle(ProducerSendThread.scala:104)
> > >
> > > at
> > >
> >
> kafka.producer.async.ProducerSendThread$$anonfun$processEvents$3.apply(ProducerSendThread.scala:87)
> > >
> > > at
> > >
> >
> kafka.producer.async.ProducerSendThread$$anonfun$processEvents$3.apply(ProducerSendThread.scala:67)
> > >
> > > at scala.collection.immutable.Stream.foreach(Stream.scala:526)
> > >
> > > at
> > >
> >
> kafka.producer.async.ProducerSendThread.processEvents(ProducerSendThread.scala:66)
> > >
> > > at
> > kafka.producer.async.ProducerSendThread.run(ProducerSendThread.scala:44)
> > >
> > > [2014-07-18 13:12:45,258] ERROR fetching topic metadata for topics
> > > [Set(secureTopic)] from broker
> > > [ArrayBuffer(id:0,host:localhost,port:9092,secure:true)] failed
> > > (kafka.utils.Utils$)
> > >
> > > kafka.common.KafkaException: fetching topic metadata for topics
> > > [Set(secureTopic)] from broker
> > > [ArrayBuffer(id:0,host:localhost,port:9092,secure:true)] failed
> > >
> > > at kafka.client.ClientUtils$.fetchTopicMetadata(ClientUtils.scala:72)
> > >
> > > at
> > >
> >
> kafka.producer.BrokerPartitionInfo.updateInfo(BrokerPartitionInfo.scala:82)
> > >
> > > at
> > >
> >
> kafka.producer.async.DefaultEventHandler$$anonfun$handle$1.apply$mcV$sp(DefaultEventHandler.scala:67)
> > >
> > > at kafka.utils.Utils$.swallow(Utils.scala:172)
> > >
> > > at kafka.utils.Logging$class.swallowError(Logging.scala:106)
> > >
> > > at kafka.utils.Utils$.swallowError(Utils.scala:45)
> > >
> > > at
> > >
> >
> kafka.producer.async.DefaultEventHandler.handle(DefaultEventHandler.scala:67)
> > >
> > > at
> > >
> >
> kafka.producer.async.ProducerSendThread.tryToHandle(ProducerSendThread.scala:104)
> > >
> > > at
> > >
> >
> kafka.producer.async.ProducerSendThread$$anonfun$processEvents$3.apply(ProducerSendThread.scala:87)
> > >
> > > at
> > >
> >
> kafka.producer.async.ProducerSendThread$$anonfun$processEvents$3.apply(ProducerSendThread.scala:67)
> > >
> > > at scala.collection.immutable.Stream.foreach(Stream.scala:526)
> > >
> > > at
> > >
> >
> kafka.producer.async.ProducerSendThread.processEvents(ProducerSendThread.scala:66)
> > >
> > > at
> > kafka.producer.async.ProducerSendThread.run(ProducerSendThread.scala:44)
> > >
> > > Caused by: java.io.EOFException: Received -1 when reading from channel,
> > > socket has likely been closed.
> > >
> > > at kafka.utils.Utils$.read(Utils.scala:381)
> > >
> > > at
> > >
> >
> kafka.network.BoundedByteBufferReceive.readFrom(BoundedByteBufferReceive.scala:67)
> > >
> > > at kafka.network.Receive$class.readCompletely(Transmission.scala:56)
> > >
> > > at
> > >
> >
> kafka.network.BoundedByteBufferReceive.readCompletely(BoundedByteBufferReceive.scala:29)
> > >
> > > at kafka.network.BlockingChannel.receive(BlockingChannel.scala:102)
> > >
> > > at kafka.producer.SyncProducer.liftedTree1$1(SyncProducer.scala:79)
> > >
> > > at
> > >
> >
> kafka.producer.SyncProducer.kafka$producer$SyncProducer$$doSend(SyncProducer.scala:76)
> > >
> > > at kafka.producer.SyncProducer.send(SyncProducer.scala:117)
> > >
> > > at kafka.client.ClientUtils$.fetchTopicMetadata(ClientUtils.scala:58)
> > >
> > > ... 12 more
> > > [2014-07-18 13:12:45,337] WARN Fetching topic metadata with correlation
> > id
> > > 1 for topics [Set(secureTopic)] from broker
> > > [id:0,host:localhost,port:9092,secure:true] failed
> > > (kafka.client.ClientUtils$)
> > >
> > > 2014-07-18 13:12:46,282] ERROR Failed to send requests for topics
> > > secureTopic with correlation ids in [0,8]
> > > (kafka.producer.async.DefaultEventHandler)
> > >
> > > [2014-07-18 13:12:46,283] ERROR Error in handling batch of 1 events
> > > (kafka.producer.async.ProducerSendThread)
> > >
> > > kafka.common.FailedToSendMessageException: Failed to send messages
> after
> > 3
> > > tries.
> > >
> > > at
> > >
> >
> kafka.producer.async.DefaultEventHandler.handle(DefaultEventHandler.scala:90)
> > >
> > > at
> > >
> >
> kafka.producer.async.ProducerSendThread.tryToHandle(ProducerSendThread.scala:104)
> > >
> > > at
> > >
> >
> kafka.producer.async.ProducerSendThread$$anonfun$processEvents$3.apply(ProducerSendThread.scala:87)
> > >
> > > at
> > >
> >
> kafka.producer.async.ProducerSendThread$$anonfun$processEvents$3.apply(ProducerSendThread.scala:67)
> > >
> > > at scala.collection.immutable.Stream.foreach(Stream.scala:526)
> > >
> > > at
> > >
> >
> kafka.producer.async.ProducerSendThread.processEvents(ProducerSendThread.scala:66)
> > >
> > > at
> > kafka.producer.async.ProducerSendThread.run(ProducerSendThread.scala:44)
> > >
> > >
> > >
> > > On Fri, Jul 18, 2014 at 11:56 AM, Joe Stein <jo...@stealth.ly>
> > wrote:
> > >
> > >> 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://www.stealth.ly
> > >>  Twitter: @allthingshadoop <http://www.twitter.com/allthingshadoop>
> > >> ********************************************/
> > >>
> > >>
> > >> On Fri, Jul 18, 2014 at 10:24 AM, Pramod Deshmukh <dpramodv@gmail.com
> >
> > >> wrote:
> > >>
> > >> > 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.OutOfMemoryError: Java heap space
> > >> >
> > >> > *broker log*
> > >> >
> > >> > INFO begin ssl handshake for localhost/
> > 127.0.0.1:50199//127.0.0.1:9092
> > >> >
> > >> >
> > >> >
> > >> >
> > >> >
> > >> > On Thu, Jul 17, 2014 at 6:07 PM, Pramod Deshmukh <
> dpramodv@gmail.com>
> > >> > wrote:
> > >> >
> > >> > > 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 <
> > >> > relango@salesforce.com>
> > >> > > wrote:
> > >> > >
> > >> > >> 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 <
> > dpramodv@gmail.com
> > >> >
> > >> > >> 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.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 config/server.properties*
> > >> > >> >
> > >> > >> >
> > >> > >> >
> > >> > >> > *broker.log:*
> > >> > >> >
> > >> > >> > [2014-07-17 15:34:46,281] INFO zookeeper state changed
> > >> (SyncConnected)
> > >> > >> > (org.I0Itec.zkclient.ZkClient)
> > >> > >> >
> > >> > >> > [2014-07-17 15:34:46,523] INFO Loading log 'secure.test-0'
> > >> > >> > (kafka.log.LogManager)
> > >> > >> >
> > >> > >> > [2014-07-17 15:34:46,558] INFO Recovering unflushed segment 0
> in
> > >> log
> > >> > >> > secure.test-0. (kafka.log.Log)
> > >> > >> >
> > >> > >> > [2014-07-17 15:34:46,571] INFO Completed load of log
> > secure.test-0
> > >> > with
> > >> > >> log
> > >> > >> > end offset 0 (kafka.log.Log)
> > >> > >> >
> > >> > >> > [2014-07-17 15:34:46,582] INFO Starting log cleanup with a
> period
> > >> of
> > >> > >> 60000
> > >> > >> > ms. (kafka.log.LogManager)
> > >> > >> >
> > >> > >> > [2014-07-17 15:34:46,587] INFO Starting log flusher with a
> > default
> > >> > >> period
> > >> > >> > of 9223372036854775807 ms. (kafka.log.LogManager)
> > >> > >> >
> > >> > >> > [2014-07-17 15:34:46,614] INFO Initializing secure
> authentication
> > >> > >> > (kafka.network.security.SecureAuth$)
> > >> > >> >
> > >> > >> > [2014-07-17 15:34:46,678] INFO Secure authentication
> > initialization
> > >> > has
> > >> > >> > been successfully completed
> (kafka.network.security.SecureAuth$)
> > >> > >> >
> > >> > >> > [2014-07-17 15:34:46,691] INFO Awaiting socket connections on
> > >> > >> 0.0.0.0:9092
> > >> > >> > .
> > >> > >> > (kafka.network.Acceptor)
> > >> > >> >
> > >> > >> > [2014-07-17 15:34:46,692] INFO [Socket Server on Broker 0],
> > Started
> > >> > >> > (kafka.network.SocketServer)
> > >> > >> >
> > >> > >> > [2014-07-17 15:34:46,794] INFO Will not load MX4J,
> mx4j-tools.jar
> > >> is
> > >> > >> not in
> > >> > >> > the classpath (kafka.utils.Mx4jLoader$)
> > >> > >> >
> > >> > >> > [2014-07-17 15:34:46,837] INFO 0 successfully elected as leader
> > >> > >> > (kafka.server.ZookeeperLeaderElector)
> > >> > >> >
> > >> > >> > [2014-07-17 15:34:47,057] INFO Registered broker 0 at path
> > >> > >> /brokers/ids/0
> > >> > >> > with address 10.1.100.130:9092. (kafka.utils.ZkUtils$)
> > >> > >> >
> > >> > >> > [2014-07-17 15:34:47,059] INFO New leader is 0
> > >> > >> > (kafka.server.ZookeeperLeaderElector$LeaderChangeListener)
> > >> > >> >
> > >> > >> > *[2014-07-17 15:34:47,068] INFO [Kafka Server 0], started
> > >> > >> > (kafka.server.KafkaServer)*
> > >> > >> >
> > >> > >> > *[2014-07-17 15:34:47,383] INFO begin ssl handshake for
> > >> > >> > /10.1.100.130:9092//10.1.100.130:51685
> > >> > >> > <http://10.1.100.130:9092//10.1.100.130:51685>
> > >> > >> > (kafka.network.security.SSLSocketChannel)*
> > >> > >> >
> > >> > >> > *[2014-07-17 15:34:47,392] INFO begin ssl handshake for
> > >> > >> > 10.1.100.130/10.1.100.130:51685//10.1.100.130:9092
> > >> > >> > <http://10.1.100.130/10.1.100.130:51685//10.1.100.130:9092>
> > >> > >> > (kafka.network.security.SSLSocketChannel)*
> > >> > >> >
> > >> > >> > *[2014-07-17 15:34:47,465] INFO finished ssl handshake for
> > >> > >> > 10.1.100.130/10.1.100.130:51685//10.1.100.130:9092
> > >> > >> > <http://10.1.100.130/10.1.100.130:51685//10.1.100.130:9092>
> > >> > >> > (kafka.network.security.SSLSocketChannel)*
> > >> > >> >
> > >> > >> > *[2014-07-17 15:34:47,465] INFO finished ssl handshake for
> > >> > >> > /10.1.100.130:9092//10.1.100.130:51685
> > >> > >> > <http://10.1.100.130:9092//10.1.100.130:51685>
> > >> > >> > (kafka.network.security.SSLSocketChannel)*
> > >> > >> >
> > >> > >> > *[2014-07-17 15:34:47,617] INFO [ReplicaFetcherManager on
> broker
> > 0]
> > >> > >> Removed
> > >> > >> > fetcher for partitions  (kafka.server.ReplicaFetcherManager)*
> > >> > >> >
> > >> > >> > *[2014-07-17 15:34:47,627] INFO [ReplicaFetcherManager on
> broker
> > 0]
> > >> > >> Added
> > >> > >> > fetcher for partitions List()
> > (kafka.server.ReplicaFetcherManager)*
> > >> > >> >
> > >> > >> > *[2014-07-17 15:34:47,656] INFO [ReplicaFetcherManager on
> broker
> > 0]
> > >> > >> Removed
> > >> > >> > fetcher for partitions [secure.test,0]
> > >> > >> > (kafka.server.ReplicaFetcherManager)*
> > >> > >> >
> > >> > >> > [2014-07-17 15:37:15,970] INFO begin ssl handshake for
> > >> > >> > 10.1.100.130/10.1.100.130:51689//10.1.100.130:9092
> > >> > >> > (kafka.network.security.SSLSocketChannel)
> > >> > >> >
> > >> > >> > [2014-07-17 15:37:16,075] INFO begin ssl handshake for
> > >> > >> > 10.1.100.130/10.1.100.130:51690//10.1.100.130:9092
> > >> > >> > (kafka.network.security.SSLSocketChannel)
> > >> > >> >
> > >> > >> > [2014-07-17 15:37:16,434] INFO begin ssl handshake for
> > >> > >> > 10.1.100.130/10.1.100.130:51691//10.1.100.130:9092
> > >> > >> > (kafka.network.security.SSLSocketChannel)
> > >> > >> >
> > >> > >> > [2014-07-17 15:37:16,530] INFO begin ssl handshake for
> > >> > >> > 10.1.100.130/10.1.100.130:51692//10.1.100.130:9092
> > >> > >> > (kafka.network.security.SSLSocketChannel)
> > >> > >> >
> > >> > >> > [2014-07-17 15:37:16,743] INFO begin ssl handshake for
> > >> > >> > 10.1.100.130/10.1.100.130:51693//10.1.100.130:9092
> > >> > >> > (kafka.network.security.SSLSocketChannel)
> > >> > >> >
> > >> > >> > [2014-07-17 15:37:16,834] INFO begin ssl handshake for
> > >> > >> > 10.1.100.130/10.1.100.130:51694//10.1.100.130:9092
> > >> > >> > (kafka.network.security.SSLSocketChannel)
> > >> > >> >
> > >> > >> > [2014-07-17 15:37:17,043] INFO begin ssl handshake for
> > >> > >> > 10.1.100.130/10.1.100.130:51695//10.1.100.130:9092
> > >> > >> > (kafka.network.security.SSLSocketChannel)
> > >> > >> >
> > >> > >> > [2014-07-17 15:37:17,137] INFO begin ssl handshake for
> > >> > >> > 10.1.100.130/10.1.100.130:51696//10.1.100.130:9092
> > >> > >> > (kafka.network.security.SSLSocketChannel)
> > >> > >> >
> > >> > >> > [2014-07-17 15:37:17,342] INFO begin ssl handshake for
> > >> > >> > 10.1.100.130/10.1.100.130:51697//10.1.100.130:9092
> > >> > >> > (kafka.network.security.SSLSocketChannel)
> > >> > >> >
> > >> > >> >
> > >> > >> > *Start producer*
> > >> > >> >
> > >> > >> > *bin/kafka-console-producer.sh --broker-list 10.1.100.130:9092
> > >> :true
> > >> > >> > --topic
> > >> > >> > secure.test*
> > >> > >> >
> > >> > >> >
> > >> > >> > *producer.log:*
> > >> > >> >
> > >> > >> > bin/kafka-console-producer.sh --broker-list 10.1.100.130:9092
> > :true
> > >> > >> --topic
> > >> > >> > secure.test
> > >> > >> >
> > >> > >> > [2014-07-17 15:37:46,889] WARN Property topic is not valid
> > >> > >> > (kafka.utils.VerifiableProperties)
> > >> > >> >
> > >> > >> > Hello Secure Kafka
> > >> > >> >
> > >> > >> > *[2014-07-17 15:38:14,186] ERROR OOME with size 352518400
> > >> > >> > (kafka.network.BoundedByteBufferReceive)*
> > >> > >> >
> > >> > >> > *java.lang.OutOfMemoryError: Java heap space*
> > >> > >> >
> > >> > >> > at java.nio.HeapByteBuffer.<init>(HeapByteBuffer.java:57)
> > >> > >> >
> > >> > >> > at java.nio.ByteBuffer.allocate(ByteBuffer.java:331)
> > >> > >> >
> > >> > >> > at
> > >> > >> >
> > >> > >> >
> > >> > >>
> > >> >
> > >>
> >
> kafka.network.BoundedByteBufferReceive.byteBufferAllocate(BoundedByteBufferReceive.scala:80)
> > >> > >> >
> > >> > >> > at
> > >> > >> >
> > >> > >> >
> > >> > >>
> > >> >
> > >>
> >
> kafka.network.BoundedByteBufferReceive.readFrom(BoundedByteBufferReceive.scala:63)
> > >> > >> >
> > >> > >> > at
> > >> kafka.network.Receive$class.readCompletely(Transmission.scala:56)
> > >> > >> >
> > >> > >> > at
> > >> > >> >
> > >> > >> >
> > >> > >>
> > >> >
> > >>
> >
> kafka.network.BoundedByteBufferReceive.readCompletely(BoundedByteBufferReceive.scala:29)
> > >> > >> >
> > >> > >> > at
> > kafka.network.BlockingChannel.receive(BlockingChannel.scala:102)
> > >> > >> >
> > >> > >> > at
> > kafka.producer.SyncProducer.liftedTree1$1(SyncProducer.scala:79)
> > >> > >> >
> > >> > >> > at
> > >> > >> >
> > >> > >> >
> > >> > >>
> > >> >
> > >>
> >
> kafka.producer.SyncProducer.kafka$producer$SyncProducer$$doSend(SyncProducer.scala:76)
> > >> > >> >
> > >> > >> > at kafka.producer.SyncProducer.send(SyncProducer.scala:117)
> > >> > >> >
> > >> > >> > at
> > >> kafka.client.ClientUtils$.fetchTopicMetadata(ClientUtils.scala:58)
> > >> > >> >
> > >> > >> > at
> > >> > >> >
> > >> > >>
> > >> >
> > >>
> >
> kafka.producer.BrokerPartitionInfo.updateInfo(BrokerPartitionInfo.scala:82)
> > >> > >> >
> > >> > >> > at
> > >> > >> >
> > >> > >> >
> > >> > >>
> > >> >
> > >>
> >
> kafka.producer.async.DefaultEventHandler$$anonfun$handle$1.apply$mcV$sp(DefaultEventHandler.scala:67)
> > >> > >> >
> > >> > >> > at kafka.utils.Utils$.swallow(Utils.scala:172)
> > >> > >> >
> > >> > >> > at kafka.utils.Logging$class.swallowError(Logging.scala:106)
> > >> > >> >
> > >> > >> > at kafka.utils.Utils$.swallowError(Utils.scala:45)
> > >> > >> >
> > >> > >> > at
> > >> > >> >
> > >> > >> >
> > >> > >>
> > >> >
> > >>
> >
> kafka.producer.async.DefaultEventHandler.handle(DefaultEventHandler.scala:67)
> > >> > >> >
> > >> > >> > at
> > >> > >> >
> > >> > >> >
> > >> > >>
> > >> >
> > >>
> >
> kafka.producer.async.ProducerSendThread.tryToHandle(ProducerSendThread.scala:104)
> > >> > >> >
> > >> > >> > at
> > >> > >> >
> > >> > >> >
> > >> > >>
> > >> >
> > >>
> >
> kafka.producer.async.ProducerSendThread$$anonfun$processEvents$3.apply(ProducerSendThread.scala:87)
> > >> > >> >
> > >> > >> > at
> > >> > >> >
> > >> > >> >
> > >> > >>
> > >> >
> > >>
> >
> kafka.producer.async.ProducerSendThread$$anonfun$processEvents$3.apply(ProducerSendThread.scala:67)
> > >> > >> >
> > >> > >> > at scala.collection.immutable.Stream.foreach(Stream.scala:526)
> > >> > >> >
> > >> > >> > at
> > >> > >> >
> > >> > >> >
> > >> > >>
> > >> >
> > >>
> >
> kafka.producer.async.ProducerSendThread.processEvents(ProducerSendThread.scala:66)
> > >> > >> >
> > >> > >> > at
> > >> > >>
> > >>
> kafka.producer.async.ProducerSendThread.run(ProducerSendThread.scala:44)
> > >> > >> >
> > >> > >> >
> > >> > >> >
> > >> > >> > On Wed, Jul 16, 2014 at 6:07 PM, Rajasekar Elango <
> > >> > >> relango@salesforce.com>
> > >> > >> > wrote:
> > >> > >> >
> > >> > >> > > Pramod,
> > >> > >> > >
> > >> > >> > >
> > >> > >> > > I presented secure kafka configuration and usage at last meet
> > >> up. So
> > >> > >> hope
> > >> > >> > > this
> > >> > >> > > video recording <http://www.ustream.tv/recorded/48396701
> >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 <
> > >> > dpramodv@gmail.com>
> > >> > >> > > wrote:
> > >> > >> > >
> > >> > >> > > > 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 <
> > >> > >> dpramodv@gmail.com>
> > >> > >> > > > 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, Jul 16, 2014 at 3:57 PM, Joe Stein <
> > >> > joe.stein@stealth.ly>
> > >> > >> > > 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
> > >> > >> (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 Stein
> > >> > >> > > > >>  Founder, Principal Consultant
> > >> > >> > > > >>  Big Data Open Source Security LLC
> > >> > >> > > > >>  http://www.stealth.ly
> > >> > >> > > > >>  Twitter: @allthingshadoop <
> > >> > >> http://www.twitter.com/allthingshadoop>
> > >> > >> > > > >> ********************************************/
> > >> > >> > > > >>
> > >> > >> > > > >>
> > >> > >> > > > >> On Wed, Jul 16, 2014 at 3:49 PM, Pramod Deshmukh <
> > >> > >> > dpramodv@gmail.com>
> > >> > >> > > > >> wrote:
> > >> > >> > > > >>
> > >> > >> > > > >> > 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 installed sbt and scala and followed your
> > >> instructions
> > >> > >> for
> > >> > >> > > > >> different
> > >> > >> > > > >> > scala versions. I was able to bring zookeeper up but
> > >> brokers
> > >> > >> fail
> > >> > >> > to
> > >> > >> > > > >> start
> > >> > >> > > > >> > with error
> > >> > >> > > > >> >
> > >> > >> > > > >> > Error: Could not find or load main class kafka.Kafka
> > >> > >> > > > >> >
> > >> > >> > > > >> > I think I am doing something wrong. Can you please
> help
> > >> me?
> > >> > >> > > > >> >
> > >> > >> > > > >> > Our current production setup is with 2.8.0 and want to
> > >> stick
> > >> > to
> > >> > >> > it.
> > >> > >> > > > >> >
> > >> > >> > > > >> > Thanks,
> > >> > >> > > > >> >
> > >> > >> > > > >> > Pramod
> > >> > >> > > > >> >
> > >> > >> > > > >> >
> > >> > >> > > > >> > On Tue, Jun 3, 2014 at 3:57 PM, Joe Stein <
> > >> > >> joe.stein@stealth.ly>
> > >> > >> > > > wrote:
> > >> > >> > > > >> >
> > >> > >> > > > >> > > 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.  This comes from what folks are doing
> > "in
> > >> the
> > >> > >> > wild"
> > >> > >> > > to
> > >> > >> > > > >> > > workaround and implement security with Kafka as it
> is
> > >> today
> > >> > >> and
> > >> > >> > > also
> > >> > >> > > > >> > what I
> > >> > >> > > > >> > > have discovered from organizations about their
> > >> blockers. It
> > >> > >> also
> > >> > >> > > > >> picks up
> > >> > >> > > > >> > > from the wiki (which I should have time to update
> > later
> > >> in
> > >> > >> the
> > >> > >> > > week
> > >> > >> > > > >> based
> > >> > >> > > > >> > > on the below and feedback from the thread).
> > >> > >> > > > >> > >
> > >> > >> > > > >> > > 1) Transport Layer Security (i.e. SSL)
> > >> > >> > > > >> > >
> > >> > >> > > > >> > > This also includes client authentication in addition
> > to
> > >> > >> > in-transit
> > >> > >> > > > >> > security
> > >> > >> > > > >> > > layer.  This work has been picked up here
> > >> > >> > > > >> > > https://issues.apache.org/jira/browse/KAFKA-1477
> and
> > do
> > >> > >> > > appreciate
> > >> > >> > > > >> any
> > >> > >> > > > >> > > thoughts, comments, feedback, tomatoes, whatever for
> > >> this
> > >> > >> patch.
> > >> > >> > >  It
> > >> > >> > > > >> is a
> > >> > >> > > > >> > > pickup from the fork of the work first done here
> > >> > >> > > > >> > >
> https://github.com/relango/kafka/tree/kafka_security.
> > >> > >> > > > >> > >
> > >> > >> > > > >> > > 2) Data encryption at rest.
> > >> > >> > > > >> > >
> > >> > >> > > > >> > > This is very important and something that can be
> > >> > facilitated
> > >> > >> > > within
> > >> > >> > > > >> the
> > >> > >> > > > >> > > wire protocol. It requires an additional map data
> > >> structure
> > >> > >> for
> > >> > >> > > the
> > >> > >> > > > >> > > "encrypted [data encryption key]". With this map
> > >> (either in
> > >> > >> your
> > >> > >> > > > >> object
> > >> > >> > > > >> > or
> > >> > >> > > > >> > > in the wire protocol) you can store the dynamically
> > >> > generated
> > >> > >> > > > >> symmetric
> > >> > >> > > > >> > key
> > >> > >> > > > >> > > (for each message) and then encrypt the data using
> > that
> > >> > >> > > dynamically
> > >> > >> > > > >> > > generated key.  You then encrypt the encryption key
> > >> using
> > >> > >> each
> > >> > >> > > > public
> > >> > >> > > > >> key
> > >> > >> > > > >> > > for whom is expected to be able to decrypt the
> > >> encryption
> > >> > >> key to
> > >> > >> > > > then
> > >> > >> > > > >> > > decrypt the message.  For each public key encrypted
> > >> > symmetric
> > >> > >> > key
> > >> > >> > > > >> (which
> > >> > >> > > > >> > is
> > >> > >> > > > >> > > now the "encrypted [data encryption key]" along with
> > >> which
> > >> > >> > public
> > >> > >> > > > key
> > >> > >> > > > >> it
> > >> > >> > > > >> > > was encrypted with for (so a map of [publicKey] =
> > >> > >> > > > >> > > encryptedDataEncryptionKey) as a chain.   Other
> > patterns
> > >> > can
> > >> > >> be
> > >> > >> > > > >> > implemented
> > >> > >> > > > >> > > but this is a pretty standard digital enveloping [0]
> > >> > pattern
> > >> > >> > with
> > >> > >> > > > >> only 1
> > >> > >> > > > >> > > field added. Other patterns should be able to use
> that
> > >> > field
> > >> > >> > to-do
> > >> > >> > > > >> their
> > >> > >> > > > >> > > implementation too.
> > >> > >> > > > >> > >
> > >> > >> > > > >> > > 3) Non-repudiation and long term non-repudiation.
> > >> > >> > > > >> > >
> > >> > >> > > > >> > > Non-repudiation is proving data hasn't changed.
>  This
> > is
> > >> > >> often
> > >> > >> > (if
> > >> > >> > > > not
> > >> > >> > > > >> > > always) done with x509 public certificates (chained
> > to a
> > >> > >> > > certificate
> > >> > >> > > > >> > > authority).
> > >> > >> > > > >> > >
> > >> > >> > > > >> > > Long term non-repudiation is what happens when the
> > >> > >> certificates
> > >> > >> > of
> > >> > >> > > > the
> > >> > >> > > > >> > > certificate authority are expired (or revoked) and
> > >> > everything
> > >> > >> > ever
> > >> > >> > > > >> signed
> > >> > >> > > > >> > > (ever) with that certificate's public key then
> becomes
> > >> "no
> > >> > >> > longer
> > >> > >> > > > >> > provable
> > >> > >> > > > >> > > as ever being authentic".  That is where RFC3126 [1]
> > and
> > >> > >> RFC3161
> > >> > >> > > [2]
> > >> > >> > > > >> come
> > >> > >> > > > >> > > in (or worm drives [hardware], etc).
> > >> > >> > > > >> > >
> > >> > >> > > > >> > > For either (or both) of these it is an operation of
> > the
> > >> > >> > encryptor
> > >> > >> > > to
> > >> > >> > > > >> > > sign/hash the data (with or without third party
> > trusted
> > >> > >> timestap
> > >> > >> > > of
> > >> > >> > > > >> the
> > >> > >> > > > >> > > signing event) and encrypt that with their own
> private
> > >> key
> > >> > >> and
> > >> > >> > > > >> distribute
> > >> > >> > > > >> > > the results (before and after encrypting if
> required)
> > >> along
> > >> > >> with
> > >> > >> > > > their
> > >> > >> > > > >> > > public key. This structure is a bit more complex but
> > >> > >> feasible,
> > >> > >> > it
> > >> > >> > > > is a
> > >> > >> > > > >> > map
> > >> > >> > > > >> > > of digital signature formats and the chain of dig
> sig
> > >> > >> > > attestations.
> > >> > >> > > > >>  The
> > >> > >> > > > >> > > map's key being the method (i.e. CRC32, PKCS7 [3],
> > >> > XmlDigSig
> > >> > >> > [4])
> > >> > >> > > > and
> > >> > >> > > > >> > then
> > >> > >> > > > >> > > a list of map where that key is "purpose" of
> signature
> > >> > (what
> > >> > >> > your
> > >> > >> > > > >> > attesting
> > >> > >> > > > >> > > too).  As a sibling field to the list another field
> > for
> > >> > "the
> > >> > >> > > > >> attester" as
> > >> > >> > > > >> > > bytes (e.g. their PKCS12 [5] for the map of PKCS7
> > >> > >> signatures).
> > >> > >> > > > >> > >
> > >> > >> > > > >> > > 4) Authorization
> > >> > >> > > > >> > >
> > >> > >> > > > >> > > We should have a policy of "404" for data, topics,
> > >> > partitions
> > >> > >> > > (etc)
> > >> > >> > > > if
> > >> > >> > > > >> > > authenticated connections do not have access.  In
> > >> "secure
> > >> > >> mode"
> > >> > >> > > any
> > >> > >> > > > >> non
> > >> > >> > > > >> > > authenticated connections should get a "404" type
> > >> message
> > >> > on
> > >> > >> > > > >> everything.
> > >> > >> > > > >> > > Knowing "something is there" is a security risk in
> > many
> > >> > uses
> > >> > >> > > cases.
> > >> > >> > > > >>  So
> > >> > >> > > > >> > if
> > >> > >> > > > >> > > you don't have access you don't even see it.  Baking
> > >> "that"
> > >> > >> into
> > >> > >> > > > Kafka
> > >> > >> > > > >> > > along with some interface for entitlement (access
> > >> > management)
> > >> > >> > > > systems
> > >> > >> > > > >> > > (pretty standard) is all that I think needs to be
> done
> > >> to
> > >> > the
> > >> > >> > core
> > >> > >> > > > >> > project.
> > >> > >> > > > >> > >  I want to tackle item later in the year after
> summer
> > >> after
> > >> > >> the
> > >> > >> > > > other
> > >> > >> > > > >> > three
> > >> > >> > > > >> > > are complete.
> > >> > >> > > > >> > >
> > >> > >> > > > >> > > I look forward to thoughts on this and anyone else
> > >> > >> interested in
> > >> > >> > > > >> working
> > >> > >> > > > >> > > with us on these items.
> > >> > >> > > > >> > >
> > >> > >> > > > >> > > [0]
> > >> > >> > > > >> > >
> > >> > >> > > > >> > >
> > >> > >> > > > >> >
> > >> > >> > > > >>
> > >> > >> > > >
> > >> > >> > >
> > >> > >> >
> > >> > >>
> > >> >
> > >>
> >
> http://www.emc.com/emc-plus/rsa-labs/standards-initiatives/what-is-a-digital-envelope.htm
> > >> > >> > > > >> > > [1] http://tools.ietf.org/html/rfc3126
> > >> > >> > > > >> > > [2] http://tools.ietf.org/html/rfc3161
> > >> > >> > > > >> > > [3]
> > >> > >> > > > >> > >
> > >> > >> > > > >> > >
> > >> > >> > > > >> >
> > >> > >> > > > >>
> > >> > >> > > >
> > >> > >> > >
> > >> > >> >
> > >> > >>
> > >> >
> > >>
> >
> http://www.emc.com/emc-plus/rsa-labs/standards-initiatives/pkcs-7-cryptographic-message-syntax-standar.htm
> > >> > >> > > > >> > > [4] http://en.wikipedia.org/wiki/XML_Signature
> > >> > >> > > > >> > > [5] http://en.wikipedia.org/wiki/PKCS_12
> > >> > >> > > > >> > >
> > >> > >> > > > >> > > /*******************************************
> > >> > >> > > > >> > >  Joe Stein
> > >> > >> > > > >> > >  Founder, Principal Consultant
> > >> > >> > > > >> > >  Big Data Open Source Security LLC
> > >> > >> > > > >> > >  http://www.stealth.ly
> > >> > >> > > > >> > >  Twitter: @allthingshadoop <
> > >> > >> > > http://www.twitter.com/allthingshadoop>
> > >> > >> > > > >> > > ********************************************/
> > >> > >> > > > >> > >
> > >> > >> > > > >> >
> > >> > >> > > > >>
> > >> > >> > > > >
> > >> > >> > > > >
> > >> > >> > > >
> > >> > >> > >
> > >> > >> > >
> > >> > >> > >
> > >> > >> > > --
> > >> > >> > > Thanks,
> > >> > >> > > Raja.
> > >> > >> > >
> > >> > >> >
> > >> > >>
> > >> > >>
> > >> > >>
> > >> > >> --
> > >> > >> Thanks,
> > >> > >> Raja.
> > >> > >>
> > >> > >
> > >> > >
> > >> >
> > >>
> > >
> > >
> >
>

Re: [DISCUSS] Kafka Security Specific Features

Posted by Chris Neal <cw...@gmail.com>.
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:

bin/kafka-console-producer.sh --broker-list n5:9092:true --topic test

to this:

bin/kafka-console-producer.sh --broker-list n5:9092 --secure
--client.security.file config/client.security.properties --topic test

I was able to push messages to the topic, although I got a WARN about the
property "topic" not being valid, even though it is required.

Also, the Producer reported this warning to me:

[2014-07-23 20:45:24,509] WARN Attempt to reinitialize auth context
(kafka.network.security.SecureAuth$)

and the broker gave me this:
[2014-07-23 20:45:24,114] INFO begin ssl handshake for
n5.example.com/192.168.1.144:48817//192.168.1.144:9092
(kafka.network.security.SSLSocketChannel)
[2014-07-23 20:45:24,374] INFO finished ssl handshake for
n5.example.com/192.168.1.144:48817//192.168.1.144:9092
(kafka.network.security.SSLSocketChannel)
[2014-07-23 20:45:24,493] INFO Closing socket connection to
n5.example.com/192.168.1.144. (kafka.network.Processor)
[2014-07-23 20:45:24,555] INFO begin ssl handshake for
n5.example.com/192.168.1.144:48818//192.168.1.144:9092
(kafka.network.security.SSLSocketChannel)
[2014-07-23 20:45:24,566] INFO finished ssl handshake for
n5.example.com/192.168.1.144:48818//192.168.1.144:9092
(kafka.network.security.SSLSocketChannel)

It's like it did the SSL piece twice :)

Subsequent puts to the topic did not exhibit this behavior though:

root@n5[937]:~/kafka_2.10-0-8-2-0.1.0.0> bin/kafka-console-producer.sh
--broker-list n5:9092 --secure --client.security.file
config/client.security.properties --topic test
[2014-07-23 20:45:17,530] WARN Property topic is not valid
(kafka.utils.VerifiableProperties)
1
[2014-07-23 20:45:24,509] WARN Attempt to reinitialize auth context
(kafka.network.security.SecureAuth$)
2
3
4

Consuming worked with these options:

root@n5[918]:~/kafka_2.10-0-8-2-0.1.0.0> bin/kafka-console-consumer.sh
--topic test --zookeeper n5:2181 --from-beginning --security.config.file
config/client.security.properties
1
2
3
4
^CConsumed 5 messages

I hope that helps!
Chris


On Tue, Jul 22, 2014 at 2:10 PM, Pramod Deshmukh <dp...@gmail.com> wrote:

> 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 not valid
> (kafka.utils.VerifiableProperties)
>
> Hare Krishna
>
> [2014-07-18 13:12:45,256] WARN Fetching topic metadata with correlation id
> 0 for topics [Set(secureTopic)] from broker
> [id:0,host:localhost,port:9092,secure:true] failed
> (kafka.client.ClientUtils$)
>
> java.io.EOFException: Received -1 when reading from channel, socket has
> likely been closed.
>
> at kafka.utils.Utils$.read(Utils.scala:381)
>
> at
>
> kafka.network.BoundedByteBufferReceive.readFrom(BoundedByteBufferReceive.scala:67)
>
> at kafka.network.Receive$class.readCompletely(Transmission.scala:56)
>
> at
>
> kafka.network.BoundedByteBufferReceive.readCompletely(BoundedByteBufferReceive.scala:29)
>
> at kafka.network.BlockingChannel.receive(BlockingChannel.scala:102)
>
> at kafka.producer.SyncProducer.liftedTree1$1(SyncProducer.scala:79)
>
> at
>
> kafka.producer.SyncProducer.kafka$producer$SyncProducer$$doSend(SyncProducer.scala:76)
>
> at kafka.producer.SyncProducer.send(SyncProducer.scala:117)
>
> at kafka.client.ClientUtils$.fetchTopicMetadata(ClientUtils.scala:58)
>
> at
> kafka.producer.BrokerPartitionInfo.updateInfo(BrokerPartitionInfo.scala:82)
>
> at
>
> kafka.producer.async.DefaultEventHandler$$anonfun$handle$1.apply$mcV$sp(DefaultEventHandler.scala:67)
>
> at kafka.utils.Utils$.swallow(Utils.scala:172)
>
> at kafka.utils.Logging$class.swallowError(Logging.scala:106)
>
> at kafka.utils.Utils$.swallowError(Utils.scala:45)
>
> at
>
> kafka.producer.async.DefaultEventHandler.handle(DefaultEventHandler.scala:67)
>
> at
>
> kafka.producer.async.ProducerSendThread.tryToHandle(ProducerSendThread.scala:104)
>
> at
>
> kafka.producer.async.ProducerSendThread$$anonfun$processEvents$3.apply(ProducerSendThread.scala:87)
>
> at
>
> kafka.producer.async.ProducerSendThread$$anonfun$processEvents$3.apply(ProducerSendThread.scala:67)
>
> at scala.collection.immutable.Stream.foreach(Stream.scala:526)
>
> at
>
> kafka.producer.async.ProducerSendThread.processEvents(ProducerSendThread.scala:66)
>
> at kafka.producer.async.ProducerSendThread.run(ProducerSendThread.scala:44)
>
>
> On Fri, Jul 18, 2014 at 1:20 PM, Pramod Deshmukh <dp...@gmail.com>
> wrote:
>
> > 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 topic "secureTopic".
> >
> > pdeshmukh$ bin/kafka-topics.sh --list --zookeeper localhost:2181
> >
> > secure.test
> >
> > secureTopic
> >
> > >> Run producer, tried both localhost:9092:true and localhost:9092
> >
> > pdeshmukh$ bin/kafka-console-producer.sh --broker-list
> localhost:9092:true
> > --topic secureTopic
> >
> > [2014-07-18 13:12:29,817] WARN Property topic is not valid
> > (kafka.utils.VerifiableProperties)
> >
> > Hare Krishna
> >
> > [2014-07-18 13:12:45,256] WARN Fetching topic metadata with correlation
> id
> > 0 for topics [Set(secureTopic)] from broker
> > [id:0,host:localhost,port:9092,secure:true] failed
> > (kafka.client.ClientUtils$)
> >
> > java.io.EOFException: Received -1 when reading from channel, socket has
> > likely been closed.
> >
> > at kafka.utils.Utils$.read(Utils.scala:381)
> >
> > at
> >
> kafka.network.BoundedByteBufferReceive.readFrom(BoundedByteBufferReceive.scala:67)
> >
> > at kafka.network.Receive$class.readCompletely(Transmission.scala:56)
> >
> > at
> >
> kafka.network.BoundedByteBufferReceive.readCompletely(BoundedByteBufferReceive.scala:29)
> >
> > at kafka.network.BlockingChannel.receive(BlockingChannel.scala:102)
> >
> > at kafka.producer.SyncProducer.liftedTree1$1(SyncProducer.scala:79)
> >
> > at
> >
> kafka.producer.SyncProducer.kafka$producer$SyncProducer$$doSend(SyncProducer.scala:76)
> >
> > at kafka.producer.SyncProducer.send(SyncProducer.scala:117)
> >
> > at kafka.client.ClientUtils$.fetchTopicMetadata(ClientUtils.scala:58)
> >
> > at
> >
> kafka.producer.BrokerPartitionInfo.updateInfo(BrokerPartitionInfo.scala:82)
> >
> > at
> >
> kafka.producer.async.DefaultEventHandler$$anonfun$handle$1.apply$mcV$sp(DefaultEventHandler.scala:67)
> >
> > at kafka.utils.Utils$.swallow(Utils.scala:172)
> >
> > at kafka.utils.Logging$class.swallowError(Logging.scala:106)
> >
> > at kafka.utils.Utils$.swallowError(Utils.scala:45)
> >
> > at
> >
> kafka.producer.async.DefaultEventHandler.handle(DefaultEventHandler.scala:67)
> >
> > at
> >
> kafka.producer.async.ProducerSendThread.tryToHandle(ProducerSendThread.scala:104)
> >
> > at
> >
> kafka.producer.async.ProducerSendThread$$anonfun$processEvents$3.apply(ProducerSendThread.scala:87)
> >
> > at
> >
> kafka.producer.async.ProducerSendThread$$anonfun$processEvents$3.apply(ProducerSendThread.scala:67)
> >
> > at scala.collection.immutable.Stream.foreach(Stream.scala:526)
> >
> > at
> >
> kafka.producer.async.ProducerSendThread.processEvents(ProducerSendThread.scala:66)
> >
> > at
> kafka.producer.async.ProducerSendThread.run(ProducerSendThread.scala:44)
> >
> > [2014-07-18 13:12:45,258] ERROR fetching topic metadata for topics
> > [Set(secureTopic)] from broker
> > [ArrayBuffer(id:0,host:localhost,port:9092,secure:true)] failed
> > (kafka.utils.Utils$)
> >
> > kafka.common.KafkaException: fetching topic metadata for topics
> > [Set(secureTopic)] from broker
> > [ArrayBuffer(id:0,host:localhost,port:9092,secure:true)] failed
> >
> > at kafka.client.ClientUtils$.fetchTopicMetadata(ClientUtils.scala:72)
> >
> > at
> >
> kafka.producer.BrokerPartitionInfo.updateInfo(BrokerPartitionInfo.scala:82)
> >
> > at
> >
> kafka.producer.async.DefaultEventHandler$$anonfun$handle$1.apply$mcV$sp(DefaultEventHandler.scala:67)
> >
> > at kafka.utils.Utils$.swallow(Utils.scala:172)
> >
> > at kafka.utils.Logging$class.swallowError(Logging.scala:106)
> >
> > at kafka.utils.Utils$.swallowError(Utils.scala:45)
> >
> > at
> >
> kafka.producer.async.DefaultEventHandler.handle(DefaultEventHandler.scala:67)
> >
> > at
> >
> kafka.producer.async.ProducerSendThread.tryToHandle(ProducerSendThread.scala:104)
> >
> > at
> >
> kafka.producer.async.ProducerSendThread$$anonfun$processEvents$3.apply(ProducerSendThread.scala:87)
> >
> > at
> >
> kafka.producer.async.ProducerSendThread$$anonfun$processEvents$3.apply(ProducerSendThread.scala:67)
> >
> > at scala.collection.immutable.Stream.foreach(Stream.scala:526)
> >
> > at
> >
> kafka.producer.async.ProducerSendThread.processEvents(ProducerSendThread.scala:66)
> >
> > at
> kafka.producer.async.ProducerSendThread.run(ProducerSendThread.scala:44)
> >
> > Caused by: java.io.EOFException: Received -1 when reading from channel,
> > socket has likely been closed.
> >
> > at kafka.utils.Utils$.read(Utils.scala:381)
> >
> > at
> >
> kafka.network.BoundedByteBufferReceive.readFrom(BoundedByteBufferReceive.scala:67)
> >
> > at kafka.network.Receive$class.readCompletely(Transmission.scala:56)
> >
> > at
> >
> kafka.network.BoundedByteBufferReceive.readCompletely(BoundedByteBufferReceive.scala:29)
> >
> > at kafka.network.BlockingChannel.receive(BlockingChannel.scala:102)
> >
> > at kafka.producer.SyncProducer.liftedTree1$1(SyncProducer.scala:79)
> >
> > at
> >
> kafka.producer.SyncProducer.kafka$producer$SyncProducer$$doSend(SyncProducer.scala:76)
> >
> > at kafka.producer.SyncProducer.send(SyncProducer.scala:117)
> >
> > at kafka.client.ClientUtils$.fetchTopicMetadata(ClientUtils.scala:58)
> >
> > ... 12 more
> > [2014-07-18 13:12:45,337] WARN Fetching topic metadata with correlation
> id
> > 1 for topics [Set(secureTopic)] from broker
> > [id:0,host:localhost,port:9092,secure:true] failed
> > (kafka.client.ClientUtils$)
> >
> > 2014-07-18 13:12:46,282] ERROR Failed to send requests for topics
> > secureTopic with correlation ids in [0,8]
> > (kafka.producer.async.DefaultEventHandler)
> >
> > [2014-07-18 13:12:46,283] ERROR Error in handling batch of 1 events
> > (kafka.producer.async.ProducerSendThread)
> >
> > kafka.common.FailedToSendMessageException: Failed to send messages after
> 3
> > tries.
> >
> > at
> >
> kafka.producer.async.DefaultEventHandler.handle(DefaultEventHandler.scala:90)
> >
> > at
> >
> kafka.producer.async.ProducerSendThread.tryToHandle(ProducerSendThread.scala:104)
> >
> > at
> >
> kafka.producer.async.ProducerSendThread$$anonfun$processEvents$3.apply(ProducerSendThread.scala:87)
> >
> > at
> >
> kafka.producer.async.ProducerSendThread$$anonfun$processEvents$3.apply(ProducerSendThread.scala:67)
> >
> > at scala.collection.immutable.Stream.foreach(Stream.scala:526)
> >
> > at
> >
> kafka.producer.async.ProducerSendThread.processEvents(ProducerSendThread.scala:66)
> >
> > at
> kafka.producer.async.ProducerSendThread.run(ProducerSendThread.scala:44)
> >
> >
> >
> > On Fri, Jul 18, 2014 at 11:56 AM, Joe Stein <jo...@stealth.ly>
> wrote:
> >
> >> 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://www.stealth.ly
> >>  Twitter: @allthingshadoop <http://www.twitter.com/allthingshadoop>
> >> ********************************************/
> >>
> >>
> >> On Fri, Jul 18, 2014 at 10:24 AM, Pramod Deshmukh <dp...@gmail.com>
> >> wrote:
> >>
> >> > 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.OutOfMemoryError: Java heap space
> >> >
> >> > *broker log*
> >> >
> >> > INFO begin ssl handshake for localhost/
> 127.0.0.1:50199//127.0.0.1:9092
> >> >
> >> >
> >> >
> >> >
> >> >
> >> > On Thu, Jul 17, 2014 at 6:07 PM, Pramod Deshmukh <dp...@gmail.com>
> >> > wrote:
> >> >
> >> > > 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 <
> >> > relango@salesforce.com>
> >> > > wrote:
> >> > >
> >> > >> 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 <
> dpramodv@gmail.com
> >> >
> >> > >> 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.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 config/server.properties*
> >> > >> >
> >> > >> >
> >> > >> >
> >> > >> > *broker.log:*
> >> > >> >
> >> > >> > [2014-07-17 15:34:46,281] INFO zookeeper state changed
> >> (SyncConnected)
> >> > >> > (org.I0Itec.zkclient.ZkClient)
> >> > >> >
> >> > >> > [2014-07-17 15:34:46,523] INFO Loading log 'secure.test-0'
> >> > >> > (kafka.log.LogManager)
> >> > >> >
> >> > >> > [2014-07-17 15:34:46,558] INFO Recovering unflushed segment 0 in
> >> log
> >> > >> > secure.test-0. (kafka.log.Log)
> >> > >> >
> >> > >> > [2014-07-17 15:34:46,571] INFO Completed load of log
> secure.test-0
> >> > with
> >> > >> log
> >> > >> > end offset 0 (kafka.log.Log)
> >> > >> >
> >> > >> > [2014-07-17 15:34:46,582] INFO Starting log cleanup with a period
> >> of
> >> > >> 60000
> >> > >> > ms. (kafka.log.LogManager)
> >> > >> >
> >> > >> > [2014-07-17 15:34:46,587] INFO Starting log flusher with a
> default
> >> > >> period
> >> > >> > of 9223372036854775807 ms. (kafka.log.LogManager)
> >> > >> >
> >> > >> > [2014-07-17 15:34:46,614] INFO Initializing secure authentication
> >> > >> > (kafka.network.security.SecureAuth$)
> >> > >> >
> >> > >> > [2014-07-17 15:34:46,678] INFO Secure authentication
> initialization
> >> > has
> >> > >> > been successfully completed (kafka.network.security.SecureAuth$)
> >> > >> >
> >> > >> > [2014-07-17 15:34:46,691] INFO Awaiting socket connections on
> >> > >> 0.0.0.0:9092
> >> > >> > .
> >> > >> > (kafka.network.Acceptor)
> >> > >> >
> >> > >> > [2014-07-17 15:34:46,692] INFO [Socket Server on Broker 0],
> Started
> >> > >> > (kafka.network.SocketServer)
> >> > >> >
> >> > >> > [2014-07-17 15:34:46,794] INFO Will not load MX4J, mx4j-tools.jar
> >> is
> >> > >> not in
> >> > >> > the classpath (kafka.utils.Mx4jLoader$)
> >> > >> >
> >> > >> > [2014-07-17 15:34:46,837] INFO 0 successfully elected as leader
> >> > >> > (kafka.server.ZookeeperLeaderElector)
> >> > >> >
> >> > >> > [2014-07-17 15:34:47,057] INFO Registered broker 0 at path
> >> > >> /brokers/ids/0
> >> > >> > with address 10.1.100.130:9092. (kafka.utils.ZkUtils$)
> >> > >> >
> >> > >> > [2014-07-17 15:34:47,059] INFO New leader is 0
> >> > >> > (kafka.server.ZookeeperLeaderElector$LeaderChangeListener)
> >> > >> >
> >> > >> > *[2014-07-17 15:34:47,068] INFO [Kafka Server 0], started
> >> > >> > (kafka.server.KafkaServer)*
> >> > >> >
> >> > >> > *[2014-07-17 15:34:47,383] INFO begin ssl handshake for
> >> > >> > /10.1.100.130:9092//10.1.100.130:51685
> >> > >> > <http://10.1.100.130:9092//10.1.100.130:51685>
> >> > >> > (kafka.network.security.SSLSocketChannel)*
> >> > >> >
> >> > >> > *[2014-07-17 15:34:47,392] INFO begin ssl handshake for
> >> > >> > 10.1.100.130/10.1.100.130:51685//10.1.100.130:9092
> >> > >> > <http://10.1.100.130/10.1.100.130:51685//10.1.100.130:9092>
> >> > >> > (kafka.network.security.SSLSocketChannel)*
> >> > >> >
> >> > >> > *[2014-07-17 15:34:47,465] INFO finished ssl handshake for
> >> > >> > 10.1.100.130/10.1.100.130:51685//10.1.100.130:9092
> >> > >> > <http://10.1.100.130/10.1.100.130:51685//10.1.100.130:9092>
> >> > >> > (kafka.network.security.SSLSocketChannel)*
> >> > >> >
> >> > >> > *[2014-07-17 15:34:47,465] INFO finished ssl handshake for
> >> > >> > /10.1.100.130:9092//10.1.100.130:51685
> >> > >> > <http://10.1.100.130:9092//10.1.100.130:51685>
> >> > >> > (kafka.network.security.SSLSocketChannel)*
> >> > >> >
> >> > >> > *[2014-07-17 15:34:47,617] INFO [ReplicaFetcherManager on broker
> 0]
> >> > >> Removed
> >> > >> > fetcher for partitions  (kafka.server.ReplicaFetcherManager)*
> >> > >> >
> >> > >> > *[2014-07-17 15:34:47,627] INFO [ReplicaFetcherManager on broker
> 0]
> >> > >> Added
> >> > >> > fetcher for partitions List()
> (kafka.server.ReplicaFetcherManager)*
> >> > >> >
> >> > >> > *[2014-07-17 15:34:47,656] INFO [ReplicaFetcherManager on broker
> 0]
> >> > >> Removed
> >> > >> > fetcher for partitions [secure.test,0]
> >> > >> > (kafka.server.ReplicaFetcherManager)*
> >> > >> >
> >> > >> > [2014-07-17 15:37:15,970] INFO begin ssl handshake for
> >> > >> > 10.1.100.130/10.1.100.130:51689//10.1.100.130:9092
> >> > >> > (kafka.network.security.SSLSocketChannel)
> >> > >> >
> >> > >> > [2014-07-17 15:37:16,075] INFO begin ssl handshake for
> >> > >> > 10.1.100.130/10.1.100.130:51690//10.1.100.130:9092
> >> > >> > (kafka.network.security.SSLSocketChannel)
> >> > >> >
> >> > >> > [2014-07-17 15:37:16,434] INFO begin ssl handshake for
> >> > >> > 10.1.100.130/10.1.100.130:51691//10.1.100.130:9092
> >> > >> > (kafka.network.security.SSLSocketChannel)
> >> > >> >
> >> > >> > [2014-07-17 15:37:16,530] INFO begin ssl handshake for
> >> > >> > 10.1.100.130/10.1.100.130:51692//10.1.100.130:9092
> >> > >> > (kafka.network.security.SSLSocketChannel)
> >> > >> >
> >> > >> > [2014-07-17 15:37:16,743] INFO begin ssl handshake for
> >> > >> > 10.1.100.130/10.1.100.130:51693//10.1.100.130:9092
> >> > >> > (kafka.network.security.SSLSocketChannel)
> >> > >> >
> >> > >> > [2014-07-17 15:37:16,834] INFO begin ssl handshake for
> >> > >> > 10.1.100.130/10.1.100.130:51694//10.1.100.130:9092
> >> > >> > (kafka.network.security.SSLSocketChannel)
> >> > >> >
> >> > >> > [2014-07-17 15:37:17,043] INFO begin ssl handshake for
> >> > >> > 10.1.100.130/10.1.100.130:51695//10.1.100.130:9092
> >> > >> > (kafka.network.security.SSLSocketChannel)
> >> > >> >
> >> > >> > [2014-07-17 15:37:17,137] INFO begin ssl handshake for
> >> > >> > 10.1.100.130/10.1.100.130:51696//10.1.100.130:9092
> >> > >> > (kafka.network.security.SSLSocketChannel)
> >> > >> >
> >> > >> > [2014-07-17 15:37:17,342] INFO begin ssl handshake for
> >> > >> > 10.1.100.130/10.1.100.130:51697//10.1.100.130:9092
> >> > >> > (kafka.network.security.SSLSocketChannel)
> >> > >> >
> >> > >> >
> >> > >> > *Start producer*
> >> > >> >
> >> > >> > *bin/kafka-console-producer.sh --broker-list 10.1.100.130:9092
> >> :true
> >> > >> > --topic
> >> > >> > secure.test*
> >> > >> >
> >> > >> >
> >> > >> > *producer.log:*
> >> > >> >
> >> > >> > bin/kafka-console-producer.sh --broker-list 10.1.100.130:9092
> :true
> >> > >> --topic
> >> > >> > secure.test
> >> > >> >
> >> > >> > [2014-07-17 15:37:46,889] WARN Property topic is not valid
> >> > >> > (kafka.utils.VerifiableProperties)
> >> > >> >
> >> > >> > Hello Secure Kafka
> >> > >> >
> >> > >> > *[2014-07-17 15:38:14,186] ERROR OOME with size 352518400
> >> > >> > (kafka.network.BoundedByteBufferReceive)*
> >> > >> >
> >> > >> > *java.lang.OutOfMemoryError: Java heap space*
> >> > >> >
> >> > >> > at java.nio.HeapByteBuffer.<init>(HeapByteBuffer.java:57)
> >> > >> >
> >> > >> > at java.nio.ByteBuffer.allocate(ByteBuffer.java:331)
> >> > >> >
> >> > >> > at
> >> > >> >
> >> > >> >
> >> > >>
> >> >
> >>
> kafka.network.BoundedByteBufferReceive.byteBufferAllocate(BoundedByteBufferReceive.scala:80)
> >> > >> >
> >> > >> > at
> >> > >> >
> >> > >> >
> >> > >>
> >> >
> >>
> kafka.network.BoundedByteBufferReceive.readFrom(BoundedByteBufferReceive.scala:63)
> >> > >> >
> >> > >> > at
> >> kafka.network.Receive$class.readCompletely(Transmission.scala:56)
> >> > >> >
> >> > >> > at
> >> > >> >
> >> > >> >
> >> > >>
> >> >
> >>
> kafka.network.BoundedByteBufferReceive.readCompletely(BoundedByteBufferReceive.scala:29)
> >> > >> >
> >> > >> > at
> kafka.network.BlockingChannel.receive(BlockingChannel.scala:102)
> >> > >> >
> >> > >> > at
> kafka.producer.SyncProducer.liftedTree1$1(SyncProducer.scala:79)
> >> > >> >
> >> > >> > at
> >> > >> >
> >> > >> >
> >> > >>
> >> >
> >>
> kafka.producer.SyncProducer.kafka$producer$SyncProducer$$doSend(SyncProducer.scala:76)
> >> > >> >
> >> > >> > at kafka.producer.SyncProducer.send(SyncProducer.scala:117)
> >> > >> >
> >> > >> > at
> >> kafka.client.ClientUtils$.fetchTopicMetadata(ClientUtils.scala:58)
> >> > >> >
> >> > >> > at
> >> > >> >
> >> > >>
> >> >
> >>
> kafka.producer.BrokerPartitionInfo.updateInfo(BrokerPartitionInfo.scala:82)
> >> > >> >
> >> > >> > at
> >> > >> >
> >> > >> >
> >> > >>
> >> >
> >>
> kafka.producer.async.DefaultEventHandler$$anonfun$handle$1.apply$mcV$sp(DefaultEventHandler.scala:67)
> >> > >> >
> >> > >> > at kafka.utils.Utils$.swallow(Utils.scala:172)
> >> > >> >
> >> > >> > at kafka.utils.Logging$class.swallowError(Logging.scala:106)
> >> > >> >
> >> > >> > at kafka.utils.Utils$.swallowError(Utils.scala:45)
> >> > >> >
> >> > >> > at
> >> > >> >
> >> > >> >
> >> > >>
> >> >
> >>
> kafka.producer.async.DefaultEventHandler.handle(DefaultEventHandler.scala:67)
> >> > >> >
> >> > >> > at
> >> > >> >
> >> > >> >
> >> > >>
> >> >
> >>
> kafka.producer.async.ProducerSendThread.tryToHandle(ProducerSendThread.scala:104)
> >> > >> >
> >> > >> > at
> >> > >> >
> >> > >> >
> >> > >>
> >> >
> >>
> kafka.producer.async.ProducerSendThread$$anonfun$processEvents$3.apply(ProducerSendThread.scala:87)
> >> > >> >
> >> > >> > at
> >> > >> >
> >> > >> >
> >> > >>
> >> >
> >>
> kafka.producer.async.ProducerSendThread$$anonfun$processEvents$3.apply(ProducerSendThread.scala:67)
> >> > >> >
> >> > >> > at scala.collection.immutable.Stream.foreach(Stream.scala:526)
> >> > >> >
> >> > >> > at
> >> > >> >
> >> > >> >
> >> > >>
> >> >
> >>
> kafka.producer.async.ProducerSendThread.processEvents(ProducerSendThread.scala:66)
> >> > >> >
> >> > >> > at
> >> > >>
> >> kafka.producer.async.ProducerSendThread.run(ProducerSendThread.scala:44)
> >> > >> >
> >> > >> >
> >> > >> >
> >> > >> > On Wed, Jul 16, 2014 at 6:07 PM, Rajasekar Elango <
> >> > >> relango@salesforce.com>
> >> > >> > wrote:
> >> > >> >
> >> > >> > > Pramod,
> >> > >> > >
> >> > >> > >
> >> > >> > > I presented secure kafka configuration and usage at last meet
> >> up. So
> >> > >> hope
> >> > >> > > this
> >> > >> > > video recording <http://www.ustream.tv/recorded/48396701>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 <
> >> > dpramodv@gmail.com>
> >> > >> > > wrote:
> >> > >> > >
> >> > >> > > > 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 <
> >> > >> dpramodv@gmail.com>
> >> > >> > > > 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, Jul 16, 2014 at 3:57 PM, Joe Stein <
> >> > joe.stein@stealth.ly>
> >> > >> > > 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
> >> > >> (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 Stein
> >> > >> > > > >>  Founder, Principal Consultant
> >> > >> > > > >>  Big Data Open Source Security LLC
> >> > >> > > > >>  http://www.stealth.ly
> >> > >> > > > >>  Twitter: @allthingshadoop <
> >> > >> http://www.twitter.com/allthingshadoop>
> >> > >> > > > >> ********************************************/
> >> > >> > > > >>
> >> > >> > > > >>
> >> > >> > > > >> On Wed, Jul 16, 2014 at 3:49 PM, Pramod Deshmukh <
> >> > >> > dpramodv@gmail.com>
> >> > >> > > > >> wrote:
> >> > >> > > > >>
> >> > >> > > > >> > 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 installed sbt and scala and followed your
> >> instructions
> >> > >> for
> >> > >> > > > >> different
> >> > >> > > > >> > scala versions. I was able to bring zookeeper up but
> >> brokers
> >> > >> fail
> >> > >> > to
> >> > >> > > > >> start
> >> > >> > > > >> > with error
> >> > >> > > > >> >
> >> > >> > > > >> > Error: Could not find or load main class kafka.Kafka
> >> > >> > > > >> >
> >> > >> > > > >> > I think I am doing something wrong. Can you please help
> >> me?
> >> > >> > > > >> >
> >> > >> > > > >> > Our current production setup is with 2.8.0 and want to
> >> stick
> >> > to
> >> > >> > it.
> >> > >> > > > >> >
> >> > >> > > > >> > Thanks,
> >> > >> > > > >> >
> >> > >> > > > >> > Pramod
> >> > >> > > > >> >
> >> > >> > > > >> >
> >> > >> > > > >> > On Tue, Jun 3, 2014 at 3:57 PM, Joe Stein <
> >> > >> joe.stein@stealth.ly>
> >> > >> > > > wrote:
> >> > >> > > > >> >
> >> > >> > > > >> > > 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.  This comes from what folks are doing
> "in
> >> the
> >> > >> > wild"
> >> > >> > > to
> >> > >> > > > >> > > workaround and implement security with Kafka as it is
> >> today
> >> > >> and
> >> > >> > > also
> >> > >> > > > >> > what I
> >> > >> > > > >> > > have discovered from organizations about their
> >> blockers. It
> >> > >> also
> >> > >> > > > >> picks up
> >> > >> > > > >> > > from the wiki (which I should have time to update
> later
> >> in
> >> > >> the
> >> > >> > > week
> >> > >> > > > >> based
> >> > >> > > > >> > > on the below and feedback from the thread).
> >> > >> > > > >> > >
> >> > >> > > > >> > > 1) Transport Layer Security (i.e. SSL)
> >> > >> > > > >> > >
> >> > >> > > > >> > > This also includes client authentication in addition
> to
> >> > >> > in-transit
> >> > >> > > > >> > security
> >> > >> > > > >> > > layer.  This work has been picked up here
> >> > >> > > > >> > > https://issues.apache.org/jira/browse/KAFKA-1477 and
> do
> >> > >> > > appreciate
> >> > >> > > > >> any
> >> > >> > > > >> > > thoughts, comments, feedback, tomatoes, whatever for
> >> this
> >> > >> patch.
> >> > >> > >  It
> >> > >> > > > >> is a
> >> > >> > > > >> > > pickup from the fork of the work first done here
> >> > >> > > > >> > > https://github.com/relango/kafka/tree/kafka_security.
> >> > >> > > > >> > >
> >> > >> > > > >> > > 2) Data encryption at rest.
> >> > >> > > > >> > >
> >> > >> > > > >> > > This is very important and something that can be
> >> > facilitated
> >> > >> > > within
> >> > >> > > > >> the
> >> > >> > > > >> > > wire protocol. It requires an additional map data
> >> structure
> >> > >> for
> >> > >> > > the
> >> > >> > > > >> > > "encrypted [data encryption key]". With this map
> >> (either in
> >> > >> your
> >> > >> > > > >> object
> >> > >> > > > >> > or
> >> > >> > > > >> > > in the wire protocol) you can store the dynamically
> >> > generated
> >> > >> > > > >> symmetric
> >> > >> > > > >> > key
> >> > >> > > > >> > > (for each message) and then encrypt the data using
> that
> >> > >> > > dynamically
> >> > >> > > > >> > > generated key.  You then encrypt the encryption key
> >> using
> >> > >> each
> >> > >> > > > public
> >> > >> > > > >> key
> >> > >> > > > >> > > for whom is expected to be able to decrypt the
> >> encryption
> >> > >> key to
> >> > >> > > > then
> >> > >> > > > >> > > decrypt the message.  For each public key encrypted
> >> > symmetric
> >> > >> > key
> >> > >> > > > >> (which
> >> > >> > > > >> > is
> >> > >> > > > >> > > now the "encrypted [data encryption key]" along with
> >> which
> >> > >> > public
> >> > >> > > > key
> >> > >> > > > >> it
> >> > >> > > > >> > > was encrypted with for (so a map of [publicKey] =
> >> > >> > > > >> > > encryptedDataEncryptionKey) as a chain.   Other
> patterns
> >> > can
> >> > >> be
> >> > >> > > > >> > implemented
> >> > >> > > > >> > > but this is a pretty standard digital enveloping [0]
> >> > pattern
> >> > >> > with
> >> > >> > > > >> only 1
> >> > >> > > > >> > > field added. Other patterns should be able to use that
> >> > field
> >> > >> > to-do
> >> > >> > > > >> their
> >> > >> > > > >> > > implementation too.
> >> > >> > > > >> > >
> >> > >> > > > >> > > 3) Non-repudiation and long term non-repudiation.
> >> > >> > > > >> > >
> >> > >> > > > >> > > Non-repudiation is proving data hasn't changed.  This
> is
> >> > >> often
> >> > >> > (if
> >> > >> > > > not
> >> > >> > > > >> > > always) done with x509 public certificates (chained
> to a
> >> > >> > > certificate
> >> > >> > > > >> > > authority).
> >> > >> > > > >> > >
> >> > >> > > > >> > > Long term non-repudiation is what happens when the
> >> > >> certificates
> >> > >> > of
> >> > >> > > > the
> >> > >> > > > >> > > certificate authority are expired (or revoked) and
> >> > everything
> >> > >> > ever
> >> > >> > > > >> signed
> >> > >> > > > >> > > (ever) with that certificate's public key then becomes
> >> "no
> >> > >> > longer
> >> > >> > > > >> > provable
> >> > >> > > > >> > > as ever being authentic".  That is where RFC3126 [1]
> and
> >> > >> RFC3161
> >> > >> > > [2]
> >> > >> > > > >> come
> >> > >> > > > >> > > in (or worm drives [hardware], etc).
> >> > >> > > > >> > >
> >> > >> > > > >> > > For either (or both) of these it is an operation of
> the
> >> > >> > encryptor
> >> > >> > > to
> >> > >> > > > >> > > sign/hash the data (with or without third party
> trusted
> >> > >> timestap
> >> > >> > > of
> >> > >> > > > >> the
> >> > >> > > > >> > > signing event) and encrypt that with their own private
> >> key
> >> > >> and
> >> > >> > > > >> distribute
> >> > >> > > > >> > > the results (before and after encrypting if required)
> >> along
> >> > >> with
> >> > >> > > > their
> >> > >> > > > >> > > public key. This structure is a bit more complex but
> >> > >> feasible,
> >> > >> > it
> >> > >> > > > is a
> >> > >> > > > >> > map
> >> > >> > > > >> > > of digital signature formats and the chain of dig sig
> >> > >> > > attestations.
> >> > >> > > > >>  The
> >> > >> > > > >> > > map's key being the method (i.e. CRC32, PKCS7 [3],
> >> > XmlDigSig
> >> > >> > [4])
> >> > >> > > > and
> >> > >> > > > >> > then
> >> > >> > > > >> > > a list of map where that key is "purpose" of signature
> >> > (what
> >> > >> > your
> >> > >> > > > >> > attesting
> >> > >> > > > >> > > too).  As a sibling field to the list another field
> for
> >> > "the
> >> > >> > > > >> attester" as
> >> > >> > > > >> > > bytes (e.g. their PKCS12 [5] for the map of PKCS7
> >> > >> signatures).
> >> > >> > > > >> > >
> >> > >> > > > >> > > 4) Authorization
> >> > >> > > > >> > >
> >> > >> > > > >> > > We should have a policy of "404" for data, topics,
> >> > partitions
> >> > >> > > (etc)
> >> > >> > > > if
> >> > >> > > > >> > > authenticated connections do not have access.  In
> >> "secure
> >> > >> mode"
> >> > >> > > any
> >> > >> > > > >> non
> >> > >> > > > >> > > authenticated connections should get a "404" type
> >> message
> >> > on
> >> > >> > > > >> everything.
> >> > >> > > > >> > > Knowing "something is there" is a security risk in
> many
> >> > uses
> >> > >> > > cases.
> >> > >> > > > >>  So
> >> > >> > > > >> > if
> >> > >> > > > >> > > you don't have access you don't even see it.  Baking
> >> "that"
> >> > >> into
> >> > >> > > > Kafka
> >> > >> > > > >> > > along with some interface for entitlement (access
> >> > management)
> >> > >> > > > systems
> >> > >> > > > >> > > (pretty standard) is all that I think needs to be done
> >> to
> >> > the
> >> > >> > core
> >> > >> > > > >> > project.
> >> > >> > > > >> > >  I want to tackle item later in the year after summer
> >> after
> >> > >> the
> >> > >> > > > other
> >> > >> > > > >> > three
> >> > >> > > > >> > > are complete.
> >> > >> > > > >> > >
> >> > >> > > > >> > > I look forward to thoughts on this and anyone else
> >> > >> interested in
> >> > >> > > > >> working
> >> > >> > > > >> > > with us on these items.
> >> > >> > > > >> > >
> >> > >> > > > >> > > [0]
> >> > >> > > > >> > >
> >> > >> > > > >> > >
> >> > >> > > > >> >
> >> > >> > > > >>
> >> > >> > > >
> >> > >> > >
> >> > >> >
> >> > >>
> >> >
> >>
> http://www.emc.com/emc-plus/rsa-labs/standards-initiatives/what-is-a-digital-envelope.htm
> >> > >> > > > >> > > [1] http://tools.ietf.org/html/rfc3126
> >> > >> > > > >> > > [2] http://tools.ietf.org/html/rfc3161
> >> > >> > > > >> > > [3]
> >> > >> > > > >> > >
> >> > >> > > > >> > >
> >> > >> > > > >> >
> >> > >> > > > >>
> >> > >> > > >
> >> > >> > >
> >> > >> >
> >> > >>
> >> >
> >>
> http://www.emc.com/emc-plus/rsa-labs/standards-initiatives/pkcs-7-cryptographic-message-syntax-standar.htm
> >> > >> > > > >> > > [4] http://en.wikipedia.org/wiki/XML_Signature
> >> > >> > > > >> > > [5] http://en.wikipedia.org/wiki/PKCS_12
> >> > >> > > > >> > >
> >> > >> > > > >> > > /*******************************************
> >> > >> > > > >> > >  Joe Stein
> >> > >> > > > >> > >  Founder, Principal Consultant
> >> > >> > > > >> > >  Big Data Open Source Security LLC
> >> > >> > > > >> > >  http://www.stealth.ly
> >> > >> > > > >> > >  Twitter: @allthingshadoop <
> >> > >> > > http://www.twitter.com/allthingshadoop>
> >> > >> > > > >> > > ********************************************/
> >> > >> > > > >> > >
> >> > >> > > > >> >
> >> > >> > > > >>
> >> > >> > > > >
> >> > >> > > > >
> >> > >> > > >
> >> > >> > >
> >> > >> > >
> >> > >> > >
> >> > >> > > --
> >> > >> > > Thanks,
> >> > >> > > Raja.
> >> > >> > >
> >> > >> >
> >> > >>
> >> > >>
> >> > >>
> >> > >> --
> >> > >> Thanks,
> >> > >> Raja.
> >> > >>
> >> > >
> >> > >
> >> >
> >>
> >
> >
>

Re: [DISCUSS] Kafka Security Specific Features

Posted by Pramod Deshmukh <dp...@gmail.com>.
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 not valid
(kafka.utils.VerifiableProperties)

Hare Krishna

[2014-07-18 13:12:45,256] WARN Fetching topic metadata with correlation id
0 for topics [Set(secureTopic)] from broker
[id:0,host:localhost,port:9092,secure:true] failed
(kafka.client.ClientUtils$)

java.io.EOFException: Received -1 when reading from channel, socket has
likely been closed.

at kafka.utils.Utils$.read(Utils.scala:381)

at
kafka.network.BoundedByteBufferReceive.readFrom(BoundedByteBufferReceive.scala:67)

at kafka.network.Receive$class.readCompletely(Transmission.scala:56)

at
kafka.network.BoundedByteBufferReceive.readCompletely(BoundedByteBufferReceive.scala:29)

at kafka.network.BlockingChannel.receive(BlockingChannel.scala:102)

at kafka.producer.SyncProducer.liftedTree1$1(SyncProducer.scala:79)

at
kafka.producer.SyncProducer.kafka$producer$SyncProducer$$doSend(SyncProducer.scala:76)

at kafka.producer.SyncProducer.send(SyncProducer.scala:117)

at kafka.client.ClientUtils$.fetchTopicMetadata(ClientUtils.scala:58)

at
kafka.producer.BrokerPartitionInfo.updateInfo(BrokerPartitionInfo.scala:82)

at
kafka.producer.async.DefaultEventHandler$$anonfun$handle$1.apply$mcV$sp(DefaultEventHandler.scala:67)

at kafka.utils.Utils$.swallow(Utils.scala:172)

at kafka.utils.Logging$class.swallowError(Logging.scala:106)

at kafka.utils.Utils$.swallowError(Utils.scala:45)

at
kafka.producer.async.DefaultEventHandler.handle(DefaultEventHandler.scala:67)

at
kafka.producer.async.ProducerSendThread.tryToHandle(ProducerSendThread.scala:104)

at
kafka.producer.async.ProducerSendThread$$anonfun$processEvents$3.apply(ProducerSendThread.scala:87)

at
kafka.producer.async.ProducerSendThread$$anonfun$processEvents$3.apply(ProducerSendThread.scala:67)

at scala.collection.immutable.Stream.foreach(Stream.scala:526)

at
kafka.producer.async.ProducerSendThread.processEvents(ProducerSendThread.scala:66)

at kafka.producer.async.ProducerSendThread.run(ProducerSendThread.scala:44)


On Fri, Jul 18, 2014 at 1:20 PM, Pramod Deshmukh <dp...@gmail.com> wrote:

> 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 topic "secureTopic".
>
> pdeshmukh$ bin/kafka-topics.sh --list --zookeeper localhost:2181
>
> secure.test
>
> secureTopic
>
> >> Run producer, tried both localhost:9092:true and localhost:9092
>
> pdeshmukh$ bin/kafka-console-producer.sh --broker-list localhost:9092:true
> --topic secureTopic
>
> [2014-07-18 13:12:29,817] WARN Property topic is not valid
> (kafka.utils.VerifiableProperties)
>
> Hare Krishna
>
> [2014-07-18 13:12:45,256] WARN Fetching topic metadata with correlation id
> 0 for topics [Set(secureTopic)] from broker
> [id:0,host:localhost,port:9092,secure:true] failed
> (kafka.client.ClientUtils$)
>
> java.io.EOFException: Received -1 when reading from channel, socket has
> likely been closed.
>
> at kafka.utils.Utils$.read(Utils.scala:381)
>
> at
> kafka.network.BoundedByteBufferReceive.readFrom(BoundedByteBufferReceive.scala:67)
>
> at kafka.network.Receive$class.readCompletely(Transmission.scala:56)
>
> at
> kafka.network.BoundedByteBufferReceive.readCompletely(BoundedByteBufferReceive.scala:29)
>
> at kafka.network.BlockingChannel.receive(BlockingChannel.scala:102)
>
> at kafka.producer.SyncProducer.liftedTree1$1(SyncProducer.scala:79)
>
> at
> kafka.producer.SyncProducer.kafka$producer$SyncProducer$$doSend(SyncProducer.scala:76)
>
> at kafka.producer.SyncProducer.send(SyncProducer.scala:117)
>
> at kafka.client.ClientUtils$.fetchTopicMetadata(ClientUtils.scala:58)
>
> at
> kafka.producer.BrokerPartitionInfo.updateInfo(BrokerPartitionInfo.scala:82)
>
> at
> kafka.producer.async.DefaultEventHandler$$anonfun$handle$1.apply$mcV$sp(DefaultEventHandler.scala:67)
>
> at kafka.utils.Utils$.swallow(Utils.scala:172)
>
> at kafka.utils.Logging$class.swallowError(Logging.scala:106)
>
> at kafka.utils.Utils$.swallowError(Utils.scala:45)
>
> at
> kafka.producer.async.DefaultEventHandler.handle(DefaultEventHandler.scala:67)
>
> at
> kafka.producer.async.ProducerSendThread.tryToHandle(ProducerSendThread.scala:104)
>
> at
> kafka.producer.async.ProducerSendThread$$anonfun$processEvents$3.apply(ProducerSendThread.scala:87)
>
> at
> kafka.producer.async.ProducerSendThread$$anonfun$processEvents$3.apply(ProducerSendThread.scala:67)
>
> at scala.collection.immutable.Stream.foreach(Stream.scala:526)
>
> at
> kafka.producer.async.ProducerSendThread.processEvents(ProducerSendThread.scala:66)
>
> at kafka.producer.async.ProducerSendThread.run(ProducerSendThread.scala:44)
>
> [2014-07-18 13:12:45,258] ERROR fetching topic metadata for topics
> [Set(secureTopic)] from broker
> [ArrayBuffer(id:0,host:localhost,port:9092,secure:true)] failed
> (kafka.utils.Utils$)
>
> kafka.common.KafkaException: fetching topic metadata for topics
> [Set(secureTopic)] from broker
> [ArrayBuffer(id:0,host:localhost,port:9092,secure:true)] failed
>
> at kafka.client.ClientUtils$.fetchTopicMetadata(ClientUtils.scala:72)
>
> at
> kafka.producer.BrokerPartitionInfo.updateInfo(BrokerPartitionInfo.scala:82)
>
> at
> kafka.producer.async.DefaultEventHandler$$anonfun$handle$1.apply$mcV$sp(DefaultEventHandler.scala:67)
>
> at kafka.utils.Utils$.swallow(Utils.scala:172)
>
> at kafka.utils.Logging$class.swallowError(Logging.scala:106)
>
> at kafka.utils.Utils$.swallowError(Utils.scala:45)
>
> at
> kafka.producer.async.DefaultEventHandler.handle(DefaultEventHandler.scala:67)
>
> at
> kafka.producer.async.ProducerSendThread.tryToHandle(ProducerSendThread.scala:104)
>
> at
> kafka.producer.async.ProducerSendThread$$anonfun$processEvents$3.apply(ProducerSendThread.scala:87)
>
> at
> kafka.producer.async.ProducerSendThread$$anonfun$processEvents$3.apply(ProducerSendThread.scala:67)
>
> at scala.collection.immutable.Stream.foreach(Stream.scala:526)
>
> at
> kafka.producer.async.ProducerSendThread.processEvents(ProducerSendThread.scala:66)
>
> at kafka.producer.async.ProducerSendThread.run(ProducerSendThread.scala:44)
>
> Caused by: java.io.EOFException: Received -1 when reading from channel,
> socket has likely been closed.
>
> at kafka.utils.Utils$.read(Utils.scala:381)
>
> at
> kafka.network.BoundedByteBufferReceive.readFrom(BoundedByteBufferReceive.scala:67)
>
> at kafka.network.Receive$class.readCompletely(Transmission.scala:56)
>
> at
> kafka.network.BoundedByteBufferReceive.readCompletely(BoundedByteBufferReceive.scala:29)
>
> at kafka.network.BlockingChannel.receive(BlockingChannel.scala:102)
>
> at kafka.producer.SyncProducer.liftedTree1$1(SyncProducer.scala:79)
>
> at
> kafka.producer.SyncProducer.kafka$producer$SyncProducer$$doSend(SyncProducer.scala:76)
>
> at kafka.producer.SyncProducer.send(SyncProducer.scala:117)
>
> at kafka.client.ClientUtils$.fetchTopicMetadata(ClientUtils.scala:58)
>
> ... 12 more
> [2014-07-18 13:12:45,337] WARN Fetching topic metadata with correlation id
> 1 for topics [Set(secureTopic)] from broker
> [id:0,host:localhost,port:9092,secure:true] failed
> (kafka.client.ClientUtils$)
>
> 2014-07-18 13:12:46,282] ERROR Failed to send requests for topics
> secureTopic with correlation ids in [0,8]
> (kafka.producer.async.DefaultEventHandler)
>
> [2014-07-18 13:12:46,283] ERROR Error in handling batch of 1 events
> (kafka.producer.async.ProducerSendThread)
>
> kafka.common.FailedToSendMessageException: Failed to send messages after 3
> tries.
>
> at
> kafka.producer.async.DefaultEventHandler.handle(DefaultEventHandler.scala:90)
>
> at
> kafka.producer.async.ProducerSendThread.tryToHandle(ProducerSendThread.scala:104)
>
> at
> kafka.producer.async.ProducerSendThread$$anonfun$processEvents$3.apply(ProducerSendThread.scala:87)
>
> at
> kafka.producer.async.ProducerSendThread$$anonfun$processEvents$3.apply(ProducerSendThread.scala:67)
>
> at scala.collection.immutable.Stream.foreach(Stream.scala:526)
>
> at
> kafka.producer.async.ProducerSendThread.processEvents(ProducerSendThread.scala:66)
>
> at kafka.producer.async.ProducerSendThread.run(ProducerSendThread.scala:44)
>
>
>
> On Fri, Jul 18, 2014 at 11:56 AM, Joe Stein <jo...@stealth.ly> wrote:
>
>> 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://www.stealth.ly
>>  Twitter: @allthingshadoop <http://www.twitter.com/allthingshadoop>
>> ********************************************/
>>
>>
>> On Fri, Jul 18, 2014 at 10:24 AM, Pramod Deshmukh <dp...@gmail.com>
>> wrote:
>>
>> > 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.OutOfMemoryError: Java heap space
>> >
>> > *broker log*
>> >
>> > INFO begin ssl handshake for localhost/127.0.0.1:50199//127.0.0.1:9092
>> >
>> >
>> >
>> >
>> >
>> > On Thu, Jul 17, 2014 at 6:07 PM, Pramod Deshmukh <dp...@gmail.com>
>> > wrote:
>> >
>> > > 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 <
>> > relango@salesforce.com>
>> > > wrote:
>> > >
>> > >> 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 <dpramodv@gmail.com
>> >
>> > >> 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.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 config/server.properties*
>> > >> >
>> > >> >
>> > >> >
>> > >> > *broker.log:*
>> > >> >
>> > >> > [2014-07-17 15:34:46,281] INFO zookeeper state changed
>> (SyncConnected)
>> > >> > (org.I0Itec.zkclient.ZkClient)
>> > >> >
>> > >> > [2014-07-17 15:34:46,523] INFO Loading log 'secure.test-0'
>> > >> > (kafka.log.LogManager)
>> > >> >
>> > >> > [2014-07-17 15:34:46,558] INFO Recovering unflushed segment 0 in
>> log
>> > >> > secure.test-0. (kafka.log.Log)
>> > >> >
>> > >> > [2014-07-17 15:34:46,571] INFO Completed load of log secure.test-0
>> > with
>> > >> log
>> > >> > end offset 0 (kafka.log.Log)
>> > >> >
>> > >> > [2014-07-17 15:34:46,582] INFO Starting log cleanup with a period
>> of
>> > >> 60000
>> > >> > ms. (kafka.log.LogManager)
>> > >> >
>> > >> > [2014-07-17 15:34:46,587] INFO Starting log flusher with a default
>> > >> period
>> > >> > of 9223372036854775807 ms. (kafka.log.LogManager)
>> > >> >
>> > >> > [2014-07-17 15:34:46,614] INFO Initializing secure authentication
>> > >> > (kafka.network.security.SecureAuth$)
>> > >> >
>> > >> > [2014-07-17 15:34:46,678] INFO Secure authentication initialization
>> > has
>> > >> > been successfully completed (kafka.network.security.SecureAuth$)
>> > >> >
>> > >> > [2014-07-17 15:34:46,691] INFO Awaiting socket connections on
>> > >> 0.0.0.0:9092
>> > >> > .
>> > >> > (kafka.network.Acceptor)
>> > >> >
>> > >> > [2014-07-17 15:34:46,692] INFO [Socket Server on Broker 0], Started
>> > >> > (kafka.network.SocketServer)
>> > >> >
>> > >> > [2014-07-17 15:34:46,794] INFO Will not load MX4J, mx4j-tools.jar
>> is
>> > >> not in
>> > >> > the classpath (kafka.utils.Mx4jLoader$)
>> > >> >
>> > >> > [2014-07-17 15:34:46,837] INFO 0 successfully elected as leader
>> > >> > (kafka.server.ZookeeperLeaderElector)
>> > >> >
>> > >> > [2014-07-17 15:34:47,057] INFO Registered broker 0 at path
>> > >> /brokers/ids/0
>> > >> > with address 10.1.100.130:9092. (kafka.utils.ZkUtils$)
>> > >> >
>> > >> > [2014-07-17 15:34:47,059] INFO New leader is 0
>> > >> > (kafka.server.ZookeeperLeaderElector$LeaderChangeListener)
>> > >> >
>> > >> > *[2014-07-17 15:34:47,068] INFO [Kafka Server 0], started
>> > >> > (kafka.server.KafkaServer)*
>> > >> >
>> > >> > *[2014-07-17 15:34:47,383] INFO begin ssl handshake for
>> > >> > /10.1.100.130:9092//10.1.100.130:51685
>> > >> > <http://10.1.100.130:9092//10.1.100.130:51685>
>> > >> > (kafka.network.security.SSLSocketChannel)*
>> > >> >
>> > >> > *[2014-07-17 15:34:47,392] INFO begin ssl handshake for
>> > >> > 10.1.100.130/10.1.100.130:51685//10.1.100.130:9092
>> > >> > <http://10.1.100.130/10.1.100.130:51685//10.1.100.130:9092>
>> > >> > (kafka.network.security.SSLSocketChannel)*
>> > >> >
>> > >> > *[2014-07-17 15:34:47,465] INFO finished ssl handshake for
>> > >> > 10.1.100.130/10.1.100.130:51685//10.1.100.130:9092
>> > >> > <http://10.1.100.130/10.1.100.130:51685//10.1.100.130:9092>
>> > >> > (kafka.network.security.SSLSocketChannel)*
>> > >> >
>> > >> > *[2014-07-17 15:34:47,465] INFO finished ssl handshake for
>> > >> > /10.1.100.130:9092//10.1.100.130:51685
>> > >> > <http://10.1.100.130:9092//10.1.100.130:51685>
>> > >> > (kafka.network.security.SSLSocketChannel)*
>> > >> >
>> > >> > *[2014-07-17 15:34:47,617] INFO [ReplicaFetcherManager on broker 0]
>> > >> Removed
>> > >> > fetcher for partitions  (kafka.server.ReplicaFetcherManager)*
>> > >> >
>> > >> > *[2014-07-17 15:34:47,627] INFO [ReplicaFetcherManager on broker 0]
>> > >> Added
>> > >> > fetcher for partitions List() (kafka.server.ReplicaFetcherManager)*
>> > >> >
>> > >> > *[2014-07-17 15:34:47,656] INFO [ReplicaFetcherManager on broker 0]
>> > >> Removed
>> > >> > fetcher for partitions [secure.test,0]
>> > >> > (kafka.server.ReplicaFetcherManager)*
>> > >> >
>> > >> > [2014-07-17 15:37:15,970] INFO begin ssl handshake for
>> > >> > 10.1.100.130/10.1.100.130:51689//10.1.100.130:9092
>> > >> > (kafka.network.security.SSLSocketChannel)
>> > >> >
>> > >> > [2014-07-17 15:37:16,075] INFO begin ssl handshake for
>> > >> > 10.1.100.130/10.1.100.130:51690//10.1.100.130:9092
>> > >> > (kafka.network.security.SSLSocketChannel)
>> > >> >
>> > >> > [2014-07-17 15:37:16,434] INFO begin ssl handshake for
>> > >> > 10.1.100.130/10.1.100.130:51691//10.1.100.130:9092
>> > >> > (kafka.network.security.SSLSocketChannel)
>> > >> >
>> > >> > [2014-07-17 15:37:16,530] INFO begin ssl handshake for
>> > >> > 10.1.100.130/10.1.100.130:51692//10.1.100.130:9092
>> > >> > (kafka.network.security.SSLSocketChannel)
>> > >> >
>> > >> > [2014-07-17 15:37:16,743] INFO begin ssl handshake for
>> > >> > 10.1.100.130/10.1.100.130:51693//10.1.100.130:9092
>> > >> > (kafka.network.security.SSLSocketChannel)
>> > >> >
>> > >> > [2014-07-17 15:37:16,834] INFO begin ssl handshake for
>> > >> > 10.1.100.130/10.1.100.130:51694//10.1.100.130:9092
>> > >> > (kafka.network.security.SSLSocketChannel)
>> > >> >
>> > >> > [2014-07-17 15:37:17,043] INFO begin ssl handshake for
>> > >> > 10.1.100.130/10.1.100.130:51695//10.1.100.130:9092
>> > >> > (kafka.network.security.SSLSocketChannel)
>> > >> >
>> > >> > [2014-07-17 15:37:17,137] INFO begin ssl handshake for
>> > >> > 10.1.100.130/10.1.100.130:51696//10.1.100.130:9092
>> > >> > (kafka.network.security.SSLSocketChannel)
>> > >> >
>> > >> > [2014-07-17 15:37:17,342] INFO begin ssl handshake for
>> > >> > 10.1.100.130/10.1.100.130:51697//10.1.100.130:9092
>> > >> > (kafka.network.security.SSLSocketChannel)
>> > >> >
>> > >> >
>> > >> > *Start producer*
>> > >> >
>> > >> > *bin/kafka-console-producer.sh --broker-list 10.1.100.130:9092
>> :true
>> > >> > --topic
>> > >> > secure.test*
>> > >> >
>> > >> >
>> > >> > *producer.log:*
>> > >> >
>> > >> > bin/kafka-console-producer.sh --broker-list 10.1.100.130:9092:true
>> > >> --topic
>> > >> > secure.test
>> > >> >
>> > >> > [2014-07-17 15:37:46,889] WARN Property topic is not valid
>> > >> > (kafka.utils.VerifiableProperties)
>> > >> >
>> > >> > Hello Secure Kafka
>> > >> >
>> > >> > *[2014-07-17 15:38:14,186] ERROR OOME with size 352518400
>> > >> > (kafka.network.BoundedByteBufferReceive)*
>> > >> >
>> > >> > *java.lang.OutOfMemoryError: Java heap space*
>> > >> >
>> > >> > at java.nio.HeapByteBuffer.<init>(HeapByteBuffer.java:57)
>> > >> >
>> > >> > at java.nio.ByteBuffer.allocate(ByteBuffer.java:331)
>> > >> >
>> > >> > at
>> > >> >
>> > >> >
>> > >>
>> >
>> kafka.network.BoundedByteBufferReceive.byteBufferAllocate(BoundedByteBufferReceive.scala:80)
>> > >> >
>> > >> > at
>> > >> >
>> > >> >
>> > >>
>> >
>> kafka.network.BoundedByteBufferReceive.readFrom(BoundedByteBufferReceive.scala:63)
>> > >> >
>> > >> > at
>> kafka.network.Receive$class.readCompletely(Transmission.scala:56)
>> > >> >
>> > >> > at
>> > >> >
>> > >> >
>> > >>
>> >
>> kafka.network.BoundedByteBufferReceive.readCompletely(BoundedByteBufferReceive.scala:29)
>> > >> >
>> > >> > at kafka.network.BlockingChannel.receive(BlockingChannel.scala:102)
>> > >> >
>> > >> > at kafka.producer.SyncProducer.liftedTree1$1(SyncProducer.scala:79)
>> > >> >
>> > >> > at
>> > >> >
>> > >> >
>> > >>
>> >
>> kafka.producer.SyncProducer.kafka$producer$SyncProducer$$doSend(SyncProducer.scala:76)
>> > >> >
>> > >> > at kafka.producer.SyncProducer.send(SyncProducer.scala:117)
>> > >> >
>> > >> > at
>> kafka.client.ClientUtils$.fetchTopicMetadata(ClientUtils.scala:58)
>> > >> >
>> > >> > at
>> > >> >
>> > >>
>> >
>> kafka.producer.BrokerPartitionInfo.updateInfo(BrokerPartitionInfo.scala:82)
>> > >> >
>> > >> > at
>> > >> >
>> > >> >
>> > >>
>> >
>> kafka.producer.async.DefaultEventHandler$$anonfun$handle$1.apply$mcV$sp(DefaultEventHandler.scala:67)
>> > >> >
>> > >> > at kafka.utils.Utils$.swallow(Utils.scala:172)
>> > >> >
>> > >> > at kafka.utils.Logging$class.swallowError(Logging.scala:106)
>> > >> >
>> > >> > at kafka.utils.Utils$.swallowError(Utils.scala:45)
>> > >> >
>> > >> > at
>> > >> >
>> > >> >
>> > >>
>> >
>> kafka.producer.async.DefaultEventHandler.handle(DefaultEventHandler.scala:67)
>> > >> >
>> > >> > at
>> > >> >
>> > >> >
>> > >>
>> >
>> kafka.producer.async.ProducerSendThread.tryToHandle(ProducerSendThread.scala:104)
>> > >> >
>> > >> > at
>> > >> >
>> > >> >
>> > >>
>> >
>> kafka.producer.async.ProducerSendThread$$anonfun$processEvents$3.apply(ProducerSendThread.scala:87)
>> > >> >
>> > >> > at
>> > >> >
>> > >> >
>> > >>
>> >
>> kafka.producer.async.ProducerSendThread$$anonfun$processEvents$3.apply(ProducerSendThread.scala:67)
>> > >> >
>> > >> > at scala.collection.immutable.Stream.foreach(Stream.scala:526)
>> > >> >
>> > >> > at
>> > >> >
>> > >> >
>> > >>
>> >
>> kafka.producer.async.ProducerSendThread.processEvents(ProducerSendThread.scala:66)
>> > >> >
>> > >> > at
>> > >>
>> kafka.producer.async.ProducerSendThread.run(ProducerSendThread.scala:44)
>> > >> >
>> > >> >
>> > >> >
>> > >> > On Wed, Jul 16, 2014 at 6:07 PM, Rajasekar Elango <
>> > >> relango@salesforce.com>
>> > >> > wrote:
>> > >> >
>> > >> > > Pramod,
>> > >> > >
>> > >> > >
>> > >> > > I presented secure kafka configuration and usage at last meet
>> up. So
>> > >> hope
>> > >> > > this
>> > >> > > video recording <http://www.ustream.tv/recorded/48396701>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 <
>> > dpramodv@gmail.com>
>> > >> > > wrote:
>> > >> > >
>> > >> > > > 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 <
>> > >> dpramodv@gmail.com>
>> > >> > > > 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, Jul 16, 2014 at 3:57 PM, Joe Stein <
>> > joe.stein@stealth.ly>
>> > >> > > 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
>> > >> (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 Stein
>> > >> > > > >>  Founder, Principal Consultant
>> > >> > > > >>  Big Data Open Source Security LLC
>> > >> > > > >>  http://www.stealth.ly
>> > >> > > > >>  Twitter: @allthingshadoop <
>> > >> http://www.twitter.com/allthingshadoop>
>> > >> > > > >> ********************************************/
>> > >> > > > >>
>> > >> > > > >>
>> > >> > > > >> On Wed, Jul 16, 2014 at 3:49 PM, Pramod Deshmukh <
>> > >> > dpramodv@gmail.com>
>> > >> > > > >> wrote:
>> > >> > > > >>
>> > >> > > > >> > 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 installed sbt and scala and followed your
>> instructions
>> > >> for
>> > >> > > > >> different
>> > >> > > > >> > scala versions. I was able to bring zookeeper up but
>> brokers
>> > >> fail
>> > >> > to
>> > >> > > > >> start
>> > >> > > > >> > with error
>> > >> > > > >> >
>> > >> > > > >> > Error: Could not find or load main class kafka.Kafka
>> > >> > > > >> >
>> > >> > > > >> > I think I am doing something wrong. Can you please help
>> me?
>> > >> > > > >> >
>> > >> > > > >> > Our current production setup is with 2.8.0 and want to
>> stick
>> > to
>> > >> > it.
>> > >> > > > >> >
>> > >> > > > >> > Thanks,
>> > >> > > > >> >
>> > >> > > > >> > Pramod
>> > >> > > > >> >
>> > >> > > > >> >
>> > >> > > > >> > On Tue, Jun 3, 2014 at 3:57 PM, Joe Stein <
>> > >> joe.stein@stealth.ly>
>> > >> > > > wrote:
>> > >> > > > >> >
>> > >> > > > >> > > 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.  This comes from what folks are doing "in
>> the
>> > >> > wild"
>> > >> > > to
>> > >> > > > >> > > workaround and implement security with Kafka as it is
>> today
>> > >> and
>> > >> > > also
>> > >> > > > >> > what I
>> > >> > > > >> > > have discovered from organizations about their
>> blockers. It
>> > >> also
>> > >> > > > >> picks up
>> > >> > > > >> > > from the wiki (which I should have time to update later
>> in
>> > >> the
>> > >> > > week
>> > >> > > > >> based
>> > >> > > > >> > > on the below and feedback from the thread).
>> > >> > > > >> > >
>> > >> > > > >> > > 1) Transport Layer Security (i.e. SSL)
>> > >> > > > >> > >
>> > >> > > > >> > > This also includes client authentication in addition to
>> > >> > in-transit
>> > >> > > > >> > security
>> > >> > > > >> > > layer.  This work has been picked up here
>> > >> > > > >> > > https://issues.apache.org/jira/browse/KAFKA-1477 and do
>> > >> > > appreciate
>> > >> > > > >> any
>> > >> > > > >> > > thoughts, comments, feedback, tomatoes, whatever for
>> this
>> > >> patch.
>> > >> > >  It
>> > >> > > > >> is a
>> > >> > > > >> > > pickup from the fork of the work first done here
>> > >> > > > >> > > https://github.com/relango/kafka/tree/kafka_security.
>> > >> > > > >> > >
>> > >> > > > >> > > 2) Data encryption at rest.
>> > >> > > > >> > >
>> > >> > > > >> > > This is very important and something that can be
>> > facilitated
>> > >> > > within
>> > >> > > > >> the
>> > >> > > > >> > > wire protocol. It requires an additional map data
>> structure
>> > >> for
>> > >> > > the
>> > >> > > > >> > > "encrypted [data encryption key]". With this map
>> (either in
>> > >> your
>> > >> > > > >> object
>> > >> > > > >> > or
>> > >> > > > >> > > in the wire protocol) you can store the dynamically
>> > generated
>> > >> > > > >> symmetric
>> > >> > > > >> > key
>> > >> > > > >> > > (for each message) and then encrypt the data using that
>> > >> > > dynamically
>> > >> > > > >> > > generated key.  You then encrypt the encryption key
>> using
>> > >> each
>> > >> > > > public
>> > >> > > > >> key
>> > >> > > > >> > > for whom is expected to be able to decrypt the
>> encryption
>> > >> key to
>> > >> > > > then
>> > >> > > > >> > > decrypt the message.  For each public key encrypted
>> > symmetric
>> > >> > key
>> > >> > > > >> (which
>> > >> > > > >> > is
>> > >> > > > >> > > now the "encrypted [data encryption key]" along with
>> which
>> > >> > public
>> > >> > > > key
>> > >> > > > >> it
>> > >> > > > >> > > was encrypted with for (so a map of [publicKey] =
>> > >> > > > >> > > encryptedDataEncryptionKey) as a chain.   Other patterns
>> > can
>> > >> be
>> > >> > > > >> > implemented
>> > >> > > > >> > > but this is a pretty standard digital enveloping [0]
>> > pattern
>> > >> > with
>> > >> > > > >> only 1
>> > >> > > > >> > > field added. Other patterns should be able to use that
>> > field
>> > >> > to-do
>> > >> > > > >> their
>> > >> > > > >> > > implementation too.
>> > >> > > > >> > >
>> > >> > > > >> > > 3) Non-repudiation and long term non-repudiation.
>> > >> > > > >> > >
>> > >> > > > >> > > Non-repudiation is proving data hasn't changed.  This is
>> > >> often
>> > >> > (if
>> > >> > > > not
>> > >> > > > >> > > always) done with x509 public certificates (chained to a
>> > >> > > certificate
>> > >> > > > >> > > authority).
>> > >> > > > >> > >
>> > >> > > > >> > > Long term non-repudiation is what happens when the
>> > >> certificates
>> > >> > of
>> > >> > > > the
>> > >> > > > >> > > certificate authority are expired (or revoked) and
>> > everything
>> > >> > ever
>> > >> > > > >> signed
>> > >> > > > >> > > (ever) with that certificate's public key then becomes
>> "no
>> > >> > longer
>> > >> > > > >> > provable
>> > >> > > > >> > > as ever being authentic".  That is where RFC3126 [1] and
>> > >> RFC3161
>> > >> > > [2]
>> > >> > > > >> come
>> > >> > > > >> > > in (or worm drives [hardware], etc).
>> > >> > > > >> > >
>> > >> > > > >> > > For either (or both) of these it is an operation of the
>> > >> > encryptor
>> > >> > > to
>> > >> > > > >> > > sign/hash the data (with or without third party trusted
>> > >> timestap
>> > >> > > of
>> > >> > > > >> the
>> > >> > > > >> > > signing event) and encrypt that with their own private
>> key
>> > >> and
>> > >> > > > >> distribute
>> > >> > > > >> > > the results (before and after encrypting if required)
>> along
>> > >> with
>> > >> > > > their
>> > >> > > > >> > > public key. This structure is a bit more complex but
>> > >> feasible,
>> > >> > it
>> > >> > > > is a
>> > >> > > > >> > map
>> > >> > > > >> > > of digital signature formats and the chain of dig sig
>> > >> > > attestations.
>> > >> > > > >>  The
>> > >> > > > >> > > map's key being the method (i.e. CRC32, PKCS7 [3],
>> > XmlDigSig
>> > >> > [4])
>> > >> > > > and
>> > >> > > > >> > then
>> > >> > > > >> > > a list of map where that key is "purpose" of signature
>> > (what
>> > >> > your
>> > >> > > > >> > attesting
>> > >> > > > >> > > too).  As a sibling field to the list another field for
>> > "the
>> > >> > > > >> attester" as
>> > >> > > > >> > > bytes (e.g. their PKCS12 [5] for the map of PKCS7
>> > >> signatures).
>> > >> > > > >> > >
>> > >> > > > >> > > 4) Authorization
>> > >> > > > >> > >
>> > >> > > > >> > > We should have a policy of "404" for data, topics,
>> > partitions
>> > >> > > (etc)
>> > >> > > > if
>> > >> > > > >> > > authenticated connections do not have access.  In
>> "secure
>> > >> mode"
>> > >> > > any
>> > >> > > > >> non
>> > >> > > > >> > > authenticated connections should get a "404" type
>> message
>> > on
>> > >> > > > >> everything.
>> > >> > > > >> > > Knowing "something is there" is a security risk in many
>> > uses
>> > >> > > cases.
>> > >> > > > >>  So
>> > >> > > > >> > if
>> > >> > > > >> > > you don't have access you don't even see it.  Baking
>> "that"
>> > >> into
>> > >> > > > Kafka
>> > >> > > > >> > > along with some interface for entitlement (access
>> > management)
>> > >> > > > systems
>> > >> > > > >> > > (pretty standard) is all that I think needs to be done
>> to
>> > the
>> > >> > core
>> > >> > > > >> > project.
>> > >> > > > >> > >  I want to tackle item later in the year after summer
>> after
>> > >> the
>> > >> > > > other
>> > >> > > > >> > three
>> > >> > > > >> > > are complete.
>> > >> > > > >> > >
>> > >> > > > >> > > I look forward to thoughts on this and anyone else
>> > >> interested in
>> > >> > > > >> working
>> > >> > > > >> > > with us on these items.
>> > >> > > > >> > >
>> > >> > > > >> > > [0]
>> > >> > > > >> > >
>> > >> > > > >> > >
>> > >> > > > >> >
>> > >> > > > >>
>> > >> > > >
>> > >> > >
>> > >> >
>> > >>
>> >
>> http://www.emc.com/emc-plus/rsa-labs/standards-initiatives/what-is-a-digital-envelope.htm
>> > >> > > > >> > > [1] http://tools.ietf.org/html/rfc3126
>> > >> > > > >> > > [2] http://tools.ietf.org/html/rfc3161
>> > >> > > > >> > > [3]
>> > >> > > > >> > >
>> > >> > > > >> > >
>> > >> > > > >> >
>> > >> > > > >>
>> > >> > > >
>> > >> > >
>> > >> >
>> > >>
>> >
>> http://www.emc.com/emc-plus/rsa-labs/standards-initiatives/pkcs-7-cryptographic-message-syntax-standar.htm
>> > >> > > > >> > > [4] http://en.wikipedia.org/wiki/XML_Signature
>> > >> > > > >> > > [5] http://en.wikipedia.org/wiki/PKCS_12
>> > >> > > > >> > >
>> > >> > > > >> > > /*******************************************
>> > >> > > > >> > >  Joe Stein
>> > >> > > > >> > >  Founder, Principal Consultant
>> > >> > > > >> > >  Big Data Open Source Security LLC
>> > >> > > > >> > >  http://www.stealth.ly
>> > >> > > > >> > >  Twitter: @allthingshadoop <
>> > >> > > http://www.twitter.com/allthingshadoop>
>> > >> > > > >> > > ********************************************/
>> > >> > > > >> > >
>> > >> > > > >> >
>> > >> > > > >>
>> > >> > > > >
>> > >> > > > >
>> > >> > > >
>> > >> > >
>> > >> > >
>> > >> > >
>> > >> > > --
>> > >> > > Thanks,
>> > >> > > Raja.
>> > >> > >
>> > >> >
>> > >>
>> > >>
>> > >>
>> > >> --
>> > >> Thanks,
>> > >> Raja.
>> > >>
>> > >
>> > >
>> >
>>
>
>

Re: [DISCUSS] Kafka Security Specific Features

Posted by Pramod Deshmukh <dp...@gmail.com>.
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 not valid
(kafka.utils.VerifiableProperties)

Hare Krishna

[2014-07-18 13:12:45,256] WARN Fetching topic metadata with correlation id
0 for topics [Set(secureTopic)] from broker
[id:0,host:localhost,port:9092,secure:true] failed
(kafka.client.ClientUtils$)

java.io.EOFException: Received -1 when reading from channel, socket has
likely been closed.

at kafka.utils.Utils$.read(Utils.scala:381)

at
kafka.network.BoundedByteBufferReceive.readFrom(BoundedByteBufferReceive.scala:67)

at kafka.network.Receive$class.readCompletely(Transmission.scala:56)

at
kafka.network.BoundedByteBufferReceive.readCompletely(BoundedByteBufferReceive.scala:29)

at kafka.network.BlockingChannel.receive(BlockingChannel.scala:102)

at kafka.producer.SyncProducer.liftedTree1$1(SyncProducer.scala:79)

at
kafka.producer.SyncProducer.kafka$producer$SyncProducer$$doSend(SyncProducer.scala:76)

at kafka.producer.SyncProducer.send(SyncProducer.scala:117)

at kafka.client.ClientUtils$.fetchTopicMetadata(ClientUtils.scala:58)

at
kafka.producer.BrokerPartitionInfo.updateInfo(BrokerPartitionInfo.scala:82)

at
kafka.producer.async.DefaultEventHandler$$anonfun$handle$1.apply$mcV$sp(DefaultEventHandler.scala:67)

at kafka.utils.Utils$.swallow(Utils.scala:172)

at kafka.utils.Logging$class.swallowError(Logging.scala:106)

at kafka.utils.Utils$.swallowError(Utils.scala:45)

at
kafka.producer.async.DefaultEventHandler.handle(DefaultEventHandler.scala:67)

at
kafka.producer.async.ProducerSendThread.tryToHandle(ProducerSendThread.scala:104)

at
kafka.producer.async.ProducerSendThread$$anonfun$processEvents$3.apply(ProducerSendThread.scala:87)

at
kafka.producer.async.ProducerSendThread$$anonfun$processEvents$3.apply(ProducerSendThread.scala:67)

at scala.collection.immutable.Stream.foreach(Stream.scala:526)

at
kafka.producer.async.ProducerSendThread.processEvents(ProducerSendThread.scala:66)

at kafka.producer.async.ProducerSendThread.run(ProducerSendThread.scala:44)


On Fri, Jul 18, 2014 at 1:20 PM, Pramod Deshmukh <dp...@gmail.com> wrote:

> 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 topic "secureTopic".
>
> pdeshmukh$ bin/kafka-topics.sh --list --zookeeper localhost:2181
>
> secure.test
>
> secureTopic
>
> >> Run producer, tried both localhost:9092:true and localhost:9092
>
> pdeshmukh$ bin/kafka-console-producer.sh --broker-list localhost:9092:true
> --topic secureTopic
>
> [2014-07-18 13:12:29,817] WARN Property topic is not valid
> (kafka.utils.VerifiableProperties)
>
> Hare Krishna
>
> [2014-07-18 13:12:45,256] WARN Fetching topic metadata with correlation id
> 0 for topics [Set(secureTopic)] from broker
> [id:0,host:localhost,port:9092,secure:true] failed
> (kafka.client.ClientUtils$)
>
> java.io.EOFException: Received -1 when reading from channel, socket has
> likely been closed.
>
> at kafka.utils.Utils$.read(Utils.scala:381)
>
> at
> kafka.network.BoundedByteBufferReceive.readFrom(BoundedByteBufferReceive.scala:67)
>
> at kafka.network.Receive$class.readCompletely(Transmission.scala:56)
>
> at
> kafka.network.BoundedByteBufferReceive.readCompletely(BoundedByteBufferReceive.scala:29)
>
> at kafka.network.BlockingChannel.receive(BlockingChannel.scala:102)
>
> at kafka.producer.SyncProducer.liftedTree1$1(SyncProducer.scala:79)
>
> at
> kafka.producer.SyncProducer.kafka$producer$SyncProducer$$doSend(SyncProducer.scala:76)
>
> at kafka.producer.SyncProducer.send(SyncProducer.scala:117)
>
> at kafka.client.ClientUtils$.fetchTopicMetadata(ClientUtils.scala:58)
>
> at
> kafka.producer.BrokerPartitionInfo.updateInfo(BrokerPartitionInfo.scala:82)
>
> at
> kafka.producer.async.DefaultEventHandler$$anonfun$handle$1.apply$mcV$sp(DefaultEventHandler.scala:67)
>
> at kafka.utils.Utils$.swallow(Utils.scala:172)
>
> at kafka.utils.Logging$class.swallowError(Logging.scala:106)
>
> at kafka.utils.Utils$.swallowError(Utils.scala:45)
>
> at
> kafka.producer.async.DefaultEventHandler.handle(DefaultEventHandler.scala:67)
>
> at
> kafka.producer.async.ProducerSendThread.tryToHandle(ProducerSendThread.scala:104)
>
> at
> kafka.producer.async.ProducerSendThread$$anonfun$processEvents$3.apply(ProducerSendThread.scala:87)
>
> at
> kafka.producer.async.ProducerSendThread$$anonfun$processEvents$3.apply(ProducerSendThread.scala:67)
>
> at scala.collection.immutable.Stream.foreach(Stream.scala:526)
>
> at
> kafka.producer.async.ProducerSendThread.processEvents(ProducerSendThread.scala:66)
>
> at kafka.producer.async.ProducerSendThread.run(ProducerSendThread.scala:44)
>
> [2014-07-18 13:12:45,258] ERROR fetching topic metadata for topics
> [Set(secureTopic)] from broker
> [ArrayBuffer(id:0,host:localhost,port:9092,secure:true)] failed
> (kafka.utils.Utils$)
>
> kafka.common.KafkaException: fetching topic metadata for topics
> [Set(secureTopic)] from broker
> [ArrayBuffer(id:0,host:localhost,port:9092,secure:true)] failed
>
> at kafka.client.ClientUtils$.fetchTopicMetadata(ClientUtils.scala:72)
>
> at
> kafka.producer.BrokerPartitionInfo.updateInfo(BrokerPartitionInfo.scala:82)
>
> at
> kafka.producer.async.DefaultEventHandler$$anonfun$handle$1.apply$mcV$sp(DefaultEventHandler.scala:67)
>
> at kafka.utils.Utils$.swallow(Utils.scala:172)
>
> at kafka.utils.Logging$class.swallowError(Logging.scala:106)
>
> at kafka.utils.Utils$.swallowError(Utils.scala:45)
>
> at
> kafka.producer.async.DefaultEventHandler.handle(DefaultEventHandler.scala:67)
>
> at
> kafka.producer.async.ProducerSendThread.tryToHandle(ProducerSendThread.scala:104)
>
> at
> kafka.producer.async.ProducerSendThread$$anonfun$processEvents$3.apply(ProducerSendThread.scala:87)
>
> at
> kafka.producer.async.ProducerSendThread$$anonfun$processEvents$3.apply(ProducerSendThread.scala:67)
>
> at scala.collection.immutable.Stream.foreach(Stream.scala:526)
>
> at
> kafka.producer.async.ProducerSendThread.processEvents(ProducerSendThread.scala:66)
>
> at kafka.producer.async.ProducerSendThread.run(ProducerSendThread.scala:44)
>
> Caused by: java.io.EOFException: Received -1 when reading from channel,
> socket has likely been closed.
>
> at kafka.utils.Utils$.read(Utils.scala:381)
>
> at
> kafka.network.BoundedByteBufferReceive.readFrom(BoundedByteBufferReceive.scala:67)
>
> at kafka.network.Receive$class.readCompletely(Transmission.scala:56)
>
> at
> kafka.network.BoundedByteBufferReceive.readCompletely(BoundedByteBufferReceive.scala:29)
>
> at kafka.network.BlockingChannel.receive(BlockingChannel.scala:102)
>
> at kafka.producer.SyncProducer.liftedTree1$1(SyncProducer.scala:79)
>
> at
> kafka.producer.SyncProducer.kafka$producer$SyncProducer$$doSend(SyncProducer.scala:76)
>
> at kafka.producer.SyncProducer.send(SyncProducer.scala:117)
>
> at kafka.client.ClientUtils$.fetchTopicMetadata(ClientUtils.scala:58)
>
> ... 12 more
> [2014-07-18 13:12:45,337] WARN Fetching topic metadata with correlation id
> 1 for topics [Set(secureTopic)] from broker
> [id:0,host:localhost,port:9092,secure:true] failed
> (kafka.client.ClientUtils$)
>
> 2014-07-18 13:12:46,282] ERROR Failed to send requests for topics
> secureTopic with correlation ids in [0,8]
> (kafka.producer.async.DefaultEventHandler)
>
> [2014-07-18 13:12:46,283] ERROR Error in handling batch of 1 events
> (kafka.producer.async.ProducerSendThread)
>
> kafka.common.FailedToSendMessageException: Failed to send messages after 3
> tries.
>
> at
> kafka.producer.async.DefaultEventHandler.handle(DefaultEventHandler.scala:90)
>
> at
> kafka.producer.async.ProducerSendThread.tryToHandle(ProducerSendThread.scala:104)
>
> at
> kafka.producer.async.ProducerSendThread$$anonfun$processEvents$3.apply(ProducerSendThread.scala:87)
>
> at
> kafka.producer.async.ProducerSendThread$$anonfun$processEvents$3.apply(ProducerSendThread.scala:67)
>
> at scala.collection.immutable.Stream.foreach(Stream.scala:526)
>
> at
> kafka.producer.async.ProducerSendThread.processEvents(ProducerSendThread.scala:66)
>
> at kafka.producer.async.ProducerSendThread.run(ProducerSendThread.scala:44)
>
>
>
> On Fri, Jul 18, 2014 at 11:56 AM, Joe Stein <jo...@stealth.ly> wrote:
>
>> 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://www.stealth.ly
>>  Twitter: @allthingshadoop <http://www.twitter.com/allthingshadoop>
>> ********************************************/
>>
>>
>> On Fri, Jul 18, 2014 at 10:24 AM, Pramod Deshmukh <dp...@gmail.com>
>> wrote:
>>
>> > 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.OutOfMemoryError: Java heap space
>> >
>> > *broker log*
>> >
>> > INFO begin ssl handshake for localhost/127.0.0.1:50199//127.0.0.1:9092
>> >
>> >
>> >
>> >
>> >
>> > On Thu, Jul 17, 2014 at 6:07 PM, Pramod Deshmukh <dp...@gmail.com>
>> > wrote:
>> >
>> > > 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 <
>> > relango@salesforce.com>
>> > > wrote:
>> > >
>> > >> 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 <dpramodv@gmail.com
>> >
>> > >> 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.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 config/server.properties*
>> > >> >
>> > >> >
>> > >> >
>> > >> > *broker.log:*
>> > >> >
>> > >> > [2014-07-17 15:34:46,281] INFO zookeeper state changed
>> (SyncConnected)
>> > >> > (org.I0Itec.zkclient.ZkClient)
>> > >> >
>> > >> > [2014-07-17 15:34:46,523] INFO Loading log 'secure.test-0'
>> > >> > (kafka.log.LogManager)
>> > >> >
>> > >> > [2014-07-17 15:34:46,558] INFO Recovering unflushed segment 0 in
>> log
>> > >> > secure.test-0. (kafka.log.Log)
>> > >> >
>> > >> > [2014-07-17 15:34:46,571] INFO Completed load of log secure.test-0
>> > with
>> > >> log
>> > >> > end offset 0 (kafka.log.Log)
>> > >> >
>> > >> > [2014-07-17 15:34:46,582] INFO Starting log cleanup with a period
>> of
>> > >> 60000
>> > >> > ms. (kafka.log.LogManager)
>> > >> >
>> > >> > [2014-07-17 15:34:46,587] INFO Starting log flusher with a default
>> > >> period
>> > >> > of 9223372036854775807 ms. (kafka.log.LogManager)
>> > >> >
>> > >> > [2014-07-17 15:34:46,614] INFO Initializing secure authentication
>> > >> > (kafka.network.security.SecureAuth$)
>> > >> >
>> > >> > [2014-07-17 15:34:46,678] INFO Secure authentication initialization
>> > has
>> > >> > been successfully completed (kafka.network.security.SecureAuth$)
>> > >> >
>> > >> > [2014-07-17 15:34:46,691] INFO Awaiting socket connections on
>> > >> 0.0.0.0:9092
>> > >> > .
>> > >> > (kafka.network.Acceptor)
>> > >> >
>> > >> > [2014-07-17 15:34:46,692] INFO [Socket Server on Broker 0], Started
>> > >> > (kafka.network.SocketServer)
>> > >> >
>> > >> > [2014-07-17 15:34:46,794] INFO Will not load MX4J, mx4j-tools.jar
>> is
>> > >> not in
>> > >> > the classpath (kafka.utils.Mx4jLoader$)
>> > >> >
>> > >> > [2014-07-17 15:34:46,837] INFO 0 successfully elected as leader
>> > >> > (kafka.server.ZookeeperLeaderElector)
>> > >> >
>> > >> > [2014-07-17 15:34:47,057] INFO Registered broker 0 at path
>> > >> /brokers/ids/0
>> > >> > with address 10.1.100.130:9092. (kafka.utils.ZkUtils$)
>> > >> >
>> > >> > [2014-07-17 15:34:47,059] INFO New leader is 0
>> > >> > (kafka.server.ZookeeperLeaderElector$LeaderChangeListener)
>> > >> >
>> > >> > *[2014-07-17 15:34:47,068] INFO [Kafka Server 0], started
>> > >> > (kafka.server.KafkaServer)*
>> > >> >
>> > >> > *[2014-07-17 15:34:47,383] INFO begin ssl handshake for
>> > >> > /10.1.100.130:9092//10.1.100.130:51685
>> > >> > <http://10.1.100.130:9092//10.1.100.130:51685>
>> > >> > (kafka.network.security.SSLSocketChannel)*
>> > >> >
>> > >> > *[2014-07-17 15:34:47,392] INFO begin ssl handshake for
>> > >> > 10.1.100.130/10.1.100.130:51685//10.1.100.130:9092
>> > >> > <http://10.1.100.130/10.1.100.130:51685//10.1.100.130:9092>
>> > >> > (kafka.network.security.SSLSocketChannel)*
>> > >> >
>> > >> > *[2014-07-17 15:34:47,465] INFO finished ssl handshake for
>> > >> > 10.1.100.130/10.1.100.130:51685//10.1.100.130:9092
>> > >> > <http://10.1.100.130/10.1.100.130:51685//10.1.100.130:9092>
>> > >> > (kafka.network.security.SSLSocketChannel)*
>> > >> >
>> > >> > *[2014-07-17 15:34:47,465] INFO finished ssl handshake for
>> > >> > /10.1.100.130:9092//10.1.100.130:51685
>> > >> > <http://10.1.100.130:9092//10.1.100.130:51685>
>> > >> > (kafka.network.security.SSLSocketChannel)*
>> > >> >
>> > >> > *[2014-07-17 15:34:47,617] INFO [ReplicaFetcherManager on broker 0]
>> > >> Removed
>> > >> > fetcher for partitions  (kafka.server.ReplicaFetcherManager)*
>> > >> >
>> > >> > *[2014-07-17 15:34:47,627] INFO [ReplicaFetcherManager on broker 0]
>> > >> Added
>> > >> > fetcher for partitions List() (kafka.server.ReplicaFetcherManager)*
>> > >> >
>> > >> > *[2014-07-17 15:34:47,656] INFO [ReplicaFetcherManager on broker 0]
>> > >> Removed
>> > >> > fetcher for partitions [secure.test,0]
>> > >> > (kafka.server.ReplicaFetcherManager)*
>> > >> >
>> > >> > [2014-07-17 15:37:15,970] INFO begin ssl handshake for
>> > >> > 10.1.100.130/10.1.100.130:51689//10.1.100.130:9092
>> > >> > (kafka.network.security.SSLSocketChannel)
>> > >> >
>> > >> > [2014-07-17 15:37:16,075] INFO begin ssl handshake for
>> > >> > 10.1.100.130/10.1.100.130:51690//10.1.100.130:9092
>> > >> > (kafka.network.security.SSLSocketChannel)
>> > >> >
>> > >> > [2014-07-17 15:37:16,434] INFO begin ssl handshake for
>> > >> > 10.1.100.130/10.1.100.130:51691//10.1.100.130:9092
>> > >> > (kafka.network.security.SSLSocketChannel)
>> > >> >
>> > >> > [2014-07-17 15:37:16,530] INFO begin ssl handshake for
>> > >> > 10.1.100.130/10.1.100.130:51692//10.1.100.130:9092
>> > >> > (kafka.network.security.SSLSocketChannel)
>> > >> >
>> > >> > [2014-07-17 15:37:16,743] INFO begin ssl handshake for
>> > >> > 10.1.100.130/10.1.100.130:51693//10.1.100.130:9092
>> > >> > (kafka.network.security.SSLSocketChannel)
>> > >> >
>> > >> > [2014-07-17 15:37:16,834] INFO begin ssl handshake for
>> > >> > 10.1.100.130/10.1.100.130:51694//10.1.100.130:9092
>> > >> > (kafka.network.security.SSLSocketChannel)
>> > >> >
>> > >> > [2014-07-17 15:37:17,043] INFO begin ssl handshake for
>> > >> > 10.1.100.130/10.1.100.130:51695//10.1.100.130:9092
>> > >> > (kafka.network.security.SSLSocketChannel)
>> > >> >
>> > >> > [2014-07-17 15:37:17,137] INFO begin ssl handshake for
>> > >> > 10.1.100.130/10.1.100.130:51696//10.1.100.130:9092
>> > >> > (kafka.network.security.SSLSocketChannel)
>> > >> >
>> > >> > [2014-07-17 15:37:17,342] INFO begin ssl handshake for
>> > >> > 10.1.100.130/10.1.100.130:51697//10.1.100.130:9092
>> > >> > (kafka.network.security.SSLSocketChannel)
>> > >> >
>> > >> >
>> > >> > *Start producer*
>> > >> >
>> > >> > *bin/kafka-console-producer.sh --broker-list 10.1.100.130:9092
>> :true
>> > >> > --topic
>> > >> > secure.test*
>> > >> >
>> > >> >
>> > >> > *producer.log:*
>> > >> >
>> > >> > bin/kafka-console-producer.sh --broker-list 10.1.100.130:9092:true
>> > >> --topic
>> > >> > secure.test
>> > >> >
>> > >> > [2014-07-17 15:37:46,889] WARN Property topic is not valid
>> > >> > (kafka.utils.VerifiableProperties)
>> > >> >
>> > >> > Hello Secure Kafka
>> > >> >
>> > >> > *[2014-07-17 15:38:14,186] ERROR OOME with size 352518400
>> > >> > (kafka.network.BoundedByteBufferReceive)*
>> > >> >
>> > >> > *java.lang.OutOfMemoryError: Java heap space*
>> > >> >
>> > >> > at java.nio.HeapByteBuffer.<init>(HeapByteBuffer.java:57)
>> > >> >
>> > >> > at java.nio.ByteBuffer.allocate(ByteBuffer.java:331)
>> > >> >
>> > >> > at
>> > >> >
>> > >> >
>> > >>
>> >
>> kafka.network.BoundedByteBufferReceive.byteBufferAllocate(BoundedByteBufferReceive.scala:80)
>> > >> >
>> > >> > at
>> > >> >
>> > >> >
>> > >>
>> >
>> kafka.network.BoundedByteBufferReceive.readFrom(BoundedByteBufferReceive.scala:63)
>> > >> >
>> > >> > at
>> kafka.network.Receive$class.readCompletely(Transmission.scala:56)
>> > >> >
>> > >> > at
>> > >> >
>> > >> >
>> > >>
>> >
>> kafka.network.BoundedByteBufferReceive.readCompletely(BoundedByteBufferReceive.scala:29)
>> > >> >
>> > >> > at kafka.network.BlockingChannel.receive(BlockingChannel.scala:102)
>> > >> >
>> > >> > at kafka.producer.SyncProducer.liftedTree1$1(SyncProducer.scala:79)
>> > >> >
>> > >> > at
>> > >> >
>> > >> >
>> > >>
>> >
>> kafka.producer.SyncProducer.kafka$producer$SyncProducer$$doSend(SyncProducer.scala:76)
>> > >> >
>> > >> > at kafka.producer.SyncProducer.send(SyncProducer.scala:117)
>> > >> >
>> > >> > at
>> kafka.client.ClientUtils$.fetchTopicMetadata(ClientUtils.scala:58)
>> > >> >
>> > >> > at
>> > >> >
>> > >>
>> >
>> kafka.producer.BrokerPartitionInfo.updateInfo(BrokerPartitionInfo.scala:82)
>> > >> >
>> > >> > at
>> > >> >
>> > >> >
>> > >>
>> >
>> kafka.producer.async.DefaultEventHandler$$anonfun$handle$1.apply$mcV$sp(DefaultEventHandler.scala:67)
>> > >> >
>> > >> > at kafka.utils.Utils$.swallow(Utils.scala:172)
>> > >> >
>> > >> > at kafka.utils.Logging$class.swallowError(Logging.scala:106)
>> > >> >
>> > >> > at kafka.utils.Utils$.swallowError(Utils.scala:45)
>> > >> >
>> > >> > at
>> > >> >
>> > >> >
>> > >>
>> >
>> kafka.producer.async.DefaultEventHandler.handle(DefaultEventHandler.scala:67)
>> > >> >
>> > >> > at
>> > >> >
>> > >> >
>> > >>
>> >
>> kafka.producer.async.ProducerSendThread.tryToHandle(ProducerSendThread.scala:104)
>> > >> >
>> > >> > at
>> > >> >
>> > >> >
>> > >>
>> >
>> kafka.producer.async.ProducerSendThread$$anonfun$processEvents$3.apply(ProducerSendThread.scala:87)
>> > >> >
>> > >> > at
>> > >> >
>> > >> >
>> > >>
>> >
>> kafka.producer.async.ProducerSendThread$$anonfun$processEvents$3.apply(ProducerSendThread.scala:67)
>> > >> >
>> > >> > at scala.collection.immutable.Stream.foreach(Stream.scala:526)
>> > >> >
>> > >> > at
>> > >> >
>> > >> >
>> > >>
>> >
>> kafka.producer.async.ProducerSendThread.processEvents(ProducerSendThread.scala:66)
>> > >> >
>> > >> > at
>> > >>
>> kafka.producer.async.ProducerSendThread.run(ProducerSendThread.scala:44)
>> > >> >
>> > >> >
>> > >> >
>> > >> > On Wed, Jul 16, 2014 at 6:07 PM, Rajasekar Elango <
>> > >> relango@salesforce.com>
>> > >> > wrote:
>> > >> >
>> > >> > > Pramod,
>> > >> > >
>> > >> > >
>> > >> > > I presented secure kafka configuration and usage at last meet
>> up. So
>> > >> hope
>> > >> > > this
>> > >> > > video recording <http://www.ustream.tv/recorded/48396701>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 <
>> > dpramodv@gmail.com>
>> > >> > > wrote:
>> > >> > >
>> > >> > > > 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 <
>> > >> dpramodv@gmail.com>
>> > >> > > > 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, Jul 16, 2014 at 3:57 PM, Joe Stein <
>> > joe.stein@stealth.ly>
>> > >> > > 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
>> > >> (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 Stein
>> > >> > > > >>  Founder, Principal Consultant
>> > >> > > > >>  Big Data Open Source Security LLC
>> > >> > > > >>  http://www.stealth.ly
>> > >> > > > >>  Twitter: @allthingshadoop <
>> > >> http://www.twitter.com/allthingshadoop>
>> > >> > > > >> ********************************************/
>> > >> > > > >>
>> > >> > > > >>
>> > >> > > > >> On Wed, Jul 16, 2014 at 3:49 PM, Pramod Deshmukh <
>> > >> > dpramodv@gmail.com>
>> > >> > > > >> wrote:
>> > >> > > > >>
>> > >> > > > >> > 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 installed sbt and scala and followed your
>> instructions
>> > >> for
>> > >> > > > >> different
>> > >> > > > >> > scala versions. I was able to bring zookeeper up but
>> brokers
>> > >> fail
>> > >> > to
>> > >> > > > >> start
>> > >> > > > >> > with error
>> > >> > > > >> >
>> > >> > > > >> > Error: Could not find or load main class kafka.Kafka
>> > >> > > > >> >
>> > >> > > > >> > I think I am doing something wrong. Can you please help
>> me?
>> > >> > > > >> >
>> > >> > > > >> > Our current production setup is with 2.8.0 and want to
>> stick
>> > to
>> > >> > it.
>> > >> > > > >> >
>> > >> > > > >> > Thanks,
>> > >> > > > >> >
>> > >> > > > >> > Pramod
>> > >> > > > >> >
>> > >> > > > >> >
>> > >> > > > >> > On Tue, Jun 3, 2014 at 3:57 PM, Joe Stein <
>> > >> joe.stein@stealth.ly>
>> > >> > > > wrote:
>> > >> > > > >> >
>> > >> > > > >> > > 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.  This comes from what folks are doing "in
>> the
>> > >> > wild"
>> > >> > > to
>> > >> > > > >> > > workaround and implement security with Kafka as it is
>> today
>> > >> and
>> > >> > > also
>> > >> > > > >> > what I
>> > >> > > > >> > > have discovered from organizations about their
>> blockers. It
>> > >> also
>> > >> > > > >> picks up
>> > >> > > > >> > > from the wiki (which I should have time to update later
>> in
>> > >> the
>> > >> > > week
>> > >> > > > >> based
>> > >> > > > >> > > on the below and feedback from the thread).
>> > >> > > > >> > >
>> > >> > > > >> > > 1) Transport Layer Security (i.e. SSL)
>> > >> > > > >> > >
>> > >> > > > >> > > This also includes client authentication in addition to
>> > >> > in-transit
>> > >> > > > >> > security
>> > >> > > > >> > > layer.  This work has been picked up here
>> > >> > > > >> > > https://issues.apache.org/jira/browse/KAFKA-1477 and do
>> > >> > > appreciate
>> > >> > > > >> any
>> > >> > > > >> > > thoughts, comments, feedback, tomatoes, whatever for
>> this
>> > >> patch.
>> > >> > >  It
>> > >> > > > >> is a
>> > >> > > > >> > > pickup from the fork of the work first done here
>> > >> > > > >> > > https://github.com/relango/kafka/tree/kafka_security.
>> > >> > > > >> > >
>> > >> > > > >> > > 2) Data encryption at rest.
>> > >> > > > >> > >
>> > >> > > > >> > > This is very important and something that can be
>> > facilitated
>> > >> > > within
>> > >> > > > >> the
>> > >> > > > >> > > wire protocol. It requires an additional map data
>> structure
>> > >> for
>> > >> > > the
>> > >> > > > >> > > "encrypted [data encryption key]". With this map
>> (either in
>> > >> your
>> > >> > > > >> object
>> > >> > > > >> > or
>> > >> > > > >> > > in the wire protocol) you can store the dynamically
>> > generated
>> > >> > > > >> symmetric
>> > >> > > > >> > key
>> > >> > > > >> > > (for each message) and then encrypt the data using that
>> > >> > > dynamically
>> > >> > > > >> > > generated key.  You then encrypt the encryption key
>> using
>> > >> each
>> > >> > > > public
>> > >> > > > >> key
>> > >> > > > >> > > for whom is expected to be able to decrypt the
>> encryption
>> > >> key to
>> > >> > > > then
>> > >> > > > >> > > decrypt the message.  For each public key encrypted
>> > symmetric
>> > >> > key
>> > >> > > > >> (which
>> > >> > > > >> > is
>> > >> > > > >> > > now the "encrypted [data encryption key]" along with
>> which
>> > >> > public
>> > >> > > > key
>> > >> > > > >> it
>> > >> > > > >> > > was encrypted with for (so a map of [publicKey] =
>> > >> > > > >> > > encryptedDataEncryptionKey) as a chain.   Other patterns
>> > can
>> > >> be
>> > >> > > > >> > implemented
>> > >> > > > >> > > but this is a pretty standard digital enveloping [0]
>> > pattern
>> > >> > with
>> > >> > > > >> only 1
>> > >> > > > >> > > field added. Other patterns should be able to use that
>> > field
>> > >> > to-do
>> > >> > > > >> their
>> > >> > > > >> > > implementation too.
>> > >> > > > >> > >
>> > >> > > > >> > > 3) Non-repudiation and long term non-repudiation.
>> > >> > > > >> > >
>> > >> > > > >> > > Non-repudiation is proving data hasn't changed.  This is
>> > >> often
>> > >> > (if
>> > >> > > > not
>> > >> > > > >> > > always) done with x509 public certificates (chained to a
>> > >> > > certificate
>> > >> > > > >> > > authority).
>> > >> > > > >> > >
>> > >> > > > >> > > Long term non-repudiation is what happens when the
>> > >> certificates
>> > >> > of
>> > >> > > > the
>> > >> > > > >> > > certificate authority are expired (or revoked) and
>> > everything
>> > >> > ever
>> > >> > > > >> signed
>> > >> > > > >> > > (ever) with that certificate's public key then becomes
>> "no
>> > >> > longer
>> > >> > > > >> > provable
>> > >> > > > >> > > as ever being authentic".  That is where RFC3126 [1] and
>> > >> RFC3161
>> > >> > > [2]
>> > >> > > > >> come
>> > >> > > > >> > > in (or worm drives [hardware], etc).
>> > >> > > > >> > >
>> > >> > > > >> > > For either (or both) of these it is an operation of the
>> > >> > encryptor
>> > >> > > to
>> > >> > > > >> > > sign/hash the data (with or without third party trusted
>> > >> timestap
>> > >> > > of
>> > >> > > > >> the
>> > >> > > > >> > > signing event) and encrypt that with their own private
>> key
>> > >> and
>> > >> > > > >> distribute
>> > >> > > > >> > > the results (before and after encrypting if required)
>> along
>> > >> with
>> > >> > > > their
>> > >> > > > >> > > public key. This structure is a bit more complex but
>> > >> feasible,
>> > >> > it
>> > >> > > > is a
>> > >> > > > >> > map
>> > >> > > > >> > > of digital signature formats and the chain of dig sig
>> > >> > > attestations.
>> > >> > > > >>  The
>> > >> > > > >> > > map's key being the method (i.e. CRC32, PKCS7 [3],
>> > XmlDigSig
>> > >> > [4])
>> > >> > > > and
>> > >> > > > >> > then
>> > >> > > > >> > > a list of map where that key is "purpose" of signature
>> > (what
>> > >> > your
>> > >> > > > >> > attesting
>> > >> > > > >> > > too).  As a sibling field to the list another field for
>> > "the
>> > >> > > > >> attester" as
>> > >> > > > >> > > bytes (e.g. their PKCS12 [5] for the map of PKCS7
>> > >> signatures).
>> > >> > > > >> > >
>> > >> > > > >> > > 4) Authorization
>> > >> > > > >> > >
>> > >> > > > >> > > We should have a policy of "404" for data, topics,
>> > partitions
>> > >> > > (etc)
>> > >> > > > if
>> > >> > > > >> > > authenticated connections do not have access.  In
>> "secure
>> > >> mode"
>> > >> > > any
>> > >> > > > >> non
>> > >> > > > >> > > authenticated connections should get a "404" type
>> message
>> > on
>> > >> > > > >> everything.
>> > >> > > > >> > > Knowing "something is there" is a security risk in many
>> > uses
>> > >> > > cases.
>> > >> > > > >>  So
>> > >> > > > >> > if
>> > >> > > > >> > > you don't have access you don't even see it.  Baking
>> "that"
>> > >> into
>> > >> > > > Kafka
>> > >> > > > >> > > along with some interface for entitlement (access
>> > management)
>> > >> > > > systems
>> > >> > > > >> > > (pretty standard) is all that I think needs to be done
>> to
>> > the
>> > >> > core
>> > >> > > > >> > project.
>> > >> > > > >> > >  I want to tackle item later in the year after summer
>> after
>> > >> the
>> > >> > > > other
>> > >> > > > >> > three
>> > >> > > > >> > > are complete.
>> > >> > > > >> > >
>> > >> > > > >> > > I look forward to thoughts on this and anyone else
>> > >> interested in
>> > >> > > > >> working
>> > >> > > > >> > > with us on these items.
>> > >> > > > >> > >
>> > >> > > > >> > > [0]
>> > >> > > > >> > >
>> > >> > > > >> > >
>> > >> > > > >> >
>> > >> > > > >>
>> > >> > > >
>> > >> > >
>> > >> >
>> > >>
>> >
>> http://www.emc.com/emc-plus/rsa-labs/standards-initiatives/what-is-a-digital-envelope.htm
>> > >> > > > >> > > [1] http://tools.ietf.org/html/rfc3126
>> > >> > > > >> > > [2] http://tools.ietf.org/html/rfc3161
>> > >> > > > >> > > [3]
>> > >> > > > >> > >
>> > >> > > > >> > >
>> > >> > > > >> >
>> > >> > > > >>
>> > >> > > >
>> > >> > >
>> > >> >
>> > >>
>> >
>> http://www.emc.com/emc-plus/rsa-labs/standards-initiatives/pkcs-7-cryptographic-message-syntax-standar.htm
>> > >> > > > >> > > [4] http://en.wikipedia.org/wiki/XML_Signature
>> > >> > > > >> > > [5] http://en.wikipedia.org/wiki/PKCS_12
>> > >> > > > >> > >
>> > >> > > > >> > > /*******************************************
>> > >> > > > >> > >  Joe Stein
>> > >> > > > >> > >  Founder, Principal Consultant
>> > >> > > > >> > >  Big Data Open Source Security LLC
>> > >> > > > >> > >  http://www.stealth.ly
>> > >> > > > >> > >  Twitter: @allthingshadoop <
>> > >> > > http://www.twitter.com/allthingshadoop>
>> > >> > > > >> > > ********************************************/
>> > >> > > > >> > >
>> > >> > > > >> >
>> > >> > > > >>
>> > >> > > > >
>> > >> > > > >
>> > >> > > >
>> > >> > >
>> > >> > >
>> > >> > >
>> > >> > > --
>> > >> > > Thanks,
>> > >> > > Raja.
>> > >> > >
>> > >> >
>> > >>
>> > >>
>> > >>
>> > >> --
>> > >> Thanks,
>> > >> Raja.
>> > >>
>> > >
>> > >
>> >
>>
>
>

Re: [DISCUSS] Kafka Security Specific Features

Posted by Pramod Deshmukh <dp...@gmail.com>.
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 topic "secureTopic".

pdeshmukh$ bin/kafka-topics.sh --list --zookeeper localhost:2181

secure.test

secureTopic

>> Run producer, tried both localhost:9092:true and localhost:9092

pdeshmukh$ bin/kafka-console-producer.sh --broker-list localhost:9092:true
--topic secureTopic

[2014-07-18 13:12:29,817] WARN Property topic is not valid
(kafka.utils.VerifiableProperties)

Hare Krishna

[2014-07-18 13:12:45,256] WARN Fetching topic metadata with correlation id
0 for topics [Set(secureTopic)] from broker
[id:0,host:localhost,port:9092,secure:true] failed
(kafka.client.ClientUtils$)

java.io.EOFException: Received -1 when reading from channel, socket has
likely been closed.

at kafka.utils.Utils$.read(Utils.scala:381)

at
kafka.network.BoundedByteBufferReceive.readFrom(BoundedByteBufferReceive.scala:67)

at kafka.network.Receive$class.readCompletely(Transmission.scala:56)

at
kafka.network.BoundedByteBufferReceive.readCompletely(BoundedByteBufferReceive.scala:29)

at kafka.network.BlockingChannel.receive(BlockingChannel.scala:102)

at kafka.producer.SyncProducer.liftedTree1$1(SyncProducer.scala:79)

at
kafka.producer.SyncProducer.kafka$producer$SyncProducer$$doSend(SyncProducer.scala:76)

at kafka.producer.SyncProducer.send(SyncProducer.scala:117)

at kafka.client.ClientUtils$.fetchTopicMetadata(ClientUtils.scala:58)

at
kafka.producer.BrokerPartitionInfo.updateInfo(BrokerPartitionInfo.scala:82)

at
kafka.producer.async.DefaultEventHandler$$anonfun$handle$1.apply$mcV$sp(DefaultEventHandler.scala:67)

at kafka.utils.Utils$.swallow(Utils.scala:172)

at kafka.utils.Logging$class.swallowError(Logging.scala:106)

at kafka.utils.Utils$.swallowError(Utils.scala:45)

at
kafka.producer.async.DefaultEventHandler.handle(DefaultEventHandler.scala:67)

at
kafka.producer.async.ProducerSendThread.tryToHandle(ProducerSendThread.scala:104)

at
kafka.producer.async.ProducerSendThread$$anonfun$processEvents$3.apply(ProducerSendThread.scala:87)

at
kafka.producer.async.ProducerSendThread$$anonfun$processEvents$3.apply(ProducerSendThread.scala:67)

at scala.collection.immutable.Stream.foreach(Stream.scala:526)

at
kafka.producer.async.ProducerSendThread.processEvents(ProducerSendThread.scala:66)

at kafka.producer.async.ProducerSendThread.run(ProducerSendThread.scala:44)

[2014-07-18 13:12:45,258] ERROR fetching topic metadata for topics
[Set(secureTopic)] from broker
[ArrayBuffer(id:0,host:localhost,port:9092,secure:true)] failed
(kafka.utils.Utils$)

kafka.common.KafkaException: fetching topic metadata for topics
[Set(secureTopic)] from broker
[ArrayBuffer(id:0,host:localhost,port:9092,secure:true)] failed

at kafka.client.ClientUtils$.fetchTopicMetadata(ClientUtils.scala:72)

at
kafka.producer.BrokerPartitionInfo.updateInfo(BrokerPartitionInfo.scala:82)

at
kafka.producer.async.DefaultEventHandler$$anonfun$handle$1.apply$mcV$sp(DefaultEventHandler.scala:67)

at kafka.utils.Utils$.swallow(Utils.scala:172)

at kafka.utils.Logging$class.swallowError(Logging.scala:106)

at kafka.utils.Utils$.swallowError(Utils.scala:45)

at
kafka.producer.async.DefaultEventHandler.handle(DefaultEventHandler.scala:67)

at
kafka.producer.async.ProducerSendThread.tryToHandle(ProducerSendThread.scala:104)

at
kafka.producer.async.ProducerSendThread$$anonfun$processEvents$3.apply(ProducerSendThread.scala:87)

at
kafka.producer.async.ProducerSendThread$$anonfun$processEvents$3.apply(ProducerSendThread.scala:67)

at scala.collection.immutable.Stream.foreach(Stream.scala:526)

at
kafka.producer.async.ProducerSendThread.processEvents(ProducerSendThread.scala:66)

at kafka.producer.async.ProducerSendThread.run(ProducerSendThread.scala:44)

Caused by: java.io.EOFException: Received -1 when reading from channel,
socket has likely been closed.

at kafka.utils.Utils$.read(Utils.scala:381)

at
kafka.network.BoundedByteBufferReceive.readFrom(BoundedByteBufferReceive.scala:67)

at kafka.network.Receive$class.readCompletely(Transmission.scala:56)

at
kafka.network.BoundedByteBufferReceive.readCompletely(BoundedByteBufferReceive.scala:29)

at kafka.network.BlockingChannel.receive(BlockingChannel.scala:102)

at kafka.producer.SyncProducer.liftedTree1$1(SyncProducer.scala:79)

at
kafka.producer.SyncProducer.kafka$producer$SyncProducer$$doSend(SyncProducer.scala:76)

at kafka.producer.SyncProducer.send(SyncProducer.scala:117)

at kafka.client.ClientUtils$.fetchTopicMetadata(ClientUtils.scala:58)

... 12 more
[2014-07-18 13:12:45,337] WARN Fetching topic metadata with correlation id
1 for topics [Set(secureTopic)] from broker
[id:0,host:localhost,port:9092,secure:true] failed
(kafka.client.ClientUtils$)

2014-07-18 13:12:46,282] ERROR Failed to send requests for topics
secureTopic with correlation ids in [0,8]
(kafka.producer.async.DefaultEventHandler)

[2014-07-18 13:12:46,283] ERROR Error in handling batch of 1 events
(kafka.producer.async.ProducerSendThread)

kafka.common.FailedToSendMessageException: Failed to send messages after 3
tries.

at
kafka.producer.async.DefaultEventHandler.handle(DefaultEventHandler.scala:90)

at
kafka.producer.async.ProducerSendThread.tryToHandle(ProducerSendThread.scala:104)

at
kafka.producer.async.ProducerSendThread$$anonfun$processEvents$3.apply(ProducerSendThread.scala:87)

at
kafka.producer.async.ProducerSendThread$$anonfun$processEvents$3.apply(ProducerSendThread.scala:67)

at scala.collection.immutable.Stream.foreach(Stream.scala:526)

at
kafka.producer.async.ProducerSendThread.processEvents(ProducerSendThread.scala:66)

at kafka.producer.async.ProducerSendThread.run(ProducerSendThread.scala:44)



On Fri, Jul 18, 2014 at 11:56 AM, Joe Stein <jo...@stealth.ly> wrote:

> 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://www.stealth.ly
>  Twitter: @allthingshadoop <http://www.twitter.com/allthingshadoop>
> ********************************************/
>
>
> On Fri, Jul 18, 2014 at 10:24 AM, Pramod Deshmukh <dp...@gmail.com>
> wrote:
>
> > 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.OutOfMemoryError: Java heap space
> >
> > *broker log*
> >
> > INFO begin ssl handshake for localhost/127.0.0.1:50199//127.0.0.1:9092
> >
> >
> >
> >
> >
> > On Thu, Jul 17, 2014 at 6:07 PM, Pramod Deshmukh <dp...@gmail.com>
> > wrote:
> >
> > > 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 <
> > relango@salesforce.com>
> > > wrote:
> > >
> > >> 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 <dp...@gmail.com>
> > >> 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.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 config/server.properties*
> > >> >
> > >> >
> > >> >
> > >> > *broker.log:*
> > >> >
> > >> > [2014-07-17 15:34:46,281] INFO zookeeper state changed
> (SyncConnected)
> > >> > (org.I0Itec.zkclient.ZkClient)
> > >> >
> > >> > [2014-07-17 15:34:46,523] INFO Loading log 'secure.test-0'
> > >> > (kafka.log.LogManager)
> > >> >
> > >> > [2014-07-17 15:34:46,558] INFO Recovering unflushed segment 0 in log
> > >> > secure.test-0. (kafka.log.Log)
> > >> >
> > >> > [2014-07-17 15:34:46,571] INFO Completed load of log secure.test-0
> > with
> > >> log
> > >> > end offset 0 (kafka.log.Log)
> > >> >
> > >> > [2014-07-17 15:34:46,582] INFO Starting log cleanup with a period of
> > >> 60000
> > >> > ms. (kafka.log.LogManager)
> > >> >
> > >> > [2014-07-17 15:34:46,587] INFO Starting log flusher with a default
> > >> period
> > >> > of 9223372036854775807 ms. (kafka.log.LogManager)
> > >> >
> > >> > [2014-07-17 15:34:46,614] INFO Initializing secure authentication
> > >> > (kafka.network.security.SecureAuth$)
> > >> >
> > >> > [2014-07-17 15:34:46,678] INFO Secure authentication initialization
> > has
> > >> > been successfully completed (kafka.network.security.SecureAuth$)
> > >> >
> > >> > [2014-07-17 15:34:46,691] INFO Awaiting socket connections on
> > >> 0.0.0.0:9092
> > >> > .
> > >> > (kafka.network.Acceptor)
> > >> >
> > >> > [2014-07-17 15:34:46,692] INFO [Socket Server on Broker 0], Started
> > >> > (kafka.network.SocketServer)
> > >> >
> > >> > [2014-07-17 15:34:46,794] INFO Will not load MX4J, mx4j-tools.jar is
> > >> not in
> > >> > the classpath (kafka.utils.Mx4jLoader$)
> > >> >
> > >> > [2014-07-17 15:34:46,837] INFO 0 successfully elected as leader
> > >> > (kafka.server.ZookeeperLeaderElector)
> > >> >
> > >> > [2014-07-17 15:34:47,057] INFO Registered broker 0 at path
> > >> /brokers/ids/0
> > >> > with address 10.1.100.130:9092. (kafka.utils.ZkUtils$)
> > >> >
> > >> > [2014-07-17 15:34:47,059] INFO New leader is 0
> > >> > (kafka.server.ZookeeperLeaderElector$LeaderChangeListener)
> > >> >
> > >> > *[2014-07-17 15:34:47,068] INFO [Kafka Server 0], started
> > >> > (kafka.server.KafkaServer)*
> > >> >
> > >> > *[2014-07-17 15:34:47,383] INFO begin ssl handshake for
> > >> > /10.1.100.130:9092//10.1.100.130:51685
> > >> > <http://10.1.100.130:9092//10.1.100.130:51685>
> > >> > (kafka.network.security.SSLSocketChannel)*
> > >> >
> > >> > *[2014-07-17 15:34:47,392] INFO begin ssl handshake for
> > >> > 10.1.100.130/10.1.100.130:51685//10.1.100.130:9092
> > >> > <http://10.1.100.130/10.1.100.130:51685//10.1.100.130:9092>
> > >> > (kafka.network.security.SSLSocketChannel)*
> > >> >
> > >> > *[2014-07-17 15:34:47,465] INFO finished ssl handshake for
> > >> > 10.1.100.130/10.1.100.130:51685//10.1.100.130:9092
> > >> > <http://10.1.100.130/10.1.100.130:51685//10.1.100.130:9092>
> > >> > (kafka.network.security.SSLSocketChannel)*
> > >> >
> > >> > *[2014-07-17 15:34:47,465] INFO finished ssl handshake for
> > >> > /10.1.100.130:9092//10.1.100.130:51685
> > >> > <http://10.1.100.130:9092//10.1.100.130:51685>
> > >> > (kafka.network.security.SSLSocketChannel)*
> > >> >
> > >> > *[2014-07-17 15:34:47,617] INFO [ReplicaFetcherManager on broker 0]
> > >> Removed
> > >> > fetcher for partitions  (kafka.server.ReplicaFetcherManager)*
> > >> >
> > >> > *[2014-07-17 15:34:47,627] INFO [ReplicaFetcherManager on broker 0]
> > >> Added
> > >> > fetcher for partitions List() (kafka.server.ReplicaFetcherManager)*
> > >> >
> > >> > *[2014-07-17 15:34:47,656] INFO [ReplicaFetcherManager on broker 0]
> > >> Removed
> > >> > fetcher for partitions [secure.test,0]
> > >> > (kafka.server.ReplicaFetcherManager)*
> > >> >
> > >> > [2014-07-17 15:37:15,970] INFO begin ssl handshake for
> > >> > 10.1.100.130/10.1.100.130:51689//10.1.100.130:9092
> > >> > (kafka.network.security.SSLSocketChannel)
> > >> >
> > >> > [2014-07-17 15:37:16,075] INFO begin ssl handshake for
> > >> > 10.1.100.130/10.1.100.130:51690//10.1.100.130:9092
> > >> > (kafka.network.security.SSLSocketChannel)
> > >> >
> > >> > [2014-07-17 15:37:16,434] INFO begin ssl handshake for
> > >> > 10.1.100.130/10.1.100.130:51691//10.1.100.130:9092
> > >> > (kafka.network.security.SSLSocketChannel)
> > >> >
> > >> > [2014-07-17 15:37:16,530] INFO begin ssl handshake for
> > >> > 10.1.100.130/10.1.100.130:51692//10.1.100.130:9092
> > >> > (kafka.network.security.SSLSocketChannel)
> > >> >
> > >> > [2014-07-17 15:37:16,743] INFO begin ssl handshake for
> > >> > 10.1.100.130/10.1.100.130:51693//10.1.100.130:9092
> > >> > (kafka.network.security.SSLSocketChannel)
> > >> >
> > >> > [2014-07-17 15:37:16,834] INFO begin ssl handshake for
> > >> > 10.1.100.130/10.1.100.130:51694//10.1.100.130:9092
> > >> > (kafka.network.security.SSLSocketChannel)
> > >> >
> > >> > [2014-07-17 15:37:17,043] INFO begin ssl handshake for
> > >> > 10.1.100.130/10.1.100.130:51695//10.1.100.130:9092
> > >> > (kafka.network.security.SSLSocketChannel)
> > >> >
> > >> > [2014-07-17 15:37:17,137] INFO begin ssl handshake for
> > >> > 10.1.100.130/10.1.100.130:51696//10.1.100.130:9092
> > >> > (kafka.network.security.SSLSocketChannel)
> > >> >
> > >> > [2014-07-17 15:37:17,342] INFO begin ssl handshake for
> > >> > 10.1.100.130/10.1.100.130:51697//10.1.100.130:9092
> > >> > (kafka.network.security.SSLSocketChannel)
> > >> >
> > >> >
> > >> > *Start producer*
> > >> >
> > >> > *bin/kafka-console-producer.sh --broker-list 10.1.100.130:9092:true
> > >> > --topic
> > >> > secure.test*
> > >> >
> > >> >
> > >> > *producer.log:*
> > >> >
> > >> > bin/kafka-console-producer.sh --broker-list 10.1.100.130:9092:true
> > >> --topic
> > >> > secure.test
> > >> >
> > >> > [2014-07-17 15:37:46,889] WARN Property topic is not valid
> > >> > (kafka.utils.VerifiableProperties)
> > >> >
> > >> > Hello Secure Kafka
> > >> >
> > >> > *[2014-07-17 15:38:14,186] ERROR OOME with size 352518400
> > >> > (kafka.network.BoundedByteBufferReceive)*
> > >> >
> > >> > *java.lang.OutOfMemoryError: Java heap space*
> > >> >
> > >> > at java.nio.HeapByteBuffer.<init>(HeapByteBuffer.java:57)
> > >> >
> > >> > at java.nio.ByteBuffer.allocate(ByteBuffer.java:331)
> > >> >
> > >> > at
> > >> >
> > >> >
> > >>
> >
> kafka.network.BoundedByteBufferReceive.byteBufferAllocate(BoundedByteBufferReceive.scala:80)
> > >> >
> > >> > at
> > >> >
> > >> >
> > >>
> >
> kafka.network.BoundedByteBufferReceive.readFrom(BoundedByteBufferReceive.scala:63)
> > >> >
> > >> > at kafka.network.Receive$class.readCompletely(Transmission.scala:56)
> > >> >
> > >> > at
> > >> >
> > >> >
> > >>
> >
> kafka.network.BoundedByteBufferReceive.readCompletely(BoundedByteBufferReceive.scala:29)
> > >> >
> > >> > at kafka.network.BlockingChannel.receive(BlockingChannel.scala:102)
> > >> >
> > >> > at kafka.producer.SyncProducer.liftedTree1$1(SyncProducer.scala:79)
> > >> >
> > >> > at
> > >> >
> > >> >
> > >>
> >
> kafka.producer.SyncProducer.kafka$producer$SyncProducer$$doSend(SyncProducer.scala:76)
> > >> >
> > >> > at kafka.producer.SyncProducer.send(SyncProducer.scala:117)
> > >> >
> > >> > at
> kafka.client.ClientUtils$.fetchTopicMetadata(ClientUtils.scala:58)
> > >> >
> > >> > at
> > >> >
> > >>
> >
> kafka.producer.BrokerPartitionInfo.updateInfo(BrokerPartitionInfo.scala:82)
> > >> >
> > >> > at
> > >> >
> > >> >
> > >>
> >
> kafka.producer.async.DefaultEventHandler$$anonfun$handle$1.apply$mcV$sp(DefaultEventHandler.scala:67)
> > >> >
> > >> > at kafka.utils.Utils$.swallow(Utils.scala:172)
> > >> >
> > >> > at kafka.utils.Logging$class.swallowError(Logging.scala:106)
> > >> >
> > >> > at kafka.utils.Utils$.swallowError(Utils.scala:45)
> > >> >
> > >> > at
> > >> >
> > >> >
> > >>
> >
> kafka.producer.async.DefaultEventHandler.handle(DefaultEventHandler.scala:67)
> > >> >
> > >> > at
> > >> >
> > >> >
> > >>
> >
> kafka.producer.async.ProducerSendThread.tryToHandle(ProducerSendThread.scala:104)
> > >> >
> > >> > at
> > >> >
> > >> >
> > >>
> >
> kafka.producer.async.ProducerSendThread$$anonfun$processEvents$3.apply(ProducerSendThread.scala:87)
> > >> >
> > >> > at
> > >> >
> > >> >
> > >>
> >
> kafka.producer.async.ProducerSendThread$$anonfun$processEvents$3.apply(ProducerSendThread.scala:67)
> > >> >
> > >> > at scala.collection.immutable.Stream.foreach(Stream.scala:526)
> > >> >
> > >> > at
> > >> >
> > >> >
> > >>
> >
> kafka.producer.async.ProducerSendThread.processEvents(ProducerSendThread.scala:66)
> > >> >
> > >> > at
> > >>
> kafka.producer.async.ProducerSendThread.run(ProducerSendThread.scala:44)
> > >> >
> > >> >
> > >> >
> > >> > On Wed, Jul 16, 2014 at 6:07 PM, Rajasekar Elango <
> > >> relango@salesforce.com>
> > >> > wrote:
> > >> >
> > >> > > Pramod,
> > >> > >
> > >> > >
> > >> > > I presented secure kafka configuration and usage at last meet up.
> So
> > >> hope
> > >> > > this
> > >> > > video recording <http://www.ustream.tv/recorded/48396701>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 <
> > dpramodv@gmail.com>
> > >> > > wrote:
> > >> > >
> > >> > > > 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 <
> > >> dpramodv@gmail.com>
> > >> > > > 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, Jul 16, 2014 at 3:57 PM, Joe Stein <
> > joe.stein@stealth.ly>
> > >> > > 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
> > >> (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 Stein
> > >> > > > >>  Founder, Principal Consultant
> > >> > > > >>  Big Data Open Source Security LLC
> > >> > > > >>  http://www.stealth.ly
> > >> > > > >>  Twitter: @allthingshadoop <
> > >> http://www.twitter.com/allthingshadoop>
> > >> > > > >> ********************************************/
> > >> > > > >>
> > >> > > > >>
> > >> > > > >> On Wed, Jul 16, 2014 at 3:49 PM, Pramod Deshmukh <
> > >> > dpramodv@gmail.com>
> > >> > > > >> wrote:
> > >> > > > >>
> > >> > > > >> > 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 installed sbt and scala and followed your
> instructions
> > >> for
> > >> > > > >> different
> > >> > > > >> > scala versions. I was able to bring zookeeper up but
> brokers
> > >> fail
> > >> > to
> > >> > > > >> start
> > >> > > > >> > with error
> > >> > > > >> >
> > >> > > > >> > Error: Could not find or load main class kafka.Kafka
> > >> > > > >> >
> > >> > > > >> > I think I am doing something wrong. Can you please help me?
> > >> > > > >> >
> > >> > > > >> > Our current production setup is with 2.8.0 and want to
> stick
> > to
> > >> > it.
> > >> > > > >> >
> > >> > > > >> > Thanks,
> > >> > > > >> >
> > >> > > > >> > Pramod
> > >> > > > >> >
> > >> > > > >> >
> > >> > > > >> > On Tue, Jun 3, 2014 at 3:57 PM, Joe Stein <
> > >> joe.stein@stealth.ly>
> > >> > > > wrote:
> > >> > > > >> >
> > >> > > > >> > > 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.  This comes from what folks are doing "in
> the
> > >> > wild"
> > >> > > to
> > >> > > > >> > > workaround and implement security with Kafka as it is
> today
> > >> and
> > >> > > also
> > >> > > > >> > what I
> > >> > > > >> > > have discovered from organizations about their blockers.
> It
> > >> also
> > >> > > > >> picks up
> > >> > > > >> > > from the wiki (which I should have time to update later
> in
> > >> the
> > >> > > week
> > >> > > > >> based
> > >> > > > >> > > on the below and feedback from the thread).
> > >> > > > >> > >
> > >> > > > >> > > 1) Transport Layer Security (i.e. SSL)
> > >> > > > >> > >
> > >> > > > >> > > This also includes client authentication in addition to
> > >> > in-transit
> > >> > > > >> > security
> > >> > > > >> > > layer.  This work has been picked up here
> > >> > > > >> > > https://issues.apache.org/jira/browse/KAFKA-1477 and do
> > >> > > appreciate
> > >> > > > >> any
> > >> > > > >> > > thoughts, comments, feedback, tomatoes, whatever for this
> > >> patch.
> > >> > >  It
> > >> > > > >> is a
> > >> > > > >> > > pickup from the fork of the work first done here
> > >> > > > >> > > https://github.com/relango/kafka/tree/kafka_security.
> > >> > > > >> > >
> > >> > > > >> > > 2) Data encryption at rest.
> > >> > > > >> > >
> > >> > > > >> > > This is very important and something that can be
> > facilitated
> > >> > > within
> > >> > > > >> the
> > >> > > > >> > > wire protocol. It requires an additional map data
> structure
> > >> for
> > >> > > the
> > >> > > > >> > > "encrypted [data encryption key]". With this map (either
> in
> > >> your
> > >> > > > >> object
> > >> > > > >> > or
> > >> > > > >> > > in the wire protocol) you can store the dynamically
> > generated
> > >> > > > >> symmetric
> > >> > > > >> > key
> > >> > > > >> > > (for each message) and then encrypt the data using that
> > >> > > dynamically
> > >> > > > >> > > generated key.  You then encrypt the encryption key using
> > >> each
> > >> > > > public
> > >> > > > >> key
> > >> > > > >> > > for whom is expected to be able to decrypt the encryption
> > >> key to
> > >> > > > then
> > >> > > > >> > > decrypt the message.  For each public key encrypted
> > symmetric
> > >> > key
> > >> > > > >> (which
> > >> > > > >> > is
> > >> > > > >> > > now the "encrypted [data encryption key]" along with
> which
> > >> > public
> > >> > > > key
> > >> > > > >> it
> > >> > > > >> > > was encrypted with for (so a map of [publicKey] =
> > >> > > > >> > > encryptedDataEncryptionKey) as a chain.   Other patterns
> > can
> > >> be
> > >> > > > >> > implemented
> > >> > > > >> > > but this is a pretty standard digital enveloping [0]
> > pattern
> > >> > with
> > >> > > > >> only 1
> > >> > > > >> > > field added. Other patterns should be able to use that
> > field
> > >> > to-do
> > >> > > > >> their
> > >> > > > >> > > implementation too.
> > >> > > > >> > >
> > >> > > > >> > > 3) Non-repudiation and long term non-repudiation.
> > >> > > > >> > >
> > >> > > > >> > > Non-repudiation is proving data hasn't changed.  This is
> > >> often
> > >> > (if
> > >> > > > not
> > >> > > > >> > > always) done with x509 public certificates (chained to a
> > >> > > certificate
> > >> > > > >> > > authority).
> > >> > > > >> > >
> > >> > > > >> > > Long term non-repudiation is what happens when the
> > >> certificates
> > >> > of
> > >> > > > the
> > >> > > > >> > > certificate authority are expired (or revoked) and
> > everything
> > >> > ever
> > >> > > > >> signed
> > >> > > > >> > > (ever) with that certificate's public key then becomes
> "no
> > >> > longer
> > >> > > > >> > provable
> > >> > > > >> > > as ever being authentic".  That is where RFC3126 [1] and
> > >> RFC3161
> > >> > > [2]
> > >> > > > >> come
> > >> > > > >> > > in (or worm drives [hardware], etc).
> > >> > > > >> > >
> > >> > > > >> > > For either (or both) of these it is an operation of the
> > >> > encryptor
> > >> > > to
> > >> > > > >> > > sign/hash the data (with or without third party trusted
> > >> timestap
> > >> > > of
> > >> > > > >> the
> > >> > > > >> > > signing event) and encrypt that with their own private
> key
> > >> and
> > >> > > > >> distribute
> > >> > > > >> > > the results (before and after encrypting if required)
> along
> > >> with
> > >> > > > their
> > >> > > > >> > > public key. This structure is a bit more complex but
> > >> feasible,
> > >> > it
> > >> > > > is a
> > >> > > > >> > map
> > >> > > > >> > > of digital signature formats and the chain of dig sig
> > >> > > attestations.
> > >> > > > >>  The
> > >> > > > >> > > map's key being the method (i.e. CRC32, PKCS7 [3],
> > XmlDigSig
> > >> > [4])
> > >> > > > and
> > >> > > > >> > then
> > >> > > > >> > > a list of map where that key is "purpose" of signature
> > (what
> > >> > your
> > >> > > > >> > attesting
> > >> > > > >> > > too).  As a sibling field to the list another field for
> > "the
> > >> > > > >> attester" as
> > >> > > > >> > > bytes (e.g. their PKCS12 [5] for the map of PKCS7
> > >> signatures).
> > >> > > > >> > >
> > >> > > > >> > > 4) Authorization
> > >> > > > >> > >
> > >> > > > >> > > We should have a policy of "404" for data, topics,
> > partitions
> > >> > > (etc)
> > >> > > > if
> > >> > > > >> > > authenticated connections do not have access.  In "secure
> > >> mode"
> > >> > > any
> > >> > > > >> non
> > >> > > > >> > > authenticated connections should get a "404" type message
> > on
> > >> > > > >> everything.
> > >> > > > >> > > Knowing "something is there" is a security risk in many
> > uses
> > >> > > cases.
> > >> > > > >>  So
> > >> > > > >> > if
> > >> > > > >> > > you don't have access you don't even see it.  Baking
> "that"
> > >> into
> > >> > > > Kafka
> > >> > > > >> > > along with some interface for entitlement (access
> > management)
> > >> > > > systems
> > >> > > > >> > > (pretty standard) is all that I think needs to be done to
> > the
> > >> > core
> > >> > > > >> > project.
> > >> > > > >> > >  I want to tackle item later in the year after summer
> after
> > >> the
> > >> > > > other
> > >> > > > >> > three
> > >> > > > >> > > are complete.
> > >> > > > >> > >
> > >> > > > >> > > I look forward to thoughts on this and anyone else
> > >> interested in
> > >> > > > >> working
> > >> > > > >> > > with us on these items.
> > >> > > > >> > >
> > >> > > > >> > > [0]
> > >> > > > >> > >
> > >> > > > >> > >
> > >> > > > >> >
> > >> > > > >>
> > >> > > >
> > >> > >
> > >> >
> > >>
> >
> http://www.emc.com/emc-plus/rsa-labs/standards-initiatives/what-is-a-digital-envelope.htm
> > >> > > > >> > > [1] http://tools.ietf.org/html/rfc3126
> > >> > > > >> > > [2] http://tools.ietf.org/html/rfc3161
> > >> > > > >> > > [3]
> > >> > > > >> > >
> > >> > > > >> > >
> > >> > > > >> >
> > >> > > > >>
> > >> > > >
> > >> > >
> > >> >
> > >>
> >
> http://www.emc.com/emc-plus/rsa-labs/standards-initiatives/pkcs-7-cryptographic-message-syntax-standar.htm
> > >> > > > >> > > [4] http://en.wikipedia.org/wiki/XML_Signature
> > >> > > > >> > > [5] http://en.wikipedia.org/wiki/PKCS_12
> > >> > > > >> > >
> > >> > > > >> > > /*******************************************
> > >> > > > >> > >  Joe Stein
> > >> > > > >> > >  Founder, Principal Consultant
> > >> > > > >> > >  Big Data Open Source Security LLC
> > >> > > > >> > >  http://www.stealth.ly
> > >> > > > >> > >  Twitter: @allthingshadoop <
> > >> > > http://www.twitter.com/allthingshadoop>
> > >> > > > >> > > ********************************************/
> > >> > > > >> > >
> > >> > > > >> >
> > >> > > > >>
> > >> > > > >
> > >> > > > >
> > >> > > >
> > >> > >
> > >> > >
> > >> > >
> > >> > > --
> > >> > > Thanks,
> > >> > > Raja.
> > >> > >
> > >> >
> > >>
> > >>
> > >>
> > >> --
> > >> Thanks,
> > >> Raja.
> > >>
> > >
> > >
> >
>

Re: [DISCUSS] Kafka Security Specific Features

Posted by Pramod Deshmukh <dp...@gmail.com>.
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 topic "secureTopic".

pdeshmukh$ bin/kafka-topics.sh --list --zookeeper localhost:2181

secure.test

secureTopic

>> Run producer, tried both localhost:9092:true and localhost:9092

pdeshmukh$ bin/kafka-console-producer.sh --broker-list localhost:9092:true
--topic secureTopic

[2014-07-18 13:12:29,817] WARN Property topic is not valid
(kafka.utils.VerifiableProperties)

Hare Krishna

[2014-07-18 13:12:45,256] WARN Fetching topic metadata with correlation id
0 for topics [Set(secureTopic)] from broker
[id:0,host:localhost,port:9092,secure:true] failed
(kafka.client.ClientUtils$)

java.io.EOFException: Received -1 when reading from channel, socket has
likely been closed.

at kafka.utils.Utils$.read(Utils.scala:381)

at
kafka.network.BoundedByteBufferReceive.readFrom(BoundedByteBufferReceive.scala:67)

at kafka.network.Receive$class.readCompletely(Transmission.scala:56)

at
kafka.network.BoundedByteBufferReceive.readCompletely(BoundedByteBufferReceive.scala:29)

at kafka.network.BlockingChannel.receive(BlockingChannel.scala:102)

at kafka.producer.SyncProducer.liftedTree1$1(SyncProducer.scala:79)

at
kafka.producer.SyncProducer.kafka$producer$SyncProducer$$doSend(SyncProducer.scala:76)

at kafka.producer.SyncProducer.send(SyncProducer.scala:117)

at kafka.client.ClientUtils$.fetchTopicMetadata(ClientUtils.scala:58)

at
kafka.producer.BrokerPartitionInfo.updateInfo(BrokerPartitionInfo.scala:82)

at
kafka.producer.async.DefaultEventHandler$$anonfun$handle$1.apply$mcV$sp(DefaultEventHandler.scala:67)

at kafka.utils.Utils$.swallow(Utils.scala:172)

at kafka.utils.Logging$class.swallowError(Logging.scala:106)

at kafka.utils.Utils$.swallowError(Utils.scala:45)

at
kafka.producer.async.DefaultEventHandler.handle(DefaultEventHandler.scala:67)

at
kafka.producer.async.ProducerSendThread.tryToHandle(ProducerSendThread.scala:104)

at
kafka.producer.async.ProducerSendThread$$anonfun$processEvents$3.apply(ProducerSendThread.scala:87)

at
kafka.producer.async.ProducerSendThread$$anonfun$processEvents$3.apply(ProducerSendThread.scala:67)

at scala.collection.immutable.Stream.foreach(Stream.scala:526)

at
kafka.producer.async.ProducerSendThread.processEvents(ProducerSendThread.scala:66)

at kafka.producer.async.ProducerSendThread.run(ProducerSendThread.scala:44)

[2014-07-18 13:12:45,258] ERROR fetching topic metadata for topics
[Set(secureTopic)] from broker
[ArrayBuffer(id:0,host:localhost,port:9092,secure:true)] failed
(kafka.utils.Utils$)

kafka.common.KafkaException: fetching topic metadata for topics
[Set(secureTopic)] from broker
[ArrayBuffer(id:0,host:localhost,port:9092,secure:true)] failed

at kafka.client.ClientUtils$.fetchTopicMetadata(ClientUtils.scala:72)

at
kafka.producer.BrokerPartitionInfo.updateInfo(BrokerPartitionInfo.scala:82)

at
kafka.producer.async.DefaultEventHandler$$anonfun$handle$1.apply$mcV$sp(DefaultEventHandler.scala:67)

at kafka.utils.Utils$.swallow(Utils.scala:172)

at kafka.utils.Logging$class.swallowError(Logging.scala:106)

at kafka.utils.Utils$.swallowError(Utils.scala:45)

at
kafka.producer.async.DefaultEventHandler.handle(DefaultEventHandler.scala:67)

at
kafka.producer.async.ProducerSendThread.tryToHandle(ProducerSendThread.scala:104)

at
kafka.producer.async.ProducerSendThread$$anonfun$processEvents$3.apply(ProducerSendThread.scala:87)

at
kafka.producer.async.ProducerSendThread$$anonfun$processEvents$3.apply(ProducerSendThread.scala:67)

at scala.collection.immutable.Stream.foreach(Stream.scala:526)

at
kafka.producer.async.ProducerSendThread.processEvents(ProducerSendThread.scala:66)

at kafka.producer.async.ProducerSendThread.run(ProducerSendThread.scala:44)

Caused by: java.io.EOFException: Received -1 when reading from channel,
socket has likely been closed.

at kafka.utils.Utils$.read(Utils.scala:381)

at
kafka.network.BoundedByteBufferReceive.readFrom(BoundedByteBufferReceive.scala:67)

at kafka.network.Receive$class.readCompletely(Transmission.scala:56)

at
kafka.network.BoundedByteBufferReceive.readCompletely(BoundedByteBufferReceive.scala:29)

at kafka.network.BlockingChannel.receive(BlockingChannel.scala:102)

at kafka.producer.SyncProducer.liftedTree1$1(SyncProducer.scala:79)

at
kafka.producer.SyncProducer.kafka$producer$SyncProducer$$doSend(SyncProducer.scala:76)

at kafka.producer.SyncProducer.send(SyncProducer.scala:117)

at kafka.client.ClientUtils$.fetchTopicMetadata(ClientUtils.scala:58)

... 12 more
[2014-07-18 13:12:45,337] WARN Fetching topic metadata with correlation id
1 for topics [Set(secureTopic)] from broker
[id:0,host:localhost,port:9092,secure:true] failed
(kafka.client.ClientUtils$)

2014-07-18 13:12:46,282] ERROR Failed to send requests for topics
secureTopic with correlation ids in [0,8]
(kafka.producer.async.DefaultEventHandler)

[2014-07-18 13:12:46,283] ERROR Error in handling batch of 1 events
(kafka.producer.async.ProducerSendThread)

kafka.common.FailedToSendMessageException: Failed to send messages after 3
tries.

at
kafka.producer.async.DefaultEventHandler.handle(DefaultEventHandler.scala:90)

at
kafka.producer.async.ProducerSendThread.tryToHandle(ProducerSendThread.scala:104)

at
kafka.producer.async.ProducerSendThread$$anonfun$processEvents$3.apply(ProducerSendThread.scala:87)

at
kafka.producer.async.ProducerSendThread$$anonfun$processEvents$3.apply(ProducerSendThread.scala:67)

at scala.collection.immutable.Stream.foreach(Stream.scala:526)

at
kafka.producer.async.ProducerSendThread.processEvents(ProducerSendThread.scala:66)

at kafka.producer.async.ProducerSendThread.run(ProducerSendThread.scala:44)



On Fri, Jul 18, 2014 at 11:56 AM, Joe Stein <jo...@stealth.ly> wrote:

> 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://www.stealth.ly
>  Twitter: @allthingshadoop <http://www.twitter.com/allthingshadoop>
> ********************************************/
>
>
> On Fri, Jul 18, 2014 at 10:24 AM, Pramod Deshmukh <dp...@gmail.com>
> wrote:
>
> > 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.OutOfMemoryError: Java heap space
> >
> > *broker log*
> >
> > INFO begin ssl handshake for localhost/127.0.0.1:50199//127.0.0.1:9092
> >
> >
> >
> >
> >
> > On Thu, Jul 17, 2014 at 6:07 PM, Pramod Deshmukh <dp...@gmail.com>
> > wrote:
> >
> > > 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 <
> > relango@salesforce.com>
> > > wrote:
> > >
> > >> 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 <dp...@gmail.com>
> > >> 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.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 config/server.properties*
> > >> >
> > >> >
> > >> >
> > >> > *broker.log:*
> > >> >
> > >> > [2014-07-17 15:34:46,281] INFO zookeeper state changed
> (SyncConnected)
> > >> > (org.I0Itec.zkclient.ZkClient)
> > >> >
> > >> > [2014-07-17 15:34:46,523] INFO Loading log 'secure.test-0'
> > >> > (kafka.log.LogManager)
> > >> >
> > >> > [2014-07-17 15:34:46,558] INFO Recovering unflushed segment 0 in log
> > >> > secure.test-0. (kafka.log.Log)
> > >> >
> > >> > [2014-07-17 15:34:46,571] INFO Completed load of log secure.test-0
> > with
> > >> log
> > >> > end offset 0 (kafka.log.Log)
> > >> >
> > >> > [2014-07-17 15:34:46,582] INFO Starting log cleanup with a period of
> > >> 60000
> > >> > ms. (kafka.log.LogManager)
> > >> >
> > >> > [2014-07-17 15:34:46,587] INFO Starting log flusher with a default
> > >> period
> > >> > of 9223372036854775807 ms. (kafka.log.LogManager)
> > >> >
> > >> > [2014-07-17 15:34:46,614] INFO Initializing secure authentication
> > >> > (kafka.network.security.SecureAuth$)
> > >> >
> > >> > [2014-07-17 15:34:46,678] INFO Secure authentication initialization
> > has
> > >> > been successfully completed (kafka.network.security.SecureAuth$)
> > >> >
> > >> > [2014-07-17 15:34:46,691] INFO Awaiting socket connections on
> > >> 0.0.0.0:9092
> > >> > .
> > >> > (kafka.network.Acceptor)
> > >> >
> > >> > [2014-07-17 15:34:46,692] INFO [Socket Server on Broker 0], Started
> > >> > (kafka.network.SocketServer)
> > >> >
> > >> > [2014-07-17 15:34:46,794] INFO Will not load MX4J, mx4j-tools.jar is
> > >> not in
> > >> > the classpath (kafka.utils.Mx4jLoader$)
> > >> >
> > >> > [2014-07-17 15:34:46,837] INFO 0 successfully elected as leader
> > >> > (kafka.server.ZookeeperLeaderElector)
> > >> >
> > >> > [2014-07-17 15:34:47,057] INFO Registered broker 0 at path
> > >> /brokers/ids/0
> > >> > with address 10.1.100.130:9092. (kafka.utils.ZkUtils$)
> > >> >
> > >> > [2014-07-17 15:34:47,059] INFO New leader is 0
> > >> > (kafka.server.ZookeeperLeaderElector$LeaderChangeListener)
> > >> >
> > >> > *[2014-07-17 15:34:47,068] INFO [Kafka Server 0], started
> > >> > (kafka.server.KafkaServer)*
> > >> >
> > >> > *[2014-07-17 15:34:47,383] INFO begin ssl handshake for
> > >> > /10.1.100.130:9092//10.1.100.130:51685
> > >> > <http://10.1.100.130:9092//10.1.100.130:51685>
> > >> > (kafka.network.security.SSLSocketChannel)*
> > >> >
> > >> > *[2014-07-17 15:34:47,392] INFO begin ssl handshake for
> > >> > 10.1.100.130/10.1.100.130:51685//10.1.100.130:9092
> > >> > <http://10.1.100.130/10.1.100.130:51685//10.1.100.130:9092>
> > >> > (kafka.network.security.SSLSocketChannel)*
> > >> >
> > >> > *[2014-07-17 15:34:47,465] INFO finished ssl handshake for
> > >> > 10.1.100.130/10.1.100.130:51685//10.1.100.130:9092
> > >> > <http://10.1.100.130/10.1.100.130:51685//10.1.100.130:9092>
> > >> > (kafka.network.security.SSLSocketChannel)*
> > >> >
> > >> > *[2014-07-17 15:34:47,465] INFO finished ssl handshake for
> > >> > /10.1.100.130:9092//10.1.100.130:51685
> > >> > <http://10.1.100.130:9092//10.1.100.130:51685>
> > >> > (kafka.network.security.SSLSocketChannel)*
> > >> >
> > >> > *[2014-07-17 15:34:47,617] INFO [ReplicaFetcherManager on broker 0]
> > >> Removed
> > >> > fetcher for partitions  (kafka.server.ReplicaFetcherManager)*
> > >> >
> > >> > *[2014-07-17 15:34:47,627] INFO [ReplicaFetcherManager on broker 0]
> > >> Added
> > >> > fetcher for partitions List() (kafka.server.ReplicaFetcherManager)*
> > >> >
> > >> > *[2014-07-17 15:34:47,656] INFO [ReplicaFetcherManager on broker 0]
> > >> Removed
> > >> > fetcher for partitions [secure.test,0]
> > >> > (kafka.server.ReplicaFetcherManager)*
> > >> >
> > >> > [2014-07-17 15:37:15,970] INFO begin ssl handshake for
> > >> > 10.1.100.130/10.1.100.130:51689//10.1.100.130:9092
> > >> > (kafka.network.security.SSLSocketChannel)
> > >> >
> > >> > [2014-07-17 15:37:16,075] INFO begin ssl handshake for
> > >> > 10.1.100.130/10.1.100.130:51690//10.1.100.130:9092
> > >> > (kafka.network.security.SSLSocketChannel)
> > >> >
> > >> > [2014-07-17 15:37:16,434] INFO begin ssl handshake for
> > >> > 10.1.100.130/10.1.100.130:51691//10.1.100.130:9092
> > >> > (kafka.network.security.SSLSocketChannel)
> > >> >
> > >> > [2014-07-17 15:37:16,530] INFO begin ssl handshake for
> > >> > 10.1.100.130/10.1.100.130:51692//10.1.100.130:9092
> > >> > (kafka.network.security.SSLSocketChannel)
> > >> >
> > >> > [2014-07-17 15:37:16,743] INFO begin ssl handshake for
> > >> > 10.1.100.130/10.1.100.130:51693//10.1.100.130:9092
> > >> > (kafka.network.security.SSLSocketChannel)
> > >> >
> > >> > [2014-07-17 15:37:16,834] INFO begin ssl handshake for
> > >> > 10.1.100.130/10.1.100.130:51694//10.1.100.130:9092
> > >> > (kafka.network.security.SSLSocketChannel)
> > >> >
> > >> > [2014-07-17 15:37:17,043] INFO begin ssl handshake for
> > >> > 10.1.100.130/10.1.100.130:51695//10.1.100.130:9092
> > >> > (kafka.network.security.SSLSocketChannel)
> > >> >
> > >> > [2014-07-17 15:37:17,137] INFO begin ssl handshake for
> > >> > 10.1.100.130/10.1.100.130:51696//10.1.100.130:9092
> > >> > (kafka.network.security.SSLSocketChannel)
> > >> >
> > >> > [2014-07-17 15:37:17,342] INFO begin ssl handshake for
> > >> > 10.1.100.130/10.1.100.130:51697//10.1.100.130:9092
> > >> > (kafka.network.security.SSLSocketChannel)
> > >> >
> > >> >
> > >> > *Start producer*
> > >> >
> > >> > *bin/kafka-console-producer.sh --broker-list 10.1.100.130:9092:true
> > >> > --topic
> > >> > secure.test*
> > >> >
> > >> >
> > >> > *producer.log:*
> > >> >
> > >> > bin/kafka-console-producer.sh --broker-list 10.1.100.130:9092:true
> > >> --topic
> > >> > secure.test
> > >> >
> > >> > [2014-07-17 15:37:46,889] WARN Property topic is not valid
> > >> > (kafka.utils.VerifiableProperties)
> > >> >
> > >> > Hello Secure Kafka
> > >> >
> > >> > *[2014-07-17 15:38:14,186] ERROR OOME with size 352518400
> > >> > (kafka.network.BoundedByteBufferReceive)*
> > >> >
> > >> > *java.lang.OutOfMemoryError: Java heap space*
> > >> >
> > >> > at java.nio.HeapByteBuffer.<init>(HeapByteBuffer.java:57)
> > >> >
> > >> > at java.nio.ByteBuffer.allocate(ByteBuffer.java:331)
> > >> >
> > >> > at
> > >> >
> > >> >
> > >>
> >
> kafka.network.BoundedByteBufferReceive.byteBufferAllocate(BoundedByteBufferReceive.scala:80)
> > >> >
> > >> > at
> > >> >
> > >> >
> > >>
> >
> kafka.network.BoundedByteBufferReceive.readFrom(BoundedByteBufferReceive.scala:63)
> > >> >
> > >> > at kafka.network.Receive$class.readCompletely(Transmission.scala:56)
> > >> >
> > >> > at
> > >> >
> > >> >
> > >>
> >
> kafka.network.BoundedByteBufferReceive.readCompletely(BoundedByteBufferReceive.scala:29)
> > >> >
> > >> > at kafka.network.BlockingChannel.receive(BlockingChannel.scala:102)
> > >> >
> > >> > at kafka.producer.SyncProducer.liftedTree1$1(SyncProducer.scala:79)
> > >> >
> > >> > at
> > >> >
> > >> >
> > >>
> >
> kafka.producer.SyncProducer.kafka$producer$SyncProducer$$doSend(SyncProducer.scala:76)
> > >> >
> > >> > at kafka.producer.SyncProducer.send(SyncProducer.scala:117)
> > >> >
> > >> > at
> kafka.client.ClientUtils$.fetchTopicMetadata(ClientUtils.scala:58)
> > >> >
> > >> > at
> > >> >
> > >>
> >
> kafka.producer.BrokerPartitionInfo.updateInfo(BrokerPartitionInfo.scala:82)
> > >> >
> > >> > at
> > >> >
> > >> >
> > >>
> >
> kafka.producer.async.DefaultEventHandler$$anonfun$handle$1.apply$mcV$sp(DefaultEventHandler.scala:67)
> > >> >
> > >> > at kafka.utils.Utils$.swallow(Utils.scala:172)
> > >> >
> > >> > at kafka.utils.Logging$class.swallowError(Logging.scala:106)
> > >> >
> > >> > at kafka.utils.Utils$.swallowError(Utils.scala:45)
> > >> >
> > >> > at
> > >> >
> > >> >
> > >>
> >
> kafka.producer.async.DefaultEventHandler.handle(DefaultEventHandler.scala:67)
> > >> >
> > >> > at
> > >> >
> > >> >
> > >>
> >
> kafka.producer.async.ProducerSendThread.tryToHandle(ProducerSendThread.scala:104)
> > >> >
> > >> > at
> > >> >
> > >> >
> > >>
> >
> kafka.producer.async.ProducerSendThread$$anonfun$processEvents$3.apply(ProducerSendThread.scala:87)
> > >> >
> > >> > at
> > >> >
> > >> >
> > >>
> >
> kafka.producer.async.ProducerSendThread$$anonfun$processEvents$3.apply(ProducerSendThread.scala:67)
> > >> >
> > >> > at scala.collection.immutable.Stream.foreach(Stream.scala:526)
> > >> >
> > >> > at
> > >> >
> > >> >
> > >>
> >
> kafka.producer.async.ProducerSendThread.processEvents(ProducerSendThread.scala:66)
> > >> >
> > >> > at
> > >>
> kafka.producer.async.ProducerSendThread.run(ProducerSendThread.scala:44)
> > >> >
> > >> >
> > >> >
> > >> > On Wed, Jul 16, 2014 at 6:07 PM, Rajasekar Elango <
> > >> relango@salesforce.com>
> > >> > wrote:
> > >> >
> > >> > > Pramod,
> > >> > >
> > >> > >
> > >> > > I presented secure kafka configuration and usage at last meet up.
> So
> > >> hope
> > >> > > this
> > >> > > video recording <http://www.ustream.tv/recorded/48396701>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 <
> > dpramodv@gmail.com>
> > >> > > wrote:
> > >> > >
> > >> > > > 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 <
> > >> dpramodv@gmail.com>
> > >> > > > 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, Jul 16, 2014 at 3:57 PM, Joe Stein <
> > joe.stein@stealth.ly>
> > >> > > 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
> > >> (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 Stein
> > >> > > > >>  Founder, Principal Consultant
> > >> > > > >>  Big Data Open Source Security LLC
> > >> > > > >>  http://www.stealth.ly
> > >> > > > >>  Twitter: @allthingshadoop <
> > >> http://www.twitter.com/allthingshadoop>
> > >> > > > >> ********************************************/
> > >> > > > >>
> > >> > > > >>
> > >> > > > >> On Wed, Jul 16, 2014 at 3:49 PM, Pramod Deshmukh <
> > >> > dpramodv@gmail.com>
> > >> > > > >> wrote:
> > >> > > > >>
> > >> > > > >> > 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 installed sbt and scala and followed your
> instructions
> > >> for
> > >> > > > >> different
> > >> > > > >> > scala versions. I was able to bring zookeeper up but
> brokers
> > >> fail
> > >> > to
> > >> > > > >> start
> > >> > > > >> > with error
> > >> > > > >> >
> > >> > > > >> > Error: Could not find or load main class kafka.Kafka
> > >> > > > >> >
> > >> > > > >> > I think I am doing something wrong. Can you please help me?
> > >> > > > >> >
> > >> > > > >> > Our current production setup is with 2.8.0 and want to
> stick
> > to
> > >> > it.
> > >> > > > >> >
> > >> > > > >> > Thanks,
> > >> > > > >> >
> > >> > > > >> > Pramod
> > >> > > > >> >
> > >> > > > >> >
> > >> > > > >> > On Tue, Jun 3, 2014 at 3:57 PM, Joe Stein <
> > >> joe.stein@stealth.ly>
> > >> > > > wrote:
> > >> > > > >> >
> > >> > > > >> > > 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.  This comes from what folks are doing "in
> the
> > >> > wild"
> > >> > > to
> > >> > > > >> > > workaround and implement security with Kafka as it is
> today
> > >> and
> > >> > > also
> > >> > > > >> > what I
> > >> > > > >> > > have discovered from organizations about their blockers.
> It
> > >> also
> > >> > > > >> picks up
> > >> > > > >> > > from the wiki (which I should have time to update later
> in
> > >> the
> > >> > > week
> > >> > > > >> based
> > >> > > > >> > > on the below and feedback from the thread).
> > >> > > > >> > >
> > >> > > > >> > > 1) Transport Layer Security (i.e. SSL)
> > >> > > > >> > >
> > >> > > > >> > > This also includes client authentication in addition to
> > >> > in-transit
> > >> > > > >> > security
> > >> > > > >> > > layer.  This work has been picked up here
> > >> > > > >> > > https://issues.apache.org/jira/browse/KAFKA-1477 and do
> > >> > > appreciate
> > >> > > > >> any
> > >> > > > >> > > thoughts, comments, feedback, tomatoes, whatever for this
> > >> patch.
> > >> > >  It
> > >> > > > >> is a
> > >> > > > >> > > pickup from the fork of the work first done here
> > >> > > > >> > > https://github.com/relango/kafka/tree/kafka_security.
> > >> > > > >> > >
> > >> > > > >> > > 2) Data encryption at rest.
> > >> > > > >> > >
> > >> > > > >> > > This is very important and something that can be
> > facilitated
> > >> > > within
> > >> > > > >> the
> > >> > > > >> > > wire protocol. It requires an additional map data
> structure
> > >> for
> > >> > > the
> > >> > > > >> > > "encrypted [data encryption key]". With this map (either
> in
> > >> your
> > >> > > > >> object
> > >> > > > >> > or
> > >> > > > >> > > in the wire protocol) you can store the dynamically
> > generated
> > >> > > > >> symmetric
> > >> > > > >> > key
> > >> > > > >> > > (for each message) and then encrypt the data using that
> > >> > > dynamically
> > >> > > > >> > > generated key.  You then encrypt the encryption key using
> > >> each
> > >> > > > public
> > >> > > > >> key
> > >> > > > >> > > for whom is expected to be able to decrypt the encryption
> > >> key to
> > >> > > > then
> > >> > > > >> > > decrypt the message.  For each public key encrypted
> > symmetric
> > >> > key
> > >> > > > >> (which
> > >> > > > >> > is
> > >> > > > >> > > now the "encrypted [data encryption key]" along with
> which
> > >> > public
> > >> > > > key
> > >> > > > >> it
> > >> > > > >> > > was encrypted with for (so a map of [publicKey] =
> > >> > > > >> > > encryptedDataEncryptionKey) as a chain.   Other patterns
> > can
> > >> be
> > >> > > > >> > implemented
> > >> > > > >> > > but this is a pretty standard digital enveloping [0]
> > pattern
> > >> > with
> > >> > > > >> only 1
> > >> > > > >> > > field added. Other patterns should be able to use that
> > field
> > >> > to-do
> > >> > > > >> their
> > >> > > > >> > > implementation too.
> > >> > > > >> > >
> > >> > > > >> > > 3) Non-repudiation and long term non-repudiation.
> > >> > > > >> > >
> > >> > > > >> > > Non-repudiation is proving data hasn't changed.  This is
> > >> often
> > >> > (if
> > >> > > > not
> > >> > > > >> > > always) done with x509 public certificates (chained to a
> > >> > > certificate
> > >> > > > >> > > authority).
> > >> > > > >> > >
> > >> > > > >> > > Long term non-repudiation is what happens when the
> > >> certificates
> > >> > of
> > >> > > > the
> > >> > > > >> > > certificate authority are expired (or revoked) and
> > everything
> > >> > ever
> > >> > > > >> signed
> > >> > > > >> > > (ever) with that certificate's public key then becomes
> "no
> > >> > longer
> > >> > > > >> > provable
> > >> > > > >> > > as ever being authentic".  That is where RFC3126 [1] and
> > >> RFC3161
> > >> > > [2]
> > >> > > > >> come
> > >> > > > >> > > in (or worm drives [hardware], etc).
> > >> > > > >> > >
> > >> > > > >> > > For either (or both) of these it is an operation of the
> > >> > encryptor
> > >> > > to
> > >> > > > >> > > sign/hash the data (with or without third party trusted
> > >> timestap
> > >> > > of
> > >> > > > >> the
> > >> > > > >> > > signing event) and encrypt that with their own private
> key
> > >> and
> > >> > > > >> distribute
> > >> > > > >> > > the results (before and after encrypting if required)
> along
> > >> with
> > >> > > > their
> > >> > > > >> > > public key. This structure is a bit more complex but
> > >> feasible,
> > >> > it
> > >> > > > is a
> > >> > > > >> > map
> > >> > > > >> > > of digital signature formats and the chain of dig sig
> > >> > > attestations.
> > >> > > > >>  The
> > >> > > > >> > > map's key being the method (i.e. CRC32, PKCS7 [3],
> > XmlDigSig
> > >> > [4])
> > >> > > > and
> > >> > > > >> > then
> > >> > > > >> > > a list of map where that key is "purpose" of signature
> > (what
> > >> > your
> > >> > > > >> > attesting
> > >> > > > >> > > too).  As a sibling field to the list another field for
> > "the
> > >> > > > >> attester" as
> > >> > > > >> > > bytes (e.g. their PKCS12 [5] for the map of PKCS7
> > >> signatures).
> > >> > > > >> > >
> > >> > > > >> > > 4) Authorization
> > >> > > > >> > >
> > >> > > > >> > > We should have a policy of "404" for data, topics,
> > partitions
> > >> > > (etc)
> > >> > > > if
> > >> > > > >> > > authenticated connections do not have access.  In "secure
> > >> mode"
> > >> > > any
> > >> > > > >> non
> > >> > > > >> > > authenticated connections should get a "404" type message
> > on
> > >> > > > >> everything.
> > >> > > > >> > > Knowing "something is there" is a security risk in many
> > uses
> > >> > > cases.
> > >> > > > >>  So
> > >> > > > >> > if
> > >> > > > >> > > you don't have access you don't even see it.  Baking
> "that"
> > >> into
> > >> > > > Kafka
> > >> > > > >> > > along with some interface for entitlement (access
> > management)
> > >> > > > systems
> > >> > > > >> > > (pretty standard) is all that I think needs to be done to
> > the
> > >> > core
> > >> > > > >> > project.
> > >> > > > >> > >  I want to tackle item later in the year after summer
> after
> > >> the
> > >> > > > other
> > >> > > > >> > three
> > >> > > > >> > > are complete.
> > >> > > > >> > >
> > >> > > > >> > > I look forward to thoughts on this and anyone else
> > >> interested in
> > >> > > > >> working
> > >> > > > >> > > with us on these items.
> > >> > > > >> > >
> > >> > > > >> > > [0]
> > >> > > > >> > >
> > >> > > > >> > >
> > >> > > > >> >
> > >> > > > >>
> > >> > > >
> > >> > >
> > >> >
> > >>
> >
> http://www.emc.com/emc-plus/rsa-labs/standards-initiatives/what-is-a-digital-envelope.htm
> > >> > > > >> > > [1] http://tools.ietf.org/html/rfc3126
> > >> > > > >> > > [2] http://tools.ietf.org/html/rfc3161
> > >> > > > >> > > [3]
> > >> > > > >> > >
> > >> > > > >> > >
> > >> > > > >> >
> > >> > > > >>
> > >> > > >
> > >> > >
> > >> >
> > >>
> >
> http://www.emc.com/emc-plus/rsa-labs/standards-initiatives/pkcs-7-cryptographic-message-syntax-standar.htm
> > >> > > > >> > > [4] http://en.wikipedia.org/wiki/XML_Signature
> > >> > > > >> > > [5] http://en.wikipedia.org/wiki/PKCS_12
> > >> > > > >> > >
> > >> > > > >> > > /*******************************************
> > >> > > > >> > >  Joe Stein
> > >> > > > >> > >  Founder, Principal Consultant
> > >> > > > >> > >  Big Data Open Source Security LLC
> > >> > > > >> > >  http://www.stealth.ly
> > >> > > > >> > >  Twitter: @allthingshadoop <
> > >> > > http://www.twitter.com/allthingshadoop>
> > >> > > > >> > > ********************************************/
> > >> > > > >> > >
> > >> > > > >> >
> > >> > > > >>
> > >> > > > >
> > >> > > > >
> > >> > > >
> > >> > >
> > >> > >
> > >> > >
> > >> > > --
> > >> > > Thanks,
> > >> > > Raja.
> > >> > >
> > >> >
> > >>
> > >>
> > >>
> > >> --
> > >> Thanks,
> > >> Raja.
> > >>
> > >
> > >
> >
>

Re: [DISCUSS] Kafka Security Specific Features

Posted by Joe Stein <jo...@stealth.ly>.
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://www.stealth.ly
 Twitter: @allthingshadoop <http://www.twitter.com/allthingshadoop>
********************************************/


On Fri, Jul 18, 2014 at 10:24 AM, Pramod Deshmukh <dp...@gmail.com>
wrote:

> 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.OutOfMemoryError: Java heap space
>
> *broker log*
>
> INFO begin ssl handshake for localhost/127.0.0.1:50199//127.0.0.1:9092
>
>
>
>
>
> On Thu, Jul 17, 2014 at 6:07 PM, Pramod Deshmukh <dp...@gmail.com>
> wrote:
>
> > 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 <
> relango@salesforce.com>
> > wrote:
> >
> >> 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 <dp...@gmail.com>
> >> 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.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 config/server.properties*
> >> >
> >> >
> >> >
> >> > *broker.log:*
> >> >
> >> > [2014-07-17 15:34:46,281] INFO zookeeper state changed (SyncConnected)
> >> > (org.I0Itec.zkclient.ZkClient)
> >> >
> >> > [2014-07-17 15:34:46,523] INFO Loading log 'secure.test-0'
> >> > (kafka.log.LogManager)
> >> >
> >> > [2014-07-17 15:34:46,558] INFO Recovering unflushed segment 0 in log
> >> > secure.test-0. (kafka.log.Log)
> >> >
> >> > [2014-07-17 15:34:46,571] INFO Completed load of log secure.test-0
> with
> >> log
> >> > end offset 0 (kafka.log.Log)
> >> >
> >> > [2014-07-17 15:34:46,582] INFO Starting log cleanup with a period of
> >> 60000
> >> > ms. (kafka.log.LogManager)
> >> >
> >> > [2014-07-17 15:34:46,587] INFO Starting log flusher with a default
> >> period
> >> > of 9223372036854775807 ms. (kafka.log.LogManager)
> >> >
> >> > [2014-07-17 15:34:46,614] INFO Initializing secure authentication
> >> > (kafka.network.security.SecureAuth$)
> >> >
> >> > [2014-07-17 15:34:46,678] INFO Secure authentication initialization
> has
> >> > been successfully completed (kafka.network.security.SecureAuth$)
> >> >
> >> > [2014-07-17 15:34:46,691] INFO Awaiting socket connections on
> >> 0.0.0.0:9092
> >> > .
> >> > (kafka.network.Acceptor)
> >> >
> >> > [2014-07-17 15:34:46,692] INFO [Socket Server on Broker 0], Started
> >> > (kafka.network.SocketServer)
> >> >
> >> > [2014-07-17 15:34:46,794] INFO Will not load MX4J, mx4j-tools.jar is
> >> not in
> >> > the classpath (kafka.utils.Mx4jLoader$)
> >> >
> >> > [2014-07-17 15:34:46,837] INFO 0 successfully elected as leader
> >> > (kafka.server.ZookeeperLeaderElector)
> >> >
> >> > [2014-07-17 15:34:47,057] INFO Registered broker 0 at path
> >> /brokers/ids/0
> >> > with address 10.1.100.130:9092. (kafka.utils.ZkUtils$)
> >> >
> >> > [2014-07-17 15:34:47,059] INFO New leader is 0
> >> > (kafka.server.ZookeeperLeaderElector$LeaderChangeListener)
> >> >
> >> > *[2014-07-17 15:34:47,068] INFO [Kafka Server 0], started
> >> > (kafka.server.KafkaServer)*
> >> >
> >> > *[2014-07-17 15:34:47,383] INFO begin ssl handshake for
> >> > /10.1.100.130:9092//10.1.100.130:51685
> >> > <http://10.1.100.130:9092//10.1.100.130:51685>
> >> > (kafka.network.security.SSLSocketChannel)*
> >> >
> >> > *[2014-07-17 15:34:47,392] INFO begin ssl handshake for
> >> > 10.1.100.130/10.1.100.130:51685//10.1.100.130:9092
> >> > <http://10.1.100.130/10.1.100.130:51685//10.1.100.130:9092>
> >> > (kafka.network.security.SSLSocketChannel)*
> >> >
> >> > *[2014-07-17 15:34:47,465] INFO finished ssl handshake for
> >> > 10.1.100.130/10.1.100.130:51685//10.1.100.130:9092
> >> > <http://10.1.100.130/10.1.100.130:51685//10.1.100.130:9092>
> >> > (kafka.network.security.SSLSocketChannel)*
> >> >
> >> > *[2014-07-17 15:34:47,465] INFO finished ssl handshake for
> >> > /10.1.100.130:9092//10.1.100.130:51685
> >> > <http://10.1.100.130:9092//10.1.100.130:51685>
> >> > (kafka.network.security.SSLSocketChannel)*
> >> >
> >> > *[2014-07-17 15:34:47,617] INFO [ReplicaFetcherManager on broker 0]
> >> Removed
> >> > fetcher for partitions  (kafka.server.ReplicaFetcherManager)*
> >> >
> >> > *[2014-07-17 15:34:47,627] INFO [ReplicaFetcherManager on broker 0]
> >> Added
> >> > fetcher for partitions List() (kafka.server.ReplicaFetcherManager)*
> >> >
> >> > *[2014-07-17 15:34:47,656] INFO [ReplicaFetcherManager on broker 0]
> >> Removed
> >> > fetcher for partitions [secure.test,0]
> >> > (kafka.server.ReplicaFetcherManager)*
> >> >
> >> > [2014-07-17 15:37:15,970] INFO begin ssl handshake for
> >> > 10.1.100.130/10.1.100.130:51689//10.1.100.130:9092
> >> > (kafka.network.security.SSLSocketChannel)
> >> >
> >> > [2014-07-17 15:37:16,075] INFO begin ssl handshake for
> >> > 10.1.100.130/10.1.100.130:51690//10.1.100.130:9092
> >> > (kafka.network.security.SSLSocketChannel)
> >> >
> >> > [2014-07-17 15:37:16,434] INFO begin ssl handshake for
> >> > 10.1.100.130/10.1.100.130:51691//10.1.100.130:9092
> >> > (kafka.network.security.SSLSocketChannel)
> >> >
> >> > [2014-07-17 15:37:16,530] INFO begin ssl handshake for
> >> > 10.1.100.130/10.1.100.130:51692//10.1.100.130:9092
> >> > (kafka.network.security.SSLSocketChannel)
> >> >
> >> > [2014-07-17 15:37:16,743] INFO begin ssl handshake for
> >> > 10.1.100.130/10.1.100.130:51693//10.1.100.130:9092
> >> > (kafka.network.security.SSLSocketChannel)
> >> >
> >> > [2014-07-17 15:37:16,834] INFO begin ssl handshake for
> >> > 10.1.100.130/10.1.100.130:51694//10.1.100.130:9092
> >> > (kafka.network.security.SSLSocketChannel)
> >> >
> >> > [2014-07-17 15:37:17,043] INFO begin ssl handshake for
> >> > 10.1.100.130/10.1.100.130:51695//10.1.100.130:9092
> >> > (kafka.network.security.SSLSocketChannel)
> >> >
> >> > [2014-07-17 15:37:17,137] INFO begin ssl handshake for
> >> > 10.1.100.130/10.1.100.130:51696//10.1.100.130:9092
> >> > (kafka.network.security.SSLSocketChannel)
> >> >
> >> > [2014-07-17 15:37:17,342] INFO begin ssl handshake for
> >> > 10.1.100.130/10.1.100.130:51697//10.1.100.130:9092
> >> > (kafka.network.security.SSLSocketChannel)
> >> >
> >> >
> >> > *Start producer*
> >> >
> >> > *bin/kafka-console-producer.sh --broker-list 10.1.100.130:9092:true
> >> > --topic
> >> > secure.test*
> >> >
> >> >
> >> > *producer.log:*
> >> >
> >> > bin/kafka-console-producer.sh --broker-list 10.1.100.130:9092:true
> >> --topic
> >> > secure.test
> >> >
> >> > [2014-07-17 15:37:46,889] WARN Property topic is not valid
> >> > (kafka.utils.VerifiableProperties)
> >> >
> >> > Hello Secure Kafka
> >> >
> >> > *[2014-07-17 15:38:14,186] ERROR OOME with size 352518400
> >> > (kafka.network.BoundedByteBufferReceive)*
> >> >
> >> > *java.lang.OutOfMemoryError: Java heap space*
> >> >
> >> > at java.nio.HeapByteBuffer.<init>(HeapByteBuffer.java:57)
> >> >
> >> > at java.nio.ByteBuffer.allocate(ByteBuffer.java:331)
> >> >
> >> > at
> >> >
> >> >
> >>
> kafka.network.BoundedByteBufferReceive.byteBufferAllocate(BoundedByteBufferReceive.scala:80)
> >> >
> >> > at
> >> >
> >> >
> >>
> kafka.network.BoundedByteBufferReceive.readFrom(BoundedByteBufferReceive.scala:63)
> >> >
> >> > at kafka.network.Receive$class.readCompletely(Transmission.scala:56)
> >> >
> >> > at
> >> >
> >> >
> >>
> kafka.network.BoundedByteBufferReceive.readCompletely(BoundedByteBufferReceive.scala:29)
> >> >
> >> > at kafka.network.BlockingChannel.receive(BlockingChannel.scala:102)
> >> >
> >> > at kafka.producer.SyncProducer.liftedTree1$1(SyncProducer.scala:79)
> >> >
> >> > at
> >> >
> >> >
> >>
> kafka.producer.SyncProducer.kafka$producer$SyncProducer$$doSend(SyncProducer.scala:76)
> >> >
> >> > at kafka.producer.SyncProducer.send(SyncProducer.scala:117)
> >> >
> >> > at kafka.client.ClientUtils$.fetchTopicMetadata(ClientUtils.scala:58)
> >> >
> >> > at
> >> >
> >>
> kafka.producer.BrokerPartitionInfo.updateInfo(BrokerPartitionInfo.scala:82)
> >> >
> >> > at
> >> >
> >> >
> >>
> kafka.producer.async.DefaultEventHandler$$anonfun$handle$1.apply$mcV$sp(DefaultEventHandler.scala:67)
> >> >
> >> > at kafka.utils.Utils$.swallow(Utils.scala:172)
> >> >
> >> > at kafka.utils.Logging$class.swallowError(Logging.scala:106)
> >> >
> >> > at kafka.utils.Utils$.swallowError(Utils.scala:45)
> >> >
> >> > at
> >> >
> >> >
> >>
> kafka.producer.async.DefaultEventHandler.handle(DefaultEventHandler.scala:67)
> >> >
> >> > at
> >> >
> >> >
> >>
> kafka.producer.async.ProducerSendThread.tryToHandle(ProducerSendThread.scala:104)
> >> >
> >> > at
> >> >
> >> >
> >>
> kafka.producer.async.ProducerSendThread$$anonfun$processEvents$3.apply(ProducerSendThread.scala:87)
> >> >
> >> > at
> >> >
> >> >
> >>
> kafka.producer.async.ProducerSendThread$$anonfun$processEvents$3.apply(ProducerSendThread.scala:67)
> >> >
> >> > at scala.collection.immutable.Stream.foreach(Stream.scala:526)
> >> >
> >> > at
> >> >
> >> >
> >>
> kafka.producer.async.ProducerSendThread.processEvents(ProducerSendThread.scala:66)
> >> >
> >> > at
> >> kafka.producer.async.ProducerSendThread.run(ProducerSendThread.scala:44)
> >> >
> >> >
> >> >
> >> > On Wed, Jul 16, 2014 at 6:07 PM, Rajasekar Elango <
> >> relango@salesforce.com>
> >> > wrote:
> >> >
> >> > > Pramod,
> >> > >
> >> > >
> >> > > I presented secure kafka configuration and usage at last meet up. So
> >> hope
> >> > > this
> >> > > video recording <http://www.ustream.tv/recorded/48396701>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 <
> dpramodv@gmail.com>
> >> > > wrote:
> >> > >
> >> > > > 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 <
> >> dpramodv@gmail.com>
> >> > > > 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, Jul 16, 2014 at 3:57 PM, Joe Stein <
> joe.stein@stealth.ly>
> >> > > 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
> >> (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 Stein
> >> > > > >>  Founder, Principal Consultant
> >> > > > >>  Big Data Open Source Security LLC
> >> > > > >>  http://www.stealth.ly
> >> > > > >>  Twitter: @allthingshadoop <
> >> http://www.twitter.com/allthingshadoop>
> >> > > > >> ********************************************/
> >> > > > >>
> >> > > > >>
> >> > > > >> On Wed, Jul 16, 2014 at 3:49 PM, Pramod Deshmukh <
> >> > dpramodv@gmail.com>
> >> > > > >> wrote:
> >> > > > >>
> >> > > > >> > 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 installed sbt and scala and followed your instructions
> >> for
> >> > > > >> different
> >> > > > >> > scala versions. I was able to bring zookeeper up but brokers
> >> fail
> >> > to
> >> > > > >> start
> >> > > > >> > with error
> >> > > > >> >
> >> > > > >> > Error: Could not find or load main class kafka.Kafka
> >> > > > >> >
> >> > > > >> > I think I am doing something wrong. Can you please help me?
> >> > > > >> >
> >> > > > >> > Our current production setup is with 2.8.0 and want to stick
> to
> >> > it.
> >> > > > >> >
> >> > > > >> > Thanks,
> >> > > > >> >
> >> > > > >> > Pramod
> >> > > > >> >
> >> > > > >> >
> >> > > > >> > On Tue, Jun 3, 2014 at 3:57 PM, Joe Stein <
> >> joe.stein@stealth.ly>
> >> > > > wrote:
> >> > > > >> >
> >> > > > >> > > 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.  This comes from what folks are doing "in the
> >> > wild"
> >> > > to
> >> > > > >> > > workaround and implement security with Kafka as it is today
> >> and
> >> > > also
> >> > > > >> > what I
> >> > > > >> > > have discovered from organizations about their blockers. It
> >> also
> >> > > > >> picks up
> >> > > > >> > > from the wiki (which I should have time to update later in
> >> the
> >> > > week
> >> > > > >> based
> >> > > > >> > > on the below and feedback from the thread).
> >> > > > >> > >
> >> > > > >> > > 1) Transport Layer Security (i.e. SSL)
> >> > > > >> > >
> >> > > > >> > > This also includes client authentication in addition to
> >> > in-transit
> >> > > > >> > security
> >> > > > >> > > layer.  This work has been picked up here
> >> > > > >> > > https://issues.apache.org/jira/browse/KAFKA-1477 and do
> >> > > appreciate
> >> > > > >> any
> >> > > > >> > > thoughts, comments, feedback, tomatoes, whatever for this
> >> patch.
> >> > >  It
> >> > > > >> is a
> >> > > > >> > > pickup from the fork of the work first done here
> >> > > > >> > > https://github.com/relango/kafka/tree/kafka_security.
> >> > > > >> > >
> >> > > > >> > > 2) Data encryption at rest.
> >> > > > >> > >
> >> > > > >> > > This is very important and something that can be
> facilitated
> >> > > within
> >> > > > >> the
> >> > > > >> > > wire protocol. It requires an additional map data structure
> >> for
> >> > > the
> >> > > > >> > > "encrypted [data encryption key]". With this map (either in
> >> your
> >> > > > >> object
> >> > > > >> > or
> >> > > > >> > > in the wire protocol) you can store the dynamically
> generated
> >> > > > >> symmetric
> >> > > > >> > key
> >> > > > >> > > (for each message) and then encrypt the data using that
> >> > > dynamically
> >> > > > >> > > generated key.  You then encrypt the encryption key using
> >> each
> >> > > > public
> >> > > > >> key
> >> > > > >> > > for whom is expected to be able to decrypt the encryption
> >> key to
> >> > > > then
> >> > > > >> > > decrypt the message.  For each public key encrypted
> symmetric
> >> > key
> >> > > > >> (which
> >> > > > >> > is
> >> > > > >> > > now the "encrypted [data encryption key]" along with which
> >> > public
> >> > > > key
> >> > > > >> it
> >> > > > >> > > was encrypted with for (so a map of [publicKey] =
> >> > > > >> > > encryptedDataEncryptionKey) as a chain.   Other patterns
> can
> >> be
> >> > > > >> > implemented
> >> > > > >> > > but this is a pretty standard digital enveloping [0]
> pattern
> >> > with
> >> > > > >> only 1
> >> > > > >> > > field added. Other patterns should be able to use that
> field
> >> > to-do
> >> > > > >> their
> >> > > > >> > > implementation too.
> >> > > > >> > >
> >> > > > >> > > 3) Non-repudiation and long term non-repudiation.
> >> > > > >> > >
> >> > > > >> > > Non-repudiation is proving data hasn't changed.  This is
> >> often
> >> > (if
> >> > > > not
> >> > > > >> > > always) done with x509 public certificates (chained to a
> >> > > certificate
> >> > > > >> > > authority).
> >> > > > >> > >
> >> > > > >> > > Long term non-repudiation is what happens when the
> >> certificates
> >> > of
> >> > > > the
> >> > > > >> > > certificate authority are expired (or revoked) and
> everything
> >> > ever
> >> > > > >> signed
> >> > > > >> > > (ever) with that certificate's public key then becomes "no
> >> > longer
> >> > > > >> > provable
> >> > > > >> > > as ever being authentic".  That is where RFC3126 [1] and
> >> RFC3161
> >> > > [2]
> >> > > > >> come
> >> > > > >> > > in (or worm drives [hardware], etc).
> >> > > > >> > >
> >> > > > >> > > For either (or both) of these it is an operation of the
> >> > encryptor
> >> > > to
> >> > > > >> > > sign/hash the data (with or without third party trusted
> >> timestap
> >> > > of
> >> > > > >> the
> >> > > > >> > > signing event) and encrypt that with their own private key
> >> and
> >> > > > >> distribute
> >> > > > >> > > the results (before and after encrypting if required) along
> >> with
> >> > > > their
> >> > > > >> > > public key. This structure is a bit more complex but
> >> feasible,
> >> > it
> >> > > > is a
> >> > > > >> > map
> >> > > > >> > > of digital signature formats and the chain of dig sig
> >> > > attestations.
> >> > > > >>  The
> >> > > > >> > > map's key being the method (i.e. CRC32, PKCS7 [3],
> XmlDigSig
> >> > [4])
> >> > > > and
> >> > > > >> > then
> >> > > > >> > > a list of map where that key is "purpose" of signature
> (what
> >> > your
> >> > > > >> > attesting
> >> > > > >> > > too).  As a sibling field to the list another field for
> "the
> >> > > > >> attester" as
> >> > > > >> > > bytes (e.g. their PKCS12 [5] for the map of PKCS7
> >> signatures).
> >> > > > >> > >
> >> > > > >> > > 4) Authorization
> >> > > > >> > >
> >> > > > >> > > We should have a policy of "404" for data, topics,
> partitions
> >> > > (etc)
> >> > > > if
> >> > > > >> > > authenticated connections do not have access.  In "secure
> >> mode"
> >> > > any
> >> > > > >> non
> >> > > > >> > > authenticated connections should get a "404" type message
> on
> >> > > > >> everything.
> >> > > > >> > > Knowing "something is there" is a security risk in many
> uses
> >> > > cases.
> >> > > > >>  So
> >> > > > >> > if
> >> > > > >> > > you don't have access you don't even see it.  Baking "that"
> >> into
> >> > > > Kafka
> >> > > > >> > > along with some interface for entitlement (access
> management)
> >> > > > systems
> >> > > > >> > > (pretty standard) is all that I think needs to be done to
> the
> >> > core
> >> > > > >> > project.
> >> > > > >> > >  I want to tackle item later in the year after summer after
> >> the
> >> > > > other
> >> > > > >> > three
> >> > > > >> > > are complete.
> >> > > > >> > >
> >> > > > >> > > I look forward to thoughts on this and anyone else
> >> interested in
> >> > > > >> working
> >> > > > >> > > with us on these items.
> >> > > > >> > >
> >> > > > >> > > [0]
> >> > > > >> > >
> >> > > > >> > >
> >> > > > >> >
> >> > > > >>
> >> > > >
> >> > >
> >> >
> >>
> http://www.emc.com/emc-plus/rsa-labs/standards-initiatives/what-is-a-digital-envelope.htm
> >> > > > >> > > [1] http://tools.ietf.org/html/rfc3126
> >> > > > >> > > [2] http://tools.ietf.org/html/rfc3161
> >> > > > >> > > [3]
> >> > > > >> > >
> >> > > > >> > >
> >> > > > >> >
> >> > > > >>
> >> > > >
> >> > >
> >> >
> >>
> http://www.emc.com/emc-plus/rsa-labs/standards-initiatives/pkcs-7-cryptographic-message-syntax-standar.htm
> >> > > > >> > > [4] http://en.wikipedia.org/wiki/XML_Signature
> >> > > > >> > > [5] http://en.wikipedia.org/wiki/PKCS_12
> >> > > > >> > >
> >> > > > >> > > /*******************************************
> >> > > > >> > >  Joe Stein
> >> > > > >> > >  Founder, Principal Consultant
> >> > > > >> > >  Big Data Open Source Security LLC
> >> > > > >> > >  http://www.stealth.ly
> >> > > > >> > >  Twitter: @allthingshadoop <
> >> > > http://www.twitter.com/allthingshadoop>
> >> > > > >> > > ********************************************/
> >> > > > >> > >
> >> > > > >> >
> >> > > > >>
> >> > > > >
> >> > > > >
> >> > > >
> >> > >
> >> > >
> >> > >
> >> > > --
> >> > > Thanks,
> >> > > Raja.
> >> > >
> >> >
> >>
> >>
> >>
> >> --
> >> Thanks,
> >> Raja.
> >>
> >
> >
>

Re: [DISCUSS] Kafka Security Specific Features

Posted by Joe Stein <jo...@stealth.ly>.
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://www.stealth.ly
 Twitter: @allthingshadoop <http://www.twitter.com/allthingshadoop>
********************************************/


On Fri, Jul 18, 2014 at 10:24 AM, Pramod Deshmukh <dp...@gmail.com>
wrote:

> 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.OutOfMemoryError: Java heap space
>
> *broker log*
>
> INFO begin ssl handshake for localhost/127.0.0.1:50199//127.0.0.1:9092
>
>
>
>
>
> On Thu, Jul 17, 2014 at 6:07 PM, Pramod Deshmukh <dp...@gmail.com>
> wrote:
>
> > 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 <
> relango@salesforce.com>
> > wrote:
> >
> >> 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 <dp...@gmail.com>
> >> 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.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 config/server.properties*
> >> >
> >> >
> >> >
> >> > *broker.log:*
> >> >
> >> > [2014-07-17 15:34:46,281] INFO zookeeper state changed (SyncConnected)
> >> > (org.I0Itec.zkclient.ZkClient)
> >> >
> >> > [2014-07-17 15:34:46,523] INFO Loading log 'secure.test-0'
> >> > (kafka.log.LogManager)
> >> >
> >> > [2014-07-17 15:34:46,558] INFO Recovering unflushed segment 0 in log
> >> > secure.test-0. (kafka.log.Log)
> >> >
> >> > [2014-07-17 15:34:46,571] INFO Completed load of log secure.test-0
> with
> >> log
> >> > end offset 0 (kafka.log.Log)
> >> >
> >> > [2014-07-17 15:34:46,582] INFO Starting log cleanup with a period of
> >> 60000
> >> > ms. (kafka.log.LogManager)
> >> >
> >> > [2014-07-17 15:34:46,587] INFO Starting log flusher with a default
> >> period
> >> > of 9223372036854775807 ms. (kafka.log.LogManager)
> >> >
> >> > [2014-07-17 15:34:46,614] INFO Initializing secure authentication
> >> > (kafka.network.security.SecureAuth$)
> >> >
> >> > [2014-07-17 15:34:46,678] INFO Secure authentication initialization
> has
> >> > been successfully completed (kafka.network.security.SecureAuth$)
> >> >
> >> > [2014-07-17 15:34:46,691] INFO Awaiting socket connections on
> >> 0.0.0.0:9092
> >> > .
> >> > (kafka.network.Acceptor)
> >> >
> >> > [2014-07-17 15:34:46,692] INFO [Socket Server on Broker 0], Started
> >> > (kafka.network.SocketServer)
> >> >
> >> > [2014-07-17 15:34:46,794] INFO Will not load MX4J, mx4j-tools.jar is
> >> not in
> >> > the classpath (kafka.utils.Mx4jLoader$)
> >> >
> >> > [2014-07-17 15:34:46,837] INFO 0 successfully elected as leader
> >> > (kafka.server.ZookeeperLeaderElector)
> >> >
> >> > [2014-07-17 15:34:47,057] INFO Registered broker 0 at path
> >> /brokers/ids/0
> >> > with address 10.1.100.130:9092. (kafka.utils.ZkUtils$)
> >> >
> >> > [2014-07-17 15:34:47,059] INFO New leader is 0
> >> > (kafka.server.ZookeeperLeaderElector$LeaderChangeListener)
> >> >
> >> > *[2014-07-17 15:34:47,068] INFO [Kafka Server 0], started
> >> > (kafka.server.KafkaServer)*
> >> >
> >> > *[2014-07-17 15:34:47,383] INFO begin ssl handshake for
> >> > /10.1.100.130:9092//10.1.100.130:51685
> >> > <http://10.1.100.130:9092//10.1.100.130:51685>
> >> > (kafka.network.security.SSLSocketChannel)*
> >> >
> >> > *[2014-07-17 15:34:47,392] INFO begin ssl handshake for
> >> > 10.1.100.130/10.1.100.130:51685//10.1.100.130:9092
> >> > <http://10.1.100.130/10.1.100.130:51685//10.1.100.130:9092>
> >> > (kafka.network.security.SSLSocketChannel)*
> >> >
> >> > *[2014-07-17 15:34:47,465] INFO finished ssl handshake for
> >> > 10.1.100.130/10.1.100.130:51685//10.1.100.130:9092
> >> > <http://10.1.100.130/10.1.100.130:51685//10.1.100.130:9092>
> >> > (kafka.network.security.SSLSocketChannel)*
> >> >
> >> > *[2014-07-17 15:34:47,465] INFO finished ssl handshake for
> >> > /10.1.100.130:9092//10.1.100.130:51685
> >> > <http://10.1.100.130:9092//10.1.100.130:51685>
> >> > (kafka.network.security.SSLSocketChannel)*
> >> >
> >> > *[2014-07-17 15:34:47,617] INFO [ReplicaFetcherManager on broker 0]
> >> Removed
> >> > fetcher for partitions  (kafka.server.ReplicaFetcherManager)*
> >> >
> >> > *[2014-07-17 15:34:47,627] INFO [ReplicaFetcherManager on broker 0]
> >> Added
> >> > fetcher for partitions List() (kafka.server.ReplicaFetcherManager)*
> >> >
> >> > *[2014-07-17 15:34:47,656] INFO [ReplicaFetcherManager on broker 0]
> >> Removed
> >> > fetcher for partitions [secure.test,0]
> >> > (kafka.server.ReplicaFetcherManager)*
> >> >
> >> > [2014-07-17 15:37:15,970] INFO begin ssl handshake for
> >> > 10.1.100.130/10.1.100.130:51689//10.1.100.130:9092
> >> > (kafka.network.security.SSLSocketChannel)
> >> >
> >> > [2014-07-17 15:37:16,075] INFO begin ssl handshake for
> >> > 10.1.100.130/10.1.100.130:51690//10.1.100.130:9092
> >> > (kafka.network.security.SSLSocketChannel)
> >> >
> >> > [2014-07-17 15:37:16,434] INFO begin ssl handshake for
> >> > 10.1.100.130/10.1.100.130:51691//10.1.100.130:9092
> >> > (kafka.network.security.SSLSocketChannel)
> >> >
> >> > [2014-07-17 15:37:16,530] INFO begin ssl handshake for
> >> > 10.1.100.130/10.1.100.130:51692//10.1.100.130:9092
> >> > (kafka.network.security.SSLSocketChannel)
> >> >
> >> > [2014-07-17 15:37:16,743] INFO begin ssl handshake for
> >> > 10.1.100.130/10.1.100.130:51693//10.1.100.130:9092
> >> > (kafka.network.security.SSLSocketChannel)
> >> >
> >> > [2014-07-17 15:37:16,834] INFO begin ssl handshake for
> >> > 10.1.100.130/10.1.100.130:51694//10.1.100.130:9092
> >> > (kafka.network.security.SSLSocketChannel)
> >> >
> >> > [2014-07-17 15:37:17,043] INFO begin ssl handshake for
> >> > 10.1.100.130/10.1.100.130:51695//10.1.100.130:9092
> >> > (kafka.network.security.SSLSocketChannel)
> >> >
> >> > [2014-07-17 15:37:17,137] INFO begin ssl handshake for
> >> > 10.1.100.130/10.1.100.130:51696//10.1.100.130:9092
> >> > (kafka.network.security.SSLSocketChannel)
> >> >
> >> > [2014-07-17 15:37:17,342] INFO begin ssl handshake for
> >> > 10.1.100.130/10.1.100.130:51697//10.1.100.130:9092
> >> > (kafka.network.security.SSLSocketChannel)
> >> >
> >> >
> >> > *Start producer*
> >> >
> >> > *bin/kafka-console-producer.sh --broker-list 10.1.100.130:9092:true
> >> > --topic
> >> > secure.test*
> >> >
> >> >
> >> > *producer.log:*
> >> >
> >> > bin/kafka-console-producer.sh --broker-list 10.1.100.130:9092:true
> >> --topic
> >> > secure.test
> >> >
> >> > [2014-07-17 15:37:46,889] WARN Property topic is not valid
> >> > (kafka.utils.VerifiableProperties)
> >> >
> >> > Hello Secure Kafka
> >> >
> >> > *[2014-07-17 15:38:14,186] ERROR OOME with size 352518400
> >> > (kafka.network.BoundedByteBufferReceive)*
> >> >
> >> > *java.lang.OutOfMemoryError: Java heap space*
> >> >
> >> > at java.nio.HeapByteBuffer.<init>(HeapByteBuffer.java:57)
> >> >
> >> > at java.nio.ByteBuffer.allocate(ByteBuffer.java:331)
> >> >
> >> > at
> >> >
> >> >
> >>
> kafka.network.BoundedByteBufferReceive.byteBufferAllocate(BoundedByteBufferReceive.scala:80)
> >> >
> >> > at
> >> >
> >> >
> >>
> kafka.network.BoundedByteBufferReceive.readFrom(BoundedByteBufferReceive.scala:63)
> >> >
> >> > at kafka.network.Receive$class.readCompletely(Transmission.scala:56)
> >> >
> >> > at
> >> >
> >> >
> >>
> kafka.network.BoundedByteBufferReceive.readCompletely(BoundedByteBufferReceive.scala:29)
> >> >
> >> > at kafka.network.BlockingChannel.receive(BlockingChannel.scala:102)
> >> >
> >> > at kafka.producer.SyncProducer.liftedTree1$1(SyncProducer.scala:79)
> >> >
> >> > at
> >> >
> >> >
> >>
> kafka.producer.SyncProducer.kafka$producer$SyncProducer$$doSend(SyncProducer.scala:76)
> >> >
> >> > at kafka.producer.SyncProducer.send(SyncProducer.scala:117)
> >> >
> >> > at kafka.client.ClientUtils$.fetchTopicMetadata(ClientUtils.scala:58)
> >> >
> >> > at
> >> >
> >>
> kafka.producer.BrokerPartitionInfo.updateInfo(BrokerPartitionInfo.scala:82)
> >> >
> >> > at
> >> >
> >> >
> >>
> kafka.producer.async.DefaultEventHandler$$anonfun$handle$1.apply$mcV$sp(DefaultEventHandler.scala:67)
> >> >
> >> > at kafka.utils.Utils$.swallow(Utils.scala:172)
> >> >
> >> > at kafka.utils.Logging$class.swallowError(Logging.scala:106)
> >> >
> >> > at kafka.utils.Utils$.swallowError(Utils.scala:45)
> >> >
> >> > at
> >> >
> >> >
> >>
> kafka.producer.async.DefaultEventHandler.handle(DefaultEventHandler.scala:67)
> >> >
> >> > at
> >> >
> >> >
> >>
> kafka.producer.async.ProducerSendThread.tryToHandle(ProducerSendThread.scala:104)
> >> >
> >> > at
> >> >
> >> >
> >>
> kafka.producer.async.ProducerSendThread$$anonfun$processEvents$3.apply(ProducerSendThread.scala:87)
> >> >
> >> > at
> >> >
> >> >
> >>
> kafka.producer.async.ProducerSendThread$$anonfun$processEvents$3.apply(ProducerSendThread.scala:67)
> >> >
> >> > at scala.collection.immutable.Stream.foreach(Stream.scala:526)
> >> >
> >> > at
> >> >
> >> >
> >>
> kafka.producer.async.ProducerSendThread.processEvents(ProducerSendThread.scala:66)
> >> >
> >> > at
> >> kafka.producer.async.ProducerSendThread.run(ProducerSendThread.scala:44)
> >> >
> >> >
> >> >
> >> > On Wed, Jul 16, 2014 at 6:07 PM, Rajasekar Elango <
> >> relango@salesforce.com>
> >> > wrote:
> >> >
> >> > > Pramod,
> >> > >
> >> > >
> >> > > I presented secure kafka configuration and usage at last meet up. So
> >> hope
> >> > > this
> >> > > video recording <http://www.ustream.tv/recorded/48396701>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 <
> dpramodv@gmail.com>
> >> > > wrote:
> >> > >
> >> > > > 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 <
> >> dpramodv@gmail.com>
> >> > > > 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, Jul 16, 2014 at 3:57 PM, Joe Stein <
> joe.stein@stealth.ly>
> >> > > 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
> >> (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 Stein
> >> > > > >>  Founder, Principal Consultant
> >> > > > >>  Big Data Open Source Security LLC
> >> > > > >>  http://www.stealth.ly
> >> > > > >>  Twitter: @allthingshadoop <
> >> http://www.twitter.com/allthingshadoop>
> >> > > > >> ********************************************/
> >> > > > >>
> >> > > > >>
> >> > > > >> On Wed, Jul 16, 2014 at 3:49 PM, Pramod Deshmukh <
> >> > dpramodv@gmail.com>
> >> > > > >> wrote:
> >> > > > >>
> >> > > > >> > 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 installed sbt and scala and followed your instructions
> >> for
> >> > > > >> different
> >> > > > >> > scala versions. I was able to bring zookeeper up but brokers
> >> fail
> >> > to
> >> > > > >> start
> >> > > > >> > with error
> >> > > > >> >
> >> > > > >> > Error: Could not find or load main class kafka.Kafka
> >> > > > >> >
> >> > > > >> > I think I am doing something wrong. Can you please help me?
> >> > > > >> >
> >> > > > >> > Our current production setup is with 2.8.0 and want to stick
> to
> >> > it.
> >> > > > >> >
> >> > > > >> > Thanks,
> >> > > > >> >
> >> > > > >> > Pramod
> >> > > > >> >
> >> > > > >> >
> >> > > > >> > On Tue, Jun 3, 2014 at 3:57 PM, Joe Stein <
> >> joe.stein@stealth.ly>
> >> > > > wrote:
> >> > > > >> >
> >> > > > >> > > 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.  This comes from what folks are doing "in the
> >> > wild"
> >> > > to
> >> > > > >> > > workaround and implement security with Kafka as it is today
> >> and
> >> > > also
> >> > > > >> > what I
> >> > > > >> > > have discovered from organizations about their blockers. It
> >> also
> >> > > > >> picks up
> >> > > > >> > > from the wiki (which I should have time to update later in
> >> the
> >> > > week
> >> > > > >> based
> >> > > > >> > > on the below and feedback from the thread).
> >> > > > >> > >
> >> > > > >> > > 1) Transport Layer Security (i.e. SSL)
> >> > > > >> > >
> >> > > > >> > > This also includes client authentication in addition to
> >> > in-transit
> >> > > > >> > security
> >> > > > >> > > layer.  This work has been picked up here
> >> > > > >> > > https://issues.apache.org/jira/browse/KAFKA-1477 and do
> >> > > appreciate
> >> > > > >> any
> >> > > > >> > > thoughts, comments, feedback, tomatoes, whatever for this
> >> patch.
> >> > >  It
> >> > > > >> is a
> >> > > > >> > > pickup from the fork of the work first done here
> >> > > > >> > > https://github.com/relango/kafka/tree/kafka_security.
> >> > > > >> > >
> >> > > > >> > > 2) Data encryption at rest.
> >> > > > >> > >
> >> > > > >> > > This is very important and something that can be
> facilitated
> >> > > within
> >> > > > >> the
> >> > > > >> > > wire protocol. It requires an additional map data structure
> >> for
> >> > > the
> >> > > > >> > > "encrypted [data encryption key]". With this map (either in
> >> your
> >> > > > >> object
> >> > > > >> > or
> >> > > > >> > > in the wire protocol) you can store the dynamically
> generated
> >> > > > >> symmetric
> >> > > > >> > key
> >> > > > >> > > (for each message) and then encrypt the data using that
> >> > > dynamically
> >> > > > >> > > generated key.  You then encrypt the encryption key using
> >> each
> >> > > > public
> >> > > > >> key
> >> > > > >> > > for whom is expected to be able to decrypt the encryption
> >> key to
> >> > > > then
> >> > > > >> > > decrypt the message.  For each public key encrypted
> symmetric
> >> > key
> >> > > > >> (which
> >> > > > >> > is
> >> > > > >> > > now the "encrypted [data encryption key]" along with which
> >> > public
> >> > > > key
> >> > > > >> it
> >> > > > >> > > was encrypted with for (so a map of [publicKey] =
> >> > > > >> > > encryptedDataEncryptionKey) as a chain.   Other patterns
> can
> >> be
> >> > > > >> > implemented
> >> > > > >> > > but this is a pretty standard digital enveloping [0]
> pattern
> >> > with
> >> > > > >> only 1
> >> > > > >> > > field added. Other patterns should be able to use that
> field
> >> > to-do
> >> > > > >> their
> >> > > > >> > > implementation too.
> >> > > > >> > >
> >> > > > >> > > 3) Non-repudiation and long term non-repudiation.
> >> > > > >> > >
> >> > > > >> > > Non-repudiation is proving data hasn't changed.  This is
> >> often
> >> > (if
> >> > > > not
> >> > > > >> > > always) done with x509 public certificates (chained to a
> >> > > certificate
> >> > > > >> > > authority).
> >> > > > >> > >
> >> > > > >> > > Long term non-repudiation is what happens when the
> >> certificates
> >> > of
> >> > > > the
> >> > > > >> > > certificate authority are expired (or revoked) and
> everything
> >> > ever
> >> > > > >> signed
> >> > > > >> > > (ever) with that certificate's public key then becomes "no
> >> > longer
> >> > > > >> > provable
> >> > > > >> > > as ever being authentic".  That is where RFC3126 [1] and
> >> RFC3161
> >> > > [2]
> >> > > > >> come
> >> > > > >> > > in (or worm drives [hardware], etc).
> >> > > > >> > >
> >> > > > >> > > For either (or both) of these it is an operation of the
> >> > encryptor
> >> > > to
> >> > > > >> > > sign/hash the data (with or without third party trusted
> >> timestap
> >> > > of
> >> > > > >> the
> >> > > > >> > > signing event) and encrypt that with their own private key
> >> and
> >> > > > >> distribute
> >> > > > >> > > the results (before and after encrypting if required) along
> >> with
> >> > > > their
> >> > > > >> > > public key. This structure is a bit more complex but
> >> feasible,
> >> > it
> >> > > > is a
> >> > > > >> > map
> >> > > > >> > > of digital signature formats and the chain of dig sig
> >> > > attestations.
> >> > > > >>  The
> >> > > > >> > > map's key being the method (i.e. CRC32, PKCS7 [3],
> XmlDigSig
> >> > [4])
> >> > > > and
> >> > > > >> > then
> >> > > > >> > > a list of map where that key is "purpose" of signature
> (what
> >> > your
> >> > > > >> > attesting
> >> > > > >> > > too).  As a sibling field to the list another field for
> "the
> >> > > > >> attester" as
> >> > > > >> > > bytes (e.g. their PKCS12 [5] for the map of PKCS7
> >> signatures).
> >> > > > >> > >
> >> > > > >> > > 4) Authorization
> >> > > > >> > >
> >> > > > >> > > We should have a policy of "404" for data, topics,
> partitions
> >> > > (etc)
> >> > > > if
> >> > > > >> > > authenticated connections do not have access.  In "secure
> >> mode"
> >> > > any
> >> > > > >> non
> >> > > > >> > > authenticated connections should get a "404" type message
> on
> >> > > > >> everything.
> >> > > > >> > > Knowing "something is there" is a security risk in many
> uses
> >> > > cases.
> >> > > > >>  So
> >> > > > >> > if
> >> > > > >> > > you don't have access you don't even see it.  Baking "that"
> >> into
> >> > > > Kafka
> >> > > > >> > > along with some interface for entitlement (access
> management)
> >> > > > systems
> >> > > > >> > > (pretty standard) is all that I think needs to be done to
> the
> >> > core
> >> > > > >> > project.
> >> > > > >> > >  I want to tackle item later in the year after summer after
> >> the
> >> > > > other
> >> > > > >> > three
> >> > > > >> > > are complete.
> >> > > > >> > >
> >> > > > >> > > I look forward to thoughts on this and anyone else
> >> interested in
> >> > > > >> working
> >> > > > >> > > with us on these items.
> >> > > > >> > >
> >> > > > >> > > [0]
> >> > > > >> > >
> >> > > > >> > >
> >> > > > >> >
> >> > > > >>
> >> > > >
> >> > >
> >> >
> >>
> http://www.emc.com/emc-plus/rsa-labs/standards-initiatives/what-is-a-digital-envelope.htm
> >> > > > >> > > [1] http://tools.ietf.org/html/rfc3126
> >> > > > >> > > [2] http://tools.ietf.org/html/rfc3161
> >> > > > >> > > [3]
> >> > > > >> > >
> >> > > > >> > >
> >> > > > >> >
> >> > > > >>
> >> > > >
> >> > >
> >> >
> >>
> http://www.emc.com/emc-plus/rsa-labs/standards-initiatives/pkcs-7-cryptographic-message-syntax-standar.htm
> >> > > > >> > > [4] http://en.wikipedia.org/wiki/XML_Signature
> >> > > > >> > > [5] http://en.wikipedia.org/wiki/PKCS_12
> >> > > > >> > >
> >> > > > >> > > /*******************************************
> >> > > > >> > >  Joe Stein
> >> > > > >> > >  Founder, Principal Consultant
> >> > > > >> > >  Big Data Open Source Security LLC
> >> > > > >> > >  http://www.stealth.ly
> >> > > > >> > >  Twitter: @allthingshadoop <
> >> > > http://www.twitter.com/allthingshadoop>
> >> > > > >> > > ********************************************/
> >> > > > >> > >
> >> > > > >> >
> >> > > > >>
> >> > > > >
> >> > > > >
> >> > > >
> >> > >
> >> > >
> >> > >
> >> > > --
> >> > > Thanks,
> >> > > Raja.
> >> > >
> >> >
> >>
> >>
> >>
> >> --
> >> Thanks,
> >> Raja.
> >>
> >
> >
>

Re: [DISCUSS] Kafka Security Specific Features

Posted by Pramod Deshmukh <dp...@gmail.com>.
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.OutOfMemoryError: Java heap space

*broker log*

INFO begin ssl handshake for localhost/127.0.0.1:50199//127.0.0.1:9092





On Thu, Jul 17, 2014 at 6:07 PM, Pramod Deshmukh <dp...@gmail.com> wrote:

> 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 <re...@salesforce.com>
> wrote:
>
>> 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 <dp...@gmail.com>
>> 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.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 config/server.properties*
>> >
>> >
>> >
>> > *broker.log:*
>> >
>> > [2014-07-17 15:34:46,281] INFO zookeeper state changed (SyncConnected)
>> > (org.I0Itec.zkclient.ZkClient)
>> >
>> > [2014-07-17 15:34:46,523] INFO Loading log 'secure.test-0'
>> > (kafka.log.LogManager)
>> >
>> > [2014-07-17 15:34:46,558] INFO Recovering unflushed segment 0 in log
>> > secure.test-0. (kafka.log.Log)
>> >
>> > [2014-07-17 15:34:46,571] INFO Completed load of log secure.test-0 with
>> log
>> > end offset 0 (kafka.log.Log)
>> >
>> > [2014-07-17 15:34:46,582] INFO Starting log cleanup with a period of
>> 60000
>> > ms. (kafka.log.LogManager)
>> >
>> > [2014-07-17 15:34:46,587] INFO Starting log flusher with a default
>> period
>> > of 9223372036854775807 ms. (kafka.log.LogManager)
>> >
>> > [2014-07-17 15:34:46,614] INFO Initializing secure authentication
>> > (kafka.network.security.SecureAuth$)
>> >
>> > [2014-07-17 15:34:46,678] INFO Secure authentication initialization has
>> > been successfully completed (kafka.network.security.SecureAuth$)
>> >
>> > [2014-07-17 15:34:46,691] INFO Awaiting socket connections on
>> 0.0.0.0:9092
>> > .
>> > (kafka.network.Acceptor)
>> >
>> > [2014-07-17 15:34:46,692] INFO [Socket Server on Broker 0], Started
>> > (kafka.network.SocketServer)
>> >
>> > [2014-07-17 15:34:46,794] INFO Will not load MX4J, mx4j-tools.jar is
>> not in
>> > the classpath (kafka.utils.Mx4jLoader$)
>> >
>> > [2014-07-17 15:34:46,837] INFO 0 successfully elected as leader
>> > (kafka.server.ZookeeperLeaderElector)
>> >
>> > [2014-07-17 15:34:47,057] INFO Registered broker 0 at path
>> /brokers/ids/0
>> > with address 10.1.100.130:9092. (kafka.utils.ZkUtils$)
>> >
>> > [2014-07-17 15:34:47,059] INFO New leader is 0
>> > (kafka.server.ZookeeperLeaderElector$LeaderChangeListener)
>> >
>> > *[2014-07-17 15:34:47,068] INFO [Kafka Server 0], started
>> > (kafka.server.KafkaServer)*
>> >
>> > *[2014-07-17 15:34:47,383] INFO begin ssl handshake for
>> > /10.1.100.130:9092//10.1.100.130:51685
>> > <http://10.1.100.130:9092//10.1.100.130:51685>
>> > (kafka.network.security.SSLSocketChannel)*
>> >
>> > *[2014-07-17 15:34:47,392] INFO begin ssl handshake for
>> > 10.1.100.130/10.1.100.130:51685//10.1.100.130:9092
>> > <http://10.1.100.130/10.1.100.130:51685//10.1.100.130:9092>
>> > (kafka.network.security.SSLSocketChannel)*
>> >
>> > *[2014-07-17 15:34:47,465] INFO finished ssl handshake for
>> > 10.1.100.130/10.1.100.130:51685//10.1.100.130:9092
>> > <http://10.1.100.130/10.1.100.130:51685//10.1.100.130:9092>
>> > (kafka.network.security.SSLSocketChannel)*
>> >
>> > *[2014-07-17 15:34:47,465] INFO finished ssl handshake for
>> > /10.1.100.130:9092//10.1.100.130:51685
>> > <http://10.1.100.130:9092//10.1.100.130:51685>
>> > (kafka.network.security.SSLSocketChannel)*
>> >
>> > *[2014-07-17 15:34:47,617] INFO [ReplicaFetcherManager on broker 0]
>> Removed
>> > fetcher for partitions  (kafka.server.ReplicaFetcherManager)*
>> >
>> > *[2014-07-17 15:34:47,627] INFO [ReplicaFetcherManager on broker 0]
>> Added
>> > fetcher for partitions List() (kafka.server.ReplicaFetcherManager)*
>> >
>> > *[2014-07-17 15:34:47,656] INFO [ReplicaFetcherManager on broker 0]
>> Removed
>> > fetcher for partitions [secure.test,0]
>> > (kafka.server.ReplicaFetcherManager)*
>> >
>> > [2014-07-17 15:37:15,970] INFO begin ssl handshake for
>> > 10.1.100.130/10.1.100.130:51689//10.1.100.130:9092
>> > (kafka.network.security.SSLSocketChannel)
>> >
>> > [2014-07-17 15:37:16,075] INFO begin ssl handshake for
>> > 10.1.100.130/10.1.100.130:51690//10.1.100.130:9092
>> > (kafka.network.security.SSLSocketChannel)
>> >
>> > [2014-07-17 15:37:16,434] INFO begin ssl handshake for
>> > 10.1.100.130/10.1.100.130:51691//10.1.100.130:9092
>> > (kafka.network.security.SSLSocketChannel)
>> >
>> > [2014-07-17 15:37:16,530] INFO begin ssl handshake for
>> > 10.1.100.130/10.1.100.130:51692//10.1.100.130:9092
>> > (kafka.network.security.SSLSocketChannel)
>> >
>> > [2014-07-17 15:37:16,743] INFO begin ssl handshake for
>> > 10.1.100.130/10.1.100.130:51693//10.1.100.130:9092
>> > (kafka.network.security.SSLSocketChannel)
>> >
>> > [2014-07-17 15:37:16,834] INFO begin ssl handshake for
>> > 10.1.100.130/10.1.100.130:51694//10.1.100.130:9092
>> > (kafka.network.security.SSLSocketChannel)
>> >
>> > [2014-07-17 15:37:17,043] INFO begin ssl handshake for
>> > 10.1.100.130/10.1.100.130:51695//10.1.100.130:9092
>> > (kafka.network.security.SSLSocketChannel)
>> >
>> > [2014-07-17 15:37:17,137] INFO begin ssl handshake for
>> > 10.1.100.130/10.1.100.130:51696//10.1.100.130:9092
>> > (kafka.network.security.SSLSocketChannel)
>> >
>> > [2014-07-17 15:37:17,342] INFO begin ssl handshake for
>> > 10.1.100.130/10.1.100.130:51697//10.1.100.130:9092
>> > (kafka.network.security.SSLSocketChannel)
>> >
>> >
>> > *Start producer*
>> >
>> > *bin/kafka-console-producer.sh --broker-list 10.1.100.130:9092:true
>> > --topic
>> > secure.test*
>> >
>> >
>> > *producer.log:*
>> >
>> > bin/kafka-console-producer.sh --broker-list 10.1.100.130:9092:true
>> --topic
>> > secure.test
>> >
>> > [2014-07-17 15:37:46,889] WARN Property topic is not valid
>> > (kafka.utils.VerifiableProperties)
>> >
>> > Hello Secure Kafka
>> >
>> > *[2014-07-17 15:38:14,186] ERROR OOME with size 352518400
>> > (kafka.network.BoundedByteBufferReceive)*
>> >
>> > *java.lang.OutOfMemoryError: Java heap space*
>> >
>> > at java.nio.HeapByteBuffer.<init>(HeapByteBuffer.java:57)
>> >
>> > at java.nio.ByteBuffer.allocate(ByteBuffer.java:331)
>> >
>> > at
>> >
>> >
>> kafka.network.BoundedByteBufferReceive.byteBufferAllocate(BoundedByteBufferReceive.scala:80)
>> >
>> > at
>> >
>> >
>> kafka.network.BoundedByteBufferReceive.readFrom(BoundedByteBufferReceive.scala:63)
>> >
>> > at kafka.network.Receive$class.readCompletely(Transmission.scala:56)
>> >
>> > at
>> >
>> >
>> kafka.network.BoundedByteBufferReceive.readCompletely(BoundedByteBufferReceive.scala:29)
>> >
>> > at kafka.network.BlockingChannel.receive(BlockingChannel.scala:102)
>> >
>> > at kafka.producer.SyncProducer.liftedTree1$1(SyncProducer.scala:79)
>> >
>> > at
>> >
>> >
>> kafka.producer.SyncProducer.kafka$producer$SyncProducer$$doSend(SyncProducer.scala:76)
>> >
>> > at kafka.producer.SyncProducer.send(SyncProducer.scala:117)
>> >
>> > at kafka.client.ClientUtils$.fetchTopicMetadata(ClientUtils.scala:58)
>> >
>> > at
>> >
>> kafka.producer.BrokerPartitionInfo.updateInfo(BrokerPartitionInfo.scala:82)
>> >
>> > at
>> >
>> >
>> kafka.producer.async.DefaultEventHandler$$anonfun$handle$1.apply$mcV$sp(DefaultEventHandler.scala:67)
>> >
>> > at kafka.utils.Utils$.swallow(Utils.scala:172)
>> >
>> > at kafka.utils.Logging$class.swallowError(Logging.scala:106)
>> >
>> > at kafka.utils.Utils$.swallowError(Utils.scala:45)
>> >
>> > at
>> >
>> >
>> kafka.producer.async.DefaultEventHandler.handle(DefaultEventHandler.scala:67)
>> >
>> > at
>> >
>> >
>> kafka.producer.async.ProducerSendThread.tryToHandle(ProducerSendThread.scala:104)
>> >
>> > at
>> >
>> >
>> kafka.producer.async.ProducerSendThread$$anonfun$processEvents$3.apply(ProducerSendThread.scala:87)
>> >
>> > at
>> >
>> >
>> kafka.producer.async.ProducerSendThread$$anonfun$processEvents$3.apply(ProducerSendThread.scala:67)
>> >
>> > at scala.collection.immutable.Stream.foreach(Stream.scala:526)
>> >
>> > at
>> >
>> >
>> kafka.producer.async.ProducerSendThread.processEvents(ProducerSendThread.scala:66)
>> >
>> > at
>> kafka.producer.async.ProducerSendThread.run(ProducerSendThread.scala:44)
>> >
>> >
>> >
>> > On Wed, Jul 16, 2014 at 6:07 PM, Rajasekar Elango <
>> relango@salesforce.com>
>> > wrote:
>> >
>> > > Pramod,
>> > >
>> > >
>> > > I presented secure kafka configuration and usage at last meet up. So
>> hope
>> > > this
>> > > video recording <http://www.ustream.tv/recorded/48396701>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 <dp...@gmail.com>
>> > > wrote:
>> > >
>> > > > 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 <
>> dpramodv@gmail.com>
>> > > > 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, Jul 16, 2014 at 3:57 PM, Joe Stein <jo...@stealth.ly>
>> > > 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
>> (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 Stein
>> > > > >>  Founder, Principal Consultant
>> > > > >>  Big Data Open Source Security LLC
>> > > > >>  http://www.stealth.ly
>> > > > >>  Twitter: @allthingshadoop <
>> http://www.twitter.com/allthingshadoop>
>> > > > >> ********************************************/
>> > > > >>
>> > > > >>
>> > > > >> On Wed, Jul 16, 2014 at 3:49 PM, Pramod Deshmukh <
>> > dpramodv@gmail.com>
>> > > > >> wrote:
>> > > > >>
>> > > > >> > 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 installed sbt and scala and followed your instructions
>> for
>> > > > >> different
>> > > > >> > scala versions. I was able to bring zookeeper up but brokers
>> fail
>> > to
>> > > > >> start
>> > > > >> > with error
>> > > > >> >
>> > > > >> > Error: Could not find or load main class kafka.Kafka
>> > > > >> >
>> > > > >> > I think I am doing something wrong. Can you please help me?
>> > > > >> >
>> > > > >> > Our current production setup is with 2.8.0 and want to stick to
>> > it.
>> > > > >> >
>> > > > >> > Thanks,
>> > > > >> >
>> > > > >> > Pramod
>> > > > >> >
>> > > > >> >
>> > > > >> > On Tue, Jun 3, 2014 at 3:57 PM, Joe Stein <
>> joe.stein@stealth.ly>
>> > > > wrote:
>> > > > >> >
>> > > > >> > > 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.  This comes from what folks are doing "in the
>> > wild"
>> > > to
>> > > > >> > > workaround and implement security with Kafka as it is today
>> and
>> > > also
>> > > > >> > what I
>> > > > >> > > have discovered from organizations about their blockers. It
>> also
>> > > > >> picks up
>> > > > >> > > from the wiki (which I should have time to update later in
>> the
>> > > week
>> > > > >> based
>> > > > >> > > on the below and feedback from the thread).
>> > > > >> > >
>> > > > >> > > 1) Transport Layer Security (i.e. SSL)
>> > > > >> > >
>> > > > >> > > This also includes client authentication in addition to
>> > in-transit
>> > > > >> > security
>> > > > >> > > layer.  This work has been picked up here
>> > > > >> > > https://issues.apache.org/jira/browse/KAFKA-1477 and do
>> > > appreciate
>> > > > >> any
>> > > > >> > > thoughts, comments, feedback, tomatoes, whatever for this
>> patch.
>> > >  It
>> > > > >> is a
>> > > > >> > > pickup from the fork of the work first done here
>> > > > >> > > https://github.com/relango/kafka/tree/kafka_security.
>> > > > >> > >
>> > > > >> > > 2) Data encryption at rest.
>> > > > >> > >
>> > > > >> > > This is very important and something that can be facilitated
>> > > within
>> > > > >> the
>> > > > >> > > wire protocol. It requires an additional map data structure
>> for
>> > > the
>> > > > >> > > "encrypted [data encryption key]". With this map (either in
>> your
>> > > > >> object
>> > > > >> > or
>> > > > >> > > in the wire protocol) you can store the dynamically generated
>> > > > >> symmetric
>> > > > >> > key
>> > > > >> > > (for each message) and then encrypt the data using that
>> > > dynamically
>> > > > >> > > generated key.  You then encrypt the encryption key using
>> each
>> > > > public
>> > > > >> key
>> > > > >> > > for whom is expected to be able to decrypt the encryption
>> key to
>> > > > then
>> > > > >> > > decrypt the message.  For each public key encrypted symmetric
>> > key
>> > > > >> (which
>> > > > >> > is
>> > > > >> > > now the "encrypted [data encryption key]" along with which
>> > public
>> > > > key
>> > > > >> it
>> > > > >> > > was encrypted with for (so a map of [publicKey] =
>> > > > >> > > encryptedDataEncryptionKey) as a chain.   Other patterns can
>> be
>> > > > >> > implemented
>> > > > >> > > but this is a pretty standard digital enveloping [0] pattern
>> > with
>> > > > >> only 1
>> > > > >> > > field added. Other patterns should be able to use that field
>> > to-do
>> > > > >> their
>> > > > >> > > implementation too.
>> > > > >> > >
>> > > > >> > > 3) Non-repudiation and long term non-repudiation.
>> > > > >> > >
>> > > > >> > > Non-repudiation is proving data hasn't changed.  This is
>> often
>> > (if
>> > > > not
>> > > > >> > > always) done with x509 public certificates (chained to a
>> > > certificate
>> > > > >> > > authority).
>> > > > >> > >
>> > > > >> > > Long term non-repudiation is what happens when the
>> certificates
>> > of
>> > > > the
>> > > > >> > > certificate authority are expired (or revoked) and everything
>> > ever
>> > > > >> signed
>> > > > >> > > (ever) with that certificate's public key then becomes "no
>> > longer
>> > > > >> > provable
>> > > > >> > > as ever being authentic".  That is where RFC3126 [1] and
>> RFC3161
>> > > [2]
>> > > > >> come
>> > > > >> > > in (or worm drives [hardware], etc).
>> > > > >> > >
>> > > > >> > > For either (or both) of these it is an operation of the
>> > encryptor
>> > > to
>> > > > >> > > sign/hash the data (with or without third party trusted
>> timestap
>> > > of
>> > > > >> the
>> > > > >> > > signing event) and encrypt that with their own private key
>> and
>> > > > >> distribute
>> > > > >> > > the results (before and after encrypting if required) along
>> with
>> > > > their
>> > > > >> > > public key. This structure is a bit more complex but
>> feasible,
>> > it
>> > > > is a
>> > > > >> > map
>> > > > >> > > of digital signature formats and the chain of dig sig
>> > > attestations.
>> > > > >>  The
>> > > > >> > > map's key being the method (i.e. CRC32, PKCS7 [3], XmlDigSig
>> > [4])
>> > > > and
>> > > > >> > then
>> > > > >> > > a list of map where that key is "purpose" of signature (what
>> > your
>> > > > >> > attesting
>> > > > >> > > too).  As a sibling field to the list another field for "the
>> > > > >> attester" as
>> > > > >> > > bytes (e.g. their PKCS12 [5] for the map of PKCS7
>> signatures).
>> > > > >> > >
>> > > > >> > > 4) Authorization
>> > > > >> > >
>> > > > >> > > We should have a policy of "404" for data, topics, partitions
>> > > (etc)
>> > > > if
>> > > > >> > > authenticated connections do not have access.  In "secure
>> mode"
>> > > any
>> > > > >> non
>> > > > >> > > authenticated connections should get a "404" type message on
>> > > > >> everything.
>> > > > >> > > Knowing "something is there" is a security risk in many uses
>> > > cases.
>> > > > >>  So
>> > > > >> > if
>> > > > >> > > you don't have access you don't even see it.  Baking "that"
>> into
>> > > > Kafka
>> > > > >> > > along with some interface for entitlement (access management)
>> > > > systems
>> > > > >> > > (pretty standard) is all that I think needs to be done to the
>> > core
>> > > > >> > project.
>> > > > >> > >  I want to tackle item later in the year after summer after
>> the
>> > > > other
>> > > > >> > three
>> > > > >> > > are complete.
>> > > > >> > >
>> > > > >> > > I look forward to thoughts on this and anyone else
>> interested in
>> > > > >> working
>> > > > >> > > with us on these items.
>> > > > >> > >
>> > > > >> > > [0]
>> > > > >> > >
>> > > > >> > >
>> > > > >> >
>> > > > >>
>> > > >
>> > >
>> >
>> http://www.emc.com/emc-plus/rsa-labs/standards-initiatives/what-is-a-digital-envelope.htm
>> > > > >> > > [1] http://tools.ietf.org/html/rfc3126
>> > > > >> > > [2] http://tools.ietf.org/html/rfc3161
>> > > > >> > > [3]
>> > > > >> > >
>> > > > >> > >
>> > > > >> >
>> > > > >>
>> > > >
>> > >
>> >
>> http://www.emc.com/emc-plus/rsa-labs/standards-initiatives/pkcs-7-cryptographic-message-syntax-standar.htm
>> > > > >> > > [4] http://en.wikipedia.org/wiki/XML_Signature
>> > > > >> > > [5] http://en.wikipedia.org/wiki/PKCS_12
>> > > > >> > >
>> > > > >> > > /*******************************************
>> > > > >> > >  Joe Stein
>> > > > >> > >  Founder, Principal Consultant
>> > > > >> > >  Big Data Open Source Security LLC
>> > > > >> > >  http://www.stealth.ly
>> > > > >> > >  Twitter: @allthingshadoop <
>> > > http://www.twitter.com/allthingshadoop>
>> > > > >> > > ********************************************/
>> > > > >> > >
>> > > > >> >
>> > > > >>
>> > > > >
>> > > > >
>> > > >
>> > >
>> > >
>> > >
>> > > --
>> > > Thanks,
>> > > Raja.
>> > >
>> >
>>
>>
>>
>> --
>> Thanks,
>> Raja.
>>
>
>

Re: [DISCUSS] Kafka Security Specific Features

Posted by Pramod Deshmukh <dp...@gmail.com>.
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.OutOfMemoryError: Java heap space

*broker log*

INFO begin ssl handshake for localhost/127.0.0.1:50199//127.0.0.1:9092





On Thu, Jul 17, 2014 at 6:07 PM, Pramod Deshmukh <dp...@gmail.com> wrote:

> 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 <re...@salesforce.com>
> wrote:
>
>> 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 <dp...@gmail.com>
>> 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.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 config/server.properties*
>> >
>> >
>> >
>> > *broker.log:*
>> >
>> > [2014-07-17 15:34:46,281] INFO zookeeper state changed (SyncConnected)
>> > (org.I0Itec.zkclient.ZkClient)
>> >
>> > [2014-07-17 15:34:46,523] INFO Loading log 'secure.test-0'
>> > (kafka.log.LogManager)
>> >
>> > [2014-07-17 15:34:46,558] INFO Recovering unflushed segment 0 in log
>> > secure.test-0. (kafka.log.Log)
>> >
>> > [2014-07-17 15:34:46,571] INFO Completed load of log secure.test-0 with
>> log
>> > end offset 0 (kafka.log.Log)
>> >
>> > [2014-07-17 15:34:46,582] INFO Starting log cleanup with a period of
>> 60000
>> > ms. (kafka.log.LogManager)
>> >
>> > [2014-07-17 15:34:46,587] INFO Starting log flusher with a default
>> period
>> > of 9223372036854775807 ms. (kafka.log.LogManager)
>> >
>> > [2014-07-17 15:34:46,614] INFO Initializing secure authentication
>> > (kafka.network.security.SecureAuth$)
>> >
>> > [2014-07-17 15:34:46,678] INFO Secure authentication initialization has
>> > been successfully completed (kafka.network.security.SecureAuth$)
>> >
>> > [2014-07-17 15:34:46,691] INFO Awaiting socket connections on
>> 0.0.0.0:9092
>> > .
>> > (kafka.network.Acceptor)
>> >
>> > [2014-07-17 15:34:46,692] INFO [Socket Server on Broker 0], Started
>> > (kafka.network.SocketServer)
>> >
>> > [2014-07-17 15:34:46,794] INFO Will not load MX4J, mx4j-tools.jar is
>> not in
>> > the classpath (kafka.utils.Mx4jLoader$)
>> >
>> > [2014-07-17 15:34:46,837] INFO 0 successfully elected as leader
>> > (kafka.server.ZookeeperLeaderElector)
>> >
>> > [2014-07-17 15:34:47,057] INFO Registered broker 0 at path
>> /brokers/ids/0
>> > with address 10.1.100.130:9092. (kafka.utils.ZkUtils$)
>> >
>> > [2014-07-17 15:34:47,059] INFO New leader is 0
>> > (kafka.server.ZookeeperLeaderElector$LeaderChangeListener)
>> >
>> > *[2014-07-17 15:34:47,068] INFO [Kafka Server 0], started
>> > (kafka.server.KafkaServer)*
>> >
>> > *[2014-07-17 15:34:47,383] INFO begin ssl handshake for
>> > /10.1.100.130:9092//10.1.100.130:51685
>> > <http://10.1.100.130:9092//10.1.100.130:51685>
>> > (kafka.network.security.SSLSocketChannel)*
>> >
>> > *[2014-07-17 15:34:47,392] INFO begin ssl handshake for
>> > 10.1.100.130/10.1.100.130:51685//10.1.100.130:9092
>> > <http://10.1.100.130/10.1.100.130:51685//10.1.100.130:9092>
>> > (kafka.network.security.SSLSocketChannel)*
>> >
>> > *[2014-07-17 15:34:47,465] INFO finished ssl handshake for
>> > 10.1.100.130/10.1.100.130:51685//10.1.100.130:9092
>> > <http://10.1.100.130/10.1.100.130:51685//10.1.100.130:9092>
>> > (kafka.network.security.SSLSocketChannel)*
>> >
>> > *[2014-07-17 15:34:47,465] INFO finished ssl handshake for
>> > /10.1.100.130:9092//10.1.100.130:51685
>> > <http://10.1.100.130:9092//10.1.100.130:51685>
>> > (kafka.network.security.SSLSocketChannel)*
>> >
>> > *[2014-07-17 15:34:47,617] INFO [ReplicaFetcherManager on broker 0]
>> Removed
>> > fetcher for partitions  (kafka.server.ReplicaFetcherManager)*
>> >
>> > *[2014-07-17 15:34:47,627] INFO [ReplicaFetcherManager on broker 0]
>> Added
>> > fetcher for partitions List() (kafka.server.ReplicaFetcherManager)*
>> >
>> > *[2014-07-17 15:34:47,656] INFO [ReplicaFetcherManager on broker 0]
>> Removed
>> > fetcher for partitions [secure.test,0]
>> > (kafka.server.ReplicaFetcherManager)*
>> >
>> > [2014-07-17 15:37:15,970] INFO begin ssl handshake for
>> > 10.1.100.130/10.1.100.130:51689//10.1.100.130:9092
>> > (kafka.network.security.SSLSocketChannel)
>> >
>> > [2014-07-17 15:37:16,075] INFO begin ssl handshake for
>> > 10.1.100.130/10.1.100.130:51690//10.1.100.130:9092
>> > (kafka.network.security.SSLSocketChannel)
>> >
>> > [2014-07-17 15:37:16,434] INFO begin ssl handshake for
>> > 10.1.100.130/10.1.100.130:51691//10.1.100.130:9092
>> > (kafka.network.security.SSLSocketChannel)
>> >
>> > [2014-07-17 15:37:16,530] INFO begin ssl handshake for
>> > 10.1.100.130/10.1.100.130:51692//10.1.100.130:9092
>> > (kafka.network.security.SSLSocketChannel)
>> >
>> > [2014-07-17 15:37:16,743] INFO begin ssl handshake for
>> > 10.1.100.130/10.1.100.130:51693//10.1.100.130:9092
>> > (kafka.network.security.SSLSocketChannel)
>> >
>> > [2014-07-17 15:37:16,834] INFO begin ssl handshake for
>> > 10.1.100.130/10.1.100.130:51694//10.1.100.130:9092
>> > (kafka.network.security.SSLSocketChannel)
>> >
>> > [2014-07-17 15:37:17,043] INFO begin ssl handshake for
>> > 10.1.100.130/10.1.100.130:51695//10.1.100.130:9092
>> > (kafka.network.security.SSLSocketChannel)
>> >
>> > [2014-07-17 15:37:17,137] INFO begin ssl handshake for
>> > 10.1.100.130/10.1.100.130:51696//10.1.100.130:9092
>> > (kafka.network.security.SSLSocketChannel)
>> >
>> > [2014-07-17 15:37:17,342] INFO begin ssl handshake for
>> > 10.1.100.130/10.1.100.130:51697//10.1.100.130:9092
>> > (kafka.network.security.SSLSocketChannel)
>> >
>> >
>> > *Start producer*
>> >
>> > *bin/kafka-console-producer.sh --broker-list 10.1.100.130:9092:true
>> > --topic
>> > secure.test*
>> >
>> >
>> > *producer.log:*
>> >
>> > bin/kafka-console-producer.sh --broker-list 10.1.100.130:9092:true
>> --topic
>> > secure.test
>> >
>> > [2014-07-17 15:37:46,889] WARN Property topic is not valid
>> > (kafka.utils.VerifiableProperties)
>> >
>> > Hello Secure Kafka
>> >
>> > *[2014-07-17 15:38:14,186] ERROR OOME with size 352518400
>> > (kafka.network.BoundedByteBufferReceive)*
>> >
>> > *java.lang.OutOfMemoryError: Java heap space*
>> >
>> > at java.nio.HeapByteBuffer.<init>(HeapByteBuffer.java:57)
>> >
>> > at java.nio.ByteBuffer.allocate(ByteBuffer.java:331)
>> >
>> > at
>> >
>> >
>> kafka.network.BoundedByteBufferReceive.byteBufferAllocate(BoundedByteBufferReceive.scala:80)
>> >
>> > at
>> >
>> >
>> kafka.network.BoundedByteBufferReceive.readFrom(BoundedByteBufferReceive.scala:63)
>> >
>> > at kafka.network.Receive$class.readCompletely(Transmission.scala:56)
>> >
>> > at
>> >
>> >
>> kafka.network.BoundedByteBufferReceive.readCompletely(BoundedByteBufferReceive.scala:29)
>> >
>> > at kafka.network.BlockingChannel.receive(BlockingChannel.scala:102)
>> >
>> > at kafka.producer.SyncProducer.liftedTree1$1(SyncProducer.scala:79)
>> >
>> > at
>> >
>> >
>> kafka.producer.SyncProducer.kafka$producer$SyncProducer$$doSend(SyncProducer.scala:76)
>> >
>> > at kafka.producer.SyncProducer.send(SyncProducer.scala:117)
>> >
>> > at kafka.client.ClientUtils$.fetchTopicMetadata(ClientUtils.scala:58)
>> >
>> > at
>> >
>> kafka.producer.BrokerPartitionInfo.updateInfo(BrokerPartitionInfo.scala:82)
>> >
>> > at
>> >
>> >
>> kafka.producer.async.DefaultEventHandler$$anonfun$handle$1.apply$mcV$sp(DefaultEventHandler.scala:67)
>> >
>> > at kafka.utils.Utils$.swallow(Utils.scala:172)
>> >
>> > at kafka.utils.Logging$class.swallowError(Logging.scala:106)
>> >
>> > at kafka.utils.Utils$.swallowError(Utils.scala:45)
>> >
>> > at
>> >
>> >
>> kafka.producer.async.DefaultEventHandler.handle(DefaultEventHandler.scala:67)
>> >
>> > at
>> >
>> >
>> kafka.producer.async.ProducerSendThread.tryToHandle(ProducerSendThread.scala:104)
>> >
>> > at
>> >
>> >
>> kafka.producer.async.ProducerSendThread$$anonfun$processEvents$3.apply(ProducerSendThread.scala:87)
>> >
>> > at
>> >
>> >
>> kafka.producer.async.ProducerSendThread$$anonfun$processEvents$3.apply(ProducerSendThread.scala:67)
>> >
>> > at scala.collection.immutable.Stream.foreach(Stream.scala:526)
>> >
>> > at
>> >
>> >
>> kafka.producer.async.ProducerSendThread.processEvents(ProducerSendThread.scala:66)
>> >
>> > at
>> kafka.producer.async.ProducerSendThread.run(ProducerSendThread.scala:44)
>> >
>> >
>> >
>> > On Wed, Jul 16, 2014 at 6:07 PM, Rajasekar Elango <
>> relango@salesforce.com>
>> > wrote:
>> >
>> > > Pramod,
>> > >
>> > >
>> > > I presented secure kafka configuration and usage at last meet up. So
>> hope
>> > > this
>> > > video recording <http://www.ustream.tv/recorded/48396701>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 <dp...@gmail.com>
>> > > wrote:
>> > >
>> > > > 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 <
>> dpramodv@gmail.com>
>> > > > 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, Jul 16, 2014 at 3:57 PM, Joe Stein <jo...@stealth.ly>
>> > > 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
>> (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 Stein
>> > > > >>  Founder, Principal Consultant
>> > > > >>  Big Data Open Source Security LLC
>> > > > >>  http://www.stealth.ly
>> > > > >>  Twitter: @allthingshadoop <
>> http://www.twitter.com/allthingshadoop>
>> > > > >> ********************************************/
>> > > > >>
>> > > > >>
>> > > > >> On Wed, Jul 16, 2014 at 3:49 PM, Pramod Deshmukh <
>> > dpramodv@gmail.com>
>> > > > >> wrote:
>> > > > >>
>> > > > >> > 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 installed sbt and scala and followed your instructions
>> for
>> > > > >> different
>> > > > >> > scala versions. I was able to bring zookeeper up but brokers
>> fail
>> > to
>> > > > >> start
>> > > > >> > with error
>> > > > >> >
>> > > > >> > Error: Could not find or load main class kafka.Kafka
>> > > > >> >
>> > > > >> > I think I am doing something wrong. Can you please help me?
>> > > > >> >
>> > > > >> > Our current production setup is with 2.8.0 and want to stick to
>> > it.
>> > > > >> >
>> > > > >> > Thanks,
>> > > > >> >
>> > > > >> > Pramod
>> > > > >> >
>> > > > >> >
>> > > > >> > On Tue, Jun 3, 2014 at 3:57 PM, Joe Stein <
>> joe.stein@stealth.ly>
>> > > > wrote:
>> > > > >> >
>> > > > >> > > 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.  This comes from what folks are doing "in the
>> > wild"
>> > > to
>> > > > >> > > workaround and implement security with Kafka as it is today
>> and
>> > > also
>> > > > >> > what I
>> > > > >> > > have discovered from organizations about their blockers. It
>> also
>> > > > >> picks up
>> > > > >> > > from the wiki (which I should have time to update later in
>> the
>> > > week
>> > > > >> based
>> > > > >> > > on the below and feedback from the thread).
>> > > > >> > >
>> > > > >> > > 1) Transport Layer Security (i.e. SSL)
>> > > > >> > >
>> > > > >> > > This also includes client authentication in addition to
>> > in-transit
>> > > > >> > security
>> > > > >> > > layer.  This work has been picked up here
>> > > > >> > > https://issues.apache.org/jira/browse/KAFKA-1477 and do
>> > > appreciate
>> > > > >> any
>> > > > >> > > thoughts, comments, feedback, tomatoes, whatever for this
>> patch.
>> > >  It
>> > > > >> is a
>> > > > >> > > pickup from the fork of the work first done here
>> > > > >> > > https://github.com/relango/kafka/tree/kafka_security.
>> > > > >> > >
>> > > > >> > > 2) Data encryption at rest.
>> > > > >> > >
>> > > > >> > > This is very important and something that can be facilitated
>> > > within
>> > > > >> the
>> > > > >> > > wire protocol. It requires an additional map data structure
>> for
>> > > the
>> > > > >> > > "encrypted [data encryption key]". With this map (either in
>> your
>> > > > >> object
>> > > > >> > or
>> > > > >> > > in the wire protocol) you can store the dynamically generated
>> > > > >> symmetric
>> > > > >> > key
>> > > > >> > > (for each message) and then encrypt the data using that
>> > > dynamically
>> > > > >> > > generated key.  You then encrypt the encryption key using
>> each
>> > > > public
>> > > > >> key
>> > > > >> > > for whom is expected to be able to decrypt the encryption
>> key to
>> > > > then
>> > > > >> > > decrypt the message.  For each public key encrypted symmetric
>> > key
>> > > > >> (which
>> > > > >> > is
>> > > > >> > > now the "encrypted [data encryption key]" along with which
>> > public
>> > > > key
>> > > > >> it
>> > > > >> > > was encrypted with for (so a map of [publicKey] =
>> > > > >> > > encryptedDataEncryptionKey) as a chain.   Other patterns can
>> be
>> > > > >> > implemented
>> > > > >> > > but this is a pretty standard digital enveloping [0] pattern
>> > with
>> > > > >> only 1
>> > > > >> > > field added. Other patterns should be able to use that field
>> > to-do
>> > > > >> their
>> > > > >> > > implementation too.
>> > > > >> > >
>> > > > >> > > 3) Non-repudiation and long term non-repudiation.
>> > > > >> > >
>> > > > >> > > Non-repudiation is proving data hasn't changed.  This is
>> often
>> > (if
>> > > > not
>> > > > >> > > always) done with x509 public certificates (chained to a
>> > > certificate
>> > > > >> > > authority).
>> > > > >> > >
>> > > > >> > > Long term non-repudiation is what happens when the
>> certificates
>> > of
>> > > > the
>> > > > >> > > certificate authority are expired (or revoked) and everything
>> > ever
>> > > > >> signed
>> > > > >> > > (ever) with that certificate's public key then becomes "no
>> > longer
>> > > > >> > provable
>> > > > >> > > as ever being authentic".  That is where RFC3126 [1] and
>> RFC3161
>> > > [2]
>> > > > >> come
>> > > > >> > > in (or worm drives [hardware], etc).
>> > > > >> > >
>> > > > >> > > For either (or both) of these it is an operation of the
>> > encryptor
>> > > to
>> > > > >> > > sign/hash the data (with or without third party trusted
>> timestap
>> > > of
>> > > > >> the
>> > > > >> > > signing event) and encrypt that with their own private key
>> and
>> > > > >> distribute
>> > > > >> > > the results (before and after encrypting if required) along
>> with
>> > > > their
>> > > > >> > > public key. This structure is a bit more complex but
>> feasible,
>> > it
>> > > > is a
>> > > > >> > map
>> > > > >> > > of digital signature formats and the chain of dig sig
>> > > attestations.
>> > > > >>  The
>> > > > >> > > map's key being the method (i.e. CRC32, PKCS7 [3], XmlDigSig
>> > [4])
>> > > > and
>> > > > >> > then
>> > > > >> > > a list of map where that key is "purpose" of signature (what
>> > your
>> > > > >> > attesting
>> > > > >> > > too).  As a sibling field to the list another field for "the
>> > > > >> attester" as
>> > > > >> > > bytes (e.g. their PKCS12 [5] for the map of PKCS7
>> signatures).
>> > > > >> > >
>> > > > >> > > 4) Authorization
>> > > > >> > >
>> > > > >> > > We should have a policy of "404" for data, topics, partitions
>> > > (etc)
>> > > > if
>> > > > >> > > authenticated connections do not have access.  In "secure
>> mode"
>> > > any
>> > > > >> non
>> > > > >> > > authenticated connections should get a "404" type message on
>> > > > >> everything.
>> > > > >> > > Knowing "something is there" is a security risk in many uses
>> > > cases.
>> > > > >>  So
>> > > > >> > if
>> > > > >> > > you don't have access you don't even see it.  Baking "that"
>> into
>> > > > Kafka
>> > > > >> > > along with some interface for entitlement (access management)
>> > > > systems
>> > > > >> > > (pretty standard) is all that I think needs to be done to the
>> > core
>> > > > >> > project.
>> > > > >> > >  I want to tackle item later in the year after summer after
>> the
>> > > > other
>> > > > >> > three
>> > > > >> > > are complete.
>> > > > >> > >
>> > > > >> > > I look forward to thoughts on this and anyone else
>> interested in
>> > > > >> working
>> > > > >> > > with us on these items.
>> > > > >> > >
>> > > > >> > > [0]
>> > > > >> > >
>> > > > >> > >
>> > > > >> >
>> > > > >>
>> > > >
>> > >
>> >
>> http://www.emc.com/emc-plus/rsa-labs/standards-initiatives/what-is-a-digital-envelope.htm
>> > > > >> > > [1] http://tools.ietf.org/html/rfc3126
>> > > > >> > > [2] http://tools.ietf.org/html/rfc3161
>> > > > >> > > [3]
>> > > > >> > >
>> > > > >> > >
>> > > > >> >
>> > > > >>
>> > > >
>> > >
>> >
>> http://www.emc.com/emc-plus/rsa-labs/standards-initiatives/pkcs-7-cryptographic-message-syntax-standar.htm
>> > > > >> > > [4] http://en.wikipedia.org/wiki/XML_Signature
>> > > > >> > > [5] http://en.wikipedia.org/wiki/PKCS_12
>> > > > >> > >
>> > > > >> > > /*******************************************
>> > > > >> > >  Joe Stein
>> > > > >> > >  Founder, Principal Consultant
>> > > > >> > >  Big Data Open Source Security LLC
>> > > > >> > >  http://www.stealth.ly
>> > > > >> > >  Twitter: @allthingshadoop <
>> > > http://www.twitter.com/allthingshadoop>
>> > > > >> > > ********************************************/
>> > > > >> > >
>> > > > >> >
>> > > > >>
>> > > > >
>> > > > >
>> > > >
>> > >
>> > >
>> > >
>> > > --
>> > > Thanks,
>> > > Raja.
>> > >
>> >
>>
>>
>>
>> --
>> Thanks,
>> Raja.
>>
>
>

Re: [DISCUSS] Kafka Security Specific Features

Posted by Pramod Deshmukh <dp...@gmail.com>.
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 <re...@salesforce.com>
wrote:

> 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 <dp...@gmail.com>
> 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.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 config/server.properties*
> >
> >
> >
> > *broker.log:*
> >
> > [2014-07-17 15:34:46,281] INFO zookeeper state changed (SyncConnected)
> > (org.I0Itec.zkclient.ZkClient)
> >
> > [2014-07-17 15:34:46,523] INFO Loading log 'secure.test-0'
> > (kafka.log.LogManager)
> >
> > [2014-07-17 15:34:46,558] INFO Recovering unflushed segment 0 in log
> > secure.test-0. (kafka.log.Log)
> >
> > [2014-07-17 15:34:46,571] INFO Completed load of log secure.test-0 with
> log
> > end offset 0 (kafka.log.Log)
> >
> > [2014-07-17 15:34:46,582] INFO Starting log cleanup with a period of
> 60000
> > ms. (kafka.log.LogManager)
> >
> > [2014-07-17 15:34:46,587] INFO Starting log flusher with a default period
> > of 9223372036854775807 ms. (kafka.log.LogManager)
> >
> > [2014-07-17 15:34:46,614] INFO Initializing secure authentication
> > (kafka.network.security.SecureAuth$)
> >
> > [2014-07-17 15:34:46,678] INFO Secure authentication initialization has
> > been successfully completed (kafka.network.security.SecureAuth$)
> >
> > [2014-07-17 15:34:46,691] INFO Awaiting socket connections on
> 0.0.0.0:9092
> > .
> > (kafka.network.Acceptor)
> >
> > [2014-07-17 15:34:46,692] INFO [Socket Server on Broker 0], Started
> > (kafka.network.SocketServer)
> >
> > [2014-07-17 15:34:46,794] INFO Will not load MX4J, mx4j-tools.jar is not
> in
> > the classpath (kafka.utils.Mx4jLoader$)
> >
> > [2014-07-17 15:34:46,837] INFO 0 successfully elected as leader
> > (kafka.server.ZookeeperLeaderElector)
> >
> > [2014-07-17 15:34:47,057] INFO Registered broker 0 at path /brokers/ids/0
> > with address 10.1.100.130:9092. (kafka.utils.ZkUtils$)
> >
> > [2014-07-17 15:34:47,059] INFO New leader is 0
> > (kafka.server.ZookeeperLeaderElector$LeaderChangeListener)
> >
> > *[2014-07-17 15:34:47,068] INFO [Kafka Server 0], started
> > (kafka.server.KafkaServer)*
> >
> > *[2014-07-17 15:34:47,383] INFO begin ssl handshake for
> > /10.1.100.130:9092//10.1.100.130:51685
> > <http://10.1.100.130:9092//10.1.100.130:51685>
> > (kafka.network.security.SSLSocketChannel)*
> >
> > *[2014-07-17 15:34:47,392] INFO begin ssl handshake for
> > 10.1.100.130/10.1.100.130:51685//10.1.100.130:9092
> > <http://10.1.100.130/10.1.100.130:51685//10.1.100.130:9092>
> > (kafka.network.security.SSLSocketChannel)*
> >
> > *[2014-07-17 15:34:47,465] INFO finished ssl handshake for
> > 10.1.100.130/10.1.100.130:51685//10.1.100.130:9092
> > <http://10.1.100.130/10.1.100.130:51685//10.1.100.130:9092>
> > (kafka.network.security.SSLSocketChannel)*
> >
> > *[2014-07-17 15:34:47,465] INFO finished ssl handshake for
> > /10.1.100.130:9092//10.1.100.130:51685
> > <http://10.1.100.130:9092//10.1.100.130:51685>
> > (kafka.network.security.SSLSocketChannel)*
> >
> > *[2014-07-17 15:34:47,617] INFO [ReplicaFetcherManager on broker 0]
> Removed
> > fetcher for partitions  (kafka.server.ReplicaFetcherManager)*
> >
> > *[2014-07-17 15:34:47,627] INFO [ReplicaFetcherManager on broker 0] Added
> > fetcher for partitions List() (kafka.server.ReplicaFetcherManager)*
> >
> > *[2014-07-17 15:34:47,656] INFO [ReplicaFetcherManager on broker 0]
> Removed
> > fetcher for partitions [secure.test,0]
> > (kafka.server.ReplicaFetcherManager)*
> >
> > [2014-07-17 15:37:15,970] INFO begin ssl handshake for
> > 10.1.100.130/10.1.100.130:51689//10.1.100.130:9092
> > (kafka.network.security.SSLSocketChannel)
> >
> > [2014-07-17 15:37:16,075] INFO begin ssl handshake for
> > 10.1.100.130/10.1.100.130:51690//10.1.100.130:9092
> > (kafka.network.security.SSLSocketChannel)
> >
> > [2014-07-17 15:37:16,434] INFO begin ssl handshake for
> > 10.1.100.130/10.1.100.130:51691//10.1.100.130:9092
> > (kafka.network.security.SSLSocketChannel)
> >
> > [2014-07-17 15:37:16,530] INFO begin ssl handshake for
> > 10.1.100.130/10.1.100.130:51692//10.1.100.130:9092
> > (kafka.network.security.SSLSocketChannel)
> >
> > [2014-07-17 15:37:16,743] INFO begin ssl handshake for
> > 10.1.100.130/10.1.100.130:51693//10.1.100.130:9092
> > (kafka.network.security.SSLSocketChannel)
> >
> > [2014-07-17 15:37:16,834] INFO begin ssl handshake for
> > 10.1.100.130/10.1.100.130:51694//10.1.100.130:9092
> > (kafka.network.security.SSLSocketChannel)
> >
> > [2014-07-17 15:37:17,043] INFO begin ssl handshake for
> > 10.1.100.130/10.1.100.130:51695//10.1.100.130:9092
> > (kafka.network.security.SSLSocketChannel)
> >
> > [2014-07-17 15:37:17,137] INFO begin ssl handshake for
> > 10.1.100.130/10.1.100.130:51696//10.1.100.130:9092
> > (kafka.network.security.SSLSocketChannel)
> >
> > [2014-07-17 15:37:17,342] INFO begin ssl handshake for
> > 10.1.100.130/10.1.100.130:51697//10.1.100.130:9092
> > (kafka.network.security.SSLSocketChannel)
> >
> >
> > *Start producer*
> >
> > *bin/kafka-console-producer.sh --broker-list 10.1.100.130:9092:true
> > --topic
> > secure.test*
> >
> >
> > *producer.log:*
> >
> > bin/kafka-console-producer.sh --broker-list 10.1.100.130:9092:true
> --topic
> > secure.test
> >
> > [2014-07-17 15:37:46,889] WARN Property topic is not valid
> > (kafka.utils.VerifiableProperties)
> >
> > Hello Secure Kafka
> >
> > *[2014-07-17 15:38:14,186] ERROR OOME with size 352518400
> > (kafka.network.BoundedByteBufferReceive)*
> >
> > *java.lang.OutOfMemoryError: Java heap space*
> >
> > at java.nio.HeapByteBuffer.<init>(HeapByteBuffer.java:57)
> >
> > at java.nio.ByteBuffer.allocate(ByteBuffer.java:331)
> >
> > at
> >
> >
> kafka.network.BoundedByteBufferReceive.byteBufferAllocate(BoundedByteBufferReceive.scala:80)
> >
> > at
> >
> >
> kafka.network.BoundedByteBufferReceive.readFrom(BoundedByteBufferReceive.scala:63)
> >
> > at kafka.network.Receive$class.readCompletely(Transmission.scala:56)
> >
> > at
> >
> >
> kafka.network.BoundedByteBufferReceive.readCompletely(BoundedByteBufferReceive.scala:29)
> >
> > at kafka.network.BlockingChannel.receive(BlockingChannel.scala:102)
> >
> > at kafka.producer.SyncProducer.liftedTree1$1(SyncProducer.scala:79)
> >
> > at
> >
> >
> kafka.producer.SyncProducer.kafka$producer$SyncProducer$$doSend(SyncProducer.scala:76)
> >
> > at kafka.producer.SyncProducer.send(SyncProducer.scala:117)
> >
> > at kafka.client.ClientUtils$.fetchTopicMetadata(ClientUtils.scala:58)
> >
> > at
> >
> kafka.producer.BrokerPartitionInfo.updateInfo(BrokerPartitionInfo.scala:82)
> >
> > at
> >
> >
> kafka.producer.async.DefaultEventHandler$$anonfun$handle$1.apply$mcV$sp(DefaultEventHandler.scala:67)
> >
> > at kafka.utils.Utils$.swallow(Utils.scala:172)
> >
> > at kafka.utils.Logging$class.swallowError(Logging.scala:106)
> >
> > at kafka.utils.Utils$.swallowError(Utils.scala:45)
> >
> > at
> >
> >
> kafka.producer.async.DefaultEventHandler.handle(DefaultEventHandler.scala:67)
> >
> > at
> >
> >
> kafka.producer.async.ProducerSendThread.tryToHandle(ProducerSendThread.scala:104)
> >
> > at
> >
> >
> kafka.producer.async.ProducerSendThread$$anonfun$processEvents$3.apply(ProducerSendThread.scala:87)
> >
> > at
> >
> >
> kafka.producer.async.ProducerSendThread$$anonfun$processEvents$3.apply(ProducerSendThread.scala:67)
> >
> > at scala.collection.immutable.Stream.foreach(Stream.scala:526)
> >
> > at
> >
> >
> kafka.producer.async.ProducerSendThread.processEvents(ProducerSendThread.scala:66)
> >
> > at
> kafka.producer.async.ProducerSendThread.run(ProducerSendThread.scala:44)
> >
> >
> >
> > On Wed, Jul 16, 2014 at 6:07 PM, Rajasekar Elango <
> relango@salesforce.com>
> > wrote:
> >
> > > Pramod,
> > >
> > >
> > > I presented secure kafka configuration and usage at last meet up. So
> hope
> > > this
> > > video recording <http://www.ustream.tv/recorded/48396701>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 <dp...@gmail.com>
> > > wrote:
> > >
> > > > 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 <dpramodv@gmail.com
> >
> > > > 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, Jul 16, 2014 at 3:57 PM, Joe Stein <jo...@stealth.ly>
> > > 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 (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 Stein
> > > > >>  Founder, Principal Consultant
> > > > >>  Big Data Open Source Security LLC
> > > > >>  http://www.stealth.ly
> > > > >>  Twitter: @allthingshadoop <
> http://www.twitter.com/allthingshadoop>
> > > > >> ********************************************/
> > > > >>
> > > > >>
> > > > >> On Wed, Jul 16, 2014 at 3:49 PM, Pramod Deshmukh <
> > dpramodv@gmail.com>
> > > > >> wrote:
> > > > >>
> > > > >> > 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 installed sbt and scala and followed your instructions
> for
> > > > >> different
> > > > >> > scala versions. I was able to bring zookeeper up but brokers
> fail
> > to
> > > > >> start
> > > > >> > with error
> > > > >> >
> > > > >> > Error: Could not find or load main class kafka.Kafka
> > > > >> >
> > > > >> > I think I am doing something wrong. Can you please help me?
> > > > >> >
> > > > >> > Our current production setup is with 2.8.0 and want to stick to
> > it.
> > > > >> >
> > > > >> > Thanks,
> > > > >> >
> > > > >> > Pramod
> > > > >> >
> > > > >> >
> > > > >> > On Tue, Jun 3, 2014 at 3:57 PM, Joe Stein <joe.stein@stealth.ly
> >
> > > > wrote:
> > > > >> >
> > > > >> > > 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.  This comes from what folks are doing "in the
> > wild"
> > > to
> > > > >> > > workaround and implement security with Kafka as it is today
> and
> > > also
> > > > >> > what I
> > > > >> > > have discovered from organizations about their blockers. It
> also
> > > > >> picks up
> > > > >> > > from the wiki (which I should have time to update later in the
> > > week
> > > > >> based
> > > > >> > > on the below and feedback from the thread).
> > > > >> > >
> > > > >> > > 1) Transport Layer Security (i.e. SSL)
> > > > >> > >
> > > > >> > > This also includes client authentication in addition to
> > in-transit
> > > > >> > security
> > > > >> > > layer.  This work has been picked up here
> > > > >> > > https://issues.apache.org/jira/browse/KAFKA-1477 and do
> > > appreciate
> > > > >> any
> > > > >> > > thoughts, comments, feedback, tomatoes, whatever for this
> patch.
> > >  It
> > > > >> is a
> > > > >> > > pickup from the fork of the work first done here
> > > > >> > > https://github.com/relango/kafka/tree/kafka_security.
> > > > >> > >
> > > > >> > > 2) Data encryption at rest.
> > > > >> > >
> > > > >> > > This is very important and something that can be facilitated
> > > within
> > > > >> the
> > > > >> > > wire protocol. It requires an additional map data structure
> for
> > > the
> > > > >> > > "encrypted [data encryption key]". With this map (either in
> your
> > > > >> object
> > > > >> > or
> > > > >> > > in the wire protocol) you can store the dynamically generated
> > > > >> symmetric
> > > > >> > key
> > > > >> > > (for each message) and then encrypt the data using that
> > > dynamically
> > > > >> > > generated key.  You then encrypt the encryption key using each
> > > > public
> > > > >> key
> > > > >> > > for whom is expected to be able to decrypt the encryption key
> to
> > > > then
> > > > >> > > decrypt the message.  For each public key encrypted symmetric
> > key
> > > > >> (which
> > > > >> > is
> > > > >> > > now the "encrypted [data encryption key]" along with which
> > public
> > > > key
> > > > >> it
> > > > >> > > was encrypted with for (so a map of [publicKey] =
> > > > >> > > encryptedDataEncryptionKey) as a chain.   Other patterns can
> be
> > > > >> > implemented
> > > > >> > > but this is a pretty standard digital enveloping [0] pattern
> > with
> > > > >> only 1
> > > > >> > > field added. Other patterns should be able to use that field
> > to-do
> > > > >> their
> > > > >> > > implementation too.
> > > > >> > >
> > > > >> > > 3) Non-repudiation and long term non-repudiation.
> > > > >> > >
> > > > >> > > Non-repudiation is proving data hasn't changed.  This is often
> > (if
> > > > not
> > > > >> > > always) done with x509 public certificates (chained to a
> > > certificate
> > > > >> > > authority).
> > > > >> > >
> > > > >> > > Long term non-repudiation is what happens when the
> certificates
> > of
> > > > the
> > > > >> > > certificate authority are expired (or revoked) and everything
> > ever
> > > > >> signed
> > > > >> > > (ever) with that certificate's public key then becomes "no
> > longer
> > > > >> > provable
> > > > >> > > as ever being authentic".  That is where RFC3126 [1] and
> RFC3161
> > > [2]
> > > > >> come
> > > > >> > > in (or worm drives [hardware], etc).
> > > > >> > >
> > > > >> > > For either (or both) of these it is an operation of the
> > encryptor
> > > to
> > > > >> > > sign/hash the data (with or without third party trusted
> timestap
> > > of
> > > > >> the
> > > > >> > > signing event) and encrypt that with their own private key and
> > > > >> distribute
> > > > >> > > the results (before and after encrypting if required) along
> with
> > > > their
> > > > >> > > public key. This structure is a bit more complex but feasible,
> > it
> > > > is a
> > > > >> > map
> > > > >> > > of digital signature formats and the chain of dig sig
> > > attestations.
> > > > >>  The
> > > > >> > > map's key being the method (i.e. CRC32, PKCS7 [3], XmlDigSig
> > [4])
> > > > and
> > > > >> > then
> > > > >> > > a list of map where that key is "purpose" of signature (what
> > your
> > > > >> > attesting
> > > > >> > > too).  As a sibling field to the list another field for "the
> > > > >> attester" as
> > > > >> > > bytes (e.g. their PKCS12 [5] for the map of PKCS7 signatures).
> > > > >> > >
> > > > >> > > 4) Authorization
> > > > >> > >
> > > > >> > > We should have a policy of "404" for data, topics, partitions
> > > (etc)
> > > > if
> > > > >> > > authenticated connections do not have access.  In "secure
> mode"
> > > any
> > > > >> non
> > > > >> > > authenticated connections should get a "404" type message on
> > > > >> everything.
> > > > >> > > Knowing "something is there" is a security risk in many uses
> > > cases.
> > > > >>  So
> > > > >> > if
> > > > >> > > you don't have access you don't even see it.  Baking "that"
> into
> > > > Kafka
> > > > >> > > along with some interface for entitlement (access management)
> > > > systems
> > > > >> > > (pretty standard) is all that I think needs to be done to the
> > core
> > > > >> > project.
> > > > >> > >  I want to tackle item later in the year after summer after
> the
> > > > other
> > > > >> > three
> > > > >> > > are complete.
> > > > >> > >
> > > > >> > > I look forward to thoughts on this and anyone else interested
> in
> > > > >> working
> > > > >> > > with us on these items.
> > > > >> > >
> > > > >> > > [0]
> > > > >> > >
> > > > >> > >
> > > > >> >
> > > > >>
> > > >
> > >
> >
> http://www.emc.com/emc-plus/rsa-labs/standards-initiatives/what-is-a-digital-envelope.htm
> > > > >> > > [1] http://tools.ietf.org/html/rfc3126
> > > > >> > > [2] http://tools.ietf.org/html/rfc3161
> > > > >> > > [3]
> > > > >> > >
> > > > >> > >
> > > > >> >
> > > > >>
> > > >
> > >
> >
> http://www.emc.com/emc-plus/rsa-labs/standards-initiatives/pkcs-7-cryptographic-message-syntax-standar.htm
> > > > >> > > [4] http://en.wikipedia.org/wiki/XML_Signature
> > > > >> > > [5] http://en.wikipedia.org/wiki/PKCS_12
> > > > >> > >
> > > > >> > > /*******************************************
> > > > >> > >  Joe Stein
> > > > >> > >  Founder, Principal Consultant
> > > > >> > >  Big Data Open Source Security LLC
> > > > >> > >  http://www.stealth.ly
> > > > >> > >  Twitter: @allthingshadoop <
> > > http://www.twitter.com/allthingshadoop>
> > > > >> > > ********************************************/
> > > > >> > >
> > > > >> >
> > > > >>
> > > > >
> > > > >
> > > >
> > >
> > >
> > >
> > > --
> > > Thanks,
> > > Raja.
> > >
> >
>
>
>
> --
> Thanks,
> Raja.
>

Re: [DISCUSS] Kafka Security Specific Features

Posted by Pramod Deshmukh <dp...@gmail.com>.
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 <re...@salesforce.com>
wrote:

> 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 <dp...@gmail.com>
> 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.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 config/server.properties*
> >
> >
> >
> > *broker.log:*
> >
> > [2014-07-17 15:34:46,281] INFO zookeeper state changed (SyncConnected)
> > (org.I0Itec.zkclient.ZkClient)
> >
> > [2014-07-17 15:34:46,523] INFO Loading log 'secure.test-0'
> > (kafka.log.LogManager)
> >
> > [2014-07-17 15:34:46,558] INFO Recovering unflushed segment 0 in log
> > secure.test-0. (kafka.log.Log)
> >
> > [2014-07-17 15:34:46,571] INFO Completed load of log secure.test-0 with
> log
> > end offset 0 (kafka.log.Log)
> >
> > [2014-07-17 15:34:46,582] INFO Starting log cleanup with a period of
> 60000
> > ms. (kafka.log.LogManager)
> >
> > [2014-07-17 15:34:46,587] INFO Starting log flusher with a default period
> > of 9223372036854775807 ms. (kafka.log.LogManager)
> >
> > [2014-07-17 15:34:46,614] INFO Initializing secure authentication
> > (kafka.network.security.SecureAuth$)
> >
> > [2014-07-17 15:34:46,678] INFO Secure authentication initialization has
> > been successfully completed (kafka.network.security.SecureAuth$)
> >
> > [2014-07-17 15:34:46,691] INFO Awaiting socket connections on
> 0.0.0.0:9092
> > .
> > (kafka.network.Acceptor)
> >
> > [2014-07-17 15:34:46,692] INFO [Socket Server on Broker 0], Started
> > (kafka.network.SocketServer)
> >
> > [2014-07-17 15:34:46,794] INFO Will not load MX4J, mx4j-tools.jar is not
> in
> > the classpath (kafka.utils.Mx4jLoader$)
> >
> > [2014-07-17 15:34:46,837] INFO 0 successfully elected as leader
> > (kafka.server.ZookeeperLeaderElector)
> >
> > [2014-07-17 15:34:47,057] INFO Registered broker 0 at path /brokers/ids/0
> > with address 10.1.100.130:9092. (kafka.utils.ZkUtils$)
> >
> > [2014-07-17 15:34:47,059] INFO New leader is 0
> > (kafka.server.ZookeeperLeaderElector$LeaderChangeListener)
> >
> > *[2014-07-17 15:34:47,068] INFO [Kafka Server 0], started
> > (kafka.server.KafkaServer)*
> >
> > *[2014-07-17 15:34:47,383] INFO begin ssl handshake for
> > /10.1.100.130:9092//10.1.100.130:51685
> > <http://10.1.100.130:9092//10.1.100.130:51685>
> > (kafka.network.security.SSLSocketChannel)*
> >
> > *[2014-07-17 15:34:47,392] INFO begin ssl handshake for
> > 10.1.100.130/10.1.100.130:51685//10.1.100.130:9092
> > <http://10.1.100.130/10.1.100.130:51685//10.1.100.130:9092>
> > (kafka.network.security.SSLSocketChannel)*
> >
> > *[2014-07-17 15:34:47,465] INFO finished ssl handshake for
> > 10.1.100.130/10.1.100.130:51685//10.1.100.130:9092
> > <http://10.1.100.130/10.1.100.130:51685//10.1.100.130:9092>
> > (kafka.network.security.SSLSocketChannel)*
> >
> > *[2014-07-17 15:34:47,465] INFO finished ssl handshake for
> > /10.1.100.130:9092//10.1.100.130:51685
> > <http://10.1.100.130:9092//10.1.100.130:51685>
> > (kafka.network.security.SSLSocketChannel)*
> >
> > *[2014-07-17 15:34:47,617] INFO [ReplicaFetcherManager on broker 0]
> Removed
> > fetcher for partitions  (kafka.server.ReplicaFetcherManager)*
> >
> > *[2014-07-17 15:34:47,627] INFO [ReplicaFetcherManager on broker 0] Added
> > fetcher for partitions List() (kafka.server.ReplicaFetcherManager)*
> >
> > *[2014-07-17 15:34:47,656] INFO [ReplicaFetcherManager on broker 0]
> Removed
> > fetcher for partitions [secure.test,0]
> > (kafka.server.ReplicaFetcherManager)*
> >
> > [2014-07-17 15:37:15,970] INFO begin ssl handshake for
> > 10.1.100.130/10.1.100.130:51689//10.1.100.130:9092
> > (kafka.network.security.SSLSocketChannel)
> >
> > [2014-07-17 15:37:16,075] INFO begin ssl handshake for
> > 10.1.100.130/10.1.100.130:51690//10.1.100.130:9092
> > (kafka.network.security.SSLSocketChannel)
> >
> > [2014-07-17 15:37:16,434] INFO begin ssl handshake for
> > 10.1.100.130/10.1.100.130:51691//10.1.100.130:9092
> > (kafka.network.security.SSLSocketChannel)
> >
> > [2014-07-17 15:37:16,530] INFO begin ssl handshake for
> > 10.1.100.130/10.1.100.130:51692//10.1.100.130:9092
> > (kafka.network.security.SSLSocketChannel)
> >
> > [2014-07-17 15:37:16,743] INFO begin ssl handshake for
> > 10.1.100.130/10.1.100.130:51693//10.1.100.130:9092
> > (kafka.network.security.SSLSocketChannel)
> >
> > [2014-07-17 15:37:16,834] INFO begin ssl handshake for
> > 10.1.100.130/10.1.100.130:51694//10.1.100.130:9092
> > (kafka.network.security.SSLSocketChannel)
> >
> > [2014-07-17 15:37:17,043] INFO begin ssl handshake for
> > 10.1.100.130/10.1.100.130:51695//10.1.100.130:9092
> > (kafka.network.security.SSLSocketChannel)
> >
> > [2014-07-17 15:37:17,137] INFO begin ssl handshake for
> > 10.1.100.130/10.1.100.130:51696//10.1.100.130:9092
> > (kafka.network.security.SSLSocketChannel)
> >
> > [2014-07-17 15:37:17,342] INFO begin ssl handshake for
> > 10.1.100.130/10.1.100.130:51697//10.1.100.130:9092
> > (kafka.network.security.SSLSocketChannel)
> >
> >
> > *Start producer*
> >
> > *bin/kafka-console-producer.sh --broker-list 10.1.100.130:9092:true
> > --topic
> > secure.test*
> >
> >
> > *producer.log:*
> >
> > bin/kafka-console-producer.sh --broker-list 10.1.100.130:9092:true
> --topic
> > secure.test
> >
> > [2014-07-17 15:37:46,889] WARN Property topic is not valid
> > (kafka.utils.VerifiableProperties)
> >
> > Hello Secure Kafka
> >
> > *[2014-07-17 15:38:14,186] ERROR OOME with size 352518400
> > (kafka.network.BoundedByteBufferReceive)*
> >
> > *java.lang.OutOfMemoryError: Java heap space*
> >
> > at java.nio.HeapByteBuffer.<init>(HeapByteBuffer.java:57)
> >
> > at java.nio.ByteBuffer.allocate(ByteBuffer.java:331)
> >
> > at
> >
> >
> kafka.network.BoundedByteBufferReceive.byteBufferAllocate(BoundedByteBufferReceive.scala:80)
> >
> > at
> >
> >
> kafka.network.BoundedByteBufferReceive.readFrom(BoundedByteBufferReceive.scala:63)
> >
> > at kafka.network.Receive$class.readCompletely(Transmission.scala:56)
> >
> > at
> >
> >
> kafka.network.BoundedByteBufferReceive.readCompletely(BoundedByteBufferReceive.scala:29)
> >
> > at kafka.network.BlockingChannel.receive(BlockingChannel.scala:102)
> >
> > at kafka.producer.SyncProducer.liftedTree1$1(SyncProducer.scala:79)
> >
> > at
> >
> >
> kafka.producer.SyncProducer.kafka$producer$SyncProducer$$doSend(SyncProducer.scala:76)
> >
> > at kafka.producer.SyncProducer.send(SyncProducer.scala:117)
> >
> > at kafka.client.ClientUtils$.fetchTopicMetadata(ClientUtils.scala:58)
> >
> > at
> >
> kafka.producer.BrokerPartitionInfo.updateInfo(BrokerPartitionInfo.scala:82)
> >
> > at
> >
> >
> kafka.producer.async.DefaultEventHandler$$anonfun$handle$1.apply$mcV$sp(DefaultEventHandler.scala:67)
> >
> > at kafka.utils.Utils$.swallow(Utils.scala:172)
> >
> > at kafka.utils.Logging$class.swallowError(Logging.scala:106)
> >
> > at kafka.utils.Utils$.swallowError(Utils.scala:45)
> >
> > at
> >
> >
> kafka.producer.async.DefaultEventHandler.handle(DefaultEventHandler.scala:67)
> >
> > at
> >
> >
> kafka.producer.async.ProducerSendThread.tryToHandle(ProducerSendThread.scala:104)
> >
> > at
> >
> >
> kafka.producer.async.ProducerSendThread$$anonfun$processEvents$3.apply(ProducerSendThread.scala:87)
> >
> > at
> >
> >
> kafka.producer.async.ProducerSendThread$$anonfun$processEvents$3.apply(ProducerSendThread.scala:67)
> >
> > at scala.collection.immutable.Stream.foreach(Stream.scala:526)
> >
> > at
> >
> >
> kafka.producer.async.ProducerSendThread.processEvents(ProducerSendThread.scala:66)
> >
> > at
> kafka.producer.async.ProducerSendThread.run(ProducerSendThread.scala:44)
> >
> >
> >
> > On Wed, Jul 16, 2014 at 6:07 PM, Rajasekar Elango <
> relango@salesforce.com>
> > wrote:
> >
> > > Pramod,
> > >
> > >
> > > I presented secure kafka configuration and usage at last meet up. So
> hope
> > > this
> > > video recording <http://www.ustream.tv/recorded/48396701>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 <dp...@gmail.com>
> > > wrote:
> > >
> > > > 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 <dpramodv@gmail.com
> >
> > > > 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, Jul 16, 2014 at 3:57 PM, Joe Stein <jo...@stealth.ly>
> > > 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 (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 Stein
> > > > >>  Founder, Principal Consultant
> > > > >>  Big Data Open Source Security LLC
> > > > >>  http://www.stealth.ly
> > > > >>  Twitter: @allthingshadoop <
> http://www.twitter.com/allthingshadoop>
> > > > >> ********************************************/
> > > > >>
> > > > >>
> > > > >> On Wed, Jul 16, 2014 at 3:49 PM, Pramod Deshmukh <
> > dpramodv@gmail.com>
> > > > >> wrote:
> > > > >>
> > > > >> > 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 installed sbt and scala and followed your instructions
> for
> > > > >> different
> > > > >> > scala versions. I was able to bring zookeeper up but brokers
> fail
> > to
> > > > >> start
> > > > >> > with error
> > > > >> >
> > > > >> > Error: Could not find or load main class kafka.Kafka
> > > > >> >
> > > > >> > I think I am doing something wrong. Can you please help me?
> > > > >> >
> > > > >> > Our current production setup is with 2.8.0 and want to stick to
> > it.
> > > > >> >
> > > > >> > Thanks,
> > > > >> >
> > > > >> > Pramod
> > > > >> >
> > > > >> >
> > > > >> > On Tue, Jun 3, 2014 at 3:57 PM, Joe Stein <joe.stein@stealth.ly
> >
> > > > wrote:
> > > > >> >
> > > > >> > > 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.  This comes from what folks are doing "in the
> > wild"
> > > to
> > > > >> > > workaround and implement security with Kafka as it is today
> and
> > > also
> > > > >> > what I
> > > > >> > > have discovered from organizations about their blockers. It
> also
> > > > >> picks up
> > > > >> > > from the wiki (which I should have time to update later in the
> > > week
> > > > >> based
> > > > >> > > on the below and feedback from the thread).
> > > > >> > >
> > > > >> > > 1) Transport Layer Security (i.e. SSL)
> > > > >> > >
> > > > >> > > This also includes client authentication in addition to
> > in-transit
> > > > >> > security
> > > > >> > > layer.  This work has been picked up here
> > > > >> > > https://issues.apache.org/jira/browse/KAFKA-1477 and do
> > > appreciate
> > > > >> any
> > > > >> > > thoughts, comments, feedback, tomatoes, whatever for this
> patch.
> > >  It
> > > > >> is a
> > > > >> > > pickup from the fork of the work first done here
> > > > >> > > https://github.com/relango/kafka/tree/kafka_security.
> > > > >> > >
> > > > >> > > 2) Data encryption at rest.
> > > > >> > >
> > > > >> > > This is very important and something that can be facilitated
> > > within
> > > > >> the
> > > > >> > > wire protocol. It requires an additional map data structure
> for
> > > the
> > > > >> > > "encrypted [data encryption key]". With this map (either in
> your
> > > > >> object
> > > > >> > or
> > > > >> > > in the wire protocol) you can store the dynamically generated
> > > > >> symmetric
> > > > >> > key
> > > > >> > > (for each message) and then encrypt the data using that
> > > dynamically
> > > > >> > > generated key.  You then encrypt the encryption key using each
> > > > public
> > > > >> key
> > > > >> > > for whom is expected to be able to decrypt the encryption key
> to
> > > > then
> > > > >> > > decrypt the message.  For each public key encrypted symmetric
> > key
> > > > >> (which
> > > > >> > is
> > > > >> > > now the "encrypted [data encryption key]" along with which
> > public
> > > > key
> > > > >> it
> > > > >> > > was encrypted with for (so a map of [publicKey] =
> > > > >> > > encryptedDataEncryptionKey) as a chain.   Other patterns can
> be
> > > > >> > implemented
> > > > >> > > but this is a pretty standard digital enveloping [0] pattern
> > with
> > > > >> only 1
> > > > >> > > field added. Other patterns should be able to use that field
> > to-do
> > > > >> their
> > > > >> > > implementation too.
> > > > >> > >
> > > > >> > > 3) Non-repudiation and long term non-repudiation.
> > > > >> > >
> > > > >> > > Non-repudiation is proving data hasn't changed.  This is often
> > (if
> > > > not
> > > > >> > > always) done with x509 public certificates (chained to a
> > > certificate
> > > > >> > > authority).
> > > > >> > >
> > > > >> > > Long term non-repudiation is what happens when the
> certificates
> > of
> > > > the
> > > > >> > > certificate authority are expired (or revoked) and everything
> > ever
> > > > >> signed
> > > > >> > > (ever) with that certificate's public key then becomes "no
> > longer
> > > > >> > provable
> > > > >> > > as ever being authentic".  That is where RFC3126 [1] and
> RFC3161
> > > [2]
> > > > >> come
> > > > >> > > in (or worm drives [hardware], etc).
> > > > >> > >
> > > > >> > > For either (or both) of these it is an operation of the
> > encryptor
> > > to
> > > > >> > > sign/hash the data (with or without third party trusted
> timestap
> > > of
> > > > >> the
> > > > >> > > signing event) and encrypt that with their own private key and
> > > > >> distribute
> > > > >> > > the results (before and after encrypting if required) along
> with
> > > > their
> > > > >> > > public key. This structure is a bit more complex but feasible,
> > it
> > > > is a
> > > > >> > map
> > > > >> > > of digital signature formats and the chain of dig sig
> > > attestations.
> > > > >>  The
> > > > >> > > map's key being the method (i.e. CRC32, PKCS7 [3], XmlDigSig
> > [4])
> > > > and
> > > > >> > then
> > > > >> > > a list of map where that key is "purpose" of signature (what
> > your
> > > > >> > attesting
> > > > >> > > too).  As a sibling field to the list another field for "the
> > > > >> attester" as
> > > > >> > > bytes (e.g. their PKCS12 [5] for the map of PKCS7 signatures).
> > > > >> > >
> > > > >> > > 4) Authorization
> > > > >> > >
> > > > >> > > We should have a policy of "404" for data, topics, partitions
> > > (etc)
> > > > if
> > > > >> > > authenticated connections do not have access.  In "secure
> mode"
> > > any
> > > > >> non
> > > > >> > > authenticated connections should get a "404" type message on
> > > > >> everything.
> > > > >> > > Knowing "something is there" is a security risk in many uses
> > > cases.
> > > > >>  So
> > > > >> > if
> > > > >> > > you don't have access you don't even see it.  Baking "that"
> into
> > > > Kafka
> > > > >> > > along with some interface for entitlement (access management)
> > > > systems
> > > > >> > > (pretty standard) is all that I think needs to be done to the
> > core
> > > > >> > project.
> > > > >> > >  I want to tackle item later in the year after summer after
> the
> > > > other
> > > > >> > three
> > > > >> > > are complete.
> > > > >> > >
> > > > >> > > I look forward to thoughts on this and anyone else interested
> in
> > > > >> working
> > > > >> > > with us on these items.
> > > > >> > >
> > > > >> > > [0]
> > > > >> > >
> > > > >> > >
> > > > >> >
> > > > >>
> > > >
> > >
> >
> http://www.emc.com/emc-plus/rsa-labs/standards-initiatives/what-is-a-digital-envelope.htm
> > > > >> > > [1] http://tools.ietf.org/html/rfc3126
> > > > >> > > [2] http://tools.ietf.org/html/rfc3161
> > > > >> > > [3]
> > > > >> > >
> > > > >> > >
> > > > >> >
> > > > >>
> > > >
> > >
> >
> http://www.emc.com/emc-plus/rsa-labs/standards-initiatives/pkcs-7-cryptographic-message-syntax-standar.htm
> > > > >> > > [4] http://en.wikipedia.org/wiki/XML_Signature
> > > > >> > > [5] http://en.wikipedia.org/wiki/PKCS_12
> > > > >> > >
> > > > >> > > /*******************************************
> > > > >> > >  Joe Stein
> > > > >> > >  Founder, Principal Consultant
> > > > >> > >  Big Data Open Source Security LLC
> > > > >> > >  http://www.stealth.ly
> > > > >> > >  Twitter: @allthingshadoop <
> > > http://www.twitter.com/allthingshadoop>
> > > > >> > > ********************************************/
> > > > >> > >
> > > > >> >
> > > > >>
> > > > >
> > > > >
> > > >
> > >
> > >
> > >
> > > --
> > > Thanks,
> > > Raja.
> > >
> >
>
>
>
> --
> Thanks,
> Raja.
>

Re: [DISCUSS] Kafka Security Specific Features

Posted by Rajasekar Elango <re...@salesforce.com>.
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 <dp...@gmail.com> 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.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 config/server.properties*
>
>
>
> *broker.log:*
>
> [2014-07-17 15:34:46,281] INFO zookeeper state changed (SyncConnected)
> (org.I0Itec.zkclient.ZkClient)
>
> [2014-07-17 15:34:46,523] INFO Loading log 'secure.test-0'
> (kafka.log.LogManager)
>
> [2014-07-17 15:34:46,558] INFO Recovering unflushed segment 0 in log
> secure.test-0. (kafka.log.Log)
>
> [2014-07-17 15:34:46,571] INFO Completed load of log secure.test-0 with log
> end offset 0 (kafka.log.Log)
>
> [2014-07-17 15:34:46,582] INFO Starting log cleanup with a period of 60000
> ms. (kafka.log.LogManager)
>
> [2014-07-17 15:34:46,587] INFO Starting log flusher with a default period
> of 9223372036854775807 ms. (kafka.log.LogManager)
>
> [2014-07-17 15:34:46,614] INFO Initializing secure authentication
> (kafka.network.security.SecureAuth$)
>
> [2014-07-17 15:34:46,678] INFO Secure authentication initialization has
> been successfully completed (kafka.network.security.SecureAuth$)
>
> [2014-07-17 15:34:46,691] INFO Awaiting socket connections on 0.0.0.0:9092
> .
> (kafka.network.Acceptor)
>
> [2014-07-17 15:34:46,692] INFO [Socket Server on Broker 0], Started
> (kafka.network.SocketServer)
>
> [2014-07-17 15:34:46,794] INFO Will not load MX4J, mx4j-tools.jar is not in
> the classpath (kafka.utils.Mx4jLoader$)
>
> [2014-07-17 15:34:46,837] INFO 0 successfully elected as leader
> (kafka.server.ZookeeperLeaderElector)
>
> [2014-07-17 15:34:47,057] INFO Registered broker 0 at path /brokers/ids/0
> with address 10.1.100.130:9092. (kafka.utils.ZkUtils$)
>
> [2014-07-17 15:34:47,059] INFO New leader is 0
> (kafka.server.ZookeeperLeaderElector$LeaderChangeListener)
>
> *[2014-07-17 15:34:47,068] INFO [Kafka Server 0], started
> (kafka.server.KafkaServer)*
>
> *[2014-07-17 15:34:47,383] INFO begin ssl handshake for
> /10.1.100.130:9092//10.1.100.130:51685
> <http://10.1.100.130:9092//10.1.100.130:51685>
> (kafka.network.security.SSLSocketChannel)*
>
> *[2014-07-17 15:34:47,392] INFO begin ssl handshake for
> 10.1.100.130/10.1.100.130:51685//10.1.100.130:9092
> <http://10.1.100.130/10.1.100.130:51685//10.1.100.130:9092>
> (kafka.network.security.SSLSocketChannel)*
>
> *[2014-07-17 15:34:47,465] INFO finished ssl handshake for
> 10.1.100.130/10.1.100.130:51685//10.1.100.130:9092
> <http://10.1.100.130/10.1.100.130:51685//10.1.100.130:9092>
> (kafka.network.security.SSLSocketChannel)*
>
> *[2014-07-17 15:34:47,465] INFO finished ssl handshake for
> /10.1.100.130:9092//10.1.100.130:51685
> <http://10.1.100.130:9092//10.1.100.130:51685>
> (kafka.network.security.SSLSocketChannel)*
>
> *[2014-07-17 15:34:47,617] INFO [ReplicaFetcherManager on broker 0] Removed
> fetcher for partitions  (kafka.server.ReplicaFetcherManager)*
>
> *[2014-07-17 15:34:47,627] INFO [ReplicaFetcherManager on broker 0] Added
> fetcher for partitions List() (kafka.server.ReplicaFetcherManager)*
>
> *[2014-07-17 15:34:47,656] INFO [ReplicaFetcherManager on broker 0] Removed
> fetcher for partitions [secure.test,0]
> (kafka.server.ReplicaFetcherManager)*
>
> [2014-07-17 15:37:15,970] INFO begin ssl handshake for
> 10.1.100.130/10.1.100.130:51689//10.1.100.130:9092
> (kafka.network.security.SSLSocketChannel)
>
> [2014-07-17 15:37:16,075] INFO begin ssl handshake for
> 10.1.100.130/10.1.100.130:51690//10.1.100.130:9092
> (kafka.network.security.SSLSocketChannel)
>
> [2014-07-17 15:37:16,434] INFO begin ssl handshake for
> 10.1.100.130/10.1.100.130:51691//10.1.100.130:9092
> (kafka.network.security.SSLSocketChannel)
>
> [2014-07-17 15:37:16,530] INFO begin ssl handshake for
> 10.1.100.130/10.1.100.130:51692//10.1.100.130:9092
> (kafka.network.security.SSLSocketChannel)
>
> [2014-07-17 15:37:16,743] INFO begin ssl handshake for
> 10.1.100.130/10.1.100.130:51693//10.1.100.130:9092
> (kafka.network.security.SSLSocketChannel)
>
> [2014-07-17 15:37:16,834] INFO begin ssl handshake for
> 10.1.100.130/10.1.100.130:51694//10.1.100.130:9092
> (kafka.network.security.SSLSocketChannel)
>
> [2014-07-17 15:37:17,043] INFO begin ssl handshake for
> 10.1.100.130/10.1.100.130:51695//10.1.100.130:9092
> (kafka.network.security.SSLSocketChannel)
>
> [2014-07-17 15:37:17,137] INFO begin ssl handshake for
> 10.1.100.130/10.1.100.130:51696//10.1.100.130:9092
> (kafka.network.security.SSLSocketChannel)
>
> [2014-07-17 15:37:17,342] INFO begin ssl handshake for
> 10.1.100.130/10.1.100.130:51697//10.1.100.130:9092
> (kafka.network.security.SSLSocketChannel)
>
>
> *Start producer*
>
> *bin/kafka-console-producer.sh --broker-list 10.1.100.130:9092:true
> --topic
> secure.test*
>
>
> *producer.log:*
>
> bin/kafka-console-producer.sh --broker-list 10.1.100.130:9092:true --topic
> secure.test
>
> [2014-07-17 15:37:46,889] WARN Property topic is not valid
> (kafka.utils.VerifiableProperties)
>
> Hello Secure Kafka
>
> *[2014-07-17 15:38:14,186] ERROR OOME with size 352518400
> (kafka.network.BoundedByteBufferReceive)*
>
> *java.lang.OutOfMemoryError: Java heap space*
>
> at java.nio.HeapByteBuffer.<init>(HeapByteBuffer.java:57)
>
> at java.nio.ByteBuffer.allocate(ByteBuffer.java:331)
>
> at
>
> kafka.network.BoundedByteBufferReceive.byteBufferAllocate(BoundedByteBufferReceive.scala:80)
>
> at
>
> kafka.network.BoundedByteBufferReceive.readFrom(BoundedByteBufferReceive.scala:63)
>
> at kafka.network.Receive$class.readCompletely(Transmission.scala:56)
>
> at
>
> kafka.network.BoundedByteBufferReceive.readCompletely(BoundedByteBufferReceive.scala:29)
>
> at kafka.network.BlockingChannel.receive(BlockingChannel.scala:102)
>
> at kafka.producer.SyncProducer.liftedTree1$1(SyncProducer.scala:79)
>
> at
>
> kafka.producer.SyncProducer.kafka$producer$SyncProducer$$doSend(SyncProducer.scala:76)
>
> at kafka.producer.SyncProducer.send(SyncProducer.scala:117)
>
> at kafka.client.ClientUtils$.fetchTopicMetadata(ClientUtils.scala:58)
>
> at
> kafka.producer.BrokerPartitionInfo.updateInfo(BrokerPartitionInfo.scala:82)
>
> at
>
> kafka.producer.async.DefaultEventHandler$$anonfun$handle$1.apply$mcV$sp(DefaultEventHandler.scala:67)
>
> at kafka.utils.Utils$.swallow(Utils.scala:172)
>
> at kafka.utils.Logging$class.swallowError(Logging.scala:106)
>
> at kafka.utils.Utils$.swallowError(Utils.scala:45)
>
> at
>
> kafka.producer.async.DefaultEventHandler.handle(DefaultEventHandler.scala:67)
>
> at
>
> kafka.producer.async.ProducerSendThread.tryToHandle(ProducerSendThread.scala:104)
>
> at
>
> kafka.producer.async.ProducerSendThread$$anonfun$processEvents$3.apply(ProducerSendThread.scala:87)
>
> at
>
> kafka.producer.async.ProducerSendThread$$anonfun$processEvents$3.apply(ProducerSendThread.scala:67)
>
> at scala.collection.immutable.Stream.foreach(Stream.scala:526)
>
> at
>
> kafka.producer.async.ProducerSendThread.processEvents(ProducerSendThread.scala:66)
>
> at kafka.producer.async.ProducerSendThread.run(ProducerSendThread.scala:44)
>
>
>
> On Wed, Jul 16, 2014 at 6:07 PM, Rajasekar Elango <re...@salesforce.com>
> wrote:
>
> > Pramod,
> >
> >
> > I presented secure kafka configuration and usage at last meet up. So hope
> > this
> > video recording <http://www.ustream.tv/recorded/48396701>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 <dp...@gmail.com>
> > wrote:
> >
> > > 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 <dp...@gmail.com>
> > > 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, Jul 16, 2014 at 3:57 PM, Joe Stein <jo...@stealth.ly>
> > 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 (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 Stein
> > > >>  Founder, Principal Consultant
> > > >>  Big Data Open Source Security LLC
> > > >>  http://www.stealth.ly
> > > >>  Twitter: @allthingshadoop <http://www.twitter.com/allthingshadoop>
> > > >> ********************************************/
> > > >>
> > > >>
> > > >> On Wed, Jul 16, 2014 at 3:49 PM, Pramod Deshmukh <
> dpramodv@gmail.com>
> > > >> wrote:
> > > >>
> > > >> > 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 installed sbt and scala and followed your instructions for
> > > >> different
> > > >> > scala versions. I was able to bring zookeeper up but brokers fail
> to
> > > >> start
> > > >> > with error
> > > >> >
> > > >> > Error: Could not find or load main class kafka.Kafka
> > > >> >
> > > >> > I think I am doing something wrong. Can you please help me?
> > > >> >
> > > >> > Our current production setup is with 2.8.0 and want to stick to
> it.
> > > >> >
> > > >> > Thanks,
> > > >> >
> > > >> > Pramod
> > > >> >
> > > >> >
> > > >> > On Tue, Jun 3, 2014 at 3:57 PM, Joe Stein <jo...@stealth.ly>
> > > wrote:
> > > >> >
> > > >> > > 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.  This comes from what folks are doing "in the
> wild"
> > to
> > > >> > > workaround and implement security with Kafka as it is today and
> > also
> > > >> > what I
> > > >> > > have discovered from organizations about their blockers. It also
> > > >> picks up
> > > >> > > from the wiki (which I should have time to update later in the
> > week
> > > >> based
> > > >> > > on the below and feedback from the thread).
> > > >> > >
> > > >> > > 1) Transport Layer Security (i.e. SSL)
> > > >> > >
> > > >> > > This also includes client authentication in addition to
> in-transit
> > > >> > security
> > > >> > > layer.  This work has been picked up here
> > > >> > > https://issues.apache.org/jira/browse/KAFKA-1477 and do
> > appreciate
> > > >> any
> > > >> > > thoughts, comments, feedback, tomatoes, whatever for this patch.
> >  It
> > > >> is a
> > > >> > > pickup from the fork of the work first done here
> > > >> > > https://github.com/relango/kafka/tree/kafka_security.
> > > >> > >
> > > >> > > 2) Data encryption at rest.
> > > >> > >
> > > >> > > This is very important and something that can be facilitated
> > within
> > > >> the
> > > >> > > wire protocol. It requires an additional map data structure for
> > the
> > > >> > > "encrypted [data encryption key]". With this map (either in your
> > > >> object
> > > >> > or
> > > >> > > in the wire protocol) you can store the dynamically generated
> > > >> symmetric
> > > >> > key
> > > >> > > (for each message) and then encrypt the data using that
> > dynamically
> > > >> > > generated key.  You then encrypt the encryption key using each
> > > public
> > > >> key
> > > >> > > for whom is expected to be able to decrypt the encryption key to
> > > then
> > > >> > > decrypt the message.  For each public key encrypted symmetric
> key
> > > >> (which
> > > >> > is
> > > >> > > now the "encrypted [data encryption key]" along with which
> public
> > > key
> > > >> it
> > > >> > > was encrypted with for (so a map of [publicKey] =
> > > >> > > encryptedDataEncryptionKey) as a chain.   Other patterns can be
> > > >> > implemented
> > > >> > > but this is a pretty standard digital enveloping [0] pattern
> with
> > > >> only 1
> > > >> > > field added. Other patterns should be able to use that field
> to-do
> > > >> their
> > > >> > > implementation too.
> > > >> > >
> > > >> > > 3) Non-repudiation and long term non-repudiation.
> > > >> > >
> > > >> > > Non-repudiation is proving data hasn't changed.  This is often
> (if
> > > not
> > > >> > > always) done with x509 public certificates (chained to a
> > certificate
> > > >> > > authority).
> > > >> > >
> > > >> > > Long term non-repudiation is what happens when the certificates
> of
> > > the
> > > >> > > certificate authority are expired (or revoked) and everything
> ever
> > > >> signed
> > > >> > > (ever) with that certificate's public key then becomes "no
> longer
> > > >> > provable
> > > >> > > as ever being authentic".  That is where RFC3126 [1] and RFC3161
> > [2]
> > > >> come
> > > >> > > in (or worm drives [hardware], etc).
> > > >> > >
> > > >> > > For either (or both) of these it is an operation of the
> encryptor
> > to
> > > >> > > sign/hash the data (with or without third party trusted timestap
> > of
> > > >> the
> > > >> > > signing event) and encrypt that with their own private key and
> > > >> distribute
> > > >> > > the results (before and after encrypting if required) along with
> > > their
> > > >> > > public key. This structure is a bit more complex but feasible,
> it
> > > is a
> > > >> > map
> > > >> > > of digital signature formats and the chain of dig sig
> > attestations.
> > > >>  The
> > > >> > > map's key being the method (i.e. CRC32, PKCS7 [3], XmlDigSig
> [4])
> > > and
> > > >> > then
> > > >> > > a list of map where that key is "purpose" of signature (what
> your
> > > >> > attesting
> > > >> > > too).  As a sibling field to the list another field for "the
> > > >> attester" as
> > > >> > > bytes (e.g. their PKCS12 [5] for the map of PKCS7 signatures).
> > > >> > >
> > > >> > > 4) Authorization
> > > >> > >
> > > >> > > We should have a policy of "404" for data, topics, partitions
> > (etc)
> > > if
> > > >> > > authenticated connections do not have access.  In "secure mode"
> > any
> > > >> non
> > > >> > > authenticated connections should get a "404" type message on
> > > >> everything.
> > > >> > > Knowing "something is there" is a security risk in many uses
> > cases.
> > > >>  So
> > > >> > if
> > > >> > > you don't have access you don't even see it.  Baking "that" into
> > > Kafka
> > > >> > > along with some interface for entitlement (access management)
> > > systems
> > > >> > > (pretty standard) is all that I think needs to be done to the
> core
> > > >> > project.
> > > >> > >  I want to tackle item later in the year after summer after the
> > > other
> > > >> > three
> > > >> > > are complete.
> > > >> > >
> > > >> > > I look forward to thoughts on this and anyone else interested in
> > > >> working
> > > >> > > with us on these items.
> > > >> > >
> > > >> > > [0]
> > > >> > >
> > > >> > >
> > > >> >
> > > >>
> > >
> >
> http://www.emc.com/emc-plus/rsa-labs/standards-initiatives/what-is-a-digital-envelope.htm
> > > >> > > [1] http://tools.ietf.org/html/rfc3126
> > > >> > > [2] http://tools.ietf.org/html/rfc3161
> > > >> > > [3]
> > > >> > >
> > > >> > >
> > > >> >
> > > >>
> > >
> >
> http://www.emc.com/emc-plus/rsa-labs/standards-initiatives/pkcs-7-cryptographic-message-syntax-standar.htm
> > > >> > > [4] http://en.wikipedia.org/wiki/XML_Signature
> > > >> > > [5] http://en.wikipedia.org/wiki/PKCS_12
> > > >> > >
> > > >> > > /*******************************************
> > > >> > >  Joe Stein
> > > >> > >  Founder, Principal Consultant
> > > >> > >  Big Data Open Source Security LLC
> > > >> > >  http://www.stealth.ly
> > > >> > >  Twitter: @allthingshadoop <
> > http://www.twitter.com/allthingshadoop>
> > > >> > > ********************************************/
> > > >> > >
> > > >> >
> > > >>
> > > >
> > > >
> > >
> >
> >
> >
> > --
> > Thanks,
> > Raja.
> >
>



-- 
Thanks,
Raja.

Re: [DISCUSS] Kafka Security Specific Features

Posted by Rajasekar Elango <re...@salesforce.com>.
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 <dp...@gmail.com> 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.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 config/server.properties*
>
>
>
> *broker.log:*
>
> [2014-07-17 15:34:46,281] INFO zookeeper state changed (SyncConnected)
> (org.I0Itec.zkclient.ZkClient)
>
> [2014-07-17 15:34:46,523] INFO Loading log 'secure.test-0'
> (kafka.log.LogManager)
>
> [2014-07-17 15:34:46,558] INFO Recovering unflushed segment 0 in log
> secure.test-0. (kafka.log.Log)
>
> [2014-07-17 15:34:46,571] INFO Completed load of log secure.test-0 with log
> end offset 0 (kafka.log.Log)
>
> [2014-07-17 15:34:46,582] INFO Starting log cleanup with a period of 60000
> ms. (kafka.log.LogManager)
>
> [2014-07-17 15:34:46,587] INFO Starting log flusher with a default period
> of 9223372036854775807 ms. (kafka.log.LogManager)
>
> [2014-07-17 15:34:46,614] INFO Initializing secure authentication
> (kafka.network.security.SecureAuth$)
>
> [2014-07-17 15:34:46,678] INFO Secure authentication initialization has
> been successfully completed (kafka.network.security.SecureAuth$)
>
> [2014-07-17 15:34:46,691] INFO Awaiting socket connections on 0.0.0.0:9092
> .
> (kafka.network.Acceptor)
>
> [2014-07-17 15:34:46,692] INFO [Socket Server on Broker 0], Started
> (kafka.network.SocketServer)
>
> [2014-07-17 15:34:46,794] INFO Will not load MX4J, mx4j-tools.jar is not in
> the classpath (kafka.utils.Mx4jLoader$)
>
> [2014-07-17 15:34:46,837] INFO 0 successfully elected as leader
> (kafka.server.ZookeeperLeaderElector)
>
> [2014-07-17 15:34:47,057] INFO Registered broker 0 at path /brokers/ids/0
> with address 10.1.100.130:9092. (kafka.utils.ZkUtils$)
>
> [2014-07-17 15:34:47,059] INFO New leader is 0
> (kafka.server.ZookeeperLeaderElector$LeaderChangeListener)
>
> *[2014-07-17 15:34:47,068] INFO [Kafka Server 0], started
> (kafka.server.KafkaServer)*
>
> *[2014-07-17 15:34:47,383] INFO begin ssl handshake for
> /10.1.100.130:9092//10.1.100.130:51685
> <http://10.1.100.130:9092//10.1.100.130:51685>
> (kafka.network.security.SSLSocketChannel)*
>
> *[2014-07-17 15:34:47,392] INFO begin ssl handshake for
> 10.1.100.130/10.1.100.130:51685//10.1.100.130:9092
> <http://10.1.100.130/10.1.100.130:51685//10.1.100.130:9092>
> (kafka.network.security.SSLSocketChannel)*
>
> *[2014-07-17 15:34:47,465] INFO finished ssl handshake for
> 10.1.100.130/10.1.100.130:51685//10.1.100.130:9092
> <http://10.1.100.130/10.1.100.130:51685//10.1.100.130:9092>
> (kafka.network.security.SSLSocketChannel)*
>
> *[2014-07-17 15:34:47,465] INFO finished ssl handshake for
> /10.1.100.130:9092//10.1.100.130:51685
> <http://10.1.100.130:9092//10.1.100.130:51685>
> (kafka.network.security.SSLSocketChannel)*
>
> *[2014-07-17 15:34:47,617] INFO [ReplicaFetcherManager on broker 0] Removed
> fetcher for partitions  (kafka.server.ReplicaFetcherManager)*
>
> *[2014-07-17 15:34:47,627] INFO [ReplicaFetcherManager on broker 0] Added
> fetcher for partitions List() (kafka.server.ReplicaFetcherManager)*
>
> *[2014-07-17 15:34:47,656] INFO [ReplicaFetcherManager on broker 0] Removed
> fetcher for partitions [secure.test,0]
> (kafka.server.ReplicaFetcherManager)*
>
> [2014-07-17 15:37:15,970] INFO begin ssl handshake for
> 10.1.100.130/10.1.100.130:51689//10.1.100.130:9092
> (kafka.network.security.SSLSocketChannel)
>
> [2014-07-17 15:37:16,075] INFO begin ssl handshake for
> 10.1.100.130/10.1.100.130:51690//10.1.100.130:9092
> (kafka.network.security.SSLSocketChannel)
>
> [2014-07-17 15:37:16,434] INFO begin ssl handshake for
> 10.1.100.130/10.1.100.130:51691//10.1.100.130:9092
> (kafka.network.security.SSLSocketChannel)
>
> [2014-07-17 15:37:16,530] INFO begin ssl handshake for
> 10.1.100.130/10.1.100.130:51692//10.1.100.130:9092
> (kafka.network.security.SSLSocketChannel)
>
> [2014-07-17 15:37:16,743] INFO begin ssl handshake for
> 10.1.100.130/10.1.100.130:51693//10.1.100.130:9092
> (kafka.network.security.SSLSocketChannel)
>
> [2014-07-17 15:37:16,834] INFO begin ssl handshake for
> 10.1.100.130/10.1.100.130:51694//10.1.100.130:9092
> (kafka.network.security.SSLSocketChannel)
>
> [2014-07-17 15:37:17,043] INFO begin ssl handshake for
> 10.1.100.130/10.1.100.130:51695//10.1.100.130:9092
> (kafka.network.security.SSLSocketChannel)
>
> [2014-07-17 15:37:17,137] INFO begin ssl handshake for
> 10.1.100.130/10.1.100.130:51696//10.1.100.130:9092
> (kafka.network.security.SSLSocketChannel)
>
> [2014-07-17 15:37:17,342] INFO begin ssl handshake for
> 10.1.100.130/10.1.100.130:51697//10.1.100.130:9092
> (kafka.network.security.SSLSocketChannel)
>
>
> *Start producer*
>
> *bin/kafka-console-producer.sh --broker-list 10.1.100.130:9092:true
> --topic
> secure.test*
>
>
> *producer.log:*
>
> bin/kafka-console-producer.sh --broker-list 10.1.100.130:9092:true --topic
> secure.test
>
> [2014-07-17 15:37:46,889] WARN Property topic is not valid
> (kafka.utils.VerifiableProperties)
>
> Hello Secure Kafka
>
> *[2014-07-17 15:38:14,186] ERROR OOME with size 352518400
> (kafka.network.BoundedByteBufferReceive)*
>
> *java.lang.OutOfMemoryError: Java heap space*
>
> at java.nio.HeapByteBuffer.<init>(HeapByteBuffer.java:57)
>
> at java.nio.ByteBuffer.allocate(ByteBuffer.java:331)
>
> at
>
> kafka.network.BoundedByteBufferReceive.byteBufferAllocate(BoundedByteBufferReceive.scala:80)
>
> at
>
> kafka.network.BoundedByteBufferReceive.readFrom(BoundedByteBufferReceive.scala:63)
>
> at kafka.network.Receive$class.readCompletely(Transmission.scala:56)
>
> at
>
> kafka.network.BoundedByteBufferReceive.readCompletely(BoundedByteBufferReceive.scala:29)
>
> at kafka.network.BlockingChannel.receive(BlockingChannel.scala:102)
>
> at kafka.producer.SyncProducer.liftedTree1$1(SyncProducer.scala:79)
>
> at
>
> kafka.producer.SyncProducer.kafka$producer$SyncProducer$$doSend(SyncProducer.scala:76)
>
> at kafka.producer.SyncProducer.send(SyncProducer.scala:117)
>
> at kafka.client.ClientUtils$.fetchTopicMetadata(ClientUtils.scala:58)
>
> at
> kafka.producer.BrokerPartitionInfo.updateInfo(BrokerPartitionInfo.scala:82)
>
> at
>
> kafka.producer.async.DefaultEventHandler$$anonfun$handle$1.apply$mcV$sp(DefaultEventHandler.scala:67)
>
> at kafka.utils.Utils$.swallow(Utils.scala:172)
>
> at kafka.utils.Logging$class.swallowError(Logging.scala:106)
>
> at kafka.utils.Utils$.swallowError(Utils.scala:45)
>
> at
>
> kafka.producer.async.DefaultEventHandler.handle(DefaultEventHandler.scala:67)
>
> at
>
> kafka.producer.async.ProducerSendThread.tryToHandle(ProducerSendThread.scala:104)
>
> at
>
> kafka.producer.async.ProducerSendThread$$anonfun$processEvents$3.apply(ProducerSendThread.scala:87)
>
> at
>
> kafka.producer.async.ProducerSendThread$$anonfun$processEvents$3.apply(ProducerSendThread.scala:67)
>
> at scala.collection.immutable.Stream.foreach(Stream.scala:526)
>
> at
>
> kafka.producer.async.ProducerSendThread.processEvents(ProducerSendThread.scala:66)
>
> at kafka.producer.async.ProducerSendThread.run(ProducerSendThread.scala:44)
>
>
>
> On Wed, Jul 16, 2014 at 6:07 PM, Rajasekar Elango <re...@salesforce.com>
> wrote:
>
> > Pramod,
> >
> >
> > I presented secure kafka configuration and usage at last meet up. So hope
> > this
> > video recording <http://www.ustream.tv/recorded/48396701>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 <dp...@gmail.com>
> > wrote:
> >
> > > 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 <dp...@gmail.com>
> > > 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, Jul 16, 2014 at 3:57 PM, Joe Stein <jo...@stealth.ly>
> > 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 (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 Stein
> > > >>  Founder, Principal Consultant
> > > >>  Big Data Open Source Security LLC
> > > >>  http://www.stealth.ly
> > > >>  Twitter: @allthingshadoop <http://www.twitter.com/allthingshadoop>
> > > >> ********************************************/
> > > >>
> > > >>
> > > >> On Wed, Jul 16, 2014 at 3:49 PM, Pramod Deshmukh <
> dpramodv@gmail.com>
> > > >> wrote:
> > > >>
> > > >> > 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 installed sbt and scala and followed your instructions for
> > > >> different
> > > >> > scala versions. I was able to bring zookeeper up but brokers fail
> to
> > > >> start
> > > >> > with error
> > > >> >
> > > >> > Error: Could not find or load main class kafka.Kafka
> > > >> >
> > > >> > I think I am doing something wrong. Can you please help me?
> > > >> >
> > > >> > Our current production setup is with 2.8.0 and want to stick to
> it.
> > > >> >
> > > >> > Thanks,
> > > >> >
> > > >> > Pramod
> > > >> >
> > > >> >
> > > >> > On Tue, Jun 3, 2014 at 3:57 PM, Joe Stein <jo...@stealth.ly>
> > > wrote:
> > > >> >
> > > >> > > 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.  This comes from what folks are doing "in the
> wild"
> > to
> > > >> > > workaround and implement security with Kafka as it is today and
> > also
> > > >> > what I
> > > >> > > have discovered from organizations about their blockers. It also
> > > >> picks up
> > > >> > > from the wiki (which I should have time to update later in the
> > week
> > > >> based
> > > >> > > on the below and feedback from the thread).
> > > >> > >
> > > >> > > 1) Transport Layer Security (i.e. SSL)
> > > >> > >
> > > >> > > This also includes client authentication in addition to
> in-transit
> > > >> > security
> > > >> > > layer.  This work has been picked up here
> > > >> > > https://issues.apache.org/jira/browse/KAFKA-1477 and do
> > appreciate
> > > >> any
> > > >> > > thoughts, comments, feedback, tomatoes, whatever for this patch.
> >  It
> > > >> is a
> > > >> > > pickup from the fork of the work first done here
> > > >> > > https://github.com/relango/kafka/tree/kafka_security.
> > > >> > >
> > > >> > > 2) Data encryption at rest.
> > > >> > >
> > > >> > > This is very important and something that can be facilitated
> > within
> > > >> the
> > > >> > > wire protocol. It requires an additional map data structure for
> > the
> > > >> > > "encrypted [data encryption key]". With this map (either in your
> > > >> object
> > > >> > or
> > > >> > > in the wire protocol) you can store the dynamically generated
> > > >> symmetric
> > > >> > key
> > > >> > > (for each message) and then encrypt the data using that
> > dynamically
> > > >> > > generated key.  You then encrypt the encryption key using each
> > > public
> > > >> key
> > > >> > > for whom is expected to be able to decrypt the encryption key to
> > > then
> > > >> > > decrypt the message.  For each public key encrypted symmetric
> key
> > > >> (which
> > > >> > is
> > > >> > > now the "encrypted [data encryption key]" along with which
> public
> > > key
> > > >> it
> > > >> > > was encrypted with for (so a map of [publicKey] =
> > > >> > > encryptedDataEncryptionKey) as a chain.   Other patterns can be
> > > >> > implemented
> > > >> > > but this is a pretty standard digital enveloping [0] pattern
> with
> > > >> only 1
> > > >> > > field added. Other patterns should be able to use that field
> to-do
> > > >> their
> > > >> > > implementation too.
> > > >> > >
> > > >> > > 3) Non-repudiation and long term non-repudiation.
> > > >> > >
> > > >> > > Non-repudiation is proving data hasn't changed.  This is often
> (if
> > > not
> > > >> > > always) done with x509 public certificates (chained to a
> > certificate
> > > >> > > authority).
> > > >> > >
> > > >> > > Long term non-repudiation is what happens when the certificates
> of
> > > the
> > > >> > > certificate authority are expired (or revoked) and everything
> ever
> > > >> signed
> > > >> > > (ever) with that certificate's public key then becomes "no
> longer
> > > >> > provable
> > > >> > > as ever being authentic".  That is where RFC3126 [1] and RFC3161
> > [2]
> > > >> come
> > > >> > > in (or worm drives [hardware], etc).
> > > >> > >
> > > >> > > For either (or both) of these it is an operation of the
> encryptor
> > to
> > > >> > > sign/hash the data (with or without third party trusted timestap
> > of
> > > >> the
> > > >> > > signing event) and encrypt that with their own private key and
> > > >> distribute
> > > >> > > the results (before and after encrypting if required) along with
> > > their
> > > >> > > public key. This structure is a bit more complex but feasible,
> it
> > > is a
> > > >> > map
> > > >> > > of digital signature formats and the chain of dig sig
> > attestations.
> > > >>  The
> > > >> > > map's key being the method (i.e. CRC32, PKCS7 [3], XmlDigSig
> [4])
> > > and
> > > >> > then
> > > >> > > a list of map where that key is "purpose" of signature (what
> your
> > > >> > attesting
> > > >> > > too).  As a sibling field to the list another field for "the
> > > >> attester" as
> > > >> > > bytes (e.g. their PKCS12 [5] for the map of PKCS7 signatures).
> > > >> > >
> > > >> > > 4) Authorization
> > > >> > >
> > > >> > > We should have a policy of "404" for data, topics, partitions
> > (etc)
> > > if
> > > >> > > authenticated connections do not have access.  In "secure mode"
> > any
> > > >> non
> > > >> > > authenticated connections should get a "404" type message on
> > > >> everything.
> > > >> > > Knowing "something is there" is a security risk in many uses
> > cases.
> > > >>  So
> > > >> > if
> > > >> > > you don't have access you don't even see it.  Baking "that" into
> > > Kafka
> > > >> > > along with some interface for entitlement (access management)
> > > systems
> > > >> > > (pretty standard) is all that I think needs to be done to the
> core
> > > >> > project.
> > > >> > >  I want to tackle item later in the year after summer after the
> > > other
> > > >> > three
> > > >> > > are complete.
> > > >> > >
> > > >> > > I look forward to thoughts on this and anyone else interested in
> > > >> working
> > > >> > > with us on these items.
> > > >> > >
> > > >> > > [0]
> > > >> > >
> > > >> > >
> > > >> >
> > > >>
> > >
> >
> http://www.emc.com/emc-plus/rsa-labs/standards-initiatives/what-is-a-digital-envelope.htm
> > > >> > > [1] http://tools.ietf.org/html/rfc3126
> > > >> > > [2] http://tools.ietf.org/html/rfc3161
> > > >> > > [3]
> > > >> > >
> > > >> > >
> > > >> >
> > > >>
> > >
> >
> http://www.emc.com/emc-plus/rsa-labs/standards-initiatives/pkcs-7-cryptographic-message-syntax-standar.htm
> > > >> > > [4] http://en.wikipedia.org/wiki/XML_Signature
> > > >> > > [5] http://en.wikipedia.org/wiki/PKCS_12
> > > >> > >
> > > >> > > /*******************************************
> > > >> > >  Joe Stein
> > > >> > >  Founder, Principal Consultant
> > > >> > >  Big Data Open Source Security LLC
> > > >> > >  http://www.stealth.ly
> > > >> > >  Twitter: @allthingshadoop <
> > http://www.twitter.com/allthingshadoop>
> > > >> > > ********************************************/
> > > >> > >
> > > >> >
> > > >>
> > > >
> > > >
> > >
> >
> >
> >
> > --
> > Thanks,
> > Raja.
> >
>



-- 
Thanks,
Raja.

Re: [DISCUSS] Kafka Security Specific Features

Posted by Pramod Deshmukh <dp...@gmail.com>.
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 config/server.properties*



*broker.log:*

[2014-07-17 15:34:46,281] INFO zookeeper state changed (SyncConnected)
(org.I0Itec.zkclient.ZkClient)

[2014-07-17 15:34:46,523] INFO Loading log 'secure.test-0'
(kafka.log.LogManager)

[2014-07-17 15:34:46,558] INFO Recovering unflushed segment 0 in log
secure.test-0. (kafka.log.Log)

[2014-07-17 15:34:46,571] INFO Completed load of log secure.test-0 with log
end offset 0 (kafka.log.Log)

[2014-07-17 15:34:46,582] INFO Starting log cleanup with a period of 60000
ms. (kafka.log.LogManager)

[2014-07-17 15:34:46,587] INFO Starting log flusher with a default period
of 9223372036854775807 ms. (kafka.log.LogManager)

[2014-07-17 15:34:46,614] INFO Initializing secure authentication
(kafka.network.security.SecureAuth$)

[2014-07-17 15:34:46,678] INFO Secure authentication initialization has
been successfully completed (kafka.network.security.SecureAuth$)

[2014-07-17 15:34:46,691] INFO Awaiting socket connections on 0.0.0.0:9092.
(kafka.network.Acceptor)

[2014-07-17 15:34:46,692] INFO [Socket Server on Broker 0], Started
(kafka.network.SocketServer)

[2014-07-17 15:34:46,794] INFO Will not load MX4J, mx4j-tools.jar is not in
the classpath (kafka.utils.Mx4jLoader$)

[2014-07-17 15:34:46,837] INFO 0 successfully elected as leader
(kafka.server.ZookeeperLeaderElector)

[2014-07-17 15:34:47,057] INFO Registered broker 0 at path /brokers/ids/0
with address 10.1.100.130:9092. (kafka.utils.ZkUtils$)

[2014-07-17 15:34:47,059] INFO New leader is 0
(kafka.server.ZookeeperLeaderElector$LeaderChangeListener)

*[2014-07-17 15:34:47,068] INFO [Kafka Server 0], started
(kafka.server.KafkaServer)*

*[2014-07-17 15:34:47,383] INFO begin ssl handshake for
/10.1.100.130:9092//10.1.100.130:51685
<http://10.1.100.130:9092//10.1.100.130:51685>
(kafka.network.security.SSLSocketChannel)*

*[2014-07-17 15:34:47,392] INFO begin ssl handshake for
10.1.100.130/10.1.100.130:51685//10.1.100.130:9092
<http://10.1.100.130/10.1.100.130:51685//10.1.100.130:9092>
(kafka.network.security.SSLSocketChannel)*

*[2014-07-17 15:34:47,465] INFO finished ssl handshake for
10.1.100.130/10.1.100.130:51685//10.1.100.130:9092
<http://10.1.100.130/10.1.100.130:51685//10.1.100.130:9092>
(kafka.network.security.SSLSocketChannel)*

*[2014-07-17 15:34:47,465] INFO finished ssl handshake for
/10.1.100.130:9092//10.1.100.130:51685
<http://10.1.100.130:9092//10.1.100.130:51685>
(kafka.network.security.SSLSocketChannel)*

*[2014-07-17 15:34:47,617] INFO [ReplicaFetcherManager on broker 0] Removed
fetcher for partitions  (kafka.server.ReplicaFetcherManager)*

*[2014-07-17 15:34:47,627] INFO [ReplicaFetcherManager on broker 0] Added
fetcher for partitions List() (kafka.server.ReplicaFetcherManager)*

*[2014-07-17 15:34:47,656] INFO [ReplicaFetcherManager on broker 0] Removed
fetcher for partitions [secure.test,0] (kafka.server.ReplicaFetcherManager)*

[2014-07-17 15:37:15,970] INFO begin ssl handshake for
10.1.100.130/10.1.100.130:51689//10.1.100.130:9092
(kafka.network.security.SSLSocketChannel)

[2014-07-17 15:37:16,075] INFO begin ssl handshake for
10.1.100.130/10.1.100.130:51690//10.1.100.130:9092
(kafka.network.security.SSLSocketChannel)

[2014-07-17 15:37:16,434] INFO begin ssl handshake for
10.1.100.130/10.1.100.130:51691//10.1.100.130:9092
(kafka.network.security.SSLSocketChannel)

[2014-07-17 15:37:16,530] INFO begin ssl handshake for
10.1.100.130/10.1.100.130:51692//10.1.100.130:9092
(kafka.network.security.SSLSocketChannel)

[2014-07-17 15:37:16,743] INFO begin ssl handshake for
10.1.100.130/10.1.100.130:51693//10.1.100.130:9092
(kafka.network.security.SSLSocketChannel)

[2014-07-17 15:37:16,834] INFO begin ssl handshake for
10.1.100.130/10.1.100.130:51694//10.1.100.130:9092
(kafka.network.security.SSLSocketChannel)

[2014-07-17 15:37:17,043] INFO begin ssl handshake for
10.1.100.130/10.1.100.130:51695//10.1.100.130:9092
(kafka.network.security.SSLSocketChannel)

[2014-07-17 15:37:17,137] INFO begin ssl handshake for
10.1.100.130/10.1.100.130:51696//10.1.100.130:9092
(kafka.network.security.SSLSocketChannel)

[2014-07-17 15:37:17,342] INFO begin ssl handshake for
10.1.100.130/10.1.100.130:51697//10.1.100.130:9092
(kafka.network.security.SSLSocketChannel)


*Start producer*

*bin/kafka-console-producer.sh --broker-list 10.1.100.130:9092:true --topic
secure.test*


*producer.log:*

bin/kafka-console-producer.sh --broker-list 10.1.100.130:9092:true --topic
secure.test

[2014-07-17 15:37:46,889] WARN Property topic is not valid
(kafka.utils.VerifiableProperties)

Hello Secure Kafka

*[2014-07-17 15:38:14,186] ERROR OOME with size 352518400
(kafka.network.BoundedByteBufferReceive)*

*java.lang.OutOfMemoryError: Java heap space*

at java.nio.HeapByteBuffer.<init>(HeapByteBuffer.java:57)

at java.nio.ByteBuffer.allocate(ByteBuffer.java:331)

at
kafka.network.BoundedByteBufferReceive.byteBufferAllocate(BoundedByteBufferReceive.scala:80)

at
kafka.network.BoundedByteBufferReceive.readFrom(BoundedByteBufferReceive.scala:63)

at kafka.network.Receive$class.readCompletely(Transmission.scala:56)

at
kafka.network.BoundedByteBufferReceive.readCompletely(BoundedByteBufferReceive.scala:29)

at kafka.network.BlockingChannel.receive(BlockingChannel.scala:102)

at kafka.producer.SyncProducer.liftedTree1$1(SyncProducer.scala:79)

at
kafka.producer.SyncProducer.kafka$producer$SyncProducer$$doSend(SyncProducer.scala:76)

at kafka.producer.SyncProducer.send(SyncProducer.scala:117)

at kafka.client.ClientUtils$.fetchTopicMetadata(ClientUtils.scala:58)

at
kafka.producer.BrokerPartitionInfo.updateInfo(BrokerPartitionInfo.scala:82)

at
kafka.producer.async.DefaultEventHandler$$anonfun$handle$1.apply$mcV$sp(DefaultEventHandler.scala:67)

at kafka.utils.Utils$.swallow(Utils.scala:172)

at kafka.utils.Logging$class.swallowError(Logging.scala:106)

at kafka.utils.Utils$.swallowError(Utils.scala:45)

at
kafka.producer.async.DefaultEventHandler.handle(DefaultEventHandler.scala:67)

at
kafka.producer.async.ProducerSendThread.tryToHandle(ProducerSendThread.scala:104)

at
kafka.producer.async.ProducerSendThread$$anonfun$processEvents$3.apply(ProducerSendThread.scala:87)

at
kafka.producer.async.ProducerSendThread$$anonfun$processEvents$3.apply(ProducerSendThread.scala:67)

at scala.collection.immutable.Stream.foreach(Stream.scala:526)

at
kafka.producer.async.ProducerSendThread.processEvents(ProducerSendThread.scala:66)

at kafka.producer.async.ProducerSendThread.run(ProducerSendThread.scala:44)



On Wed, Jul 16, 2014 at 6:07 PM, Rajasekar Elango <re...@salesforce.com>
wrote:

> Pramod,
>
>
> I presented secure kafka configuration and usage at last meet up. So hope
> this
> video recording <http://www.ustream.tv/recorded/48396701>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 <dp...@gmail.com>
> wrote:
>
> > 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 <dp...@gmail.com>
> > 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, Jul 16, 2014 at 3:57 PM, Joe Stein <jo...@stealth.ly>
> 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 (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 Stein
> > >>  Founder, Principal Consultant
> > >>  Big Data Open Source Security LLC
> > >>  http://www.stealth.ly
> > >>  Twitter: @allthingshadoop <http://www.twitter.com/allthingshadoop>
> > >> ********************************************/
> > >>
> > >>
> > >> On Wed, Jul 16, 2014 at 3:49 PM, Pramod Deshmukh <dp...@gmail.com>
> > >> wrote:
> > >>
> > >> > 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 installed sbt and scala and followed your instructions for
> > >> different
> > >> > scala versions. I was able to bring zookeeper up but brokers fail to
> > >> start
> > >> > with error
> > >> >
> > >> > Error: Could not find or load main class kafka.Kafka
> > >> >
> > >> > I think I am doing something wrong. Can you please help me?
> > >> >
> > >> > Our current production setup is with 2.8.0 and want to stick to it.
> > >> >
> > >> > Thanks,
> > >> >
> > >> > Pramod
> > >> >
> > >> >
> > >> > On Tue, Jun 3, 2014 at 3:57 PM, Joe Stein <jo...@stealth.ly>
> > wrote:
> > >> >
> > >> > > 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.  This comes from what folks are doing "in the wild"
> to
> > >> > > workaround and implement security with Kafka as it is today and
> also
> > >> > what I
> > >> > > have discovered from organizations about their blockers. It also
> > >> picks up
> > >> > > from the wiki (which I should have time to update later in the
> week
> > >> based
> > >> > > on the below and feedback from the thread).
> > >> > >
> > >> > > 1) Transport Layer Security (i.e. SSL)
> > >> > >
> > >> > > This also includes client authentication in addition to in-transit
> > >> > security
> > >> > > layer.  This work has been picked up here
> > >> > > https://issues.apache.org/jira/browse/KAFKA-1477 and do
> appreciate
> > >> any
> > >> > > thoughts, comments, feedback, tomatoes, whatever for this patch.
>  It
> > >> is a
> > >> > > pickup from the fork of the work first done here
> > >> > > https://github.com/relango/kafka/tree/kafka_security.
> > >> > >
> > >> > > 2) Data encryption at rest.
> > >> > >
> > >> > > This is very important and something that can be facilitated
> within
> > >> the
> > >> > > wire protocol. It requires an additional map data structure for
> the
> > >> > > "encrypted [data encryption key]". With this map (either in your
> > >> object
> > >> > or
> > >> > > in the wire protocol) you can store the dynamically generated
> > >> symmetric
> > >> > key
> > >> > > (for each message) and then encrypt the data using that
> dynamically
> > >> > > generated key.  You then encrypt the encryption key using each
> > public
> > >> key
> > >> > > for whom is expected to be able to decrypt the encryption key to
> > then
> > >> > > decrypt the message.  For each public key encrypted symmetric key
> > >> (which
> > >> > is
> > >> > > now the "encrypted [data encryption key]" along with which public
> > key
> > >> it
> > >> > > was encrypted with for (so a map of [publicKey] =
> > >> > > encryptedDataEncryptionKey) as a chain.   Other patterns can be
> > >> > implemented
> > >> > > but this is a pretty standard digital enveloping [0] pattern with
> > >> only 1
> > >> > > field added. Other patterns should be able to use that field to-do
> > >> their
> > >> > > implementation too.
> > >> > >
> > >> > > 3) Non-repudiation and long term non-repudiation.
> > >> > >
> > >> > > Non-repudiation is proving data hasn't changed.  This is often (if
> > not
> > >> > > always) done with x509 public certificates (chained to a
> certificate
> > >> > > authority).
> > >> > >
> > >> > > Long term non-repudiation is what happens when the certificates of
> > the
> > >> > > certificate authority are expired (or revoked) and everything ever
> > >> signed
> > >> > > (ever) with that certificate's public key then becomes "no longer
> > >> > provable
> > >> > > as ever being authentic".  That is where RFC3126 [1] and RFC3161
> [2]
> > >> come
> > >> > > in (or worm drives [hardware], etc).
> > >> > >
> > >> > > For either (or both) of these it is an operation of the encryptor
> to
> > >> > > sign/hash the data (with or without third party trusted timestap
> of
> > >> the
> > >> > > signing event) and encrypt that with their own private key and
> > >> distribute
> > >> > > the results (before and after encrypting if required) along with
> > their
> > >> > > public key. This structure is a bit more complex but feasible, it
> > is a
> > >> > map
> > >> > > of digital signature formats and the chain of dig sig
> attestations.
> > >>  The
> > >> > > map's key being the method (i.e. CRC32, PKCS7 [3], XmlDigSig [4])
> > and
> > >> > then
> > >> > > a list of map where that key is "purpose" of signature (what your
> > >> > attesting
> > >> > > too).  As a sibling field to the list another field for "the
> > >> attester" as
> > >> > > bytes (e.g. their PKCS12 [5] for the map of PKCS7 signatures).
> > >> > >
> > >> > > 4) Authorization
> > >> > >
> > >> > > We should have a policy of "404" for data, topics, partitions
> (etc)
> > if
> > >> > > authenticated connections do not have access.  In "secure mode"
> any
> > >> non
> > >> > > authenticated connections should get a "404" type message on
> > >> everything.
> > >> > > Knowing "something is there" is a security risk in many uses
> cases.
> > >>  So
> > >> > if
> > >> > > you don't have access you don't even see it.  Baking "that" into
> > Kafka
> > >> > > along with some interface for entitlement (access management)
> > systems
> > >> > > (pretty standard) is all that I think needs to be done to the core
> > >> > project.
> > >> > >  I want to tackle item later in the year after summer after the
> > other
> > >> > three
> > >> > > are complete.
> > >> > >
> > >> > > I look forward to thoughts on this and anyone else interested in
> > >> working
> > >> > > with us on these items.
> > >> > >
> > >> > > [0]
> > >> > >
> > >> > >
> > >> >
> > >>
> >
> http://www.emc.com/emc-plus/rsa-labs/standards-initiatives/what-is-a-digital-envelope.htm
> > >> > > [1] http://tools.ietf.org/html/rfc3126
> > >> > > [2] http://tools.ietf.org/html/rfc3161
> > >> > > [3]
> > >> > >
> > >> > >
> > >> >
> > >>
> >
> http://www.emc.com/emc-plus/rsa-labs/standards-initiatives/pkcs-7-cryptographic-message-syntax-standar.htm
> > >> > > [4] http://en.wikipedia.org/wiki/XML_Signature
> > >> > > [5] http://en.wikipedia.org/wiki/PKCS_12
> > >> > >
> > >> > > /*******************************************
> > >> > >  Joe Stein
> > >> > >  Founder, Principal Consultant
> > >> > >  Big Data Open Source Security LLC
> > >> > >  http://www.stealth.ly
> > >> > >  Twitter: @allthingshadoop <
> http://www.twitter.com/allthingshadoop>
> > >> > > ********************************************/
> > >> > >
> > >> >
> > >>
> > >
> > >
> >
>
>
>
> --
> Thanks,
> Raja.
>

Re: [DISCUSS] Kafka Security Specific Features

Posted by Pramod Deshmukh <dp...@gmail.com>.
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 config/server.properties*



*broker.log:*

[2014-07-17 15:34:46,281] INFO zookeeper state changed (SyncConnected)
(org.I0Itec.zkclient.ZkClient)

[2014-07-17 15:34:46,523] INFO Loading log 'secure.test-0'
(kafka.log.LogManager)

[2014-07-17 15:34:46,558] INFO Recovering unflushed segment 0 in log
secure.test-0. (kafka.log.Log)

[2014-07-17 15:34:46,571] INFO Completed load of log secure.test-0 with log
end offset 0 (kafka.log.Log)

[2014-07-17 15:34:46,582] INFO Starting log cleanup with a period of 60000
ms. (kafka.log.LogManager)

[2014-07-17 15:34:46,587] INFO Starting log flusher with a default period
of 9223372036854775807 ms. (kafka.log.LogManager)

[2014-07-17 15:34:46,614] INFO Initializing secure authentication
(kafka.network.security.SecureAuth$)

[2014-07-17 15:34:46,678] INFO Secure authentication initialization has
been successfully completed (kafka.network.security.SecureAuth$)

[2014-07-17 15:34:46,691] INFO Awaiting socket connections on 0.0.0.0:9092.
(kafka.network.Acceptor)

[2014-07-17 15:34:46,692] INFO [Socket Server on Broker 0], Started
(kafka.network.SocketServer)

[2014-07-17 15:34:46,794] INFO Will not load MX4J, mx4j-tools.jar is not in
the classpath (kafka.utils.Mx4jLoader$)

[2014-07-17 15:34:46,837] INFO 0 successfully elected as leader
(kafka.server.ZookeeperLeaderElector)

[2014-07-17 15:34:47,057] INFO Registered broker 0 at path /brokers/ids/0
with address 10.1.100.130:9092. (kafka.utils.ZkUtils$)

[2014-07-17 15:34:47,059] INFO New leader is 0
(kafka.server.ZookeeperLeaderElector$LeaderChangeListener)

*[2014-07-17 15:34:47,068] INFO [Kafka Server 0], started
(kafka.server.KafkaServer)*

*[2014-07-17 15:34:47,383] INFO begin ssl handshake for
/10.1.100.130:9092//10.1.100.130:51685
<http://10.1.100.130:9092//10.1.100.130:51685>
(kafka.network.security.SSLSocketChannel)*

*[2014-07-17 15:34:47,392] INFO begin ssl handshake for
10.1.100.130/10.1.100.130:51685//10.1.100.130:9092
<http://10.1.100.130/10.1.100.130:51685//10.1.100.130:9092>
(kafka.network.security.SSLSocketChannel)*

*[2014-07-17 15:34:47,465] INFO finished ssl handshake for
10.1.100.130/10.1.100.130:51685//10.1.100.130:9092
<http://10.1.100.130/10.1.100.130:51685//10.1.100.130:9092>
(kafka.network.security.SSLSocketChannel)*

*[2014-07-17 15:34:47,465] INFO finished ssl handshake for
/10.1.100.130:9092//10.1.100.130:51685
<http://10.1.100.130:9092//10.1.100.130:51685>
(kafka.network.security.SSLSocketChannel)*

*[2014-07-17 15:34:47,617] INFO [ReplicaFetcherManager on broker 0] Removed
fetcher for partitions  (kafka.server.ReplicaFetcherManager)*

*[2014-07-17 15:34:47,627] INFO [ReplicaFetcherManager on broker 0] Added
fetcher for partitions List() (kafka.server.ReplicaFetcherManager)*

*[2014-07-17 15:34:47,656] INFO [ReplicaFetcherManager on broker 0] Removed
fetcher for partitions [secure.test,0] (kafka.server.ReplicaFetcherManager)*

[2014-07-17 15:37:15,970] INFO begin ssl handshake for
10.1.100.130/10.1.100.130:51689//10.1.100.130:9092
(kafka.network.security.SSLSocketChannel)

[2014-07-17 15:37:16,075] INFO begin ssl handshake for
10.1.100.130/10.1.100.130:51690//10.1.100.130:9092
(kafka.network.security.SSLSocketChannel)

[2014-07-17 15:37:16,434] INFO begin ssl handshake for
10.1.100.130/10.1.100.130:51691//10.1.100.130:9092
(kafka.network.security.SSLSocketChannel)

[2014-07-17 15:37:16,530] INFO begin ssl handshake for
10.1.100.130/10.1.100.130:51692//10.1.100.130:9092
(kafka.network.security.SSLSocketChannel)

[2014-07-17 15:37:16,743] INFO begin ssl handshake for
10.1.100.130/10.1.100.130:51693//10.1.100.130:9092
(kafka.network.security.SSLSocketChannel)

[2014-07-17 15:37:16,834] INFO begin ssl handshake for
10.1.100.130/10.1.100.130:51694//10.1.100.130:9092
(kafka.network.security.SSLSocketChannel)

[2014-07-17 15:37:17,043] INFO begin ssl handshake for
10.1.100.130/10.1.100.130:51695//10.1.100.130:9092
(kafka.network.security.SSLSocketChannel)

[2014-07-17 15:37:17,137] INFO begin ssl handshake for
10.1.100.130/10.1.100.130:51696//10.1.100.130:9092
(kafka.network.security.SSLSocketChannel)

[2014-07-17 15:37:17,342] INFO begin ssl handshake for
10.1.100.130/10.1.100.130:51697//10.1.100.130:9092
(kafka.network.security.SSLSocketChannel)


*Start producer*

*bin/kafka-console-producer.sh --broker-list 10.1.100.130:9092:true --topic
secure.test*


*producer.log:*

bin/kafka-console-producer.sh --broker-list 10.1.100.130:9092:true --topic
secure.test

[2014-07-17 15:37:46,889] WARN Property topic is not valid
(kafka.utils.VerifiableProperties)

Hello Secure Kafka

*[2014-07-17 15:38:14,186] ERROR OOME with size 352518400
(kafka.network.BoundedByteBufferReceive)*

*java.lang.OutOfMemoryError: Java heap space*

at java.nio.HeapByteBuffer.<init>(HeapByteBuffer.java:57)

at java.nio.ByteBuffer.allocate(ByteBuffer.java:331)

at
kafka.network.BoundedByteBufferReceive.byteBufferAllocate(BoundedByteBufferReceive.scala:80)

at
kafka.network.BoundedByteBufferReceive.readFrom(BoundedByteBufferReceive.scala:63)

at kafka.network.Receive$class.readCompletely(Transmission.scala:56)

at
kafka.network.BoundedByteBufferReceive.readCompletely(BoundedByteBufferReceive.scala:29)

at kafka.network.BlockingChannel.receive(BlockingChannel.scala:102)

at kafka.producer.SyncProducer.liftedTree1$1(SyncProducer.scala:79)

at
kafka.producer.SyncProducer.kafka$producer$SyncProducer$$doSend(SyncProducer.scala:76)

at kafka.producer.SyncProducer.send(SyncProducer.scala:117)

at kafka.client.ClientUtils$.fetchTopicMetadata(ClientUtils.scala:58)

at
kafka.producer.BrokerPartitionInfo.updateInfo(BrokerPartitionInfo.scala:82)

at
kafka.producer.async.DefaultEventHandler$$anonfun$handle$1.apply$mcV$sp(DefaultEventHandler.scala:67)

at kafka.utils.Utils$.swallow(Utils.scala:172)

at kafka.utils.Logging$class.swallowError(Logging.scala:106)

at kafka.utils.Utils$.swallowError(Utils.scala:45)

at
kafka.producer.async.DefaultEventHandler.handle(DefaultEventHandler.scala:67)

at
kafka.producer.async.ProducerSendThread.tryToHandle(ProducerSendThread.scala:104)

at
kafka.producer.async.ProducerSendThread$$anonfun$processEvents$3.apply(ProducerSendThread.scala:87)

at
kafka.producer.async.ProducerSendThread$$anonfun$processEvents$3.apply(ProducerSendThread.scala:67)

at scala.collection.immutable.Stream.foreach(Stream.scala:526)

at
kafka.producer.async.ProducerSendThread.processEvents(ProducerSendThread.scala:66)

at kafka.producer.async.ProducerSendThread.run(ProducerSendThread.scala:44)



On Wed, Jul 16, 2014 at 6:07 PM, Rajasekar Elango <re...@salesforce.com>
wrote:

> Pramod,
>
>
> I presented secure kafka configuration and usage at last meet up. So hope
> this
> video recording <http://www.ustream.tv/recorded/48396701>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 <dp...@gmail.com>
> wrote:
>
> > 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 <dp...@gmail.com>
> > 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, Jul 16, 2014 at 3:57 PM, Joe Stein <jo...@stealth.ly>
> 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 (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 Stein
> > >>  Founder, Principal Consultant
> > >>  Big Data Open Source Security LLC
> > >>  http://www.stealth.ly
> > >>  Twitter: @allthingshadoop <http://www.twitter.com/allthingshadoop>
> > >> ********************************************/
> > >>
> > >>
> > >> On Wed, Jul 16, 2014 at 3:49 PM, Pramod Deshmukh <dp...@gmail.com>
> > >> wrote:
> > >>
> > >> > 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 installed sbt and scala and followed your instructions for
> > >> different
> > >> > scala versions. I was able to bring zookeeper up but brokers fail to
> > >> start
> > >> > with error
> > >> >
> > >> > Error: Could not find or load main class kafka.Kafka
> > >> >
> > >> > I think I am doing something wrong. Can you please help me?
> > >> >
> > >> > Our current production setup is with 2.8.0 and want to stick to it.
> > >> >
> > >> > Thanks,
> > >> >
> > >> > Pramod
> > >> >
> > >> >
> > >> > On Tue, Jun 3, 2014 at 3:57 PM, Joe Stein <jo...@stealth.ly>
> > wrote:
> > >> >
> > >> > > 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.  This comes from what folks are doing "in the wild"
> to
> > >> > > workaround and implement security with Kafka as it is today and
> also
> > >> > what I
> > >> > > have discovered from organizations about their blockers. It also
> > >> picks up
> > >> > > from the wiki (which I should have time to update later in the
> week
> > >> based
> > >> > > on the below and feedback from the thread).
> > >> > >
> > >> > > 1) Transport Layer Security (i.e. SSL)
> > >> > >
> > >> > > This also includes client authentication in addition to in-transit
> > >> > security
> > >> > > layer.  This work has been picked up here
> > >> > > https://issues.apache.org/jira/browse/KAFKA-1477 and do
> appreciate
> > >> any
> > >> > > thoughts, comments, feedback, tomatoes, whatever for this patch.
>  It
> > >> is a
> > >> > > pickup from the fork of the work first done here
> > >> > > https://github.com/relango/kafka/tree/kafka_security.
> > >> > >
> > >> > > 2) Data encryption at rest.
> > >> > >
> > >> > > This is very important and something that can be facilitated
> within
> > >> the
> > >> > > wire protocol. It requires an additional map data structure for
> the
> > >> > > "encrypted [data encryption key]". With this map (either in your
> > >> object
> > >> > or
> > >> > > in the wire protocol) you can store the dynamically generated
> > >> symmetric
> > >> > key
> > >> > > (for each message) and then encrypt the data using that
> dynamically
> > >> > > generated key.  You then encrypt the encryption key using each
> > public
> > >> key
> > >> > > for whom is expected to be able to decrypt the encryption key to
> > then
> > >> > > decrypt the message.  For each public key encrypted symmetric key
> > >> (which
> > >> > is
> > >> > > now the "encrypted [data encryption key]" along with which public
> > key
> > >> it
> > >> > > was encrypted with for (so a map of [publicKey] =
> > >> > > encryptedDataEncryptionKey) as a chain.   Other patterns can be
> > >> > implemented
> > >> > > but this is a pretty standard digital enveloping [0] pattern with
> > >> only 1
> > >> > > field added. Other patterns should be able to use that field to-do
> > >> their
> > >> > > implementation too.
> > >> > >
> > >> > > 3) Non-repudiation and long term non-repudiation.
> > >> > >
> > >> > > Non-repudiation is proving data hasn't changed.  This is often (if
> > not
> > >> > > always) done with x509 public certificates (chained to a
> certificate
> > >> > > authority).
> > >> > >
> > >> > > Long term non-repudiation is what happens when the certificates of
> > the
> > >> > > certificate authority are expired (or revoked) and everything ever
> > >> signed
> > >> > > (ever) with that certificate's public key then becomes "no longer
> > >> > provable
> > >> > > as ever being authentic".  That is where RFC3126 [1] and RFC3161
> [2]
> > >> come
> > >> > > in (or worm drives [hardware], etc).
> > >> > >
> > >> > > For either (or both) of these it is an operation of the encryptor
> to
> > >> > > sign/hash the data (with or without third party trusted timestap
> of
> > >> the
> > >> > > signing event) and encrypt that with their own private key and
> > >> distribute
> > >> > > the results (before and after encrypting if required) along with
> > their
> > >> > > public key. This structure is a bit more complex but feasible, it
> > is a
> > >> > map
> > >> > > of digital signature formats and the chain of dig sig
> attestations.
> > >>  The
> > >> > > map's key being the method (i.e. CRC32, PKCS7 [3], XmlDigSig [4])
> > and
> > >> > then
> > >> > > a list of map where that key is "purpose" of signature (what your
> > >> > attesting
> > >> > > too).  As a sibling field to the list another field for "the
> > >> attester" as
> > >> > > bytes (e.g. their PKCS12 [5] for the map of PKCS7 signatures).
> > >> > >
> > >> > > 4) Authorization
> > >> > >
> > >> > > We should have a policy of "404" for data, topics, partitions
> (etc)
> > if
> > >> > > authenticated connections do not have access.  In "secure mode"
> any
> > >> non
> > >> > > authenticated connections should get a "404" type message on
> > >> everything.
> > >> > > Knowing "something is there" is a security risk in many uses
> cases.
> > >>  So
> > >> > if
> > >> > > you don't have access you don't even see it.  Baking "that" into
> > Kafka
> > >> > > along with some interface for entitlement (access management)
> > systems
> > >> > > (pretty standard) is all that I think needs to be done to the core
> > >> > project.
> > >> > >  I want to tackle item later in the year after summer after the
> > other
> > >> > three
> > >> > > are complete.
> > >> > >
> > >> > > I look forward to thoughts on this and anyone else interested in
> > >> working
> > >> > > with us on these items.
> > >> > >
> > >> > > [0]
> > >> > >
> > >> > >
> > >> >
> > >>
> >
> http://www.emc.com/emc-plus/rsa-labs/standards-initiatives/what-is-a-digital-envelope.htm
> > >> > > [1] http://tools.ietf.org/html/rfc3126
> > >> > > [2] http://tools.ietf.org/html/rfc3161
> > >> > > [3]
> > >> > >
> > >> > >
> > >> >
> > >>
> >
> http://www.emc.com/emc-plus/rsa-labs/standards-initiatives/pkcs-7-cryptographic-message-syntax-standar.htm
> > >> > > [4] http://en.wikipedia.org/wiki/XML_Signature
> > >> > > [5] http://en.wikipedia.org/wiki/PKCS_12
> > >> > >
> > >> > > /*******************************************
> > >> > >  Joe Stein
> > >> > >  Founder, Principal Consultant
> > >> > >  Big Data Open Source Security LLC
> > >> > >  http://www.stealth.ly
> > >> > >  Twitter: @allthingshadoop <
> http://www.twitter.com/allthingshadoop>
> > >> > > ********************************************/
> > >> > >
> > >> >
> > >>
> > >
> > >
> >
>
>
>
> --
> Thanks,
> Raja.
>

Re: [DISCUSS] Kafka Security Specific Features

Posted by Rajasekar Elango <re...@salesforce.com>.
Pramod,


I presented secure kafka configuration and usage at last meet up. So hope this
video recording <http://www.ustream.tv/recorded/48396701>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 <dp...@gmail.com> wrote:

> 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 <dp...@gmail.com>
> 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, Jul 16, 2014 at 3:57 PM, Joe Stein <jo...@stealth.ly> 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 (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 Stein
> >>  Founder, Principal Consultant
> >>  Big Data Open Source Security LLC
> >>  http://www.stealth.ly
> >>  Twitter: @allthingshadoop <http://www.twitter.com/allthingshadoop>
> >> ********************************************/
> >>
> >>
> >> On Wed, Jul 16, 2014 at 3:49 PM, Pramod Deshmukh <dp...@gmail.com>
> >> wrote:
> >>
> >> > 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 installed sbt and scala and followed your instructions for
> >> different
> >> > scala versions. I was able to bring zookeeper up but brokers fail to
> >> start
> >> > with error
> >> >
> >> > Error: Could not find or load main class kafka.Kafka
> >> >
> >> > I think I am doing something wrong. Can you please help me?
> >> >
> >> > Our current production setup is with 2.8.0 and want to stick to it.
> >> >
> >> > Thanks,
> >> >
> >> > Pramod
> >> >
> >> >
> >> > On Tue, Jun 3, 2014 at 3:57 PM, Joe Stein <jo...@stealth.ly>
> wrote:
> >> >
> >> > > 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.  This comes from what folks are doing "in the wild" to
> >> > > workaround and implement security with Kafka as it is today and also
> >> > what I
> >> > > have discovered from organizations about their blockers. It also
> >> picks up
> >> > > from the wiki (which I should have time to update later in the week
> >> based
> >> > > on the below and feedback from the thread).
> >> > >
> >> > > 1) Transport Layer Security (i.e. SSL)
> >> > >
> >> > > This also includes client authentication in addition to in-transit
> >> > security
> >> > > layer.  This work has been picked up here
> >> > > https://issues.apache.org/jira/browse/KAFKA-1477 and do appreciate
> >> any
> >> > > thoughts, comments, feedback, tomatoes, whatever for this patch.  It
> >> is a
> >> > > pickup from the fork of the work first done here
> >> > > https://github.com/relango/kafka/tree/kafka_security.
> >> > >
> >> > > 2) Data encryption at rest.
> >> > >
> >> > > This is very important and something that can be facilitated within
> >> the
> >> > > wire protocol. It requires an additional map data structure for the
> >> > > "encrypted [data encryption key]". With this map (either in your
> >> object
> >> > or
> >> > > in the wire protocol) you can store the dynamically generated
> >> symmetric
> >> > key
> >> > > (for each message) and then encrypt the data using that dynamically
> >> > > generated key.  You then encrypt the encryption key using each
> public
> >> key
> >> > > for whom is expected to be able to decrypt the encryption key to
> then
> >> > > decrypt the message.  For each public key encrypted symmetric key
> >> (which
> >> > is
> >> > > now the "encrypted [data encryption key]" along with which public
> key
> >> it
> >> > > was encrypted with for (so a map of [publicKey] =
> >> > > encryptedDataEncryptionKey) as a chain.   Other patterns can be
> >> > implemented
> >> > > but this is a pretty standard digital enveloping [0] pattern with
> >> only 1
> >> > > field added. Other patterns should be able to use that field to-do
> >> their
> >> > > implementation too.
> >> > >
> >> > > 3) Non-repudiation and long term non-repudiation.
> >> > >
> >> > > Non-repudiation is proving data hasn't changed.  This is often (if
> not
> >> > > always) done with x509 public certificates (chained to a certificate
> >> > > authority).
> >> > >
> >> > > Long term non-repudiation is what happens when the certificates of
> the
> >> > > certificate authority are expired (or revoked) and everything ever
> >> signed
> >> > > (ever) with that certificate's public key then becomes "no longer
> >> > provable
> >> > > as ever being authentic".  That is where RFC3126 [1] and RFC3161 [2]
> >> come
> >> > > in (or worm drives [hardware], etc).
> >> > >
> >> > > For either (or both) of these it is an operation of the encryptor to
> >> > > sign/hash the data (with or without third party trusted timestap of
> >> the
> >> > > signing event) and encrypt that with their own private key and
> >> distribute
> >> > > the results (before and after encrypting if required) along with
> their
> >> > > public key. This structure is a bit more complex but feasible, it
> is a
> >> > map
> >> > > of digital signature formats and the chain of dig sig attestations.
> >>  The
> >> > > map's key being the method (i.e. CRC32, PKCS7 [3], XmlDigSig [4])
> and
> >> > then
> >> > > a list of map where that key is "purpose" of signature (what your
> >> > attesting
> >> > > too).  As a sibling field to the list another field for "the
> >> attester" as
> >> > > bytes (e.g. their PKCS12 [5] for the map of PKCS7 signatures).
> >> > >
> >> > > 4) Authorization
> >> > >
> >> > > We should have a policy of "404" for data, topics, partitions (etc)
> if
> >> > > authenticated connections do not have access.  In "secure mode" any
> >> non
> >> > > authenticated connections should get a "404" type message on
> >> everything.
> >> > > Knowing "something is there" is a security risk in many uses cases.
> >>  So
> >> > if
> >> > > you don't have access you don't even see it.  Baking "that" into
> Kafka
> >> > > along with some interface for entitlement (access management)
> systems
> >> > > (pretty standard) is all that I think needs to be done to the core
> >> > project.
> >> > >  I want to tackle item later in the year after summer after the
> other
> >> > three
> >> > > are complete.
> >> > >
> >> > > I look forward to thoughts on this and anyone else interested in
> >> working
> >> > > with us on these items.
> >> > >
> >> > > [0]
> >> > >
> >> > >
> >> >
> >>
> http://www.emc.com/emc-plus/rsa-labs/standards-initiatives/what-is-a-digital-envelope.htm
> >> > > [1] http://tools.ietf.org/html/rfc3126
> >> > > [2] http://tools.ietf.org/html/rfc3161
> >> > > [3]
> >> > >
> >> > >
> >> >
> >>
> http://www.emc.com/emc-plus/rsa-labs/standards-initiatives/pkcs-7-cryptographic-message-syntax-standar.htm
> >> > > [4] http://en.wikipedia.org/wiki/XML_Signature
> >> > > [5] http://en.wikipedia.org/wiki/PKCS_12
> >> > >
> >> > > /*******************************************
> >> > >  Joe Stein
> >> > >  Founder, Principal Consultant
> >> > >  Big Data Open Source Security LLC
> >> > >  http://www.stealth.ly
> >> > >  Twitter: @allthingshadoop <http://www.twitter.com/allthingshadoop>
> >> > > ********************************************/
> >> > >
> >> >
> >>
> >
> >
>



-- 
Thanks,
Raja.

Re: [DISCUSS] Kafka Security Specific Features

Posted by Rajasekar Elango <re...@salesforce.com>.
Pramod,


I presented secure kafka configuration and usage at last meet up. So hope this
video recording <http://www.ustream.tv/recorded/48396701>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 <dp...@gmail.com> wrote:

> 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 <dp...@gmail.com>
> 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, Jul 16, 2014 at 3:57 PM, Joe Stein <jo...@stealth.ly> 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 (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 Stein
> >>  Founder, Principal Consultant
> >>  Big Data Open Source Security LLC
> >>  http://www.stealth.ly
> >>  Twitter: @allthingshadoop <http://www.twitter.com/allthingshadoop>
> >> ********************************************/
> >>
> >>
> >> On Wed, Jul 16, 2014 at 3:49 PM, Pramod Deshmukh <dp...@gmail.com>
> >> wrote:
> >>
> >> > 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 installed sbt and scala and followed your instructions for
> >> different
> >> > scala versions. I was able to bring zookeeper up but brokers fail to
> >> start
> >> > with error
> >> >
> >> > Error: Could not find or load main class kafka.Kafka
> >> >
> >> > I think I am doing something wrong. Can you please help me?
> >> >
> >> > Our current production setup is with 2.8.0 and want to stick to it.
> >> >
> >> > Thanks,
> >> >
> >> > Pramod
> >> >
> >> >
> >> > On Tue, Jun 3, 2014 at 3:57 PM, Joe Stein <jo...@stealth.ly>
> wrote:
> >> >
> >> > > 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.  This comes from what folks are doing "in the wild" to
> >> > > workaround and implement security with Kafka as it is today and also
> >> > what I
> >> > > have discovered from organizations about their blockers. It also
> >> picks up
> >> > > from the wiki (which I should have time to update later in the week
> >> based
> >> > > on the below and feedback from the thread).
> >> > >
> >> > > 1) Transport Layer Security (i.e. SSL)
> >> > >
> >> > > This also includes client authentication in addition to in-transit
> >> > security
> >> > > layer.  This work has been picked up here
> >> > > https://issues.apache.org/jira/browse/KAFKA-1477 and do appreciate
> >> any
> >> > > thoughts, comments, feedback, tomatoes, whatever for this patch.  It
> >> is a
> >> > > pickup from the fork of the work first done here
> >> > > https://github.com/relango/kafka/tree/kafka_security.
> >> > >
> >> > > 2) Data encryption at rest.
> >> > >
> >> > > This is very important and something that can be facilitated within
> >> the
> >> > > wire protocol. It requires an additional map data structure for the
> >> > > "encrypted [data encryption key]". With this map (either in your
> >> object
> >> > or
> >> > > in the wire protocol) you can store the dynamically generated
> >> symmetric
> >> > key
> >> > > (for each message) and then encrypt the data using that dynamically
> >> > > generated key.  You then encrypt the encryption key using each
> public
> >> key
> >> > > for whom is expected to be able to decrypt the encryption key to
> then
> >> > > decrypt the message.  For each public key encrypted symmetric key
> >> (which
> >> > is
> >> > > now the "encrypted [data encryption key]" along with which public
> key
> >> it
> >> > > was encrypted with for (so a map of [publicKey] =
> >> > > encryptedDataEncryptionKey) as a chain.   Other patterns can be
> >> > implemented
> >> > > but this is a pretty standard digital enveloping [0] pattern with
> >> only 1
> >> > > field added. Other patterns should be able to use that field to-do
> >> their
> >> > > implementation too.
> >> > >
> >> > > 3) Non-repudiation and long term non-repudiation.
> >> > >
> >> > > Non-repudiation is proving data hasn't changed.  This is often (if
> not
> >> > > always) done with x509 public certificates (chained to a certificate
> >> > > authority).
> >> > >
> >> > > Long term non-repudiation is what happens when the certificates of
> the
> >> > > certificate authority are expired (or revoked) and everything ever
> >> signed
> >> > > (ever) with that certificate's public key then becomes "no longer
> >> > provable
> >> > > as ever being authentic".  That is where RFC3126 [1] and RFC3161 [2]
> >> come
> >> > > in (or worm drives [hardware], etc).
> >> > >
> >> > > For either (or both) of these it is an operation of the encryptor to
> >> > > sign/hash the data (with or without third party trusted timestap of
> >> the
> >> > > signing event) and encrypt that with their own private key and
> >> distribute
> >> > > the results (before and after encrypting if required) along with
> their
> >> > > public key. This structure is a bit more complex but feasible, it
> is a
> >> > map
> >> > > of digital signature formats and the chain of dig sig attestations.
> >>  The
> >> > > map's key being the method (i.e. CRC32, PKCS7 [3], XmlDigSig [4])
> and
> >> > then
> >> > > a list of map where that key is "purpose" of signature (what your
> >> > attesting
> >> > > too).  As a sibling field to the list another field for "the
> >> attester" as
> >> > > bytes (e.g. their PKCS12 [5] for the map of PKCS7 signatures).
> >> > >
> >> > > 4) Authorization
> >> > >
> >> > > We should have a policy of "404" for data, topics, partitions (etc)
> if
> >> > > authenticated connections do not have access.  In "secure mode" any
> >> non
> >> > > authenticated connections should get a "404" type message on
> >> everything.
> >> > > Knowing "something is there" is a security risk in many uses cases.
> >>  So
> >> > if
> >> > > you don't have access you don't even see it.  Baking "that" into
> Kafka
> >> > > along with some interface for entitlement (access management)
> systems
> >> > > (pretty standard) is all that I think needs to be done to the core
> >> > project.
> >> > >  I want to tackle item later in the year after summer after the
> other
> >> > three
> >> > > are complete.
> >> > >
> >> > > I look forward to thoughts on this and anyone else interested in
> >> working
> >> > > with us on these items.
> >> > >
> >> > > [0]
> >> > >
> >> > >
> >> >
> >>
> http://www.emc.com/emc-plus/rsa-labs/standards-initiatives/what-is-a-digital-envelope.htm
> >> > > [1] http://tools.ietf.org/html/rfc3126
> >> > > [2] http://tools.ietf.org/html/rfc3161
> >> > > [3]
> >> > >
> >> > >
> >> >
> >>
> http://www.emc.com/emc-plus/rsa-labs/standards-initiatives/pkcs-7-cryptographic-message-syntax-standar.htm
> >> > > [4] http://en.wikipedia.org/wiki/XML_Signature
> >> > > [5] http://en.wikipedia.org/wiki/PKCS_12
> >> > >
> >> > > /*******************************************
> >> > >  Joe Stein
> >> > >  Founder, Principal Consultant
> >> > >  Big Data Open Source Security LLC
> >> > >  http://www.stealth.ly
> >> > >  Twitter: @allthingshadoop <http://www.twitter.com/allthingshadoop>
> >> > > ********************************************/
> >> > >
> >> >
> >>
> >
> >
>



-- 
Thanks,
Raja.

Re: [DISCUSS] Kafka Security Specific Features

Posted by Pramod Deshmukh <dp...@gmail.com>.
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 <dp...@gmail.com> 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, Jul 16, 2014 at 3:57 PM, Joe Stein <jo...@stealth.ly> 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 (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 Stein
>>  Founder, Principal Consultant
>>  Big Data Open Source Security LLC
>>  http://www.stealth.ly
>>  Twitter: @allthingshadoop <http://www.twitter.com/allthingshadoop>
>> ********************************************/
>>
>>
>> On Wed, Jul 16, 2014 at 3:49 PM, Pramod Deshmukh <dp...@gmail.com>
>> wrote:
>>
>> > 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 installed sbt and scala and followed your instructions for
>> different
>> > scala versions. I was able to bring zookeeper up but brokers fail to
>> start
>> > with error
>> >
>> > Error: Could not find or load main class kafka.Kafka
>> >
>> > I think I am doing something wrong. Can you please help me?
>> >
>> > Our current production setup is with 2.8.0 and want to stick to it.
>> >
>> > Thanks,
>> >
>> > Pramod
>> >
>> >
>> > On Tue, Jun 3, 2014 at 3:57 PM, Joe Stein <jo...@stealth.ly> wrote:
>> >
>> > > 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.  This comes from what folks are doing "in the wild" to
>> > > workaround and implement security with Kafka as it is today and also
>> > what I
>> > > have discovered from organizations about their blockers. It also
>> picks up
>> > > from the wiki (which I should have time to update later in the week
>> based
>> > > on the below and feedback from the thread).
>> > >
>> > > 1) Transport Layer Security (i.e. SSL)
>> > >
>> > > This also includes client authentication in addition to in-transit
>> > security
>> > > layer.  This work has been picked up here
>> > > https://issues.apache.org/jira/browse/KAFKA-1477 and do appreciate
>> any
>> > > thoughts, comments, feedback, tomatoes, whatever for this patch.  It
>> is a
>> > > pickup from the fork of the work first done here
>> > > https://github.com/relango/kafka/tree/kafka_security.
>> > >
>> > > 2) Data encryption at rest.
>> > >
>> > > This is very important and something that can be facilitated within
>> the
>> > > wire protocol. It requires an additional map data structure for the
>> > > "encrypted [data encryption key]". With this map (either in your
>> object
>> > or
>> > > in the wire protocol) you can store the dynamically generated
>> symmetric
>> > key
>> > > (for each message) and then encrypt the data using that dynamically
>> > > generated key.  You then encrypt the encryption key using each public
>> key
>> > > for whom is expected to be able to decrypt the encryption key to then
>> > > decrypt the message.  For each public key encrypted symmetric key
>> (which
>> > is
>> > > now the "encrypted [data encryption key]" along with which public key
>> it
>> > > was encrypted with for (so a map of [publicKey] =
>> > > encryptedDataEncryptionKey) as a chain.   Other patterns can be
>> > implemented
>> > > but this is a pretty standard digital enveloping [0] pattern with
>> only 1
>> > > field added. Other patterns should be able to use that field to-do
>> their
>> > > implementation too.
>> > >
>> > > 3) Non-repudiation and long term non-repudiation.
>> > >
>> > > Non-repudiation is proving data hasn't changed.  This is often (if not
>> > > always) done with x509 public certificates (chained to a certificate
>> > > authority).
>> > >
>> > > Long term non-repudiation is what happens when the certificates of the
>> > > certificate authority are expired (or revoked) and everything ever
>> signed
>> > > (ever) with that certificate's public key then becomes "no longer
>> > provable
>> > > as ever being authentic".  That is where RFC3126 [1] and RFC3161 [2]
>> come
>> > > in (or worm drives [hardware], etc).
>> > >
>> > > For either (or both) of these it is an operation of the encryptor to
>> > > sign/hash the data (with or without third party trusted timestap of
>> the
>> > > signing event) and encrypt that with their own private key and
>> distribute
>> > > the results (before and after encrypting if required) along with their
>> > > public key. This structure is a bit more complex but feasible, it is a
>> > map
>> > > of digital signature formats and the chain of dig sig attestations.
>>  The
>> > > map's key being the method (i.e. CRC32, PKCS7 [3], XmlDigSig [4]) and
>> > then
>> > > a list of map where that key is "purpose" of signature (what your
>> > attesting
>> > > too).  As a sibling field to the list another field for "the
>> attester" as
>> > > bytes (e.g. their PKCS12 [5] for the map of PKCS7 signatures).
>> > >
>> > > 4) Authorization
>> > >
>> > > We should have a policy of "404" for data, topics, partitions (etc) if
>> > > authenticated connections do not have access.  In "secure mode" any
>> non
>> > > authenticated connections should get a "404" type message on
>> everything.
>> > > Knowing "something is there" is a security risk in many uses cases.
>>  So
>> > if
>> > > you don't have access you don't even see it.  Baking "that" into Kafka
>> > > along with some interface for entitlement (access management) systems
>> > > (pretty standard) is all that I think needs to be done to the core
>> > project.
>> > >  I want to tackle item later in the year after summer after the other
>> > three
>> > > are complete.
>> > >
>> > > I look forward to thoughts on this and anyone else interested in
>> working
>> > > with us on these items.
>> > >
>> > > [0]
>> > >
>> > >
>> >
>> http://www.emc.com/emc-plus/rsa-labs/standards-initiatives/what-is-a-digital-envelope.htm
>> > > [1] http://tools.ietf.org/html/rfc3126
>> > > [2] http://tools.ietf.org/html/rfc3161
>> > > [3]
>> > >
>> > >
>> >
>> http://www.emc.com/emc-plus/rsa-labs/standards-initiatives/pkcs-7-cryptographic-message-syntax-standar.htm
>> > > [4] http://en.wikipedia.org/wiki/XML_Signature
>> > > [5] http://en.wikipedia.org/wiki/PKCS_12
>> > >
>> > > /*******************************************
>> > >  Joe Stein
>> > >  Founder, Principal Consultant
>> > >  Big Data Open Source Security LLC
>> > >  http://www.stealth.ly
>> > >  Twitter: @allthingshadoop <http://www.twitter.com/allthingshadoop>
>> > > ********************************************/
>> > >
>> >
>>
>
>

Re: [DISCUSS] Kafka Security Specific Features

Posted by Pramod Deshmukh <dp...@gmail.com>.
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 <dp...@gmail.com> 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, Jul 16, 2014 at 3:57 PM, Joe Stein <jo...@stealth.ly> 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 (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 Stein
>>  Founder, Principal Consultant
>>  Big Data Open Source Security LLC
>>  http://www.stealth.ly
>>  Twitter: @allthingshadoop <http://www.twitter.com/allthingshadoop>
>> ********************************************/
>>
>>
>> On Wed, Jul 16, 2014 at 3:49 PM, Pramod Deshmukh <dp...@gmail.com>
>> wrote:
>>
>> > 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 installed sbt and scala and followed your instructions for
>> different
>> > scala versions. I was able to bring zookeeper up but brokers fail to
>> start
>> > with error
>> >
>> > Error: Could not find or load main class kafka.Kafka
>> >
>> > I think I am doing something wrong. Can you please help me?
>> >
>> > Our current production setup is with 2.8.0 and want to stick to it.
>> >
>> > Thanks,
>> >
>> > Pramod
>> >
>> >
>> > On Tue, Jun 3, 2014 at 3:57 PM, Joe Stein <jo...@stealth.ly> wrote:
>> >
>> > > 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.  This comes from what folks are doing "in the wild" to
>> > > workaround and implement security with Kafka as it is today and also
>> > what I
>> > > have discovered from organizations about their blockers. It also
>> picks up
>> > > from the wiki (which I should have time to update later in the week
>> based
>> > > on the below and feedback from the thread).
>> > >
>> > > 1) Transport Layer Security (i.e. SSL)
>> > >
>> > > This also includes client authentication in addition to in-transit
>> > security
>> > > layer.  This work has been picked up here
>> > > https://issues.apache.org/jira/browse/KAFKA-1477 and do appreciate
>> any
>> > > thoughts, comments, feedback, tomatoes, whatever for this patch.  It
>> is a
>> > > pickup from the fork of the work first done here
>> > > https://github.com/relango/kafka/tree/kafka_security.
>> > >
>> > > 2) Data encryption at rest.
>> > >
>> > > This is very important and something that can be facilitated within
>> the
>> > > wire protocol. It requires an additional map data structure for the
>> > > "encrypted [data encryption key]". With this map (either in your
>> object
>> > or
>> > > in the wire protocol) you can store the dynamically generated
>> symmetric
>> > key
>> > > (for each message) and then encrypt the data using that dynamically
>> > > generated key.  You then encrypt the encryption key using each public
>> key
>> > > for whom is expected to be able to decrypt the encryption key to then
>> > > decrypt the message.  For each public key encrypted symmetric key
>> (which
>> > is
>> > > now the "encrypted [data encryption key]" along with which public key
>> it
>> > > was encrypted with for (so a map of [publicKey] =
>> > > encryptedDataEncryptionKey) as a chain.   Other patterns can be
>> > implemented
>> > > but this is a pretty standard digital enveloping [0] pattern with
>> only 1
>> > > field added. Other patterns should be able to use that field to-do
>> their
>> > > implementation too.
>> > >
>> > > 3) Non-repudiation and long term non-repudiation.
>> > >
>> > > Non-repudiation is proving data hasn't changed.  This is often (if not
>> > > always) done with x509 public certificates (chained to a certificate
>> > > authority).
>> > >
>> > > Long term non-repudiation is what happens when the certificates of the
>> > > certificate authority are expired (or revoked) and everything ever
>> signed
>> > > (ever) with that certificate's public key then becomes "no longer
>> > provable
>> > > as ever being authentic".  That is where RFC3126 [1] and RFC3161 [2]
>> come
>> > > in (or worm drives [hardware], etc).
>> > >
>> > > For either (or both) of these it is an operation of the encryptor to
>> > > sign/hash the data (with or without third party trusted timestap of
>> the
>> > > signing event) and encrypt that with their own private key and
>> distribute
>> > > the results (before and after encrypting if required) along with their
>> > > public key. This structure is a bit more complex but feasible, it is a
>> > map
>> > > of digital signature formats and the chain of dig sig attestations.
>>  The
>> > > map's key being the method (i.e. CRC32, PKCS7 [3], XmlDigSig [4]) and
>> > then
>> > > a list of map where that key is "purpose" of signature (what your
>> > attesting
>> > > too).  As a sibling field to the list another field for "the
>> attester" as
>> > > bytes (e.g. their PKCS12 [5] for the map of PKCS7 signatures).
>> > >
>> > > 4) Authorization
>> > >
>> > > We should have a policy of "404" for data, topics, partitions (etc) if
>> > > authenticated connections do not have access.  In "secure mode" any
>> non
>> > > authenticated connections should get a "404" type message on
>> everything.
>> > > Knowing "something is there" is a security risk in many uses cases.
>>  So
>> > if
>> > > you don't have access you don't even see it.  Baking "that" into Kafka
>> > > along with some interface for entitlement (access management) systems
>> > > (pretty standard) is all that I think needs to be done to the core
>> > project.
>> > >  I want to tackle item later in the year after summer after the other
>> > three
>> > > are complete.
>> > >
>> > > I look forward to thoughts on this and anyone else interested in
>> working
>> > > with us on these items.
>> > >
>> > > [0]
>> > >
>> > >
>> >
>> http://www.emc.com/emc-plus/rsa-labs/standards-initiatives/what-is-a-digital-envelope.htm
>> > > [1] http://tools.ietf.org/html/rfc3126
>> > > [2] http://tools.ietf.org/html/rfc3161
>> > > [3]
>> > >
>> > >
>> >
>> http://www.emc.com/emc-plus/rsa-labs/standards-initiatives/pkcs-7-cryptographic-message-syntax-standar.htm
>> > > [4] http://en.wikipedia.org/wiki/XML_Signature
>> > > [5] http://en.wikipedia.org/wiki/PKCS_12
>> > >
>> > > /*******************************************
>> > >  Joe Stein
>> > >  Founder, Principal Consultant
>> > >  Big Data Open Source Security LLC
>> > >  http://www.stealth.ly
>> > >  Twitter: @allthingshadoop <http://www.twitter.com/allthingshadoop>
>> > > ********************************************/
>> > >
>> >
>>
>
>

Re: [DISCUSS] Kafka Security Specific Features

Posted by Pramod Deshmukh <dp...@gmail.com>.
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 <jo...@stealth.ly> 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 (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 Stein
>  Founder, Principal Consultant
>  Big Data Open Source Security LLC
>  http://www.stealth.ly
>  Twitter: @allthingshadoop <http://www.twitter.com/allthingshadoop>
> ********************************************/
>
>
> On Wed, Jul 16, 2014 at 3:49 PM, Pramod Deshmukh <dp...@gmail.com>
> wrote:
>
> > 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 installed sbt and scala and followed your instructions for
> different
> > scala versions. I was able to bring zookeeper up but brokers fail to
> start
> > with error
> >
> > Error: Could not find or load main class kafka.Kafka
> >
> > I think I am doing something wrong. Can you please help me?
> >
> > Our current production setup is with 2.8.0 and want to stick to it.
> >
> > Thanks,
> >
> > Pramod
> >
> >
> > On Tue, Jun 3, 2014 at 3:57 PM, Joe Stein <jo...@stealth.ly> wrote:
> >
> > > 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.  This comes from what folks are doing "in the wild" to
> > > workaround and implement security with Kafka as it is today and also
> > what I
> > > have discovered from organizations about their blockers. It also picks
> up
> > > from the wiki (which I should have time to update later in the week
> based
> > > on the below and feedback from the thread).
> > >
> > > 1) Transport Layer Security (i.e. SSL)
> > >
> > > This also includes client authentication in addition to in-transit
> > security
> > > layer.  This work has been picked up here
> > > https://issues.apache.org/jira/browse/KAFKA-1477 and do appreciate any
> > > thoughts, comments, feedback, tomatoes, whatever for this patch.  It
> is a
> > > pickup from the fork of the work first done here
> > > https://github.com/relango/kafka/tree/kafka_security.
> > >
> > > 2) Data encryption at rest.
> > >
> > > This is very important and something that can be facilitated within the
> > > wire protocol. It requires an additional map data structure for the
> > > "encrypted [data encryption key]". With this map (either in your object
> > or
> > > in the wire protocol) you can store the dynamically generated symmetric
> > key
> > > (for each message) and then encrypt the data using that dynamically
> > > generated key.  You then encrypt the encryption key using each public
> key
> > > for whom is expected to be able to decrypt the encryption key to then
> > > decrypt the message.  For each public key encrypted symmetric key
> (which
> > is
> > > now the "encrypted [data encryption key]" along with which public key
> it
> > > was encrypted with for (so a map of [publicKey] =
> > > encryptedDataEncryptionKey) as a chain.   Other patterns can be
> > implemented
> > > but this is a pretty standard digital enveloping [0] pattern with only
> 1
> > > field added. Other patterns should be able to use that field to-do
> their
> > > implementation too.
> > >
> > > 3) Non-repudiation and long term non-repudiation.
> > >
> > > Non-repudiation is proving data hasn't changed.  This is often (if not
> > > always) done with x509 public certificates (chained to a certificate
> > > authority).
> > >
> > > Long term non-repudiation is what happens when the certificates of the
> > > certificate authority are expired (or revoked) and everything ever
> signed
> > > (ever) with that certificate's public key then becomes "no longer
> > provable
> > > as ever being authentic".  That is where RFC3126 [1] and RFC3161 [2]
> come
> > > in (or worm drives [hardware], etc).
> > >
> > > For either (or both) of these it is an operation of the encryptor to
> > > sign/hash the data (with or without third party trusted timestap of the
> > > signing event) and encrypt that with their own private key and
> distribute
> > > the results (before and after encrypting if required) along with their
> > > public key. This structure is a bit more complex but feasible, it is a
> > map
> > > of digital signature formats and the chain of dig sig attestations.
>  The
> > > map's key being the method (i.e. CRC32, PKCS7 [3], XmlDigSig [4]) and
> > then
> > > a list of map where that key is "purpose" of signature (what your
> > attesting
> > > too).  As a sibling field to the list another field for "the attester"
> as
> > > bytes (e.g. their PKCS12 [5] for the map of PKCS7 signatures).
> > >
> > > 4) Authorization
> > >
> > > We should have a policy of "404" for data, topics, partitions (etc) if
> > > authenticated connections do not have access.  In "secure mode" any non
> > > authenticated connections should get a "404" type message on
> everything.
> > > Knowing "something is there" is a security risk in many uses cases.  So
> > if
> > > you don't have access you don't even see it.  Baking "that" into Kafka
> > > along with some interface for entitlement (access management) systems
> > > (pretty standard) is all that I think needs to be done to the core
> > project.
> > >  I want to tackle item later in the year after summer after the other
> > three
> > > are complete.
> > >
> > > I look forward to thoughts on this and anyone else interested in
> working
> > > with us on these items.
> > >
> > > [0]
> > >
> > >
> >
> http://www.emc.com/emc-plus/rsa-labs/standards-initiatives/what-is-a-digital-envelope.htm
> > > [1] http://tools.ietf.org/html/rfc3126
> > > [2] http://tools.ietf.org/html/rfc3161
> > > [3]
> > >
> > >
> >
> http://www.emc.com/emc-plus/rsa-labs/standards-initiatives/pkcs-7-cryptographic-message-syntax-standar.htm
> > > [4] http://en.wikipedia.org/wiki/XML_Signature
> > > [5] http://en.wikipedia.org/wiki/PKCS_12
> > >
> > > /*******************************************
> > >  Joe Stein
> > >  Founder, Principal Consultant
> > >  Big Data Open Source Security LLC
> > >  http://www.stealth.ly
> > >  Twitter: @allthingshadoop <http://www.twitter.com/allthingshadoop>
> > > ********************************************/
> > >
> >
>

Re: [DISCUSS] Kafka Security Specific Features

Posted by Pramod Deshmukh <dp...@gmail.com>.
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 <jo...@stealth.ly> 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 (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 Stein
>  Founder, Principal Consultant
>  Big Data Open Source Security LLC
>  http://www.stealth.ly
>  Twitter: @allthingshadoop <http://www.twitter.com/allthingshadoop>
> ********************************************/
>
>
> On Wed, Jul 16, 2014 at 3:49 PM, Pramod Deshmukh <dp...@gmail.com>
> wrote:
>
> > 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 installed sbt and scala and followed your instructions for
> different
> > scala versions. I was able to bring zookeeper up but brokers fail to
> start
> > with error
> >
> > Error: Could not find or load main class kafka.Kafka
> >
> > I think I am doing something wrong. Can you please help me?
> >
> > Our current production setup is with 2.8.0 and want to stick to it.
> >
> > Thanks,
> >
> > Pramod
> >
> >
> > On Tue, Jun 3, 2014 at 3:57 PM, Joe Stein <jo...@stealth.ly> wrote:
> >
> > > 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.  This comes from what folks are doing "in the wild" to
> > > workaround and implement security with Kafka as it is today and also
> > what I
> > > have discovered from organizations about their blockers. It also picks
> up
> > > from the wiki (which I should have time to update later in the week
> based
> > > on the below and feedback from the thread).
> > >
> > > 1) Transport Layer Security (i.e. SSL)
> > >
> > > This also includes client authentication in addition to in-transit
> > security
> > > layer.  This work has been picked up here
> > > https://issues.apache.org/jira/browse/KAFKA-1477 and do appreciate any
> > > thoughts, comments, feedback, tomatoes, whatever for this patch.  It
> is a
> > > pickup from the fork of the work first done here
> > > https://github.com/relango/kafka/tree/kafka_security.
> > >
> > > 2) Data encryption at rest.
> > >
> > > This is very important and something that can be facilitated within the
> > > wire protocol. It requires an additional map data structure for the
> > > "encrypted [data encryption key]". With this map (either in your object
> > or
> > > in the wire protocol) you can store the dynamically generated symmetric
> > key
> > > (for each message) and then encrypt the data using that dynamically
> > > generated key.  You then encrypt the encryption key using each public
> key
> > > for whom is expected to be able to decrypt the encryption key to then
> > > decrypt the message.  For each public key encrypted symmetric key
> (which
> > is
> > > now the "encrypted [data encryption key]" along with which public key
> it
> > > was encrypted with for (so a map of [publicKey] =
> > > encryptedDataEncryptionKey) as a chain.   Other patterns can be
> > implemented
> > > but this is a pretty standard digital enveloping [0] pattern with only
> 1
> > > field added. Other patterns should be able to use that field to-do
> their
> > > implementation too.
> > >
> > > 3) Non-repudiation and long term non-repudiation.
> > >
> > > Non-repudiation is proving data hasn't changed.  This is often (if not
> > > always) done with x509 public certificates (chained to a certificate
> > > authority).
> > >
> > > Long term non-repudiation is what happens when the certificates of the
> > > certificate authority are expired (or revoked) and everything ever
> signed
> > > (ever) with that certificate's public key then becomes "no longer
> > provable
> > > as ever being authentic".  That is where RFC3126 [1] and RFC3161 [2]
> come
> > > in (or worm drives [hardware], etc).
> > >
> > > For either (or both) of these it is an operation of the encryptor to
> > > sign/hash the data (with or without third party trusted timestap of the
> > > signing event) and encrypt that with their own private key and
> distribute
> > > the results (before and after encrypting if required) along with their
> > > public key. This structure is a bit more complex but feasible, it is a
> > map
> > > of digital signature formats and the chain of dig sig attestations.
>  The
> > > map's key being the method (i.e. CRC32, PKCS7 [3], XmlDigSig [4]) and
> > then
> > > a list of map where that key is "purpose" of signature (what your
> > attesting
> > > too).  As a sibling field to the list another field for "the attester"
> as
> > > bytes (e.g. their PKCS12 [5] for the map of PKCS7 signatures).
> > >
> > > 4) Authorization
> > >
> > > We should have a policy of "404" for data, topics, partitions (etc) if
> > > authenticated connections do not have access.  In "secure mode" any non
> > > authenticated connections should get a "404" type message on
> everything.
> > > Knowing "something is there" is a security risk in many uses cases.  So
> > if
> > > you don't have access you don't even see it.  Baking "that" into Kafka
> > > along with some interface for entitlement (access management) systems
> > > (pretty standard) is all that I think needs to be done to the core
> > project.
> > >  I want to tackle item later in the year after summer after the other
> > three
> > > are complete.
> > >
> > > I look forward to thoughts on this and anyone else interested in
> working
> > > with us on these items.
> > >
> > > [0]
> > >
> > >
> >
> http://www.emc.com/emc-plus/rsa-labs/standards-initiatives/what-is-a-digital-envelope.htm
> > > [1] http://tools.ietf.org/html/rfc3126
> > > [2] http://tools.ietf.org/html/rfc3161
> > > [3]
> > >
> > >
> >
> http://www.emc.com/emc-plus/rsa-labs/standards-initiatives/pkcs-7-cryptographic-message-syntax-standar.htm
> > > [4] http://en.wikipedia.org/wiki/XML_Signature
> > > [5] http://en.wikipedia.org/wiki/PKCS_12
> > >
> > > /*******************************************
> > >  Joe Stein
> > >  Founder, Principal Consultant
> > >  Big Data Open Source Security LLC
> > >  http://www.stealth.ly
> > >  Twitter: @allthingshadoop <http://www.twitter.com/allthingshadoop>
> > > ********************************************/
> > >
> >
>

Re: [DISCUSS] Kafka Security Specific Features

Posted by Joe Stein <jo...@stealth.ly>.
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 Stein
 Founder, Principal Consultant
 Big Data Open Source Security LLC
 http://www.stealth.ly
 Twitter: @allthingshadoop <http://www.twitter.com/allthingshadoop>
********************************************/


On Wed, Jul 16, 2014 at 3:49 PM, Pramod Deshmukh <dp...@gmail.com> wrote:

> 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 installed sbt and scala and followed your instructions for different
> scala versions. I was able to bring zookeeper up but brokers fail to start
> with error
>
> Error: Could not find or load main class kafka.Kafka
>
> I think I am doing something wrong. Can you please help me?
>
> Our current production setup is with 2.8.0 and want to stick to it.
>
> Thanks,
>
> Pramod
>
>
> On Tue, Jun 3, 2014 at 3:57 PM, Joe Stein <jo...@stealth.ly> wrote:
>
> > 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.  This comes from what folks are doing "in the wild" to
> > workaround and implement security with Kafka as it is today and also
> what I
> > have discovered from organizations about their blockers. It also picks up
> > from the wiki (which I should have time to update later in the week based
> > on the below and feedback from the thread).
> >
> > 1) Transport Layer Security (i.e. SSL)
> >
> > This also includes client authentication in addition to in-transit
> security
> > layer.  This work has been picked up here
> > https://issues.apache.org/jira/browse/KAFKA-1477 and do appreciate any
> > thoughts, comments, feedback, tomatoes, whatever for this patch.  It is a
> > pickup from the fork of the work first done here
> > https://github.com/relango/kafka/tree/kafka_security.
> >
> > 2) Data encryption at rest.
> >
> > This is very important and something that can be facilitated within the
> > wire protocol. It requires an additional map data structure for the
> > "encrypted [data encryption key]". With this map (either in your object
> or
> > in the wire protocol) you can store the dynamically generated symmetric
> key
> > (for each message) and then encrypt the data using that dynamically
> > generated key.  You then encrypt the encryption key using each public key
> > for whom is expected to be able to decrypt the encryption key to then
> > decrypt the message.  For each public key encrypted symmetric key (which
> is
> > now the "encrypted [data encryption key]" along with which public key it
> > was encrypted with for (so a map of [publicKey] =
> > encryptedDataEncryptionKey) as a chain.   Other patterns can be
> implemented
> > but this is a pretty standard digital enveloping [0] pattern with only 1
> > field added. Other patterns should be able to use that field to-do their
> > implementation too.
> >
> > 3) Non-repudiation and long term non-repudiation.
> >
> > Non-repudiation is proving data hasn't changed.  This is often (if not
> > always) done with x509 public certificates (chained to a certificate
> > authority).
> >
> > Long term non-repudiation is what happens when the certificates of the
> > certificate authority are expired (or revoked) and everything ever signed
> > (ever) with that certificate's public key then becomes "no longer
> provable
> > as ever being authentic".  That is where RFC3126 [1] and RFC3161 [2] come
> > in (or worm drives [hardware], etc).
> >
> > For either (or both) of these it is an operation of the encryptor to
> > sign/hash the data (with or without third party trusted timestap of the
> > signing event) and encrypt that with their own private key and distribute
> > the results (before and after encrypting if required) along with their
> > public key. This structure is a bit more complex but feasible, it is a
> map
> > of digital signature formats and the chain of dig sig attestations.  The
> > map's key being the method (i.e. CRC32, PKCS7 [3], XmlDigSig [4]) and
> then
> > a list of map where that key is "purpose" of signature (what your
> attesting
> > too).  As a sibling field to the list another field for "the attester" as
> > bytes (e.g. their PKCS12 [5] for the map of PKCS7 signatures).
> >
> > 4) Authorization
> >
> > We should have a policy of "404" for data, topics, partitions (etc) if
> > authenticated connections do not have access.  In "secure mode" any non
> > authenticated connections should get a "404" type message on everything.
> > Knowing "something is there" is a security risk in many uses cases.  So
> if
> > you don't have access you don't even see it.  Baking "that" into Kafka
> > along with some interface for entitlement (access management) systems
> > (pretty standard) is all that I think needs to be done to the core
> project.
> >  I want to tackle item later in the year after summer after the other
> three
> > are complete.
> >
> > I look forward to thoughts on this and anyone else interested in working
> > with us on these items.
> >
> > [0]
> >
> >
> http://www.emc.com/emc-plus/rsa-labs/standards-initiatives/what-is-a-digital-envelope.htm
> > [1] http://tools.ietf.org/html/rfc3126
> > [2] http://tools.ietf.org/html/rfc3161
> > [3]
> >
> >
> http://www.emc.com/emc-plus/rsa-labs/standards-initiatives/pkcs-7-cryptographic-message-syntax-standar.htm
> > [4] http://en.wikipedia.org/wiki/XML_Signature
> > [5] http://en.wikipedia.org/wiki/PKCS_12
> >
> > /*******************************************
> >  Joe Stein
> >  Founder, Principal Consultant
> >  Big Data Open Source Security LLC
> >  http://www.stealth.ly
> >  Twitter: @allthingshadoop <http://www.twitter.com/allthingshadoop>
> > ********************************************/
> >
>

Re: [DISCUSS] Kafka Security Specific Features

Posted by Joe Stein <jo...@stealth.ly>.
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 Stein
 Founder, Principal Consultant
 Big Data Open Source Security LLC
 http://www.stealth.ly
 Twitter: @allthingshadoop <http://www.twitter.com/allthingshadoop>
********************************************/


On Wed, Jul 16, 2014 at 3:49 PM, Pramod Deshmukh <dp...@gmail.com> wrote:

> 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 installed sbt and scala and followed your instructions for different
> scala versions. I was able to bring zookeeper up but brokers fail to start
> with error
>
> Error: Could not find or load main class kafka.Kafka
>
> I think I am doing something wrong. Can you please help me?
>
> Our current production setup is with 2.8.0 and want to stick to it.
>
> Thanks,
>
> Pramod
>
>
> On Tue, Jun 3, 2014 at 3:57 PM, Joe Stein <jo...@stealth.ly> wrote:
>
> > 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.  This comes from what folks are doing "in the wild" to
> > workaround and implement security with Kafka as it is today and also
> what I
> > have discovered from organizations about their blockers. It also picks up
> > from the wiki (which I should have time to update later in the week based
> > on the below and feedback from the thread).
> >
> > 1) Transport Layer Security (i.e. SSL)
> >
> > This also includes client authentication in addition to in-transit
> security
> > layer.  This work has been picked up here
> > https://issues.apache.org/jira/browse/KAFKA-1477 and do appreciate any
> > thoughts, comments, feedback, tomatoes, whatever for this patch.  It is a
> > pickup from the fork of the work first done here
> > https://github.com/relango/kafka/tree/kafka_security.
> >
> > 2) Data encryption at rest.
> >
> > This is very important and something that can be facilitated within the
> > wire protocol. It requires an additional map data structure for the
> > "encrypted [data encryption key]". With this map (either in your object
> or
> > in the wire protocol) you can store the dynamically generated symmetric
> key
> > (for each message) and then encrypt the data using that dynamically
> > generated key.  You then encrypt the encryption key using each public key
> > for whom is expected to be able to decrypt the encryption key to then
> > decrypt the message.  For each public key encrypted symmetric key (which
> is
> > now the "encrypted [data encryption key]" along with which public key it
> > was encrypted with for (so a map of [publicKey] =
> > encryptedDataEncryptionKey) as a chain.   Other patterns can be
> implemented
> > but this is a pretty standard digital enveloping [0] pattern with only 1
> > field added. Other patterns should be able to use that field to-do their
> > implementation too.
> >
> > 3) Non-repudiation and long term non-repudiation.
> >
> > Non-repudiation is proving data hasn't changed.  This is often (if not
> > always) done with x509 public certificates (chained to a certificate
> > authority).
> >
> > Long term non-repudiation is what happens when the certificates of the
> > certificate authority are expired (or revoked) and everything ever signed
> > (ever) with that certificate's public key then becomes "no longer
> provable
> > as ever being authentic".  That is where RFC3126 [1] and RFC3161 [2] come
> > in (or worm drives [hardware], etc).
> >
> > For either (or both) of these it is an operation of the encryptor to
> > sign/hash the data (with or without third party trusted timestap of the
> > signing event) and encrypt that with their own private key and distribute
> > the results (before and after encrypting if required) along with their
> > public key. This structure is a bit more complex but feasible, it is a
> map
> > of digital signature formats and the chain of dig sig attestations.  The
> > map's key being the method (i.e. CRC32, PKCS7 [3], XmlDigSig [4]) and
> then
> > a list of map where that key is "purpose" of signature (what your
> attesting
> > too).  As a sibling field to the list another field for "the attester" as
> > bytes (e.g. their PKCS12 [5] for the map of PKCS7 signatures).
> >
> > 4) Authorization
> >
> > We should have a policy of "404" for data, topics, partitions (etc) if
> > authenticated connections do not have access.  In "secure mode" any non
> > authenticated connections should get a "404" type message on everything.
> > Knowing "something is there" is a security risk in many uses cases.  So
> if
> > you don't have access you don't even see it.  Baking "that" into Kafka
> > along with some interface for entitlement (access management) systems
> > (pretty standard) is all that I think needs to be done to the core
> project.
> >  I want to tackle item later in the year after summer after the other
> three
> > are complete.
> >
> > I look forward to thoughts on this and anyone else interested in working
> > with us on these items.
> >
> > [0]
> >
> >
> http://www.emc.com/emc-plus/rsa-labs/standards-initiatives/what-is-a-digital-envelope.htm
> > [1] http://tools.ietf.org/html/rfc3126
> > [2] http://tools.ietf.org/html/rfc3161
> > [3]
> >
> >
> http://www.emc.com/emc-plus/rsa-labs/standards-initiatives/pkcs-7-cryptographic-message-syntax-standar.htm
> > [4] http://en.wikipedia.org/wiki/XML_Signature
> > [5] http://en.wikipedia.org/wiki/PKCS_12
> >
> > /*******************************************
> >  Joe Stein
> >  Founder, Principal Consultant
> >  Big Data Open Source Security LLC
> >  http://www.stealth.ly
> >  Twitter: @allthingshadoop <http://www.twitter.com/allthingshadoop>
> > ********************************************/
> >
>

Re: [DISCUSS] Kafka Security Specific Features

Posted by Pramod Deshmukh <dp...@gmail.com>.
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 installed sbt and scala and followed your instructions for different
scala versions. I was able to bring zookeeper up but brokers fail to start
with error

Error: Could not find or load main class kafka.Kafka

I think I am doing something wrong. Can you please help me?

Our current production setup is with 2.8.0 and want to stick to it.

Thanks,

Pramod


On Tue, Jun 3, 2014 at 3:57 PM, Joe Stein <jo...@stealth.ly> wrote:

> 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.  This comes from what folks are doing "in the wild" to
> workaround and implement security with Kafka as it is today and also what I
> have discovered from organizations about their blockers. It also picks up
> from the wiki (which I should have time to update later in the week based
> on the below and feedback from the thread).
>
> 1) Transport Layer Security (i.e. SSL)
>
> This also includes client authentication in addition to in-transit security
> layer.  This work has been picked up here
> https://issues.apache.org/jira/browse/KAFKA-1477 and do appreciate any
> thoughts, comments, feedback, tomatoes, whatever for this patch.  It is a
> pickup from the fork of the work first done here
> https://github.com/relango/kafka/tree/kafka_security.
>
> 2) Data encryption at rest.
>
> This is very important and something that can be facilitated within the
> wire protocol. It requires an additional map data structure for the
> "encrypted [data encryption key]". With this map (either in your object or
> in the wire protocol) you can store the dynamically generated symmetric key
> (for each message) and then encrypt the data using that dynamically
> generated key.  You then encrypt the encryption key using each public key
> for whom is expected to be able to decrypt the encryption key to then
> decrypt the message.  For each public key encrypted symmetric key (which is
> now the "encrypted [data encryption key]" along with which public key it
> was encrypted with for (so a map of [publicKey] =
> encryptedDataEncryptionKey) as a chain.   Other patterns can be implemented
> but this is a pretty standard digital enveloping [0] pattern with only 1
> field added. Other patterns should be able to use that field to-do their
> implementation too.
>
> 3) Non-repudiation and long term non-repudiation.
>
> Non-repudiation is proving data hasn't changed.  This is often (if not
> always) done with x509 public certificates (chained to a certificate
> authority).
>
> Long term non-repudiation is what happens when the certificates of the
> certificate authority are expired (or revoked) and everything ever signed
> (ever) with that certificate's public key then becomes "no longer provable
> as ever being authentic".  That is where RFC3126 [1] and RFC3161 [2] come
> in (or worm drives [hardware], etc).
>
> For either (or both) of these it is an operation of the encryptor to
> sign/hash the data (with or without third party trusted timestap of the
> signing event) and encrypt that with their own private key and distribute
> the results (before and after encrypting if required) along with their
> public key. This structure is a bit more complex but feasible, it is a map
> of digital signature formats and the chain of dig sig attestations.  The
> map's key being the method (i.e. CRC32, PKCS7 [3], XmlDigSig [4]) and then
> a list of map where that key is "purpose" of signature (what your attesting
> too).  As a sibling field to the list another field for "the attester" as
> bytes (e.g. their PKCS12 [5] for the map of PKCS7 signatures).
>
> 4) Authorization
>
> We should have a policy of "404" for data, topics, partitions (etc) if
> authenticated connections do not have access.  In "secure mode" any non
> authenticated connections should get a "404" type message on everything.
> Knowing "something is there" is a security risk in many uses cases.  So if
> you don't have access you don't even see it.  Baking "that" into Kafka
> along with some interface for entitlement (access management) systems
> (pretty standard) is all that I think needs to be done to the core project.
>  I want to tackle item later in the year after summer after the other three
> are complete.
>
> I look forward to thoughts on this and anyone else interested in working
> with us on these items.
>
> [0]
>
> http://www.emc.com/emc-plus/rsa-labs/standards-initiatives/what-is-a-digital-envelope.htm
> [1] http://tools.ietf.org/html/rfc3126
> [2] http://tools.ietf.org/html/rfc3161
> [3]
>
> http://www.emc.com/emc-plus/rsa-labs/standards-initiatives/pkcs-7-cryptographic-message-syntax-standar.htm
> [4] http://en.wikipedia.org/wiki/XML_Signature
> [5] http://en.wikipedia.org/wiki/PKCS_12
>
> /*******************************************
>  Joe Stein
>  Founder, Principal Consultant
>  Big Data Open Source Security LLC
>  http://www.stealth.ly
>  Twitter: @allthingshadoop <http://www.twitter.com/allthingshadoop>
> ********************************************/
>

Re: [DISCUSS] Kafka Security Specific Features

Posted by Pramod Deshmukh <dp...@gmail.com>.
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 installed sbt and scala and followed your instructions for different
scala versions. I was able to bring zookeeper up but brokers fail to start
with error

Error: Could not find or load main class kafka.Kafka

I think I am doing something wrong. Can you please help me?

Our current production setup is with 2.8.0 and want to stick to it.

Thanks,

Pramod


On Tue, Jun 3, 2014 at 3:57 PM, Joe Stein <jo...@stealth.ly> wrote:

> 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.  This comes from what folks are doing "in the wild" to
> workaround and implement security with Kafka as it is today and also what I
> have discovered from organizations about their blockers. It also picks up
> from the wiki (which I should have time to update later in the week based
> on the below and feedback from the thread).
>
> 1) Transport Layer Security (i.e. SSL)
>
> This also includes client authentication in addition to in-transit security
> layer.  This work has been picked up here
> https://issues.apache.org/jira/browse/KAFKA-1477 and do appreciate any
> thoughts, comments, feedback, tomatoes, whatever for this patch.  It is a
> pickup from the fork of the work first done here
> https://github.com/relango/kafka/tree/kafka_security.
>
> 2) Data encryption at rest.
>
> This is very important and something that can be facilitated within the
> wire protocol. It requires an additional map data structure for the
> "encrypted [data encryption key]". With this map (either in your object or
> in the wire protocol) you can store the dynamically generated symmetric key
> (for each message) and then encrypt the data using that dynamically
> generated key.  You then encrypt the encryption key using each public key
> for whom is expected to be able to decrypt the encryption key to then
> decrypt the message.  For each public key encrypted symmetric key (which is
> now the "encrypted [data encryption key]" along with which public key it
> was encrypted with for (so a map of [publicKey] =
> encryptedDataEncryptionKey) as a chain.   Other patterns can be implemented
> but this is a pretty standard digital enveloping [0] pattern with only 1
> field added. Other patterns should be able to use that field to-do their
> implementation too.
>
> 3) Non-repudiation and long term non-repudiation.
>
> Non-repudiation is proving data hasn't changed.  This is often (if not
> always) done with x509 public certificates (chained to a certificate
> authority).
>
> Long term non-repudiation is what happens when the certificates of the
> certificate authority are expired (or revoked) and everything ever signed
> (ever) with that certificate's public key then becomes "no longer provable
> as ever being authentic".  That is where RFC3126 [1] and RFC3161 [2] come
> in (or worm drives [hardware], etc).
>
> For either (or both) of these it is an operation of the encryptor to
> sign/hash the data (with or without third party trusted timestap of the
> signing event) and encrypt that with their own private key and distribute
> the results (before and after encrypting if required) along with their
> public key. This structure is a bit more complex but feasible, it is a map
> of digital signature formats and the chain of dig sig attestations.  The
> map's key being the method (i.e. CRC32, PKCS7 [3], XmlDigSig [4]) and then
> a list of map where that key is "purpose" of signature (what your attesting
> too).  As a sibling field to the list another field for "the attester" as
> bytes (e.g. their PKCS12 [5] for the map of PKCS7 signatures).
>
> 4) Authorization
>
> We should have a policy of "404" for data, topics, partitions (etc) if
> authenticated connections do not have access.  In "secure mode" any non
> authenticated connections should get a "404" type message on everything.
> Knowing "something is there" is a security risk in many uses cases.  So if
> you don't have access you don't even see it.  Baking "that" into Kafka
> along with some interface for entitlement (access management) systems
> (pretty standard) is all that I think needs to be done to the core project.
>  I want to tackle item later in the year after summer after the other three
> are complete.
>
> I look forward to thoughts on this and anyone else interested in working
> with us on these items.
>
> [0]
>
> http://www.emc.com/emc-plus/rsa-labs/standards-initiatives/what-is-a-digital-envelope.htm
> [1] http://tools.ietf.org/html/rfc3126
> [2] http://tools.ietf.org/html/rfc3161
> [3]
>
> http://www.emc.com/emc-plus/rsa-labs/standards-initiatives/pkcs-7-cryptographic-message-syntax-standar.htm
> [4] http://en.wikipedia.org/wiki/XML_Signature
> [5] http://en.wikipedia.org/wiki/PKCS_12
>
> /*******************************************
>  Joe Stein
>  Founder, Principal Consultant
>  Big Data Open Source Security LLC
>  http://www.stealth.ly
>  Twitter: @allthingshadoop <http://www.twitter.com/allthingshadoop>
> ********************************************/
>