yeah, we tried for this a while back (kip 388 -
https://cwiki.apache.org/confluence/display/KAFKA/KIP-388%3A+Add+observer+interface+to+record+request+and+response)

its implemented in our kafka repo (linked to above)

On Tue, Dec 3, 2019 at 8:59 PM Ignacio Solis <iso...@igso.net> wrote:
>
> At LinkedIn we run a style of "read-only" interceptor we call an observer.
> We use this for usage monitoring.
> https://github.com/linkedin/kafka/commit/a378c8980af16e3c6d3f6550868ac0fd5a58682e
>
> There is always a tension between exposing internals, creating stable
> interfaces and performance.  It's understandable that upstream feels uneasy
> about this type of change.
>
> For us, the observer has become an essential tool, so we're ok if we need
> to update code.
>
> Personally, I would like to argue that a general interceptor framework
> would have been valuable from the very beginning.  Since one didn't exist,
> we've had to find one-off solutions for a few different problems.  A few
> examples that may have fit in there include up/down convert formats,
> authorizer, quotas, transaction coordinator, idempotent producer, etc.  Ok,
> ok, I'm overreaching, but you get the idea.  The chain of processes that a
> message/request goes through are basically interceptors, and we have
> decided that instead of a generic framework, we prefer to do the one-offs.
>
> Nacho
>
>
> On Tue, Dec 3, 2019 at 8:03 AM Ismael Juma <ism...@juma.me.uk> wrote:
>
> > The main challenge is doing this without exposing a bunch of internal
> > classes. I haven't seen a proposal that handles that aspect well so far.
> >
> > Ismael
> >
> > On Tue, Dec 3, 2019 at 7:21 AM Sönke Liebau
> > <soenke.lie...@opencore.com.invalid> wrote:
> >
> > > Hi Thomas,
> > >
> > > I think that idea is worth looking at. As you say, if no interceptor is
> > > configured then the performance overhead should be negligible. Basically
> > it
> > > is then up to the user to decide if he wants tomtake the performance hit.
> > > We should make sure to think about monitoring capabilities like time
> > spent
> > > in the interceptor for records etc.
> > >
> > > The most obvious use case I think is server side schema validation, which
> > > Confluent are also offering as part of their commercial product, but
> > other
> > > ideas come to mind as well.
> > >
> > > Best regards,
> > > Sönke
> > >
> > > Thomas Aley <thomas.a...@ibm.com> schrieb am Di., 3. Dez. 2019, 10:45:
> > >
> > > > Hi M. Manna,
> > > >
> > > > Thank you for your feedback, any and all thoughts on this are
> > appreciated
> > > > from the community.
> > > >
> > > > I think it is important to distinguish that there are two parts to
> > this.
> > > > One would be a server side interceptor framework and the other would be
> > > > the interceptor implementations themselves.
> > > >
> > > > The idea would be that the Interceptor framework manifests as a plug
> > > point
> > > > in the request/response paths that by itself has negligible performance
> > > > impact as without an interceptor registered in the framework it is
> > > > essentially a no-op. This way the out-the-box behavior of the Kafka
> > > broker
> > > > remains essentially unchanged, it is only if the cluster administrator
> > > > registers an interceptor into the framework that the path of a record
> > is
> > > > intercepted. This is much like the already accepted and implemented
> > > client
> > > > interceptors - the capability exists and it is an opt-in feature.
> > > >
> > > > As with the client interceptors and indeed interception in general, the
> > > > interceptor implementations need to be thoughtfully crafted to ensure
> > > > minimal performance impact. Yes the interceptor framework could tap
> > into
> > > > nearly everything but would only be tapping into the subset of APIs
> > that
> > > > the user wishes to intercept for their use case.
> > > >
> > > > Tom Aley
> > > > thomas.a...@ibm.com
> > > >
> > > >
> > > >
> > > > From:   "M. Manna" <manme...@gmail.com>
> > > > To:     Kafka Users <us...@kafka.apache.org>
> > > > Cc:     dev@kafka.apache.org
> > > > Date:   02/12/2019 11:31
> > > > Subject:        [EXTERNAL] Re: Broker Interceptors
> > > >
> > > >
> > > >
> > > > Hi Tom,
> > > >
> > > > On Mon, 2 Dec 2019 at 09:41, Thomas Aley <thomas.a...@ibm.com> wrote:
> > > >
> > > > > Hi Kafka community,
> > > > >
> > > > > I am hoping to get some feedback and thoughts about broker
> > > interceptors.
> > > > >
> > > > > KIP-42 Added Producer and Consumer interceptors which have provided
> > > > Kafka
> > > > > users the ability to collect client side metrics and trace the path
> > of
> > > > > individual messages end-to-end.
> > > > >
> > > > > This KIP also mentioned "Adding message interceptor on the broker
> > makes
> > > > a
> > > > > lot of sense, and will add more detail to monitoring. However, the
> > > > > proposal is to do it later in a separate KIP".
> > > > >
> > > > > One of the motivations for leading with client interceptors was to
> > gain
> > > > > experience and see how useable they are before tackling the server
> > side
> > > > > implementation which would ultimately "allow us to have a more
> > > > > complete/detailed message monitoring".
> > > > >
> > > > > Broker interceptors could also provide more value than just more
> > > > complete
> > > > > and detailed monitoring such as server side schema validation, so I
> > am
> > > > > curious to learn if anyone in the community has progressed this work;
> > > > has
> > > > > ideas about other potential server side interceptor uses or has
> > > actually
> > > > > implemented something similar.
> > > > >
> > > >
> > > >  I personally feel that the cost here is the impact on performance. If
> > I
> > > > am
> > > > right, this interceptor is going to tap into nearly everything. If you
> > > > have
> > > > strong guarantee (min.in.sync.replicas = N-1) then this may incur some
> > > > delay (and let's not forget inter broker comms protection by TLS
> > config).
> > > > This may not be desirable for some systems. That said, it would be good
> > > to
> > > > know what others think about this.
> > > >
> > > > Thanks,
> > > >
> > > > >
> > > > > Regards,
> > > > >
> > > > > Tom Aley
> > > > > thomas.a...@ibm.com
> > > > > Unless stated otherwise above:
> > > > > IBM United Kingdom Limited - Registered in England and Wales with
> > > number
> > > > > 741598.
> > > > > Registered office: PO Box 41, North Harbour, Portsmouth, Hampshire
> > PO6
> > > > 3AU
> > > > >
> > > > >
> > > >
> > > >
> > > >
> > > > Unless stated otherwise above:
> > > > IBM United Kingdom Limited - Registered in England and Wales with
> > number
> > > > 741598.
> > > > Registered office: PO Box 41, North Harbour, Portsmouth, Hampshire PO6
> > > 3AU
> > > >
> > > >
> > >
> >
>
>
> --
> Nacho - Ignacio Solis - iso...@igso.net

Reply via email to