Hi Rajini.  I'm glad you agree this is a good security addition.  Regarding
your questions/comments:

1. Retries.

<<<What is the state of a client connection in between re-authentication
retries?
<<<Can it continue to process requests?
Yes, the existing connection remains intact and continues to accept
requests and process responses.  In fact, the existing connection is never
closed/replaced at any time.  The 3 APIs that constitute the authentication
"dance" (ApiVersionsRequest/Response, SaslHandshakeRequest/Response, and
SaslAuthenticateRequest/Response) traverse the existing socket connection,
and the only impact of re-authentication (aside from sending these three
API calls) is that the KafkaChannel#principal() method on the server side
ends up returning a different value once re-authentication succeeds
(because the KafkaChannel instance ends up with a new instance of
SaslServerAuthenticator, which is where the call gets delegated).

<<<Also, can you describe scenarios where retries are useful?
I have added this sentence to the KIP: "A retry is helpful when
re-authentication fails on the server side due to some temporary outage
(for example, the re-authentication will fail through no fault of the
client if the token endpoint is unavailable and the broker has not yet
cached the public key required to validate the token signature)."

2. Metrics:

<<<I think we should add separate re-authentication success/failure metrics.
Yes, I stated there are none only to be transparent about that fact more
than anything, and I am not surprised to get this feedback.  I will add
them to the KIP.

3. Implementation

<<<Hmm...
lol.  Yeah, I'm not surprised to get this feedback, either.  I came at this
totally fresh (i.e. no knowledge of this part of the code base), and I
avoided writing the integration tests (which I think might take a bit of
time) because I wanted confirmation on the approach first.  So this
feedback is excellent.  Let me try to unpack/address it, and let's see
where we end up.

<<<NetworkClient shouldn't have to deal with authentication
Actually, the only change I made to NetworkClient is to add the below
method because there is currently no way to submit a request with a
callback and have that request use the default timeout value of the client
-- currently if you specify a callback you also must explicitly specify a
timeout.  I considered adding this to the KafkaClient interface but decided
this was quicker in terms of getting the KIP and PR out there for
discussion.

public ClientRequest newClientRequest(String nodeId,

                                      AbstractRequest.Builder<?>
requestBuilder,

                                      long createdTimeMs,

                                      boolean expectResponse,

                                      RequestCompletionHandler callback) {

    return newClientRequest(nodeId, requestBuilder, createdTimeMs,
expectResponse, defaultRequestTimeoutMs, callback);

}


<<<changes to every use (AdminClient/KafkaConsumer/KafkaProducer/Broker) to

<<<deal with security protocols and SASL mechanisms will be hard to
maintain.

<<<And what about Connect and Streams - will they need changing as well?


I believe Streams and Connect are taken care of because they are built on
top of standard producers and consumers.  I nevertheless do agree with the
comment that this might be hard to maintain.  If there is ever a need for a
new use of NetworkClient -- whether as part of new inter-broker
communication functionality or as part of a client that does not leverage
the standard producer/consumer -- then that new use will have to address
the issue of how to inject the requests related to re-authentication.  I am
not happy with this, and I only arrived at it after rejecting the
"one-size-fits-all" approach that I mentioned in the "Rejected
Alternatives" section.  I think you are suggesting a third alternative,
though.  I'll address that in a moment.

First, I do want to make sure we are on the same page with what the current
changes are with respect to imports.  I realize I may be repeating stuff
you already know having looked at the PR, and I apologize if this is the
case.  I did have to make one change to checkstyle/import-control.xml to
allow org.apache.kafka.common.network.KafkaChannel to import the below
classes -- aside from that, though, every other import I added was already
"allowed".

// added to KafkaChannel

import org.apache.kafka.common.requests.RequestHeader;

import org.apache.kafka.common.requests.SaslAuthenticateRequest;

import org.apache.kafka.common.requests.SaslAuthenticateResponse;

import org.apache.kafka.common.requests.SaslHandshakeRequest;

import org.apache.kafka.common.requests.SaslHandshakeResponse;

The above needed to be imported because of the addition of these two
methods to KafkaChannel:

    /**
     * Respond to a SASL re-authentication initial handshake. This occurs
on the
     * SASL Server side of the re-authentication dance (i.e. on the broker).
     *
     * @param requestHeader
     *            the request header
     * @param saslHandshakeRequest
     *            the initial handshake request to process
     * @return the response to return to the client
     */
    public SaslHandshakeResponse
respondToReauthenticationSaslHandshakeRequest(RequestHeader requestHeader,
            SaslHandshakeRequest saslHandshakeRequest)

    /**
     * Respond to a SASL re-authentication token exchange. This occurs on
the SASL
     * Server side of the re-authentication dance (i.e. on the broker).
     *
     * @param requestHeader
     *            the request header
     * @param saslAuthenticateRequest
     *            the token exchange request to process
     * @return the response to return to the client
     */
    public SaslAuthenticateResponse
respondToReauthenticationSaslAuthenticateRequest(RequestHeader
requestHeader,
            SaslAuthenticateRequest saslAuthenticateRequest)

And then of course I route the requests to these methods via the following
code in kafka.server.KafkaApis:

  def handleSaslHandshakeRequest(request: RequestChannel.Request) {

    requestChannel.kafkaChannel(request) match {

      case Some(kafkaChannel) =>

        sendResponseMaybeThrottle(request, _ =>
kafkaChannel.respondToReauthenticationSaslHandshakeRequest(request.header,
request.body[SaslHandshakeRequest]))

      case None =>

        sendResponseMaybeThrottle(request, _ => new
SaslHandshakeResponse(Errors.ILLEGAL_SASL_STATE, Collections.emptySet()))

    }

  }


  def handleSaslAuthenticateRequest(request: RequestChannel.Request) {

    requestChannel.kafkaChannel(request) match {

      case Some(kafkaChannel) =>

        sendResponseMaybeThrottle(request, _ =>
kafkaChannel.respondToReauthenticationSaslAuthenticateRequest(request.header,
request.body[SaslAuthenticateRequest]))

      case None =>

        sendResponseMaybeThrottle(request, _ => new
SaslAuthenticateResponse(Errors.ILLEGAL_SASL_STATE,

            "SaslAuthenticate request received but could not find channel
(should not happen)"))

    }

  }

To address what I think you are proposing -- the third alternative.  Are
referring to the fact that SaslClientAuthenticator and
SaslServerAuthenticator actually talk directly over the socket, and are you
suggesting that we somehow handle re-authentication in the same way?

    // buffers used in `authenticate`

    private NetworkReceive netInBuffer;

    private Send netOutBuffer;

I did not like the fact that authentication and re-authentication ended up
working differently -- authentication via direct socket sends and
re-authentication via handling API requests in the normal
kafka.server.KafkaApis API control flow.  The nonce-based proposal
(described in the "Rejected Alternatives" section) was a way of reusing the
authentication code for re-authentication -- simply authenticate a new
socket to the same server and then transfer the principal to the original
connection.  So I am definitely in favor of having authentication and
re-authentication work the same.  I just don't know if it is possible to
make re-authentication work the way authentication works.  I imagine that
trying to inject stuff directly into the socket when that socket is
simultaneously being used for something else -- it feels like a really bad
idea that isn't viable.  It works for authentication because at that point
we know that nobody else is using the socket.  But thereafter I would
imagine all bets are off.

Am I understanding what you were suggesting?  Was the above description
helpful?  I think my instincts are similar to yours -- I'm not a a fan of
making several changes to make this work in several places, and it feels a
bit wrong to have authentication and re-authentication work differently --
I just don't see a way around it at this point.  Do you?

Ron

On Tue, Sep 4, 2018 at 7:40 AM Rajini Sivaram <rajinisiva...@gmail.com>
wrote:

> Hi Ron,
>
> Thanks for the KIP. This is going to be a really useful feature to tighten
> security.
>
> I have a few comments/questions:
>
>    1. Retries: What is the state of a client connection in between
>    re-authentication retries? Can it continue to process requests? Also,
> can
>    you describe scenarios where retries are useful?
>    2. Metrics: I think we should add separate re-authentication
>    success/failure metrics.
>    3. Implementation: Hmm... The PR looks quite a lot different from what I
>    hoped we would do. In particular, I am concerned about network and
> security
>    layer code escaping into the client/broker implementation layers.
>
> "*We leave it to the owners of the **NetworkClient instances to define how
> to inject such requests by providing an implementation of the interface to
> the SaslChannelBuilder, which in turn provides it to the *
> *SaslClientAuthenticator*"
>
> NetworkClient shouldn't have to deal with authentication and changes to
> every use (AdminClient/KafkaConsumer/KafkaProducer/Broker) to deal with
> security protocols and SASL mechanisms will be hard to maintain. And what
> about Connect and Streams - will they need changing as well?
>
> Have we considered implementing re-authentication within the
> network/security layer, like we do authentication? I think we should try
> and find a way to move channels back to authentication/re-authentication
> phase so that the rest of the codebase doesn't have special code to handle
> SASL. I haven't looked into how this can be done, but it doesn't feel
> impossible. Thoughts?
>
>
>
> On Mon, Sep 3, 2018 at 10:54 PM, Ron Dagostino <rndg...@gmail.com> wrote:
>
> > I just realized there was one place in the KIP that stated that retries
> > could occur indefinitely (when a client attempts to change identity,
> which
> > we arbitrarily disallow).  This was a mistake, a holdover from a prior
> > draft of the KIP.  This is now fixed.  Retries are never allowed
> > indefinitely.
> >
> > <<<if a connection originally authenticates as User:user1, an attempt to
> > re-authenticate as anything else (e.g. User:user2) will fail.
> > <<<Retry is allowed indefinitely in this case.
> > >>>if a connection originally authenticates as User:user1, an attempt to
> > re-authenticate as anything else (e.g. User:user2) will fail.
> > >>>Retry is allowed in this case (still subject to the expiration of the
> > original credential as described above)
> >
> > Ron
> >
> >
> > On Mon, Sep 3, 2018 at 5:35 PM Ron Dagostino <rndg...@gmail.com> wrote:
> >
> > > Thanks for the feedback, Stanislav.
> > >
> > > <<<I am not too familiar with the networking code to evaluate your
> > > solution.
> > > Yeah, I wasn't familiar with it when I started, either, and I am hoping
> > > people who are intimately familiar with it will take a close look.
> Some
> > of
> > > that code seems to be very central to the core of Kafka, and injecting
> > > re-authentication attempts into the flow of replica fetching, sending
> > > transaction markers, and producing or consuming messages is not
> > something I
> > > am convinced is acceptable under all circumstances.  To be clear,
> > though, I
> > > am not saying it is problematic, either -- I just don't have enough
> > > experience or familiarity to know.  I really do want additional eyes on
> > > this if possible.
> > >
> > > Regarding your question about retries, here's the part of the KIP that
> > > deals with those:
> > >
> > > If a re-authentication attempt should fail then the connection will be
> > >> told to retry after some delay depending on how many retries have been
> > >> attempted so far: after some small amount of time for the first retry
> > (e.g.
> > >> 1 minute), double that for the second retry, and 4 times the initial
> > delay
> > >> for every retry thereafter.  Retry attempts generally occur at least
> > until
> > >> the current credential expires (but not indefinitely – and of course
> > they
> > >> won't continue if one of them actually succeeds).  There are certain
> > errors
> > >> that result in retries not being attempted (i.e. if some internal
> state
> > >> doesn't make sense, which generally should not happen).
> > >
> > >
> > > <<<Would we retry endlessly?
> > > No.  There may be at most one retry after the client token expires.  So
> > if
> > > a token expires after an hour, and several retry attempts fail
> including
> > > one at minute 59, then one last attempt will be made at the 63-minute
> > mark.
> > >
> > > <<<Since the functionality for brokers to close connections is outside
> > the
> > > scope of this KIP, what effect would
> > > <<<success/failure in re-authentication have
> > > Practically speaking, unless authorization is based on the contents of
> > the
> > > token rather than ACLs, the ultimate success or failure of
> > > re-authentication has no effect.  The intent is definitely to follow
> this
> > > KIP with another one to add the ability for brokers to close
> connections
> > > that use expired credentials, and then at that point the client would
> > have
> > > to successfully re-authenticate to avoid the connection being forcibly
> > > closed.
> > >
> > > Ron
> > >
> > >
> > > On Mon, Sep 3, 2018 at 12:58 PM Stanislav Kozlovski <
> > > stanis...@confluent.io> wrote:
> > >
> > >> Hi Ron,
> > >>
> > >> I really like this KIP, it is very needed.
> > >> I am still looking through it but unfortunately I am not too familiar
> > with
> > >> the networking code to evaluate your solution.
> > >>
> > >> I'd like to ask what happens if re-authentication consistently fails.
> > >> Would
> > >> we retry endlessly? Since the functionality for brokers to close
> > >> connections is outside the scope of this KIP, what effect would
> > >> success/failure in re-authentication have? I think it's worth noting
> in
> > >> the
> > >> KIP
> > >>
> > >> I also think the rejected alternative of initiating a new connection
> > >> should
> > >> stay rejected. I am not aware of anything currently limiting the
> client
> > to
> > >> connect to the same broker, but I think it would be best if we kept
> > >> Kafka's
> > >> options open (e.g addition of a load balancer in the future) and not
> > >> introduce additional client-broker statefulness.
> > >>
> > >> Thanks,
> > >> Stanislav
> > >>
> > >> On Tue, Aug 28, 2018 at 5:28 PM Ron Dagostino <rndg...@gmail.com>
> > wrote:
> > >>
> > >> > Hi everyone. I created KIP 368: Allow SASL Connections to
> Periodically
> > >> > Re-Authenticate
> > >> > <
> > >> >
> > >> https://cwiki.apache.org/confluence/display/KAFKA/KIP-368%
> > 3A+Allow+SASL+Connections+to+Periodically+Re-Authenticate
> > >> > >
> > >> > (
> > >> >
> > >> >
> > >> https://cwiki.apache.org/confluence/display/KAFKA/KIP-368%
> > 3A+Allow+SASL+Connections+to+Periodically+Re-Authenticate
> > >> > ).
> > >> > The motivation for this KIP is as follows:
> > >> >
> > >> > The adoption of KIP-255: OAuth Authentication via SASL/OAUTHBEARER
> > >> > <
> > >> https://cwiki.apache.org/confluence/pages/viewpage.action?
> > pageId=75968876
> > >> >
> > >> > in
> > >> > release 2.0.0 creates the possibility of using information in the
> > bearer
> > >> > token to make authorization decisions.  Unfortunately, however,
> Kafka
> > >> > connections are long-lived, so there is no ability to change the
> > bearer
> > >> > token associated with a particular connection.  Allowing SASL
> > >> connections
> > >> > to periodically re-authenticate would resolve this.  In addition to
> > this
> > >> > motivation there are two others that are security-related.  First,
> to
> > >> > eliminate access to Kafka for connected clients, the current
> > >> requirement is
> > >> > to remove all authorizations (i.e. remove all ACLs).  This is
> > necessary
> > >> > because of the long-lived nature of the connections.  It is
> > >> operationally
> > >> > simpler to shut off access at the point of authentication, and with
> > the
> > >> > release of KIP-86: Configurable SASL Callback Handlers
> > >> > <
> > >> >
> > >> https://cwiki.apache.org/confluence/display/KAFKA/KIP-86%3A+
> > Configurable+SASL+callback+handlers
> > >> > >
> > >> > it
> > >> > is going to become more and more likely that installations will
> > >> > authenticate users against external directories (e.g. via LDAP).
> The
> > >> > ability to stop Kafka access by simply disabling an account in an
> LDAP
> > >> > directory (for example) is desirable.  The second motivating factor
> > for
> > >> > re-authentication related to security is that the use of short-lived
> > >> tokens
> > >> > is a common OAuth security recommendation, but issuing a short-lived
> > >> token
> > >> > to a Kafka client (or a broker when OAUTHBEARER is the inter-broker
> > >> > protocol) currently has no benefit because once a client is
> connected
> > >> to a
> > >> > broker the client is never challenged again and the connection may
> > >> remain
> > >> > intact beyond the token expiration time (and may remain intact
> > >> indefinitely
> > >> > under perfect circumstances).  This KIP proposes adding the ability
> > for
> > >> > clients (and brokers when OAUTHBEARER is the inter-broker protocol)
> to
> > >> > re-authenticate their connections to brokers and have the new bearer
> > >> token
> > >> > appear on their session rather than the old one.
> > >> >
> > >> > The description of this KIP is actually quite straightforward from a
> > >> > functionality perspective; from an implementation perspective,
> though,
> > >> the
> > >> > KIP is not so straightforward, so it includes a pull request with a
> > >> > proposed implementation.  See https://github.com/apache/kafk
> > a/pull/5582
> > >> .
> > >> >
> > >> > Ron
> > >> >
> > >>
> > >>
> > >> --
> > >> Best,
> > >> Stanislav
> > >>
> > >
> >
>

Reply via email to