Hi all, I just wanted to mention that there have been some minor updates to the configuration for OAuth.
The most relevant change is that I introduced an exponential backoff for when we attempt to retrieve the JWKS (JSON Web Key Set) from the OAuth/OIDC provider. Rather than hard-code the values, I introduced two new configuration options: * sasl.oauthbearer.jwks.endpoint.retry.backoff.ms: optional value in milliseconds for the amount of time to wait between HTTPS call attempts to retrieve the JWKS; only used when using an HTTP(S)-based URL for sasl.oauthbearer.jwks.endpoint.url ; defaults to 100 * sasl.oauthbearer.jwks.endpoint.retry.backoff.max.ms: optional value in milliseconds for the maximum wait for HTTPS call attempts to retrieve the JWKS; only used when using an HTTP(S)-based URL for sasl.oauthbearer.jwks.endpoint.url ; defaults to 10000 These are documented in the KIP and reflected in the code. Thanks, Kirk On Thu, Oct 7, 2021, at 1:15 PM, Rajini Sivaram wrote: > Hi Kirk, > > Thanks for the updates. Looks good. > > Just one comment on the naming of configs. For configs that are very specific > to OAUTHBEARER, can we add `sasl.oauthbearer` as the prefix, similar to > `sasl.kerberos.` that we use for Kerberos configs, e.g. > ``sasl.login.sub.claim.name```. For configs that could potentially be used by > any SASL mechanism that uses a remote server, we can keep the current naming > without the `oauthbearer`, e.g. ``sasl.login.connect.timeout.ms`. I think we > want to use the same convention for broker-side configs too, even though > broker configs may specify oauthbearer in the listener prefix so that we > remain consistent with other configs (also, we allow listener configs to be > specified without listener prefix as well). > > Regards, > > Rajini > > > On Thu, Oct 7, 2021 at 6:51 PM Kirk True <k...@mustardgrain.com> wrote: >> __ >> Hi Rajini, >> >> I've updated the KIP with your feedback. Let me know if there's anything >> still amiss. >> >> Thanks, >> Kirk >> >> On Wed, Oct 6, 2021, at 5:27 PM, Kirk True wrote: >>> Hi Rajini, >>> >>> Thank you very much for your in-depth review! You highlighted a lot of dark >>> corners :) >>> >>> > 1. The diagram shows broker startup followed by `broker requests keys >>> > from JWKS endpoint`. >>> > - Do we open broker ports only after we successfully get the keys? >>> > We >>> > need to guarantee this to ensure that clients don't see >>> > authentication >>> > failures during broker restarts. >>> > - Doesn't sound like we will persist the keys, so what is the >>> > behaviour if the OAuth server is not available? Will broker retry >>> > forever? >>> >>> In the case where the OAuth provider is unavailable, is it preferable for >>> the broker to start up in a diminished capacity or to simply fail to start >>> up at all? >>> >>> It's my understanding that a broker can support more than one form of >>> authentication. If so, should we continue start up if the other forms of >>> authentication are working? >>> >>> > 2. Client configuration includes a large number of JAAS config options >>> > like `loginRetryWaitMs` and `loginRetryMaxWaitMs`. Have we considered >>> > making them top-level configs instead? Not saying we should, but it >>> > will be >>> > good to document why we chose to do it this way. The advantage of >>> > top-level option is that it can be used for other similar login methods >>> > in future. And they become visible in logs (unlike `sasl.jaas.config` >>> > which is considered sensitive and hence not logged). The current >>> > approach keeps all the related configs together in one place, so that >>> > may >>> > be ok too, worth documenting the reasons anyway. It is useful to keep >>> > credentials in `sasl.jaas.config`, it is less clear with other configs >>> > (e.g. we have various `sasl.kerberos.` configs. >>> >>> I can look at moving the more general, non-sensitive configuration out from >>> under the JAAS configuration. Now that you mention it, I did notice that >>> the JAAS configuration was redacted in the logs. >>> >>> > 3. The extension config uses inconsistent naming ` >>> > Extension_supportFeatureX`. If we are trying to keep this consistent >>> > with the existing callback handler, should this be ` >>> > unsecuredLoginExtension_xxx` or otherwise `extension_xxx`? >>> >>> You're right, it was a half-baked attempt at consistency with the existing >>> unsecured implementation. >>> >>> I wanted to drop the "unsecuredLogin" prefix as it doesn't apply. Do you >>> have a preference for any of the following forms? >>> >>> * securedLoginExtension_xxx >>> * secureLoginExtension_xxx >>> * loginExtension_xxx >>> * extension_xxx >>> >>> > 4. We talk about re-authentication using KIP-368. Can we also describe >>> > re-login on the client-side to acquire new tokens? That should be >>> > based on >>> > expiry of the token and should happen irrespective of whether broker >>> > has >>> > enabled re-authentication. The unsecured version already supports >>> > this, so >>> > no additional work is necessary, worth mentioning nevertheless. >>> >>> I spent more time than I'd like to admit trying to trigger a client >>> side-only refresh. While the client would refresh and grab an updated token >>> from the OAuth provider, it never seemed to trigger a call to the broker to >>> re-validate. >>> >>> I'll take another look to see what I'm missing. >>> >>> > 5. KIP says: `A new key ID (kid) could appear in the header of an >>> > incoming JWT access token. Code that can retrieve the JWKS from the >>> > OAuth >>> > provider on demand will be implemented.`. What happens to the first >>> > connection that requires this? Given we can't block network thread >>> > while we >>> > do this network operation, will we fail authentications until we have >>> > refreshed keys in the background thread? >>> >>> Ugh. Another good catch :) >>> >>> There are a few cases related to the timing of a new key ID being >>> published. I'm going to try to make this sound all formal, but hopefully it >>> doesn't just come off confusing :) >>> >>> Let A = the time that the OAuth provider publishes the updated JWKS with >>> the new key ID. >>> >>> Let B = the time that the broker's internal key cache refresh is run. >>> >>> Let C = the time that the OAuth provider issues a JWT with a new key ID. >>> >>> Here are the timing cases: >>> >>> 1. A < B < C. This is the case where the JWKS publish time is far enough in >>> advance of first JWT issuance that our cache has had a chance to run and is >>> then fully refreshed and ready for the key. This is the optimal case. >>> 2. A < C < B. This is the case where the JWKS publish time happens before >>> JWT issuance, but after our last cache refresh. This is the case referred >>> to in the KIP when it says that the broker would block to look up the JWKS. >>> 3. C < A < B. This is the case where the JWKS publish time is *after* the >>> JWT issuance. I would hope this "should not happen", but I don't know the >>> timing is specified anywhere or if it happens occasionally. From the >>> broker's perspective, it's similar to case #2 except that even after >>> blocking we'd throw an error since we don't see the key in the JWKS. >>> >>> As you state, the approach of blocking while looking up the JWKS is >>> unacceptable. As such, we should probably introduce some state management >>> for dynamically retrieving a new JWKS. In short, if the JWKS doesn't >>> include the key ID, we should fail the client validation. However, we >>> should attempt to reload the JWKS "intelligently" to avoid redundant >>> lookups, prevent DOS if the client unintentionally (or maliciously) >>> continuously provides invalid key IDs. >>> >>> This obviously requires a lot more explanation in the KIP and perhaps even >>> some diagrams. >>> >>> > 6. `jwksFile` option for brokers: Couldn't this just be `file:` URI >>> > for ` >>> > jwksEndpointUri`? >>> >>> Yes, probably. I'll look into simplifying that. >>> >>> > 7. Like with clients, have we considered making some of the broker's >>> > options standalone configs rather than part of `sasl.jaas.config`? That >>> > would allow these configs to be logged, described using admin client >>> > and >>> > independently modified as dynamic configs. >>> >>> Good point. >>> >>> Would we want the configuration options to be top level? For example: >>> >>> * login.retry.wait.ms >>> >>> Or should the options be scoped to a particular listener and SASL >>> mechanism? For example: >>> >>> * listener.name.xxx.oauthbearer.sasl.login.retry.wait.ms >>> >>> > 8. In the broker-to-broker section, it will be good to document that >>> > brokers should configure `sasl.jaas.config` that includes both >>> > client-side and server-side options. >>> >>> OK. I'll update that after I've resolved which configuration options are >>> moved out to be top-level. >>> >>> > 9. Any reason why we chose to use system properties rather than command >>> > line options for OAuthCompatibilityTest? >>> >>> I think I convinced myself that it made things easier and avoided a lot of >>> boilerplate. I'll take another look. >>> >>> Thanks! >>> Kirk >>> >>> > >>> > >>> > Regards, >>> > >>> > Rajini >>> > >>> > On Thu, Sep 23, 2021 at 9:33 PM Kirk True <k...@mustardgrain.com> wrote: >>> > >>> > > Hi Manikumar, >>> > > >>> > > On Wed, Aug 25, 2021, at 8:54 PM, Manikumar wrote: >>> > > > Thanks for the reply, >>> > > > >>> > > > Can we also update the KIP about the testing approach? >>> > > >>> > > Yes, I've added that as a dedicated section in the KIP: >>> > > >>> > > >>> > > https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=186877575#KIP768:ExtendSASL/OAUTHBEARERwithSupportforOIDC-Testing >>> > > >>> > > Thanks, >>> > > Kirk >>> > > >>> > > > Thanks, >>> > > > >>> > > > On Wed, Aug 25, 2021 at 12:01 AM Kirk True <k...@mustardgrain.com> >>> > > wrote: >>> > > >> __ >>> > > >> Hi Manikumar! >>> > > >> >>> > > >> On Mon, Aug 23, 2021, at 12:59 PM, Manikumar wrote: >>> > > >>> Hi Kirk, >>> > > >>> >>> > > >>> Thanks for the KIP! >>> > > >>> >>> > > >>> 1. Do we want to support validating issuers using the issuer uri? >>> > > >> >>> > > >> Are you referring to validating the JWT was issued by a known, >>> > > configured issuer, or something more different/more dynamic? >>> > > >> >>> > > >> The current design allows for optionally requiring that the iss claim >>> > > from the JWT match a statically configured issuer from the >>> > > configuration. >>> > > See expectedIssuer under the broker configuration. >>> > > >> >>> > > >>> 2. Can the access token be reused for multiple connections from the >>> > > same >>> > > >>> client? >>> > > >> >>> > > >> That's an excellent question - I will double-check that it is reused. >>> > > Hopefully the existing client authentication mechanism supports that >>> > > level >>> > > of reuse. >>> > > >> >>> > > >>> 3. Do we support configuring separate ssl configs for connecting >>> > > >>> authorization server/jwks endpoint? >>> > > >> >>> > > >> Yes, that documentation is forthcoming soon. >>> > > >> >>> > > >> It will be a set of configuration options similar to the existing SSL >>> > > socket configuration, but specific to the HTTPS endpoint for the >>> > > OAuth/OIDC >>> > > provider connections. >>> > > >> >>> > > >>> 4. Do we want support any readable username as principal if it is >>> > > present >>> > > >>> in the token claims >>> > > >> >>> > > >> Yes. See the subClaimName and principalClaimName configuration >>> > > >> options. >>> > > Those will allow specifying a claim name other than sub for the >>> > > principal. >>> > > >> >>> > > >> Now that I'm writing this out I realize that the configuration names >>> > > are different on the client and broker for some reason 🤔 I'll change >>> > > that. >>> > > >> >>> > > >>> 5. I agree with Ron, We should move the public classes to >>> > > >>> "o.a.k.c.s.oauthbearer.secured" package. >>> > > >> >>> > > >> Sounds good. I made the change in the KIP. >>> > > >> >>> > > >>> Thanks, >>> > > >>> Manikumar >>> > > >> >>> > > >> Thanks for your feedback! >>> > > >> >>> > > >>> >>> > > >>> On Thu, Aug 19, 2021 at 11:46 PM Kirk True <k...@mustardgrain.com> >>> > > wrote: >>> > > >>> >>> > > >>> > Hi Ron, >>> > > >>> > >>> > > >>> > On Sat, Aug 14, 2021, at 11:27 AM, Ron Dagostino wrote: >>> > > >>> > > Hi Kirk -- thanks for the KIP! Having concrete implementations >>> > > >>> > > out-of-the-box will be very helpful. >>> > > >>> > > >>> > > >>> > > > As seen in this diagram, the login callback is executed on the >>> > > client >>> > > >>> > and >>> > > >>> > > the validate callback is executed on the broker. >>> > > >>> > > >>> > > >>> > > There was no diagram when I looked. Maybe there is a broken >>> > > >>> > > link >>> > > or >>> > > >>> > > something? >>> > > >>> > >>> > > >>> > I double-checked and it's showing for me on the published version >>> > > >>> > of >>> > > the >>> > > >>> > wiki, even after I've logged out. >>> > > >>> > >>> > > >>> > Would you mind checking again when you get the chance? >>> > > >>> > >>> > > >>> > > > The name of the implementation class will be >>> > > >>> > > >>> > > >>> > >>> > > org.apache.kafka.common.security.oauthbearer.internals.secured.OAuthBearerLoginCallbackHandler >>> > > >>> > > >>> > > >>> > > I think the internals package was meant for non-public stuff >>> > > >>> > > Most >>> > > of it >>> > > >>> > > seems that way, although the "unsecured" implementation is in >>> > > there -- >>> > > >>> > but >>> > > >>> > > that's maybe a grey area since it isn't meant to be used in >>> > > production >>> > > >>> > > scenarios and is mostly leveraged in unit tests. Perhaps move >>> > > >>> > > the >>> > > >>> > proposed >>> > > >>> > > class into a "o.a.k.c.s.oauthbearer.secured" package? Then any >>> > > >>> > > implementation details beyond the public stuff can live under >>> > > >>> > > the >>> > > >>> > > "...internals.secured" package that you mentioned? The same >>> > > comment >>> > > >>> > > applies to the validator callback handler class. >>> > > >>> > >>> > > >>> > In a draft I had the secured package directly under the >>> > > >>> > oauthbearer >>> > > >>> > package as you describe but I received some out-of-band feedback >>> > > >>> > to >>> > > aim for >>> > > >>> > parity with the unsecured package layout. >>> > > >>> > >>> > > >>> > I don't have a preference for either. I do agree that it seems >>> > > >>> > weird >>> > > for a >>> > > >>> > package named internals to be used in configuration since its name >>> > > implies >>> > > >>> > that things could change. >>> > > >>> > >>> > > >>> > > I'm confused by loginRetryMaxWaitMs and loginRetryWaitMs. The >>> > > former has >>> > > >>> > > "Max" in the name, but only the description of the latter >>> > > >>> > > mentions >>> > > it >>> > > >>> > being >>> > > >>> > > a max amount of time? Are the descriptions incorrect or perhaps >>> > > >>> > reversed? >>> > > >>> > >>> > > >>> > Yes. Thanks for catching that. I've added more description in a >>> > > separate >>> > > >>> > paragraph above the enumerated configurations. >>> > > >>> > >>> > > >>> > > > Ensure the encoding algorithm isn't none and matches what the >>> > > expected >>> > > >>> > > algorithm expecting >>> > > >>> > > >>> > > >>> > > "expected algorithm expecting" some kind of grammar issue? >>> > > >>> > >>> > > >>> > Haha! Yes - thanks for catching that too! >>> > > >>> > >>> > > >>> > It now reads: >>> > > >>> > >>> > > >>> > > Ensure the encoding algorithm (`alg` from the header) isn't >>> > > >>> > > `none` >>> > > and >>> > > >>> > matches the expected algorithm for the JWK ID >>> > > >>> > >>> > > >>> > > Thanks again -- very exciting! >>> > > >>> > >>> > > >>> > Thanks for the feedback!!! >>> > > >>> > >>> > > >>> > Kirk >>> > > >>> > >>> > > >>> > > >>> > > >>> > > Ron >>> > > >>> > > >>> > > >>> > > >>> > > >>> > > >>> > > >>> > > >>> > > >>> > > >>> > > >>> > > On Fri, Aug 13, 2021 at 3:53 PM Kirk True >>> > > >>> > > <k...@mustardgrain.com> >>> > > wrote: >>> > > >>> > > >>> > > >>> > > > Hi all! >>> > > >>> > > > >>> > > >>> > > > I have created a new KIP for a new OAuth/OIDC related >>> > > authentication >>> > > >>> > > > feature. >>> > > >>> > > > >>> > > >>> > > > This task is to provide a concrete implementation of the >>> > > interfaces >>> > > >>> > > > defined in KIP-255 to allow Kafka to connect to an OAuth / >>> > > >>> > > > OIDC >>> > > >>> > identity >>> > > >>> > > > provider for authentication and token retrieval. While KIP-255 >>> > > >>> > provides an >>> > > >>> > > > unsecured JWT example for development purposes, this will fill >>> > > in the >>> > > >>> > gap >>> > > >>> > > > and provide a production-grade implementation. >>> > > >>> > > > >>> > > >>> > > > Here's the KIP: >>> > > >>> > > > >>> > > >>> > > > >>> > > >>> > > > >>> > > >>> > >>> > > https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=186877575 >>> > > >>> > > > >>> > > >>> > > > Thanks! >>> > > >>> > > > Kirk >>> > > >>> > > >>> > > >>> > >>> > > >>> >>> > > >> >>> > > >>> > >>> >>