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.

Reply via email to