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 <tpal...@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" <robert.w.with...@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" <robert.w.with...@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" <pradeep...@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.with...@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 <joe.st...@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 <jay.kr...@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
>>>>>>>>>>> <joe.st...@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
>>>>>>>>>>>> ********************************************/
> 

Reply via email to