kafka-users mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Tom Bentley <tbent...@redhat.com>
Subject Re: Broker Interceptors
Date Fri, 06 Dec 2019 09:56:11 GMT
Hi,

Couldn't this be done without exposing broker internals at the slightly
higher level of AbstractRequest and AbstractResponse? Those classes are
public. If the observer interface used Java default methods then adding a
new request type would not break existing implementations. I'm thinking
something like this:

```
public interface RequestObserver {
    default void observeAny(RequestContext context, AbstractRequest
request) {}
    default void observe(RequestContext context, MetadataRequest request) {
        observeAny(context, request);
    }
    default void observe(RequestContext context, ProduceRequest request) {
        observeAny(context, request);
    }
    default void observe(RequestContext context, FetchRequest request) {
        observeAny(context, request);
    }
   ...
```

And similar for a `ResponseObserver`. Request classes would implement this
method

```
    public abstract void observeForAudit(RequestContext context,
RequestObserver requestObserver);
```

where the implementation would look like this:

```
    @Override
    public void observe(RequestContext context, RequestObserver
requestObserver) {
        requestObserver.observe(context, this);
    }
```

I think this sufficiently abstracted to allow KafkaApis.handle() and
sendResponse() to call observe() generically.

Kind regards,

Tom

On Wed, Dec 4, 2019 at 6:59 PM Lincong Li <andrewlincong@gmail.com> wrote:

> Hi Thomas,
>
> Thanks for your interest in KIP-388. As Ignacio and Radai have mentioned,
> this
> <
> https://github.com/linkedin/kafka/commit/a378c8980af16e3c6d3f6550868ac0fd5a58682e
> >
> is our (LinkedIn's) implementation of KIP-388. The implementation and
> deployment of this broker-side observer has been working very well for us
> by far. On the other hand, I totally agree with the longer-term concerns
> raised by other committers. However we still decided to implement the KIP
> idea as a hot fix in order to solve our immediate problem and meet our
> business requirements.
>
> The "Rejected Alternatives for Kafka Audit" section at the end of KIP-388
> sheds some lights on the client-side auditor/interceptor/observer (sorry
> about the potential confusion caused by these words being used
> interchangeably).
>
> Best regards,
> Lincong Li
>
> On Wed, Dec 4, 2019 at 8:15 AM Thomas Aley <Thomas.Aley@ibm.com> wrote:
>
> > Thanks for the responses. I did worry about the challenge of exposing a
> > vast number of internal classes with general interceptor framework. A
> less
> > general solution more along the lines of the producer/consumer
> > interceptors on the client would satisfy the majority of use cases. If we
> > are smart, we should be able to come up with a pattern that could be
> > extended further in future if the community sees the demand.
> >
> > Looking through the discussion thread for KIP-388, I see a lot of good
> > points to consider and I intend to dive further into this.
> >
> >
> > Tom Aley
> > thomas.aley@ibm.com
> >
> >
> >
> > From:   Ismael Juma <ismael@juma.me.uk>
> > To:     Kafka Users <users@kafka.apache.org>
> > Cc:     dev <dev@kafka.apache.org>
> > Date:   03/12/2019 16:12
> > Subject:        [EXTERNAL] Re: Broker Interceptors
> >
> >
> >
> > 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.liebau@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.Aley@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.aley@ibm.com
> > > >
> > > >
> > > >
> > > > From:   "M. Manna" <manmedia@gmail.com>
> > > > To:     Kafka Users <users@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.Aley@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.aley@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
> > > >
> > > >
> > >
> >
> >
> >
> > 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
> >
> >
>

Mime
  • Unnamed multipart/alternative (inline, None, 0 bytes)
View raw message