@Kostya - we do not support compacted topics in combination with in-house
headers. internal systems that use compacted topics (and need to be able to
send null values) use the open source producer and cannot use some of the
company infra.

yet another good use case for "official" header support :-)

On Wed, Oct 5, 2016 at 3:44 PM, Nacho Solis <nso...@linkedin.com.invalid>
wrote:

> There are pros and cons to having the headers be completely layered above
> the broker (and hence wire level messages), let's look at the base ones.
>
> A few pros (of headers being a higher layer thing):
> - No broker changes
> - No protocol changes
> - Messages with headers can work with brokers that don't support them.
> - Multiple header styles can be used (different applications and middleware
> can use different header styles, or no headers at all)
>
> However, there are a number of cons
> - Brokers can't see the headers (part of the "V" black box)
> - No unified header system
>
>
> Now let's look at some higher level issues.
>
> Kafka messages are identified by their topic and partition. There is no way
> to identify what the data is, for that matter what type. There are normally
> 2 schools of thought in this area. In one, we identify the content type
> using some field in the enclosing layer (let's call this the Type-field
> approach). This is the approach the IP takes (and Ethernet).  There is a
> field telling you what the type inside is. The other method is the
> self-contained approach. We can call this the Self-Describing approach.
>
> Kafka does not offer the Type-field approach. There is no way to describe
> the structure of what's inside the Key and Value. This means that the Key
> and Value are only able to be interpreted if you know a priori what they
> have, or if they have some way to self-identify what they are. Some
> protocols achieve this by the use of magic numbers.
>
> If we don't have a protocol level header system, you will need to rely on
> some external method to identify what type of data is being sent and
> whether it has headers or not.  Right now, the different deployments rely
> on assumptions and some setups of a schema registry or some other
> hack. Many Kafka users rely on some mapping of avro, topic name,
> registration system, etc. This in and of itself causes issues with
> extensibility, versioning and backward/forward compatibility.
>
> A good example is auditing and tracing, where you force people to modify
> their avro schemas with any change to the audit information. The same would
> be true for provenance (if, for example, we wanted a universal way to sign
> the contents of a message).
>
> The need to add structured data to messages is universal. Organizations and
> developers will find ways around it.  Somebody might implement some
> validation in their Kafka client wrappers to ensure that their developers
> are using approved schemas, others will do encapsulation, others would just
> delegate this to the application itself, why not let them worry about what
> cluster they're going to?
>
> The benefit of having a base system that is universal is great because we
> can achieve large benefits for all:
> - structured data can be carried with the message (this data could come
> from middleware, the stack or even brokers)
> - a community of plugins and addons can develop (some open source some
> commercial?)
>
> Right now what we have seen is a number of organizations having developed
> their own in house systems.
>
> To be clear, Gwen is right that it's possible for LinkedIn to add this to
> our kafka client wrappers; we just think that it would be something
> valuable to everybody.
>
>
> (Also, it would be nice if we had a way to access the headers from the
> brokers, something that is not trivial at this time with the current broker
> architecture).
>
>
> Nacho
>
>
>
>
> On Wed, Oct 5, 2016 at 2:20 PM, Gwen Shapira <g...@confluent.io> wrote:
>
> > Since LinkedIn has some kind of wrapper thingy that adds the headers,
> > where they could have added them to Apache Kafka - I'm very curious to
> > hear what drove that decision and the pros/cons of managing the
> > headers outside Kafka itself.
> >
> > On Wed, Oct 5, 2016 at 2:16 PM, K Burstev <k.burs...@yandex.com> wrote:
> > > +1
> > >
> > > This is a much needed feature, one I also have been waiting for, and
> that
> > > Kafka has been too long without.
> > >
> > > Especially since compaction the custom wrapper solution does not work
> > where
> > > you want a null payload but need the record to have headers.
> > >
> > > (It actually made me sign up to the mailing list so I could reply, as
> up
> > > until now I just browse the archives and forums)
> > >
> > >
> > > In general the KIP looks great. The solution address's all my core
> needs.
> > > Really hope this makes it to the next release after the current one.
> > >
> > >
> > > Questions:
> > >
> > > 1) Why not String,String headers?
> > >
> > > I assume reading the KIP it is for message size but surely compression
> > would
> > > greatly reduce this overhead with Strings.
> > >
> > > Many systems in the eco-sphere that kafka sits in, like JMS and Flume
> use
> > > String,String headers as such it would be easier to integrate with
> these
> > > frameworks/systems, as they can simply map across the headers.
> > >
> > >
> > > 2) Key Allocation
> > >
> > > If you do keep with int keys why not make the key allocation the
> proposed
> > > why is it an example. The example makes sense after all, and seems very
> > > sensible and clean.
> > >
> > > 3) Header Ordering
> > >
> > > I would avoid this as per your proposed between the two options and
> keep
> > > them un-ordered.
> > > There are many clients not maintained by the core community and also
> > > internally in many companies, that would need to implement it. Whilst
> > > trivial it complicates matters, its easier to just expect an unordered
> > set
> > > as will be converted to a map client side anyhow.
> > >
> > > Kostya
> > >
> > >
> > >
> > > On 04/10/2016 23:35, radai wrote:
> > >>
> > >> another potential benefit of headers is it would reduce the number of
> > API
> > >> changes required to support future features (as they could be
> > implemented
> > >> as plugins).
> > >> that would greatly accelerate the rate with which kafka can be
> extended.
> > >>
> > >> On Mon, Oct 3, 2016 at 12:46 PM, Michael Pearce <
> michael.pea...@ig.com>
> > >> wrote:
> > >>
> > >>> Opposite way around v4 instead of v3 ;)
> > >>> ________________________________________
> > >>> From: Michael Pearce
> > >>> Sent: Monday, October 3, 2016 8:45 PM
> > >>> To: dev@kafka.apache.org
> > >>> Subject: Re: [DISCUSS] KIP-82 - Add Record Headers
> > >>>
> > >>> Thanks guys for spotting this, i have updated KIP-82 to state v3
> > instead
> > >>> of v4.
> > >>>
> > >>> Mike.
> > >>> ________________________________________
> > >>> From: Becket Qin <becket....@gmail.com>
> > >>> Sent: Monday, October 3, 2016 6:18 PM
> > >>> To: dev@kafka.apache.org
> > >>> Subject: Re: [DISCUSS] KIP-82 - Add Record Headers
> > >>>
> > >>> Yes, KIP-74 has already been checked in. The new
> FetchRequest/Response
> > >>> version should be V4. :)
> > >>>
> > >>> On Mon, Oct 3, 2016 at 10:14 AM, Sean McCauliff <
> > >>> smccaul...@linkedin.com.invalid> wrote:
> > >>>
> > >>>> Change to public interfaces:
> > >>>>
> > >>>> "Add ProduceRequest/ProduceResponse V3 which uses the new message
> > >>>> format.
> > >>>> Add FetchRequest/FetchResponse V3 which uses the new message
> format."
> > >>>>
> > >>>> When I look at org.apache.kafka.common.requests.FetchResponse on
> > >>>> master I see that there is already a version 3.  Seems like this is
> > >>>> from a recent commit about implementing KIP-74.  Do we need to
> > >>>> coordinate these changes with KIP-74?
> > >>>>
> > >>>>
> > >>>> "The serialisation of the [int, bye[]] header set will on the wire
> > >>>> using a strict format"  bye[] -> byte[]
> > >>>>
> > >>>> Sean
> > >>>> --
> > >>>> Sean McCauliff
> > >>>> Staff Software Engineer
> > >>>> Kafka
> > >>>>
> > >>>> smccaul...@linkedin.com
> > >>>> linkedin.com/in/sean-mccauliff-b563192
> > >>>>
> > >>>>
> > >>>> On Fri, Sep 30, 2016 at 3:43 PM, radai <radai.rosenbl...@gmail.com>
> > >>>
> > >>> wrote:
> > >>>>>
> > >>>>> I think headers are a great idea.
> > >>>>>
> > >>>>> Right now, people who are trying to implement any sort of org-wide
> > >>>>> functionality like monitoring, tracing, profiling etc pretty much
> > have
> > >>>
> > >>> to
> > >>>>>
> > >>>>> define their own wrapper layers, which probably leads to everyone
> > >>>>> implementing their own variants of the same underlying
> functionality.
> > >>>>>
> > >>>>> I think a common base for headers would allow implementing a lot of
> > >>>
> > >>> this
> > >>>>>
> > >>>>> functionality only one in a way that different header-based
> > >>>
> > >>> capabilities
> > >>>>>
> > >>>>> could be shared and composed and open the door the a wide range of
> > >>>>
> > >>>> possible
> > >>>>>
> > >>>>> Kafka middleware that's simply impossible to write against the
> > current
> > >>>>
> > >>>> API.
> > >>>>>
> > >>>>> Here's a list of things that could be implemented as "plugins" on
> top
> > >>>
> > >>> of
> > >>>>
> > >>>> a
> > >>>>>
> > >>>>> header mechanism (full list here -
> > >>>>> https://cwiki.apache.org/confluence/display/KAFKA/A+
> > >>>>
> > >>>> Case+for+Kafka+Headers).
> > >>>>>
> > >>>>> A lot of these already exist within LinkedIn and could for example
> be
> > >>>>
> > >>>> open
> > >>>>>
> > >>>>> sourced if Kafka had headers. I'm fairly certain the same is true
> in
> > >>>>
> > >>>> other
> > >>>>>
> > >>>>> organizations using Kafka
> > >>>>>
> > >>>>>
> > >>>>>
> > >>>>> 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.
> > >>>
> > >
> >
> >
> >
> > --
> > Gwen Shapira
> > Product Manager | Confluent
> > 650.450.2760 | @gwenshap
> > Follow us: Twitter | blog
> >
>
>
>
> --
> Nacho (Ignacio) Solis
> Kafka
> nso...@linkedin.com
>

Reply via email to