Thanks James for taking the time out.

My comments per solution below you commented about. (I note you didn’t comment 
on the 3rd at all , which is the current proposal in the kip)
1) 
a. This forces all clients to have distinct knowledge of platform level 
implementation detail 
b. enforces single serialization technology for all apps payloads and platform 
headers
i. what if apps need to have different serialization e.g. app team need to use 
XML for legacy system reasons but we force at a platform to have to use avro 
because of our headers
c. If we were to have a common Kafka solution, this would force everyone onto a 
single serialization solution, I think this is something we don’t want to do?
d. this doesn’t deal with having large payloads as you’ve mentioned http in 
second solution, think of MIME multipart.
e. End2End encryption, if apps need end2end encryption then platform tooling 
cannot read the header information without decoding the message that then 
breaks reasons for having e2e encryption.
2) 
a. Container is the solution we currently use (we don’t use MIME but it looks 
like a not bad choice if you don’t care about size, or you have big enough 
payloads its small overhead)
i. I think if we don’t go with adding the headers to the message and offset , 
having an common agreed container format is the next best offering.
b. The TiVO specific HTTP MIME type message is indeed a good solution in our 
view
i. Deals with separating headers and payload
ii. Allows multipart messaging
iii. Allows payload to be encrypted yet headers not
iv. Platform tooling doesn’t care about payload and can quickly read headers
v. Well established and known container solution
c. HTTP MIME type headers (String keys) has a large byte overhead though
i. See Nacho’s and Radai’s previous points on this
d. If we agree on say a container format being MIME how does a platform team 
integrate adding its needed headers without enforcing all teams to have to be 
aware of it? Or is this actually ok?
i. Would we make a new consumer and producer Kafka API that is container aware?
e. How would this work with the likes of Kafka Streams , where as a platform 
team we want to add some meta data needed to ever message but we don’t want to 
recode these frameworks.




On 10/29/16, 8:09 AM, "James Cheng" <wushuja...@gmail.com> wrote:

    Let me talk about the container format that we are using here at TiVo to 
add headers to our Kafka messages.
    
    Just some quick terminology, so that I don't confuse everyone.
    I'm going to use "message body" to refer to the thing returned by 
ConsumerRecord.value()
    And I'm going to use "payload" to refer to your data after it has been 
serialized into bytes.
    
    To recap, during the KIP call, we talked about 3 ways to have headers in 
Kafka messages:
    1) The message body is your payload, which has headers within it.
    2) The message body is a container, which has headers in it as well your 
payload.
    3) Extend Kafka to hold headers outside of the message body. The message 
body holds your payload.
    
    1) The message body is your payload, which has headers in it
    -----------------------
    Here's an example of what this may look like, if it were rendered in JSON:
    
    {
        "headers" : {
            "Host" : "host.domain.com",
            "Service" : "PaymentProcessor",
            "Timestamp" : "2016-10-28 12:45:56"
        },
        "Field1" : "value",
        "Field2" : "value"
    }
    
    In this scenario, headers are really not anything special. They are a part 
of your payload. They may have been auto-included by some mechanism in all of 
your schemas, but they really just are part of your payload. I believe LinkedIn 
uses this mechanism. The "headers" field is a reserved word in all schemas, and 
is somehow auto-inserted into all schemas. The headers schema contains a couple 
fields like "host" and "service" and "timestamp". If LinkedIn decides that a 
new field needs to be added for company-wide infrastructure purposes, then they 
will add it to the schema of "headers", and because "headers" is included 
everywhere, then all schemas will get updated as well.
    
    Because they are simply part of your payload, you need to deserialize your 
payload in order to read the headers.
    
    3) Extend Kafka to hold headers outside of the message body. The message 
body holds your payload.
    -------------
    This is what this KIP is discussing. I will let others talk about this.
    
    2) The message body is a container, which has headers in it, as well as 
your payload.
    --------------
    At TiVo, we have standardized on a container format that looks very similar 
to HTTP. Let me jump straight to an example:
    
    ----- example below ----
    JS/1 123 1024
    Host: host.domain.com
    Service: SomethingProcessor
    Timestamp: 2016-10-28 12:45:56
    ObjectTypeInPayload: MyObjectV1
    
    {
        "Field1" : "value",
        "Field2" : "value"
    }
    ----- example above ----
    
    Ignore the first line for now. Lines 2-5 are headers. Then there is a blank 
line, and then after that is your payload.  The field "ObjectTypeInPayload" 
describes what schema applies to the payload. In order to decode your payload, 
you read the field "ObjectTypeInPayload" and use that to decide how to decode 
the payload.
    
    Okay, let's talk about details.
    The first line is "JS/1 123 1024". 
    * The JS/1 thing is the "schema" of the container. JS is the container 
type, 1 is the version number of the JS container. This particular version of 
the JS container means those 4 specific headers are present, and that the 
payload is encoded in JSON.
    * The 123 is the length in bytes of the header section. (This particular 
example probably isn't exactly 123 bytes)
    * The 1024 is the length in bytes of the payload. (This particular example 
probably isn't exactly 1024 bytes)
    The 123 and 1024 allow the deserializer to quickly jump to the different 
sections. I don't know how necessary they are. It's possible they are an over 
optimization. They are kind of a holdover from a previous wireformat here at 
TiVo where we were pipelining messages over TCP as one continuous bytestream  
(NOT using Kafka), and we needed to be able to know where one object ended and 
another started, and also be able to skip messages that we didn't care about.
    
    Let me show another made up example of this container format being used:
    
    ---- example below ----
    AV/1 123 1024
    Host: host.domain.com
    Service: SomethingProcessor
    Timestamp: 2016-10-28 12:45:56
    
    0xFF
    BYTESOFDATA
    ---- example above ----
    
    This container is of type AV/1. This means that the payload is a magic byte 
followed by a stream of bytes. The magic byte is schema registry ID which is 
used to look up the schema, which is then used to decode the rest of the bytes 
in the payload.
    
    Notice that this is a different use of the same container syntax. In this 
case, the schema ID was a byte in the payload. In the JS/1 case, the schema ID 
was stored in a header.
    
    Here is a more precise description of the container format:
    ---- container format below ----
    <tag><headers length><payload length>\r\n
    header: value\r\n
    header: value\r\n
    \r\n
    payload
    ---- container format above ----
    
    As I mentioned above, the headers length and payload length might not be 
necessary. You can also simply scan the message body until the first occurence 
of \r\n\r\n
    
    Let's talk about pros/cons.
    
    Pros:
    * Headers do not affect the payload. An addition of a header does not 
effect the schema of the payload.
    * Payload serialization can be different for different use cases. This 
container format can carry a payload that is Avro or JSON or Thrift or 
whatever. The payload is just a stream of bytes.
    * Headers can be read without deserializing the payload
    * Headers can have a schema. In the JS/1 case, I use "JS/1" to mean that 
"There are 4 required fields. Host is a string, Service is a string, Timestamp 
is a time in ISO(something) format, ObjectTypeInPayload is a String, and the 
payload is in JSON"
    * Plaintext headers with a relatively simple syntax is pretty easy to parse 
in any programming language.
    
    Cons:
    * Double serialization upon writes. In order to create the message body, 
you first have to create your payload (which means you serialize your object 
into an array of bytes) and then tack headers onto the front of it. And if you 
do the optimization where your store the length of the payload, you actually 
have to do it in this order. Which means you have to encode the payload first 
and store the whole thing in memory before creating your message body.
    * Double deserialization upon reads. You *might* need to read the headers 
so that you can figure out how to read the payload. It depends on how you use 
the container. In the JS/1 case, I had to read the ObjectIdInPayload field in 
order to deserialize the payload. However, in the AV/1 case, you did NOT have 
to read any of the headers in order to deserialize the payload.
    * What if I want my header values to be complex types? What if I wanted to 
store a header where the value was an array?  Do I start relying on stuff like 
comma-separated strings to indicate arrays? What if I wanted to store a header 
where the value was binary bytes? Do I insist that headers all must be ASCII 
encoded? I realize this conflicts with what I said above about headers being 
easy to parse. Maybe they are actually more complex that I realized. 
    * Size overhead of the container format and headers: If I have a 10 byte 
payload, but my container is 512 bytes of ascii-encoded strings, is it worth it?
    
    Alternatives:
    * I can imagine doing something similar to the above, but using Avro as the 
serialization format for the container. The avro schemas would be like the 
following (apologies if I got those wrong, I actually haven't used avro)
    
    {
        "type": "record", 
        "name": "JS",
        "fields" : [
            {"name": "Host", "type" : "string"},
            {"name": "Service", "type" : "string"},
            {"name": "Timestamp", "type" : "double"},
            {"name": "ObjectTypeInPayload", "type" : "string"},
            {"name": "payload", "type": "bytes"}
        ]
    }
    
    {
        "type": "record", 
        "name": "AV",
        "fields" : [
            {"name": "Host", "type" : "string"},
            {"name": "Service", "type" : "string"},
            {"name": "Timestamp", "type" : "double"},
            {"name": "payload", "type": "bytes"}
        ]
    }
    
    You would use avro to deserialize the container, and then potentially use a 
different deserializer for the payload. Using avro would potentially reduce the 
overhead of the container format, and let you use complex types in your 
headers. However, this would mean people would still have to use avro for 
deserializing a Kafka message body.
    
    Our experience using this at TiVo:
    * We haven't run into any problems so far.
    * We are not yet running Kafka in production, so we don't yet have a lot of 
traffic running through our brokers.
    * Even when we go to production, we expect that the amount of data that we 
have will be relatively small compared to most companies. So we're hoping that 
the overhead of the container format will be okay for our use cases.
    
    Phew, okay, that's enough for now. Let's discuss.
    
    -James
    
    > On Oct 27, 2016, at 12:19 AM, James Cheng <wushuja...@gmail.com> wrote:
    > 
    > 
    >> On Oct 25, 2016, at 10:23 PM, Michael Pearce <michael.pea...@ig.com> 
wrote:
    >> 
    >> Hi All,
    >> 
    >> In case you hadn't noticed re the compaction issue for non-null values i 
have created a separate KIP-87, if you could all contribute to its discussion 
would be much appreciated.
    >> 
    >> 
https://cwiki.apache.org/confluence/display/KAFKA/KIP-87+-+Add+Compaction+Tombstone+Flag
    >> 
    >> Secondly, focussing back on KIP-82, one of the actions agreed from the 
KIP call was for some additional alternative solution proposals on top of those 
already detailed in the KIP wiki and subsequent linked wiki pages by others in 
the group in the meeting.
    >> 
    >> I haven't seen any activity on this, does this mean there isn't any 
further and everyone in hindsight actually thinks the current proposed solution 
in the KIP is the front runner? (i assume this isn't the case, just want to 
nudge everyone)
    >> 
    > 
    > I have been meaning to respond, but I haven't had the time. In the next 
couple days, I will try to write up the container format that TiVo is using, 
and we can discuss it.
    > 
    > -James
    > 
    >> Also just copying across the kip call thread to keep everything in one 
thread to avoid a divergence of the discussion into multiple threads.
    >> 
    >> Cheers
    >> Mike
    >> 
    >> ________________________________________
    >> From: Mayuresh Gharat <gharatmayures...@gmail.com>
    >> Sent: Monday, October 24, 2016 6:17 PM
    >> To: dev@kafka.apache.org
    >> Subject: Re: Kafka KIP meeting Oct 19 at 11:00am PST
    >> 
    >> I agree with Nacho.
    >> +1 for the KIP.
    >> 
    >> Thanks,
    >> 
    >> Mayuresh
    >> 
    >> On Fri, Oct 21, 2016 at 11:46 AM, Nacho Solis 
<nso...@linkedin.com.invalid>
    >> wrote:
    >> 
    >>> I think a separate KIP is a good idea as well.  Note however that 
potential
    >>> decisions in this KIP could affect the other KIP.
    >>> 
    >>> Nacho
    >>> 
    >>> On Fri, Oct 21, 2016 at 10:23 AM, Jun Rao <j...@confluent.io> wrote:
    >>> 
    >>>> Michael,
    >>>> 
    >>>> Yes, doing a separate KIP to address the null payload issue for 
compacted
    >>>> topics is a good idea.
    >>>> 
    >>>> Thanks,
    >>>> 
    >>>> Jun
    >>>> 
    >>>> On Fri, Oct 21, 2016 at 12:57 AM, Michael Pearce 
<michael.pea...@ig.com>
    >>>> wrote:
    >>>> 
    >>>>> I had noted that what ever the solution having compaction based on 
null
    >>>>> payload was agreed isn't elegant.
    >>>>> 
    >>>>> Shall we raise another kip to : as discussed propose using a attribute
    >>>> bit
    >>>>> for delete/compaction flag as well/or instead of null value and
    >>> updating
    >>>>> compaction logic to look at that delelete/compaction attribute
    >>>>> 
    >>>>> I believe this is less contentious, so that at least we get that done
    >>>>> alleviating some concerns whilst the below gets discussed further?
    >>>>> 
    >>>>> ________________________________________
    >>>>> From: Jun Rao <j...@confluent.io>
    >>>>> Sent: Wednesday, October 19, 2016 8:56:52 PM
    >>>>> To: dev@kafka.apache.org
    >>>>> Subject: Re: Kafka KIP meeting Oct 19 at 11:00am PST
    >>>>> 
    >>>>> The following are the notes from today's KIP discussion.
    >>>>> 
    >>>>> 
    >>>>>  - KIP-82 - add record header: We agreed that there are use cases for
    >>>>>  third-party vendors building tools around Kafka. We haven't reached
    >>>> the
    >>>>>  conclusion whether the added complexity justifies the use cases. We
    >>>> will
    >>>>>  follow up on the mailing list with use cases, container format
    >>> people
    >>>>> have
    >>>>>  been using, and details on the proposal.
    >>>>> 
    >>>>> 
    >>>>> The video will be uploaded soon in https://cwiki.apache.org/
    >>>>> confluence/display/KAFKA/Kafka+Improvement+Proposals .
    >>>>> 
    >>>>> Thanks,
    >>>>> 
    >>>>> Jun
    >>>>> 
    >>>>> On Mon, Oct 17, 2016 at 10:49 AM, Jun Rao <j...@confluent.io> wrote:
    >>>>> 
    >>>>>> Hi, Everyone.,
    >>>>>> 
    >>>>>> We plan to have a Kafka KIP meeting this coming Wednesday at 11:00am
    >>>> PST.
    >>>>>> If you plan to attend but haven't received an invite, please let me
    >>>> know.
    >>>>>> The following is the tentative agenda.
    >>>>>> 
    >>>>>> Agenda:
    >>>>>> KIP-82: add record header
    >>>>>> 
    >>>>>> Thanks,
    >>>>>> 
    >>>>>> Jun
    >>>>>> 
    >>>>> The information contained in this email is strictly confidential and
    >>> for
    >>>>> the use of the addressee only, unless otherwise indicated. If you are
    >>> not
    >>>>> the intended recipient, please do not read, copy, use or disclose to
    >>>> others
    >>>>> this message or any attachment. Please also notify the sender by
    >>> replying
    >>>>> to this email or by telephone (+44(020 7896 0011) and then delete the
    >>>> email
    >>>>> and any copies of it. Opinions, conclusion (etc) that do not relate to
    >>>> the
    >>>>> official business of this company shall be understood as neither given
    >>>> nor
    >>>>> endorsed by it. IG is a trading name of IG Markets Limited (a company
    >>>>> registered in England and Wales, company number 04008957) and IG Index
    >>>>> Limited (a company registered in England and Wales, company number
    >>>>> 01190902). Registered address at Cannon Bridge House, 25 Dowgate Hill,
    >>>>> London EC4R 2YA. Both IG Markets Limited (register number 195355) and
    >>> IG
    >>>>> Index Limited (register number 114059) are authorised and regulated by
    >>>> the
    >>>>> Financial Conduct Authority.
    >>>>> 
    >>>> 
    >>> 
    >>> 
    >>> 
    >>> --
    >>> Nacho (Ignacio) Solis
    >>> Kafka
    >>> nso...@linkedin.com
    >>> 
    >> 
    >> 
    >> 
    >> --
    >> -Regards,
    >> Mayuresh R. Gharat
    >> (862) 250-7125
    >> 
    >> 
    >> ________________________________________
    >> From: Michael Pearce <michael.pea...@ig.com>
    >> Sent: Monday, October 17, 2016 7:48 AM
    >> To: dev@kafka.apache.org
    >> Subject: Re: [DISCUSS] KIP-82 - Add Record Headers
    >> 
    >> Hi Jun,
    >> 
    >> Sounds good.
    >> 
    >> Look forward to the invite.
    >> 
    >> Cheers,
    >> Mike
    >> ________________________________________
    >> From: Jun Rao <j...@confluent.io>
    >> Sent: Monday, October 17, 2016 5:55:57 AM
    >> To: dev@kafka.apache.org
    >> Subject: Re: [DISCUSS] KIP-82 - Add Record Headers
    >> 
    >> Hi, Michael,
    >> 
    >> We do have online KIP discussion meetings from time to time. How about we
    >> discuss this KIP Wed (Oct 19) at 11:00am PST? I will send out an invite 
(we
    >> typically do the meeting through Zoom and will post the video recording 
to
    >> Kafka wiki).
    >> 
    >> Thanks,
    >> 
    >> Jun
    >> 
    >> On Wed, Oct 12, 2016 at 1:22 AM, Michael Pearce <michael.pea...@ig.com>
    >> wrote:
    >> 
    >>> @Jay and Dana
    >>> 
    >>> We have internally had a few discussions of how we may address this if 
we
    >>> had a common apache kafka message wrapper for headers that can be used
    >>> client side only to, and address the compaction issue.
    >>> I have detailed this solution separately and linked from the main KIP-82
    >>> wiki.
    >>> 
    >>> Here’s a direct link –
    >>> https://cwiki.apache.org/confluence/display/KAFKA/
    >>> Headers+Value+Message+Wrapper
    >>> 
    >>> We feel this solution though doesn’t manage to address all the use cases
    >>> being mentioned still and also has some compatibility drawbacks e.g.
    >>> backwards forwards compatibility especially on different language 
clients
    >>> Also we still require with this solution, as still need to address
    >>> compaction issue / tombstones, we need to make server side changes and 
as
    >>> many message/record version changes.
    >>> 
    >>> We believe the proposed solution in KIP-82 does address all these needs
    >>> and is cleaner still, and more benefits.
    >>> Please have a read, and comment. Also if you have any improvements on 
the
    >>> proposed KIP-82 or an alternative solution/option your input is 
appreciated.
    >>> 
    >>> @All
    >>> As Joel has mentioned to get this moving along, and able to discuss more
    >>> fluidly, it would be great if we can organize to meet up virtually 
online
    >>> e.g. webex or something.
    >>> I am aware, that the majority are based in America, myself is in the UK.
    >>> @Kostya I assume you’re in Eastern Europe or Russia based on your email
    >>> address (please correct this assumption), I hope the time difference 
isn’t
    >>> too much that the below would suit you if you wish to join
    >>> 
    >>> Can I propose next Wednesday 19th October at 18:30 BST , 10:30 PST, 
20:30
    >>> MSK we try meetup online?
    >>> 
    >>> Would this date/time suit the majority?
    >>> Also what is the preferred method? I can host via Adobe Connect style
    >>> webex (which my company uses) but it isn’t the best IMHO, so more than
    >>> happy to have someone suggest a better alternative.
    >>> 
    >>> Best,
    >>> Mike
    >>> 
    >>> 
    >>> 
    >>> 
    >>> On 10/8/16, 7:26 AM, "Michael Pearce" <michael.pea...@ig.com> wrote:
    >>> 
    >>>>> I agree with the critique of compaction not having a value. I think
    >>> we should consider fixing that directly.
    >>> 
    >>>> Agree that the compaction issue is troubling: compacted "null"
    >>> deletes
    >>>   are incompatible w/ headers that must be packed into the message
    >>>   value. Are there any alternatives on compaction delete semantics that
    >>>   could address this? The KIP wiki discussion I think mostly assumes
    >>>   that compaction-delete is what it is and can't be changed/fixed.
    >>> 
    >>>   This KIP is about dealing with quite a few use cases and issues,
    >>> please see both the KIP use cases detailed by myself and also the
    >>> additional use cases wiki added by LinkedIn linked from the main KIP.
    >>> 
    >>>   The compaction is something that happily is addressed with headers,
    >>> but most defiantly isn't the sole reason or use case for them, headers
    >>> solves many issues and use cases. Thus their elegance and simplicity, 
and
    >>> why they're so common in transport mechanisms and so succesfull, as 
stated
    >>> like http, tcp, jms.
    >>> 
    >>>   ________________________________________
    >>>   From: Dana Powers <dana.pow...@gmail.com>
    >>>   Sent: Friday, October 7, 2016 11:09 PM
    >>>   To: dev@kafka.apache.org
    >>>   Subject: Re: [DISCUSS] KIP-82 - Add Record Headers
    >>> 
    >>>> I agree with the critique of compaction not having a value. I think
    >>> we should consider fixing that directly.
    >>> 
    >>>   Agree that the compaction issue is troubling: compacted "null" deletes
    >>>   are incompatible w/ headers that must be packed into the message
    >>>   value. Are there any alternatives on compaction delete semantics that
    >>>   could address this? The KIP wiki discussion I think mostly assumes
    >>>   that compaction-delete is what it is and can't be changed/fixed.
    >>> 
    >>>   -Dana
    >>> 
    >>>   On Fri, Oct 7, 2016 at 1:38 PM, Michael Pearce <michael.pea...@ig.com>
    >>> wrote:
    >>>> 
    >>>> Hi Jay,
    >>>> 
    >>>> Thanks for the comments and feedback.
    >>>> 
    >>>> I think its quite clear that if a problem keeps arising then it is
    >>> clear that it needs resolving, and addressing properly.
    >>>> 
    >>>> Fair enough at linkedIn, and historically for the very first use
    >>> cases addressing this maybe not have been a big priority. But as Kafka 
is
    >>> now Apache open source and being picked up by many including my 
company, it
    >>> is clear and evident that this is a requirement and issue that needs to 
be
    >>> now addressed to address these needs.
    >>>> 
    >>>> The fact in almost every transport mechanism including networking
    >>> layers in the enterprise ive worked in, there has always been headers i
    >>> think clearly shows their need and success for a transport mechanism.
    >>>> 
    >>>> I understand some concerns with regards to impact for others not
    >>> needing it.
    >>>> 
    >>>> What we are proposing is flexible solution that provides no overhead
    >>> on storage or network traffic layers if you chose not to use headers, 
but
    >>> does enable those who need or want it to use it.
    >>>> 
    >>>> 
    >>>> On your response to 1), there is nothing saying that it should be
    >>> put in any faster or without diligence and the same KIP process can 
still
    >>> apply for adding kafka-scope headers, having headers, just makes it 
easier
    >>> to add, without constant message and record changes. Timestamp is a 
clear
    >>> real example of actually what should be in a header (along with other
    >>> fields) but as such the whole message/record object needed to be 
changed to
    >>> add this, as will any further headers deemed needed by kafka.
    >>>> 
    >>>> On response to 2) why within my company as a platforms designer
    >>> should i enforce that all teams use the same serialization for their
    >>> payloads? But what i do need is some core cross cutting concerns and
    >>> information addressed at my platform level and i don't want to impose 
onto
    >>> my development teams. This is the same argument why byte[] is the 
exposed
    >>> value and key because as a messaging platform you dont want to impose 
that
    >>> on my company.
    >>>> 
    >>>> On response to 3) Actually this isnt true, there are many 3rd party
    >>> tools, we need to hook into our messaging flows that they only build 
onto
    >>> standardised interfaces as obviously the cost to have a custom
    >>> implementation for every company would be very high.
    >>>> APM tooling is a clear case in point, every enterprise level APM
    >>> tool on the market is able to stitch in transaction flow end 2 end over 
a
    >>> platform over http, jms because they can stitch in some "magic" data in 
a
    >>> uniform/standardised for the two mentioned they stitch this into the
    >>> headers. It is current form they cannot do this with Kafka. Providing a
    >>> standardised interface will i believe actually benefit the project as
    >>> commercial companies like these will now be able to plugin their tooling
    >>> uniformly, making it attractive and possible.
    >>>> 
    >>>> Some of you other concerns as Joel mentions these are more
    >>> implementation details, that i think should be agreed upon, but i think 
can
    >>> be addressed.
    >>>> 
    >>>> e.g. re your concern on the hashmap.
    >>>> it is more than possible not to have every record have to have a
    >>> hashmap unless it actually has a header (just like we have managed to 
do on
    >>> the serialized meesage) so if theres a concern on the in memory record 
size
    >>> for those using kafka without headers.
    >>>> 
    >>>> On your second to last comment about every team choosing their own
    >>> format, actually we do want this a little, as very first mentioned, no 
we
    >>> don't want a free for all, but some freedom to have different 
serialization
    >>> has different benefits and draw backs across our business. I can iterate
    >>> these if needed. One of the use case for headers provided by linkedIn on
    >>> top of my KIP even shows where headers could be beneficial here as a 
header
    >>> could be used to detail which data format the message is serialized to
    >>> allowing me to consume different formats.
    >>>> 
    >>>> Also we have some systems that we need to integrate that pretty near
    >>> impossible to wrap or touch their binary payloads, or we’re not allowed 
to
    >>> touch them (historic system, or inter/intra corporate)
    >>>> 
    >>>> Headers really gives as a solution to provide a pluggable platform,
    >>> and standardisation that allows users to build platforms that adapt to
    >>> their needs.
    >>>> 
    >>>> 
    >>>> Cheers
    >>>> Mike
    >>>> 
    >>>> 
    >>>> ________________________________________
    >>>> From: Jay Kreps <j...@confluent.io>
    >>>> Sent: Friday, October 7, 2016 4:45 PM
    >>>> To: dev@kafka.apache.org
    >>>> Subject: Re: [DISCUSS] KIP-82 - Add Record Headers
    >>>> 
    >>>> Hey guys,
    >>>> 
    >>>> This discussion has come up a number of times and we've always
    >>> passed.
    >>>> 
    >>>> One of things that has helped keep Kafka simple is not adding in new
    >>>> abstractions and concepts except when the proposal is really elegant
    >>> and
    >>>> makes things simpler.
    >>>> 
    >>>> Consider three use cases for headers:
    >>>> 
    >>>>  1. Kafka-scope: We want to add a feature to Kafka that needs a
    >>>>  particular field.
    >>>>  2. Company-scope: You want to add a header to be shared by
    >>> everyone in
    >>>>  your company.
    >>>>  3. World-wide scope: You are building a third party tool and want
    >>> to add
    >>>>  some kind of header.
    >>>> 
    >>>> For the case of (1) you should not use headers, you should just add
    >>> a field
    >>>> to the record format. Having a second way of encoding things doesn't
    >>> make
    >>>> sense. Occasionally people have complained that adding to the record
    >>> format
    >>>> is hard and it would be nice to just shove lots of things in
    >>> quickly. I
    >>>> think a better solution would be to make it easy to add to the record
    >>>> format, and I think we've made progress on that. I also think we
    >>> should be
    >>>> insanely focused on the simplicity of the abstraction and not adding
    >>> in new
    >>>> thingies often---we thought about time for years before adding a
    >>> timestamp
    >>>> and I guarantee you we would have goofed it up if we'd gone with the
    >>>> earlier proposals. These things end up being long term commitments
    >>> so it's
    >>>> really worth being thoughtful.
    >>>> 
    >>>> For case (2) just use the body of the message. You don't need a
    >>> globally
    >>>> agreed on definition of headers, just standardize on a header you
    >>> want to
    >>>> include in the value in your company. Since this is just used by
    >>> code in
    >>>> your company having a more standard header format doesn't really
    >>> help you.
    >>>> In fact by using something like Avro you can define exactly the
    >>> types you
    >>>> want, the required header fields, etc.
    >>>> 
    >>>> The only case that headers help is (3). This is a bit of a niche
    >>> case and i
    >>>> think is easily solved just making the reading and writing of given
    >>>> required fields pluggable to work with the header you have.
    >>>> 
    >>>> A couple of specific problems with this proposal:
    >>>> 
    >>>>  1. A global registry of numeric keys is super super ugly. This
    >>> seems
    >>>>  silly compared to the Avro (or whatever) header solution which
    >>> gives more
    >>>>  compact encoding, rich types, etc.
    >>>>  2. Using byte arrays for header values means they aren't really
    >>>>  interoperable for case (3). E.g. I can't make a UI that displays
    >>> headers,
    >>>>  or allow you to set them in config. To work with third party
    >>> headers, the
    >>>>  only case I think this really helps, you need the union of all
    >>>>  serialization schemes people have used for any tool.
    >>>>  3. For case (2) and (3) your key numbers are going to collide like
    >>>>  crazy. I don't think a global registry of magic numbers
    >>> maintained either
    >>>>  by word of mouth or checking in changes to kafka source is the
    >>> right thing
    >>>>  to do.
    >>>>  4. We are introducing a new serialization primitive which makes
    >>> fields
    >>>>  disappear conditional on the contents of other fields. This
    >>> breaks the
    >>>>  whole serialization/schema system we have today.
    >>>>  5. We're adding a hashmap to each record
    >>>>  6. This proposes making the ProducerRecord and ConsumerRecord
    >>> mutable
    >>>>  and adding setters and getters (which we try to avoid).
    >>>> 
    >>>> For context on LinkedIn: I set up the system there, but it may have
    >>> changed
    >>>> since i left. The header is maintained with the record schemas in
    >>> the avro
    >>>> schema registry and is required for all records. Essentially all
    >>> messages
    >>>> must have a field named "header" of type EventHeader which is itself
    >>> a
    >>>> record schema with a handful of fields (time, host, etc). The header
    >>>> follows the same compatibility rules as other avro fields, so it can
    >>> be
    >>>> evolved in a compatible way gradually across apps. Avro is typed and
    >>>> doesn't require deserializing the full record to read the header. The
    >>>> header information is (timestamp, host, etc) is important and needs
    >>> to
    >>>> propagate into other systems like Hadoop which don't have a concept
    >>> of
    >>>> headers for records, so I doubt it could move out of the value in
    >>> any case.
    >>>> Not allowing teams to chose a data format other than avro was
    >>> considered a
    >>>> feature, not a bug, since the whole point was to be able to share
    >>> data,
    >>>> which doesn't work if every team chooses their own format.
    >>>> 
    >>>> I agree with the critique of compaction not having a value. I think
    >>> we
    >>>> should consider fixing that directly.
    >>>> 
    >>>> -Jay
    >>>> 
    >>>> On Thu, Sep 22, 2016 at 12:31 PM, Michael Pearce <
    >>> michael.pea...@ig.com>
    >>>> wrote:
    >>>> 
    >>>>> Hi All,
    >>>>> 
    >>>>> 
    >>>>> I would like to discuss the following KIP proposal:
    >>>>> 
    >>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-
    >>>>> 82+-+Add+Record+Headers
    >>>>> 
    >>>>> 
    >>>>> 
    >>>>> I have some initial ?drafts of roughly the changes that would be
    >>> needed.
    >>>>> This is no where finalized and look forward to the discussion
    >>> especially as
    >>>>> some bits I'm personally in two minds about.
    >>>>> 
    >>>>> https://github.com/michaelandrepearce/kafka/tree/
    >>> kafka-headers-properties
    >>>>> 
    >>>>> 
    >>>>> 
    >>>>> Here is a link to a alternative option mentioned in the kip but one
    >>> i
    >>>>> would personally would discard (disadvantages mentioned in kip)
    >>>>> 
    >>>>> https://github.com/michaelandrepearce/kafka/tree/kafka-headers-full
    >>> ?
    >>>>> 
    >>>>> 
    >>>>> Thanks
    >>>>> 
    >>>>> Mike
    >>>>> 
    >>>>> 
    >>>>> 
    >>>>> 
    >>>>> 
    >>>>> The information contained in this email is strictly confidential
    >>> and for
    >>>>> the use of the addressee only, unless otherwise indicated. If you
    >>> are not
    >>>>> the intended recipient, please do not read, copy, use or disclose
    >>> to others
    >>>>> this message or any attachment. Please also notify the sender by
    >>> replying
    >>>>> to this email or by telephone (+44(020 7896 0011) and then delete
    >>> the email
    >>>>> and any copies of it. Opinions, conclusion (etc) that do not relate
    >>> to the
    >>>>> official business of this company shall be understood as neither
    >>> given nor
    >>>>> endorsed by it. IG is a trading name of IG Markets Limited (a
    >>> company
    >>>>> registered in England and Wales, company number 04008957) and IG
    >>> Index
    >>>>> Limited (a company registered in England and Wales, company number
    >>>>> 01190902). Registered address at Cannon Bridge House, 25 Dowgate
    >>> Hill,
    >>>>> London EC4R 2YA. Both IG Markets Limited (register number 195355)
    >>> and IG
    >>>>> Index Limited (register number 114059) are authorised and regulated
    >>> by the
    >>>>> Financial Conduct Authority.
    >>>>> 
    >>>> The information contained in this email is strictly confidential and
    >>> for the use of the addressee only, unless otherwise indicated. If you 
are
    >>> not the intended recipient, please do not read, copy, use or disclose to
    >>> others this message or any attachment. Please also notify the sender by
    >>> replying to this email or by telephone (+44(020 7896 0011) and then 
delete
    >>> the email and any copies of it. Opinions, conclusion (etc) that do not
    >>> relate to the official business of this company shall be understood as
    >>> neither given nor endorsed by it. IG is a trading name of IG Markets
    >>> Limited (a company registered in England and Wales, company number
    >>> 04008957) and IG Index Limited (a company registered in England and 
Wales,
    >>> company number 01190902). Registered address at Cannon Bridge House, 25
    >>> Dowgate Hill, London EC4R 2YA. Both IG Markets Limited (register number
    >>> 195355) and IG Index Limited (register number 114059) are authorised and
    >>> regulated by the Financial Conduct Authority.
    >>> 
    >>> 
    >> The information contained in this email is strictly confidential and for 
the use of the addressee only, unless otherwise indicated. If you are not the 
intended recipient, please do not read, copy, use or disclose to others this 
message or any attachment. Please also notify the sender by replying to this 
email or by telephone (+44(020 7896 0011) and then delete the email and any 
copies of it. Opinions, conclusion (etc) that do not relate to the official 
business of this company shall be understood as neither given nor endorsed by 
it. IG is a trading name of IG Markets Limited (a company registered in England 
and Wales, company number 04008957) and IG Index Limited (a company registered 
in England and Wales, company number 01190902). Registered address at Cannon 
Bridge House, 25 Dowgate Hill, London EC4R 2YA. Both IG Markets Limited 
(register number 195355) and IG Index Limited (register number 114059) are 
authorised and regulated by the Financial Conduct Authority.
    > 
    
    

Reply via email to