Hi all,

I have updated the interfaces. I managed to shrink the required number of
entities. Basically I store the event type with the event, therefore we can
cover all topic related events (create, delete, change) with one event type.
I think if on-one has objections then I'll start a vote soon.

Viktor

On Thu, Oct 29, 2020 at 5:15 PM Viktor Somogyi-Vass <viktorsomo...@gmail.com>
wrote:

> Hi Tom.
>
> Sorry for the delay.
> Answering your points:
>
> > Why is it necessary to introduce this interface to produce the audit
> trail
> > when there is logging that can already record a lot of the same
> > information, albeit in less structured form? If logging isn't adequate it
> > would be good to explain why not in the Motivation or Rejected
> Alternatives
> > section. It's worth pointing out that even the "less structured" part
> would
> > be helped by KIP-673, which proposes to change the RequestChannel's
> logging
> > to include a JSON representation of the request.
>
> We will need authorization details as would an auditor normally have them
> but a request logger doesn't as you correctly pointed out later in your
> reply. They would also appear at different lifecycle points I imagine, like
> the request logger is probably when the request enters Kafka and the
> auditor catches them before sending the response, so it can obtain all
> information (authorization, execution).
> Furthermore this auditing API would specifically target other JVM based
> components that depend on Kafka (like Ranger or Atlas) and from both side's
> perspective it's much better to expose Java level classes rather than a
> lower level (JSON) implementation. If a Java level object is exposed then
> we need to create them once during request processing which is fairly
> low-fat since we're parsing the request most of the time anyways as opposed
> to JSON which would need to be serialized first and then deserialized for
> the consumer of the API.
>
> > I'm guessing what you gain from the proposed interface is the fact that
> > it's called after the authorizer (perhaps after the request has been
> > handled: I'm unclear about the purpose of AuditInfo.error), so you could
> > generate a single record in the audit trail. That could still be achieved
> > using logging, either by correlating existing log messages or by
> proposing
> > some new logging just for this auditing purpose (perhaps with a logger
> per
> > API key so people could avoid the performance hit on the produce and
> fetch
> > paths if they weren't interested in auditing those things). Again, if
> this
> > doesn't work it would be great for the KIP to explain why.
>
> AuditInfo.error serves for capturing the possible errors that could happen
> during the authorization and execution of the request. For instance a
> partition creation request could be authorized and then rejected
> with INVALID_TOPIC_EXCEPTION because the topic is queued for deletion. In
> this case the AuditInfo.error would contain this API error thus emitting
> information about the failure of the request. With normal auditing that
> looks at only the authorization information we wouldn't detect it.
> Regarding the produce and fetch performance: for these kinds of requests I
> don't think we should enable parsing the batches themselves, just only pass
> some meta information like which topics and partitions are affected. These
> are parsed anyways for log reading and writing. Also similarly to the
> authorizer we need to require implementations to run the auditing logic on
> a different thread to minimize the performance impact.
>
> > To me there were parallels with previous discussions about broker-side
> > interceptors (
> > https://www.mail-archive.com/dev@kafka.apache.org/msg103310.html if
> you've
> > not seen it before), those seemed to founder on the unwillingness to make
> > the request internal classes into a supported API. You've tried to
> address
> > this by proposing a parallel set of classes implementing AuditEvent for
> > exposing selective details about the request. It's not really clear that
> > you really _need_ to access all that information about each request,
> rather
> > than simply recording it all, and it would also come with a significant
> > implementation and maintenance cost. If it's simply about recording all
> the
> > information in the request, then it would likely be enough to pass a
> > suitably formatted String rather than an AuditEvent, which basically
> brings
> > us back to point 1, but with some justification for not using logging.
>
> Thanks for this email thread, I haven't seen it but now I see it's a much
> bigger tree that I'm chopping :). But the point is that everyone basically
> faces a similar issue, that we need server side interceptors and
> observables. Indeed auditing can be part of such an interceptor chain and
> I've been thinking of it like this too sometimes but as it has been
> correctly assessed in the thread "we're doing the one-offs". I also admit
> that maintaining all the implementation of AuditEvent could be cumbersome
> and maybe this isn't a way. However I think we should expose more
> structured forms. If we maintain suitably formatted Strings and if the
> protocol changes for some requests it could be much harder to trace the
> needed changes back to these Strings.
> One idea that I had while reading the email is we could generate these
> classes similarly to the *Data classes (like CreateTopicsData). There would
> be a flag called "useForAuditing=true" in the JSON definition of the
> protocol that would cause the given field to be generated into a class that
> would be the implementation of AuditEvent and would be a public API. It
> would be instantiated when a request is deserialized. In my opinion it has
> the advantage that it's tightly coupled with the protocol from the
> maintenance point of view and still provides an efficient and structured
> way of accessing certain information of the request.
>
> Best,
> Viktor
>
> On Thu, Oct 1, 2020 at 4:16 PM Tom Bentley <tbent...@redhat.com> wrote:
>
>> Hi Viktor,
>>
>> Like Mickael, I can see that there's value in having an audit trail. For
>> me
>> the KIP raises a number of questions in its current form:
>>
>> Why is it necessary to introduce this interface to produce the audit trail
>> when there is logging that can already record a lot of the same
>> information, albeit in less structured form? If logging isn't adequate it
>> would be good to explain why not in the Motivation or Rejected
>> Alternatives
>> section. It's worth pointing out that even the "less structured" part
>> would
>> be helped by KIP-673, which proposes to change the RequestChannel's
>> logging
>> to include a JSON representation of the request.
>>
>> I'm guessing what you gain from the proposed interface is the fact that
>> it's called after the authorizer (perhaps after the request has been
>> handled: I'm unclear about the purpose of AuditInfo.error), so you could
>> generate a single record in the audit trail. That could still be achieved
>> using logging, either by correlating existing log messages or by proposing
>> some new logging just for this auditing purpose (perhaps with a logger per
>> API key so people could avoid the performance hit on the produce and fetch
>> paths if they weren't interested in auditing those things). Again, if this
>> doesn't work it would be great for the KIP to explain why.
>>
>> To me there were parallels with previous discussions about broker-side
>> interceptors (
>> https://www.mail-archive.com/dev@kafka.apache.org/msg103310.html if
>> you've
>> not seen it before), those seemed to founder on the unwillingness to make
>> the request internal classes into a supported API. You've tried to address
>> this by proposing a parallel set of classes implementing AuditEvent for
>> exposing selective details about the request. It's not really clear that
>> you really _need_ to access all that information about each request,
>> rather
>> than simply recording it all, and it would also come with a significant
>> implementation and maintenance cost. If it's simply about recording all
>> the
>> information in the request, then it would likely be enough to pass a
>> suitably formatted String rather than an AuditEvent, which basically
>> brings
>> us back to point 1, but with some justification for not using logging.
>>
>> Kind regards,
>>
>> Tom
>>
>> On Thu, Oct 1, 2020 at 11:30 AM Dániel Urbán <urb.dani...@gmail.com>
>> wrote:
>>
>> > Hi Viktor,
>> >
>> > I think the current state of the proposal is flexible enough to support
>> > use-cases where the response data is of interest to the auditor.
>> > This part ensures that: "... doing the auditing before sending the
>> response
>> > back ...". Additionally, event classes could be extended with additional
>> > data if needed.
>> >
>> > Overall, the KIP looks good, thanks!
>> >
>> > Daniel
>> >
>> > Viktor Somogyi-Vass <viktorsomo...@gmail.com> ezt írta (időpont: 2020.
>> > szept. 30., Sze, 17:24):
>> >
>> > > Hi Daniel,
>> > >
>> > > I think in this sense we can use the precedence set with the
>> > > KAfkaAdminClient. It has *Result and *Options classes which in this
>> > > interpretation are similar in versioning and usage as they transform
>> and
>> > > convey the responses of the protocol in a minimalistic API.
>> > > I've modified the KIP a bit and created some examples for these event
>> > > classes. For now as the implementation I think we can treat this
>> > similarly
>> > > to KIP-4 (AdminClient) which didn't push implementation for everything
>> > but
>> > > rather pushed implementing everything to subsequent KIPs as the
>> > > requirements become important. In this first KIP we can create the
>> more
>> > > important ones (listed in the "Default Implementation") section if
>> that
>> > is
>> > > fine.
>> > >
>> > > Regarding the response passing: to be honest I feel like that it's not
>> > that
>> > > strictly related to auditing but I think it's a good idea and could
>> fit
>> > > into this API. I think that we should design this current API with
>> this
>> > in
>> > > mind. Did you have any specific ideas about the implementation?
>> > >
>> > > Viktor
>> > >
>> > > On Tue, Sep 22, 2020 at 9:05 AM Dániel Urbán <urb.dani...@gmail.com>
>> > > wrote:
>> > >
>> > > > An example I had in mind was the ProduceResponse - the auditor might
>> > need
>> > > > access to the new end offset of the partitions.
>> > > > The event-based approach sounds good - new events and fields can be
>> > added
>> > > > on-demand. Do we need the same versioning strategy we use with the
>> > > > requests/responses?
>> > > >
>> > > > Daniel
>> > > >
>> > > > Viktor Somogyi-Vass <viktorsomo...@gmail.com> ezt írta (időpont:
>> 2020.
>> > > > szept. 21., H, 14:08):
>> > > >
>> > > > > Hi Daniel,
>> > > > >
>> > > > > > If the auditor needs access to the details of the action, one
>> could
>> > > > argue
>> > > > > that even the response should be passed down to the auditor.
>> > > > > At this point I don't think we need to include responses into the
>> > > > interface
>> > > > > but if you have a use-case we can consider doing that.
>> > > > >
>> > > > > > Is it feasible to convert the Java requests and responses to
>> public
>> > > > API?
>> > > > > Well I think that in this case we would need to actually
>> transform a
>> > > lot
>> > > > of
>> > > > > classes and that might be a bit too invasive. Although since the
>> > > protocol
>> > > > > itself *is* a public API it might make sense to have some kind of
>> > Java
>> > > > > representation as a public API as well.
>> > > > >
>> > > > > > If not, do we have another option to access this info in the
>> > auditor?
>> > > > > I think one option would be to do what the original KIP-567 was
>> > > > > implemented. Basically we could have an AuditEvent interface that
>> > would
>> > > > > contain request specific data. Its obvious drawback is that it
>> has to
>> > > be
>> > > > > implemented for most of the 40 something protocols but on the
>> upside
>> > > > these
>> > > > > classes shouldn't be complicated. I can try to do a PoC with this
>> to
>> > > see
>> > > > > how it looks like and whether it solves the problem. To be honest
>> I
>> > > think
>> > > > > it would be better than publishing the request classes as an API
>> > > because
>> > > > > here we're restricting access to only what is necessary.
>> > > > >
>> > > > > Thanks,
>> > > > > Viktor
>> > > > >
>> > > > >
>> > > > >
>> > > > > On Fri, Sep 18, 2020 at 8:37 AM Dániel Urbán <
>> urb.dani...@gmail.com>
>> > > > > wrote:
>> > > > >
>> > > > > > Hi,
>> > > > > >
>> > > > > > Thanks for the KIP.
>> > > > > >
>> > > > > > If the auditor needs access to the details of the action, one
>> could
>> > > > argue
>> > > > > > that even the response should be passed down to the auditor.
>> > > > > > Is it feasible to convert the Java requests and responses to
>> public
>> > > > API?
>> > > > > > If not, do we have another option to access this info in the
>> > auditor?
>> > > > > > I know that the auditor could just send proper requests through
>> the
>> > > API
>> > > > > to
>> > > > > > the brokers, but that seems like an awful lot of overhead, and
>> > could
>> > > > > > introduce timing issues as well.
>> > > > > >
>> > > > > > Daniel
>> > > > > >
>> > > > > >
>> > > > > > Viktor Somogyi-Vass <viktorsomo...@gmail.com> ezt írta
>> (időpont:
>> > > 2020.
>> > > > > > szept. 16., Sze, 17:17):
>> > > > > >
>> > > > > > > One more after-thought on your second point (AbstractRequest):
>> > the
>> > > > > > reason I
>> > > > > > > introduced it in the first place was that this way
>> implementers
>> > can
>> > > > > > access
>> > > > > > > request data. A use case can be if they want to audit a
>> change in
>> > > > > > > configuration or client quotas but not just acknowledge the
>> fact
>> > > that
>> > > > > > such
>> > > > > > > an event happened but also capture the change itself by
>> peeking
>> > > into
>> > > > > the
>> > > > > > > request. Sometimes it can be useful especially when people
>> want
>> > to
>> > > > > trace
>> > > > > > > back the order of events and what happened when and not just
>> > > > > acknowledge
>> > > > > > > that there was an event of a certain kind. I also recognize
>> that
>> > > this
>> > > > > > might
>> > > > > > > be a very loose interpretation of auditing as it's not
>> strictly
>> > > > related
>> > > > > > to
>> > > > > > > authorization but rather a way of tracing the admin actions
>> > within
>> > > > the
>> > > > > > > cluster. It even could be a different API therefore but
>> because
>> > of
>> > > > the
>> > > > > > > variety of the Kafka APIs it's very hard to give a method that
>> > fits
>> > > > > all,
>> > > > > > so
>> > > > > > > it's easier to pass down the AbstractRequest and the
>> > implementation
>> > > > can
>> > > > > > do
>> > > > > > > the extraction of valuable info. So that's why I added this in
>> > the
>> > > > > first
>> > > > > > > place and I'm interested in your thoughts.
>> > > > > > >
>> > > > > > > On Wed, Sep 16, 2020 at 4:41 PM Viktor Somogyi-Vass <
>> > > > > > > viktorsomo...@gmail.com>
>> > > > > > > wrote:
>> > > > > > >
>> > > > > > > > Hi Mickael,
>> > > > > > > >
>> > > > > > > > Thanks for reviewing the KIP.
>> > > > > > > >
>> > > > > > > > 1.) I just wanted to follow the conventions used with the
>> > > > Authorizer
>> > > > > as
>> > > > > > > it
>> > > > > > > > is built in a similar fashion, although it's true that in
>> > > > KafkaServer
>> > > > > > we
>> > > > > > > > call the configure() method and the start() in the next
>> line.
>> > > This
>> > > > > > would
>> > > > > > > be
>> > > > > > > > the same in Auditor and even simpler as there aren't any
>> > > parameters
>> > > > > to
>> > > > > > > > start(), so I can remove it. If it turns out there is a need
>> > for
>> > > > it,
>> > > > > we
>> > > > > > > can
>> > > > > > > > add it later.
>> > > > > > > >
>> > > > > > > > 2.) Yes, this is a very good point, I will remove it,
>> however
>> > in
>> > > > this
>> > > > > > > case
>> > > > > > > > I don't think we need to add the ApiKey as it is already
>> > > available
>> > > > in
>> > > > > > > > AuthorizableRequestContext.requestType(). One less parameter
>> > :).
>> > > > > > > >
>> > > > > > > > 3.) I'll add it. It will simply log important changes in the
>> > > > cluster
>> > > > > > like
>> > > > > > > > topic events (create, update, delete, partition or
>> replication
>> > > > factor
>> > > > > > > > change), ACL events, config changes, reassignment, altering
>> log
>> > > > dirs,
>> > > > > > > > offset delete, group delete with the authorization info like
>> > who
>> > > > > > > initiated
>> > > > > > > > the call, was it authorized, were there any errors. Let me
>> know
>> > > if
>> > > > > you
>> > > > > > > > think there are other APIs I should include.
>> > > > > > > >
>> > > > > > > > 4.) The builder is there mostly for easier usability but
>> > actually
>> > > > > > > thinking
>> > > > > > > > of it it doesn't help much so I removed it. The AuditInfo is
>> > > also a
>> > > > > > > helper
>> > > > > > > > class so I don't see any value in transforming it into an
>> > > interface
>> > > > > but
>> > > > > > > if
>> > > > > > > > I simplify it (by removing the builder) it will be cleaner.
>> > Would
>> > > > > that
>> > > > > > > work?
>> > > > > > > >
>> > > > > > > > I'll update the KIP to reflect my answers.
>> > > > > > > >
>> > > > > > > > Viktor
>> > > > > > > >
>> > > > > > > >
>> > > > > > > > On Mon, Sep 14, 2020 at 6:02 PM Mickael Maison <
>> > > > > > mickael.mai...@gmail.com
>> > > > > > > >
>> > > > > > > > wrote:
>> > > > > > > >
>> > > > > > > >> Hi Viktor,
>> > > > > > > >>
>> > > > > > > >> Thanks for restarting the discussion on this KIP. Being
>> able
>> > to
>> > > > > easily
>> > > > > > > >> audit usage of a Kafka cluster is a very valuable feature.
>> > > > > > > >>
>> > > > > > > >> Regarding the API, I have a few of questions:
>> > > > > > > >> 1) You introduced a start() method. I don't think any other
>> > > > > interfaces
>> > > > > > > >> have such a method. Users can do any setup they want in
>> > > > configure()
>> > > > > > > >>
>> > > > > > > >> 2) The first argument of audit is an AbstractRequest.
>> > > > Unfortunately
>> > > > > > > >> this type is not part of the public API. But actually I'm
>> not
>> > > sure
>> > > > > > > >> having the full request is really needed here. Maybe just
>> > > passing
>> > > > > the
>> > > > > > > >> Apikey would be enough as we already have all the resources
>> > from
>> > > > the
>> > > > > > > >> auditInfos field.
>> > > > > > > >>
>> > > > > > > >> 3) The KIP mentions a "LoggingAuditor" default
>> implementation.
>> > > > What
>> > > > > is
>> > > > > > > >> it doing? Can you add more details about it?
>> > > > > > > >>
>> > > > > > > >> 4) Can fields of AuditInfo be null? I can see there's a
>> > > > constructor
>> > > > > > > >> without an Errors and that sets the error field to None.
>> > > However,
>> > > > > with
>> > > > > > > >> the builder pattern, if error is not set it's null.
>> > > > > > > >>
>> > > > > > > >> 5) Should AuditInfo be an interface?
>> > > > > > > >>
>> > > > > > > >> On Mon, Sep 14, 2020 at 3:26 PM Viktor Somogyi-Vass
>> > > > > > > >> <viktorsomo...@gmail.com> wrote:
>> > > > > > > >> >
>> > > > > > > >> > Hi everyone,
>> > > > > > > >> >
>> > > > > > > >> > Changed the interface a little bit to accommodate methods
>> > > better
>> > > > > > where
>> > > > > > > >> > authorization happens for multiple operations so the
>> > > implementer
>> > > > > of
>> > > > > > > the
>> > > > > > > >> > audit interface will receive all authorizations together.
>> > > > > > > >> > I'll wait a few more days to allow people to react or
>> give
>> > > > > feedback
>> > > > > > > but
>> > > > > > > >> if
>> > > > > > > >> > there are no objections until then, I'll start a vote.
>> > > > > > > >> >
>> > > > > > > >> > Viktor
>> > > > > > > >> >
>> > > > > > > >> > On Tue, Sep 8, 2020 at 9:49 AM Viktor Somogyi-Vass <
>> > > > > > > >> viktorsomo...@gmail.com>
>> > > > > > > >> > wrote:
>> > > > > > > >> >
>> > > > > > > >> > > Hi Everyone,
>> > > > > > > >> > >
>> > > > > > > >> > > I'd like to restart the discussion on this. Since the
>> KIP
>> > > has
>> > > > > been
>> > > > > > > >> > > revamped I thought I'd start a new discussion thread.
>> > > > > > > >> > >
>> > > > > > > >> > > Link:
>> > > > > > > >> > >
>> > > > > > > >>
>> > > > > > >
>> > > > > >
>> > > > >
>> > > >
>> > >
>> >
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-567%3A+Kafka+Cluster+Audit
>> > > > > > > >> > >
>> > > > > > > >> > > Short summary:
>> > > > > > > >> > > - Would like to introduce a new interface similar to
>> the
>> > > > > > Authorizer
>> > > > > > > >> called
>> > > > > > > >> > > Auditor as follows:
>> > > > > > > >> > >     public interface Auditor {
>> > > > > > > >> > >         audit(Request r, AuthorizableRequestContext c,
>> > > > > > AclOperation
>> > > > > > > >> > > o, Map<ResourcePattern, Boolean> isAllowed,
>> > > > Map<ResourcePattern,
>> > > > > > > >> Errors>
>> > > > > > > >> > > errors);
>> > > > > > > >> > >     }
>> > > > > > > >> > > - Basically it would pass down the request and the
>> > > > authorization
>> > > > > > > >> > > information to the auditor implementation where various
>> > kind
>> > > > of
>> > > > > > > >> reporting
>> > > > > > > >> > > can be done based on the request.
>> > > > > > > >> > > - A new config would be added called "auditor" which is
>> > > > similar
>> > > > > to
>> > > > > > > the
>> > > > > > > >> > > "authorizer" config, but users can pass a list of
>> auditor
>> > > > class
>> > > > > > > names.
>> > > > > > > >> > > - The implementation is expected to be low latency
>> > similarly
>> > > > to
>> > > > > > the
>> > > > > > > >> > > Authorizer.
>> > > > > > > >> > > - A default implementation will be added that logs
>> into a
>> > > > file.
>> > > > > > > >> > >
>> > > > > > > >> > > I appreciate any feedback on this.
>> > > > > > > >> > >
>> > > > > > > >> > > Best,
>> > > > > > > >> > > Viktor
>> > > > > > > >> > >
>> > > > > > > >>
>> > > > > > > >
>> > > > > > >
>> > > > > >
>> > > > >
>> > > >
>> > >
>> >
>>
>

Reply via email to