Skip to content

Add support for different claims for authentication#5336

Open
pranav shah (prashah-confluent) wants to merge 21 commits intomasterfrom
dev_prashah_sasl_subclaim
Open

Add support for different claims for authentication#5336
pranav shah (prashah-confluent) wants to merge 21 commits intomasterfrom
dev_prashah_sasl_subclaim

Conversation

@prashah-confluent
Copy link
Member

@prashah-confluent pranav shah (prashah-confluent) commented Mar 2, 2026

PR description

Problem

For OIDC / SASL_OAUTHBEARER flows using librdkafka’s built‑in OIDC helper, we currently hard‑require that the JWT payload contain a sub claim. If the IdP omits sub or uses another claim (for example client_id) as the primary subject, librdkafka fails token acquisition with an error such as:

Failed to acquire SASL OAUTHBEARER token: Expected JSON JWT response with "sub" field

Fix

We have introduced a new config sub_claim_name using which customers can provide a different claim. The claim value will be honored and used for client side validation.

Note
Requires users to keep broker and client configs aligned for the subject claim name to avoid confusion and consistency.

Testing

Unit tests

To test the core JWT payload parsing and claim extraction logic inside rd_kafka_oidc_token_try_validate() in isolation (no broker, network, oidc server).

  1. Default Configuration & Backward Compatibility
    Implicit Default Behavior: Validate that when sub.claim.name is not provided, the system defaults to extracting the subject from the "sub" claim.
    Legacy Parity: Ensure that existing JWTs containing only the "sub" claim continue to be processed correctly, maintaining no regressions for standard OIDC flows.

  2. Explicit Claim Configuration (KIP-768 Parity)
    Custom Claim Extraction: Verify that setting sub.claim.name (e.g., to client_id) successfully extracts the value from the specified field instead of the hardcoded "sub" key.
    Configuration Precedence: Test that when both "sub" and a custom claim (e.g., "client_id") coexist in a JWT, the system honors the user-defined configuration over the OIDC default.

  3. Error Handling & Validation Logic
    Missing Claim Detection: Verify that providing a sub.claim.name that does not exist in the JWT triggers a specific "Claim not found" error, rather than failing on a hardcoded "sub".
    Empty Value Rejection: Ensure the system rejects empty string subjects with a "non-empty value required" error, addressing previous bugs where empty strings passed silently.
    Invalid Configuration Recovery: Validate that if sub.claim.name is set to an empty string, the system gracefully falls back to the OIDC default ("sub") to ensure continuous operation.

  4. librdkafka & Java Client Alignment
    Functional Parity: Ensure all scenarios are cross-validated against the Java KIP-768 implementation to guarantee identical extraction logic and exception triggers (e.g., ValidateException equivalents).

Integration tests

  • To validate end-to-end OIDC authentication against a real Kafka cluster (via trivup) with different sasl.oauthbearer.sub.claim.name configurations
  • The trivup setup requires changes for returning JWT response with multiple claims

Design Doc

@confluent-cla-assistant
Copy link

🎉 All Contributor License Agreements have been signed. Ready to merge.
Please push an empty commit if you would like to re-run the checks to verify CLA status for all contributors.

@prashah-confluent pranav shah (prashah-confluent) marked this pull request as ready for review March 4, 2026 12:19
@prashah-confluent pranav shah (prashah-confluent) requested a review from a team as a code owner March 4, 2026 12:19
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks Pranav! This is my feedback:

"JWT claim name to use as the subject (principal) when validating "
"OIDC access tokens. Must be present in the JWT payload with a "
"non-empty value. Should match the broker's "
"sasl.oauthbearer.sub.claim.name configuration for consistent "

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Suggested change
"sasl.oauthbearer.sub.claim.name configuration for consistent "
"`sasl.oauthbearer.sub.claim.name` configuration for consistent "

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

rebuild CONFIGURATION.md as well.

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Changed to "sasl.oauthbearer.sub.claim.name" and rebuilt CONFIGURATION.md

Comment on lines +733 to +735
/* Safety check to default to "sub" if not set */
if (!sub_claim_name || !*sub_claim_name)
sub_claim_name = "sub";

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

it shoud be a rd_dassert here, not reverting to the default. The configuration should be validated in rd_kafka_conf_finalize_oauthbearer_oidc and using rd_kafka_conf_is_modified on the field and then check for NULL or a an empty string or whitespace string ("\s+") you can write a function like the Java method ClaimValidationUtils.validateString.

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Added rd_dassert and as suggested added helper api which validates string only if config is modified.

do_test_produce_consumer_with_OIDC(test_name,
sub_claim_conf);
} else {
/* These variations should fail due to missing claim */

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This function should be agnostic to the failure cause (at the moment it's only this one but in future there can be other test cases).

Suggested change
/* These variations should fail due to missing claim */
/* These variations should fail */

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Modified the comment to make the function failure agnostic

CHANGELOG.md Outdated

librdkafka v2.14.0 is a feature release:

* [KIP-768](https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=186877575#KIP768:ExtendSASL/OAUTHBEARERwithSupportforOIDC-ClientConfiguration) Extend SASL/OAUTHBEARER to support OIDC claim mapping beyond the default `sub` claim [(#5336)](https://github.com/confluentinc/librdkafka/pull/5336)

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Suggested change
* [KIP-768](https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=186877575#KIP768:ExtendSASL/OAUTHBEARERwithSupportforOIDC-ClientConfiguration) Extend SASL/OAUTHBEARER to support OIDC claim mapping beyond the default `sub` claim [(#5336)](https://github.com/confluentinc/librdkafka/pull/5336)
* [KIP-768](https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=186877575#KIP768:ExtendSASL/OAUTHBEARERwithSupportforOIDC-ClientConfiguration) Extend SASL/OAUTHBEARER to support OIDC claim mapping beyond the default `sub` claim (#5336).

The link is already added by GH in the release page, so we don't put the link here.

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Removed the link

Comment on lines +1391 to +1394
{"NULL sub_claim_name defaults to 'sub'", UT_JWT_SUB_ONLY, NULL,
rd_true, "subject"},
{"Empty sub_claim_name defaults to 'sub'", UT_JWT_SUB_ONLY, "",
rd_true, "subject"},

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We can remove these two cases after putting an assert there as it must not reach this function if those preconditions aren't satisfied.

Suggested change
{"NULL sub_claim_name defaults to 'sub'", UT_JWT_SUB_ONLY, NULL,
rd_true, "subject"},
{"Empty sub_claim_name defaults to 'sub'", UT_JWT_SUB_ONLY, "",
rd_true, "subject"},

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Removed the test cases since they can't be valid at this point.

Comment on lines +548 to +550
case OIDC_CONFIGURATION_SUB_CLAIM_VARIATION_EMPTY_STRING:
test_conf_set(conf, "sasl.oauthbearer.sub.claim.name", "");
break;

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This one should fail as well. Similarly to the Java client, in case it's set it should be valid otherwise the user could want to set something but there was a bug in their code so an empty string was received instead.

Copy link
Member Author

@prashah-confluent pranav shah (prashah-confluent) Mar 13, 2026

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This should pass due to librdkafka convention which sets the config to default values if empty/NULL string is provided.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

2 participants