Hi all, If there are no more comments, I will start vote on this KIP tomorrow. Please let me know if there are any concerns.
Thanks, Rajini On Mon, Aug 21, 2017 at 9:24 AM, Rajini Sivaram <rajinisiva...@gmail.com> wrote: > Hi Jun, > > Thank you for the review. > > 1. Each token response will indicate success/failure. At the moment, the > broker simply closes the connection in the case of failure. With this KIP, > an empty token with an error_code indicating authentication failure will be > sent by the broker before closing the connection. I have made this clearer > in the KIP. > > 2. I have added a section on versioning of SaslAuthenticate and > SaslHandshake requests. > > > On Fri, Aug 18, 2017 at 2:47 PM, Jun Rao <j...@confluent.io> wrote: > >> Hi, Rajini, >> >> Thanks for the KIP. Looks good overall. Just a couple of minor comments. >> >> 1. "The final message from the broker will indicate if authentication >> succeeded or failed." Are we doing something special for the final >> message? >> I thought each token response will indicate success or failure in the >> error >> code. >> >> 2. It would be useful to document how we plan to evolve >> SaslAuthenticateRequest >> in the future. For example, if we add a new error code in >> SaslAuthenticateResponse, >> should we bump up the version of SaslAuthenticateRequest? When do we need >> to bump up SaslHandshakeRequest? >> >> Jun >> >> >> On Fri, Aug 18, 2017 at 10:59 AM, Ismael Juma <ism...@juma.me.uk> wrote: >> >> > Thanks Rajini. The changes look good. We have to update the motivation >> to >> > take into account the improvements in 0.11: >> > >> > "With the current Kafka SASL implementation, broker closes the client >> > connection if SASL authentication fails. Clients see this as a >> connection >> > failure and do not get any feedback for the reason why the connection >> was >> > closed. Producers and consumers retry, attempting to create successful >> > connections, treating authentication failures as transient failures. >> There >> > are no log entries on the client-side to indicate that any of these >> > connection failures were due to authentication failure." >> > >> > 0.11.0.x logs a warning with a hint that the disconnect is likely >> related >> > to authentication. >> > >> > Ismael >> > >> > On Tue, Aug 15, 2017 at 9:23 PM, Rajini Sivaram < >> rajinisiva...@gmail.com> >> > wrote: >> > >> > > I have updated the KIP based on the discussions so far. It will be >> good >> > if >> > > we can get some more feedback so that this can be implemented for >> 1.0.0. >> > > >> > > >> > > Thanks, >> > > >> > > Rajini >> > > >> > > >> > > On Thu, May 4, 2017 at 10:22 PM, Ismael Juma <ism...@juma.me.uk> >> wrote: >> > > >> > > > Hi Rajini, >> > > > >> > > > I think we were talking about slightly different things. I was just >> > > > referring to the fact that there are cases where we throw an >> > > > AuthorizationException back to the user without retrying from >> various >> > > > methods (poll, commitSync, etc). >> > > > >> > > > As you said, my initial preference was for not retrying at all >> because >> > it >> > > > is what you want in the common case of a misconfigured application. >> I >> > > > hadn't considered credential updates for authenticators that rely on >> > > > eventual consistency. Thinking about it some more, it seems like >> this >> > > > should be solved by the authenticator implementation as well. For >> > > example, >> > > > it could refresh the cached data for a user if authentication >> failed (a >> > > > good implementation would be a bit more involved to avoid going to >> the >> > > > underlying data source too often). >> > > > >> > > > Given that, not retrying sounds good to me. >> > > > >> > > > Ismael >> > > > >> > > > On Thu, May 4, 2017 at 4:04 PM, Rajini Sivaram < >> > rajinisiva...@gmail.com> >> > > > wrote: >> > > > >> > > > > Hi Ismael, >> > > > > >> > > > > I thought the blocking waits in the producer and consumer are >> always >> > > > > related to retrying for metadata. So an authorization exception >> that >> > > > > impacts this wait can only be due to Describe authorization >> failure - >> > > > that >> > > > > always retries? >> > > > > >> > > > > I agree that connecting to different brokers when authentication >> > fails >> > > > with >> > > > > one is not desirable. But I am not keen on retrying with a >> suitable >> > > > backoff >> > > > > until timeout either. Because that has the same problem as the >> > scenario >> > > > > that you described. The next metadata request could be to >> broker-1 to >> > > > which >> > > > > authentication succeeds and subsequent produce/consume to >> broker-0 >> > > could >> > > > > still fail. >> > > > > >> > > > > How about we just fail fast if one authentication fails - I think >> > that >> > > is >> > > > > what you were suggesting in the first place? We don't need to >> > blackout >> > > > any >> > > > > nodes beyond the reconnect backoff interval. Applications can >> still >> > > retry >> > > > > if they want to. In the case of credential updates, it will be up >> to >> > > the >> > > > > application to retry. During regular operation, a misconfigured >> > > > application >> > > > > fails fast with a meaningful exception. What do you think? >> > > > > >> > > > > Regards, >> > > > > >> > > > > Rajini >> > > > > >> > > > > >> > > > > On Thu, May 4, 2017 at 3:01 PM, Ismael Juma <ism...@juma.me.uk> >> > wrote: >> > > > > >> > > > > > H Rajini, >> > > > > > >> > > > > > Comments inline. >> > > > > > >> > > > > > On Thu, May 4, 2017 at 2:29 PM, Rajini Sivaram < >> > > > rajinisiva...@gmail.com> >> > > > > > wrote: >> > > > > > >> > > > > > > Hi Ismael, >> > > > > > > >> > > > > > > Thank you for reviewing the KIP. >> > > > > > > >> > > > > > > An authenticated client that is not authorized to access a >> topic >> > is >> > > > > never >> > > > > > > told that the operation was not authorized. This is to prevent >> > the >> > > > > client >> > > > > > > from finding out if the topic exists by sending an >> unauthorized >> > > > > request. >> > > > > > So >> > > > > > > in this case, the client will retry metadata requests with the >> > > > > configured >> > > > > > > backoff until it times out. >> > > > > > >> > > > > > >> > > > > > This is true if the user does not have Describe permission. If >> the >> > > user >> > > > > has >> > > > > > Describe access and no Read or Write access, then the user is >> > > informed >> > > > > that >> > > > > > the operation was not authorized. >> > > > > > >> > > > > > >> > > > > > > Another important distinction for authorization failures is >> that >> > > the >> > > > > > > connection is not terminated. >> > > > > > > >> > > > > > > For unauthenticated clients, we do want to inform the client >> that >> > > > > > > authentication failed. The connection is terminated by the >> > broker. >> > > > > > > Especially if the client is using SASL_SSL, we really do want >> to >> > > > avoid >> > > > > > > reconnections that result in unnecessary expensive >> handshakes. So >> > > we >> > > > > want >> > > > > > > to return an exception to the user with minimal retries. >> > > > > > > >> > > > > > >> > > > > > Agreed. >> > > > > > >> > > > > > I was thinking that it may be useful to try more than one broker >> > for >> > > > the >> > > > > > > case where brokers are being upgraded and some brokers haven't >> > yet >> > > > seen >> > > > > > the >> > > > > > > latest credentials. I suppose I was thinking that at the >> moment >> > we >> > > > keep >> > > > > > on >> > > > > > > retrying every broker forever in the consumer and suddenly if >> we >> > > stop >> > > > > > > retrying altogether, it could potentially lead to some >> unforeseen >> > > > > timing >> > > > > > > issues. Hence the suggestion to try every broker once. >> > > > > > > >> > > > > > >> > > > > > I see. Retrying forever is a side-effect of auto-topic creation, >> > but >> > > > it's >> > > > > > something we want to move away from. As mentioned, we actually >> > don't >> > > > > retry >> > > > > > at all if the user has Describe permission. >> > > > > > >> > > > > > Broker upgrades could be fixed by ensuring that the latest >> > > credentials >> > > > > are >> > > > > > loaded before the broker starts serving requests. More >> problematic >> > is >> > > > > > dealing with credential updates. This is another distinction >> when >> > > > > compared >> > > > > > to authorization. >> > > > > > >> > > > > > I am not sure if trying different brokers really helps us >> though. >> > > Say, >> > > > we >> > > > > > fail to authenticate with broker 0 and then we succeed with >> broker >> > 1. >> > > > > This >> > > > > > helps with metadata requests, but we will be in trouble when we >> try >> > > to >> > > > > > produce or consume to broker 0 (because it's the leader of some >> > > > > > partitions). So maybe we just want to retry with a suitable >> backoff >> > > > > until a >> > > > > > timeout? >> > > > > > >> > > > > > Yes, I agree that blacking out nodes forever isn't a good idea. >> > When >> > > we >> > > > > > > throw AuthenticationFailedException for the current operation >> or >> > if >> > > > > > > authentication to another broker succeeds, we can clear the >> > > blackout >> > > > so >> > > > > > > that any new request from the client can attempt reconnection >> > after >> > > > the >> > > > > > > reconnect backoff period as they do now. >> > > > > > > >> > > > > > >> > > > > > Yes, that would be better if we decide that connecting to >> different >> > > > > brokers >> > > > > > is worthwhile for the requests that can be sent to any broker. >> > > > > > >> > > > > > Ismael >> > > > > > >> > > > > > On Thu, May 4, 2017 at 2:29 PM, Rajini Sivaram < >> > > > rajinisiva...@gmail.com> >> > > > > > wrote: >> > > > > > >> > > > > > > Hi Ismael, >> > > > > > > >> > > > > > > Thank you for reviewing the KIP. >> > > > > > > >> > > > > > > An authenticated client that is not authorized to access a >> topic >> > is >> > > > > never >> > > > > > > told that the operation was not authorized. This is to prevent >> > the >> > > > > client >> > > > > > > from finding out if the topic exists by sending an >> unauthorized >> > > > > request. >> > > > > > So >> > > > > > > in this case, the client will retry metadata requests with the >> > > > > configured >> > > > > > > backoff until it times out. Another important distinction for >> > > > > > authorization >> > > > > > > failures is that the connection is not terminated. >> > > > > > > >> > > > > > > For unauthenticated clients, we do want to inform the client >> that >> > > > > > > authentication failed. The connection is terminated by the >> > broker. >> > > > > > > Especially if the client is using SASL_SSL, we really do want >> to >> > > > avoid >> > > > > > > reconnections that result in unnecessary expensive >> handshakes. So >> > > we >> > > > > want >> > > > > > > to return an exception to the user with minimal retries. >> > > > > > > >> > > > > > > I was thinking that it may be useful to try more than one >> broker >> > > for >> > > > > the >> > > > > > > case where brokers are being upgraded and some brokers haven't >> > yet >> > > > seen >> > > > > > the >> > > > > > > latest credentials. I suppose I was thinking that at the >> moment >> > we >> > > > keep >> > > > > > on >> > > > > > > retrying every broker forever in the consumer and suddenly if >> we >> > > stop >> > > > > > > retrying altogether, it could potentially lead to some >> unforeseen >> > > > > timing >> > > > > > > issues. Hence the suggestion to try every broker once. >> > > > > > > >> > > > > > > Yes, I agree that blacking out nodes forever isn't a good >> idea. >> > > When >> > > > we >> > > > > > > throw AuthenticationFailedException for the current operation >> or >> > if >> > > > > > > authentication to another broker succeeds, we can clear the >> > > blackout >> > > > so >> > > > > > > that any new request from the client can attempt reconnection >> > after >> > > > the >> > > > > > > reconnect backoff period as they do now. >> > > > > > > >> > > > > > > Regards, >> > > > > > > >> > > > > > > Rajini >> > > > > > > >> > > > > > > On Thu, May 4, 2017 at 12:51 PM, Ismael Juma < >> ism...@juma.me.uk> >> > > > > wrote: >> > > > > > > >> > > > > > > > Thanks Rajini. This is a good improvement. One question, the >> > > > proposal >> > > > > > > > states: >> > > > > > > > >> > > > > > > > Producer waitForMetadata and consumer ensureCoordinatorReady >> > will >> > > > be >> > > > > > > > > updated to throw AuthenticationFailedException if >> connections >> > > to >> > > > > all >> > > > > > > > > available brokers fail authentication. >> > > > > > > > >> > > > > > > > >> > > > > > > > Can you elaborate on the reason why we would treat >> > authentication >> > > > > > > failures >> > > > > > > > differently from authorization failures? It would be good to >> > > > > understand >> > > > > > > > under which scenario it would be beneficial to try all the >> > > brokers >> > > > > (it >> > > > > > > > seems that the proposal also suggests blacking out brokers >> > > > > permanently >> > > > > > if >> > > > > > > > we fail authentication, so that could also eventually cause >> > > > issues). >> > > > > > > > >> > > > > > > > Ismael >> > > > > > > > >> > > > > > > > >> > > > > > > > On Thu, May 4, 2017 at 12:37 PM, Rajini Sivaram < >> > > > > > rajinisiva...@gmail.com >> > > > > > > > >> > > > > > > > wrote: >> > > > > > > > >> > > > > > > > > Hi all, >> > > > > > > > > >> > > > > > > > > I have created a KIP to improve diagnostics for SASL >> > > > authentication >> > > > > > > > > failures and reduce retries and blocking when >> authentication >> > > > fails: >> > > > > > > > > >> > > > > > > > > https://cwiki.apache.org/confl >> uence/display/KAFKA/KIP-152+-+ >> > > > > > > > > Improve+diagnostics+for+SASL+authentication+failures >> > > > > > > > > >> > > > > > > > > Comments and suggestions are welcome. >> > > > > > > > > >> > > > > > > > > Thank you... >> > > > > > > > > >> > > > > > > > > Regards, >> > > > > > > > > >> > > > > > > > > Rajini >> > > > > > > > > >> > > > > > > > >> > > > > > > >> > > > > > >> > > > > >> > > > >> > > >> > >> > >