Re: [DISCUSS] KIP-82 - Add Record Headers

2016-11-18 Thread K Burstev
For what it is worth also i agree. As a user:

 1) Yes - Headers are worthwhile
 2) Yes - Headers should be a top level option

14.11.2016, 21:15, "Ignacio Solis" :
> 1) Yes - Headers are worthwhile
> 2) Yes - Headers should be a top level option
>
> On Mon, Nov 14, 2016 at 9:16 AM, Michael Pearce 
> wrote:
>
>>  Hi Roger,
>>
>>  The kip details/examples the original proposal for key spacing , not the
>>  new mentioned as per discussion namespace idea.
>>
>>  We will need to update the kip, when we get agreement this is a better
>>  approach (which seems to be the case if I have understood the general
>>  feeling in the conversation)
>>
>>  Re the variable ints, at very early stage we did think about this. I think
>>  the added complexity for the saving isn't worth it. I'd rather go with, if
>>  we want to reduce overheads and size int16 (2bytes) keys as it keeps it
>>  simple.
>>
>>  On the note of no headers, there is as per the kip as we use an attribute
>>  bit to denote if headers are present or not as such provides a zero
>>  overhead currently if headers are not used.
>>
>>  I think as radai mentions would be good first if we can get clarity if do
>>  we now have general consensus that (1) headers are worthwhile and useful,
>>  and (2) we want it as a top level entity.
>>
>>  Just to state the obvious i believe (1) headers are worthwhile and (2)
>>  agree as a top level entity.
>>
>>  Cheers
>>  Mike
>>  
>>  From: Roger Hoover 
>>  Sent: Wednesday, November 9, 2016 9:10:47 PM
>>  To: dev@kafka.apache.org
>>  Subject: Re: [DISCUSS] KIP-82 - Add Record Headers
>>
>>  Sorry for going a little in the weeds but thanks for the replies regarding
>>  varint.
>>
>>  Agreed that a prefix and {int, int} can be the same. It doesn't look like
>>  that's what the KIP is saying the "Open" section. The example shows
>>  211
>>  for New Relic and 210002 for App Dynamics implying that the New Relic
>>  organization will have only a single header id to work with. Or is 211
>>  a prefix? The main point of a namespace or prefix is to reduce the
>>  overhead of config mapping or registration depending on how
>>  namespaces/prefixes are managed.
>>
>>  Would love to hear more feedback on the higher-level questions though...
>>
>>  Cheers,
>>
>>  Roger
>>
>>  On Wed, Nov 9, 2016 at 11:38 AM, radai  wrote:
>>
>>  > I think this discussion is getting a bit into the weeds on technical
>>  > implementation details.
>>  > I'd liek to step back a minute and try and establish where we are in the
>>  > larger picture:
>>  >
>>  > (re-wording nacho's last paragraph)
>>  > 1. are we all in agreement that headers are a worthwhile and useful
>>  > addition to have? this was contested early on
>>  > 2. are we all in agreement on headers as top level entity vs headers
>>  > squirreled-away in V?
>>  >
>>  > if there are still concerns around these #2 points (#jay? #jun?)?
>>  >
>>  > (and now back to our normal programming ...)
>>  >
>>  > varints are nice. having said that, its adding complexity (see
>>  > https://github.com/addthis/stream-lib/blob/master/src/
>>  > main/java/com/clearspring/analytics/util/Varint.java
>>  > as 1st google result) and would require anyone writing other clients (C?
>>  > Python? Go? Bash? ;-) ) to get/implement the same, and for relatively
>>  > little gain (int vs string is order of magnitude, this isnt).
>>  >
>>  > int namespacing vs {int, int} namespacing are basically the same thing -
>>  > youre just namespacing an int64 and giving people while 2^32 ranges at a
>>  > time. the part i like about this is letting people have a large swath of
>>  > numbers with one registration so they dont have to come back for every
>>  > single plugin/header they want to "reserve".
>>  >
>>  >
>>  > On Wed, Nov 9, 2016 at 11:01 AM, Roger Hoover 
>>  > wrote:
>>  >
>>  > > Since some of the debate has been about overhead + performance, I'm
>>  > > wondering if we have considered a varint encoding (
>>  > > https://developers.google.com/protocol-buffers/docs/encoding#varints)
>>  > for
>>  > > the header length field (int32 in the proposal) and for header ids? If
>>  > you
>>  > > don't use headers, the overhead would be a single byte and for each
>>  > header
>>  > > id < 128 would also need only a single byte?
>>  > >
>>  > >
>>  > >
>>  > > On Wed, Nov 9, 2016 at 6:43 AM, radai 
>>  > wrote:
>>  > >
>>  > > > @magnus - and very dangerous (youre essentially downloading and
>>  > executing
>>  > > > arbitrary code off the internet on your servers ... bad idea without
>>  a
>>  > > > sandbox, even with)
>>  > > >
>>  > > > as for it being a purely administrative task - i disagree.
>>  > > >
>>  > > > i wish it would, really, because then my earlier point on the
>>  > complexity
>>  > > of
>>  > > > the remapping process would be invalid, but at linkedin, for example,
>>  > we
>>  > > > (the team im in) run kafka as a service. we dont really know what our
>>  > > users
>>

Re: [DISCUSS] KIP-82 - Add Record Headers

2016-10-05 Thread K Burstev

+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 
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 
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 

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 address

Re: [DISCUSS] KIP-82 - Add Record Headers

2016-10-05 Thread K Burstev
Thank you Radai,
 
Ints are good, I was just asking more of curiosity to understand the reasoning 
in the KIP.
 
In your reply to Gwen you gave some insight into LinkedIn's internal solution. 
We do the same, and have a custom Avro wrapper also, but suffer with the same 
issues that are detailed in the KIP. Especially around compacted topics.
 
How does LinkedIn transport headers on a compacted topic where you need to send 
a null payload to indicate a delete both for the broker for compaction reasons 
and the consumers to remove from their state but need the headers still on that 
record as they contain meta infra information used for features like single 
message tracing and routing.
 
Or currently do you also suffer the same issue as I and you cannot transport 
headers on delete record on a compacted topic.
 
To get around this we have an awful *cough* solution whereby we have to send 
our message wrapper with the headers and null content, and then we have an 
application that has to consume from all the compacted topics and when it sees 
this message it produces back in a null payload record to make the broker 
compact it out.
 
Our solution is so flaky has caused a few production issues the fact we, and 
everyone it seems has to make these kinds of solutions is terrible. I cannot 
wait to burn our solution, when we can have true first class citizen headers 
provided by Kafka.
 
Kostya

05.10.2016, 22:34, "radai" :
> Linkedin currently just forces everyone using kafka to:
> 1. use avro as payload
> 2. add company-wide headers into their avro schema
>
> this leads to problems with teams that would like to produce arbitrary
> blobs while still benefitting from company-wide infra.
>
> it also means every change to these fields is a company-wide version bump
> across literally hundreds of source repositories.
>
> moving to "standard" headers would allow much more rapid development
> iteration and roll-out of features and capabilities (not to mention enable
> arbitrary binary payloads)
>
> why not strings - space efficiency. some of our payloads are very small
> (<50 bytes) which would make the headers dominate the bandwidth.
>
> header ordering - its mostly a micro-optimization. coupled with the
> namespacing suggestion it would make it faster to check is any "system
> headers" (low ids) exist. this could later be combined with lazy parsing of
> the headers blob to minimize the overhead of server side plugins.
>
> On Wed, Oct 5, 2016 at 2:20 PM, Gwen Shapira  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  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 unord

Re: [DISCUSS] KIP-82 - Add Record Headers

2016-10-06 Thread K Burstev
@Mayuresh

Yes exactly, it is a real nasty race issue.

This is why I look forward to being able to trash our custom workaround :)

Kostya


06.10.2016, 02:36, "Mayuresh Gharat" :
> @Kostya
>
> Regarding "To get around this we have an awful *cough* solution whereby we
> have to send our message wrapper with the headers and null content, and
> then we have an application that has to consume from all the compacted
> topics and when it sees this message it produces back in a null payload
> record to make the broker compact it out."
>
>  ---> This has a race condition, right?
>
> Suppose the producer produces a message with headers and null content at
> time To to Kafka.
>
> Then the producer, at time To + 1, sends another message with headers and
> actual content to Kafka.
>
> What we expect is that the application that is consuming and then producing
> same message with null payload should happen at time To + 0.5, so that the
> message at To + 1 is not deleted.
>
> But there is no guarantee here.
>
> If the null payload goes in to Kafka at time To + 2, then essentially you
> loose the second message produced by the producer at time To + 1.
>
> Thanks,
>
> Mayuresh
>
> On Wed, Oct 5, 2016 at 6:13 PM, Joel Koshy  wrote:
>
>>  @Nacho
>>
>>  > > - Brokers can't see the headers (part of the "V" black box)>
>>  >
>>
>>  > (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).
>>  >
>>  >
>>
>>  I think this can be addressed with broker interceptors which we touched on
>>  in KIP-42
>>  >  42%3A+Add+Producer+and+Consumer+Interceptors>
>>  .
>>
>>  @Gwen
>>
>>  You are right that the wrapper thingy “works”, but there are some drawbacks
>>  that Nacho and Radai have covered in detail that I can add a few more
>>  comments to.
>>
>>  At LinkedIn, we *get by* without the proposed Kafka record headers by
>>  dumping such metadata in one or two places:
>>
>> - Most of our applications use Avro, so for the most part we can use an
>> explicit header field in the Avro schema. Topic owners are supposed to
>> include this header in their schemas.
>> - A prefix to the payload that primarily contains the schema’s ID so we
>> can deserialize the Avro. (We could use this for other use-cases as
>>  well -
>> i.e., move some of the above into this prefix blob.)
>>
>>  Dumping headers in the Avro schema pollutes the application’s data model
>>  with data/service-infra-related fields that are unrelated to the underlying
>>  topic; and forces the application to deserialize the entire blob whether or
>>  not the headers are actually used. Conversely from an infrastructure
>>  perspective, we would really like to not touch any application data. Our
>>  infiltration of the application’s schema is a major reason why many at
>>  LinkedIn sometimes assume that we (Kafka folks) are the shepherds for all
>>  things Avro :)
>>
>>  Another drawback is that all this only works if everyone in the
>>  organization is a good citizen and includes the header; and uses our
>>  wrapper libraries - which is a good practice IMO - but may not always be
>>  easy for open source projects that wish to directly use the Apache
>>  producer/client. If instead we allow these headers to be inserted via
>>  suitable interceptors outside the application payloads it would remove such
>>  issues of separation in the data model and choice of clients.
>>
>>  Radai has enumerated a number of use-cases
>>  >  Case+for+Kafka+Headers>
>>  and
>>  I’m sure the broader community will have a lot more to add. The feature as
>>  such would enable an ecosystem of plugins from different vendors that users
>>  can mix and match in their data pipelines without requiring any specific
>>  payload formats or client libraries.
>>
>>  Thanks,
>>
>>  Joel
>>
>>  > >
>>  > >
>>  > > On Wed, Oct 5, 2016 at 2:20 PM, Gwen Shapira 
>>  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.
>>  > > >
>>  >
>
> --
> -Regards,
> Mayuresh R. Gharat
> (862) 250-7125


Re: [DISCUSS] KIP-82 - Add Record Headers

2016-10-09 Thread K Burstev
I think here we have a situation like SQS in Amazon had, originally not 
supporting headers but as time and requirements increased the need becomes 
evident and finally headers or by which ever name it goes by are introduced got 
added back in 2014.
 
A blog from way back when SQS first added support for "headers" attributes 
including some further very basic use cases for why they decided to add them.
https://aws.amazon.com/blogs/aws/simple-queue-service-message-attributes/
 
I am sure they "passed" before also on adding it, but as a use cases and the 
product matures, it is inevitable they would be added and they did. I think 
Kafka is now at this stage.
 
The fact we have these wrapper work arounds is expensive and not solving our 
problems
 
* every single company re-implementing essentially the wheel to be able to send 
message meta data
* due to no common interface there cannot evolve an eco-system of 
plugins/interceptors to use them (again everyone's is custom but no doubt doing 
the same thing)
* cannot convince 3rd party commercial vendors to invest into adding support, 
as they don't want to write code against custom code written by my company as 
they get no re-use.
* work arounds cause production issues (compaction is just one noted point)

Headers really are a simple, elegant and common solution in my view and are 
addressing all of my above problems and reading the KIP many more needs and use 
cases.

It is too easy sometimes to simply say no without providing an alternative, or 
dismiss peoples real use cases. At the moment I don't see any sensible 
alternative proposition or commitment.
 
Here we have someone/a company addressing a real common need, willing to 
implement the solution it seems fairly advanced in the design also which simply 
needs the finer details discussed, I'll be honest haven't fully reviewed the 
sample code but so far it seems not very invasive, and could be in the next 
release.
 
As such this is why I am +1 for the KIP.
 
As for detail of the discussion about the actual implementation details.
 
For our headers in Kafka maybe everyone could read:
http://docs.aws.amazon.com/AWSSimpleQueueService/latest/SQSDeveloperGuide/SQSMessageAttributes.html
 
I quite like that a type for the value is passed along with the key and value, 
which means you don't need to know the type of the value ahead of time when 
consuming the header. Im not saying we have to have them, but I think it is 
worth a thought.
 
Kostya


08.10.2016, 00:37, "Nacho Solis" :
> On Fri, Oct 7, 2016 at 8:45 AM, Jay Kreps  wrote:
>
>>  This discussion has come up a number of times and we've always passed.
>
> ​Hopefully this time the arguments will be convincing enough that Kafka can
> decide to do something about it.
> ​
>
>>  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.
>
> ​I completely agree that we want things to be simple and elegant. This is
> exactly what headers provide.
>
> Headers are a clean way to extend the system without sacrificing
> performance or elegance. The are modular and backwards compatible.
>
> ​​
>
>>  Consider three use cases for headers:
>>
>>  ​​
>>   1. Kafka-scope: We want to add a feature to Kafka that needs a
>> particular field.
>
> ​This is a _great_ use case for Kafka headers. Having headers means that
> you can have features that are optional. Features that are slowly deployed
> without needing to move everybody from one protocol version to another
> protocol version. All clients don't have to change and all brokers don't
> have to change.
>
> Without headers you need to parse the messages differently. With headers
> you use the same parser.
> I assume I don't need to get into how this makes the system extensible
> without requiring others to use the same extensions you have.
>
> ​
>
>> 2. Company-scope: You want to add a header to be shared by everyone in
>> your company.
>
> ​It is completely true that for client-side things you don't need a
> architectural header system. You could just write a wrapper and
> encapsulate every message you send. You could achieve end-to-end. Even if
> this end-to-end exists, Kafka currently offers no way to identify the type
> of a message (which I wish we could change), so we have to rely on some
> magic number to identify the type. Once we have that we can have a header
> system.
>
> Avro is useful for encoding schema based systems, but it's not as useful
> for modularity and it's not universal. We have a number of use cases that
> don't use avro (and don't want to). They want to send binary data, but from
> an org perspective still need some structure to be added for accounting,
> tracing, auditing, security, etc. There is some of this data that would
> also be useful at the broker side. This is somewhat problematic at this
> point (say, using a client side wrapper).
>
>> 3. World-wide scope: Y