Skip to content

Commit

Permalink
Merge pull request #138188 from cockroachdb/blathers/backport-release…
Browse files Browse the repository at this point in the history
…-23.2.19-rc-138152

release-23.2.19-rc: jwtauthccl: add mapper for issuer url to jwks URI
souravcrl authored Jan 3, 2025
2 parents 2bfc920 + 50a0043 commit 9577e2f
Showing 7 changed files with 319 additions and 102 deletions.
3 changes: 3 additions & 0 deletions pkg/ccl/jwtauthccl/BUILD.bazel
Original file line number Diff line number Diff line change
@@ -21,8 +21,10 @@ go_library(
"//pkg/util/syncutil",
"//pkg/util/uuid",
"@com_github_cockroachdb_errors//:errors",
"@com_github_cockroachdb_redact//:redact",
"@com_github_lestrrat_go_jwx//jwk",
"@com_github_lestrrat_go_jwx//jwt",
"@org_golang_x_exp//maps",
],
)

@@ -55,6 +57,7 @@ go_test(
"//pkg/util/timeutil",
"@com_github_cockroachdb_errors//:errors",
"@com_github_cockroachdb_errors//oserror",
"@com_github_cockroachdb_redact//:redact",
"@com_github_lestrrat_go_jwx//jwa",
"@com_github_lestrrat_go_jwx//jwk",
"@com_github_lestrrat_go_jwx//jwt",
56 changes: 29 additions & 27 deletions pkg/ccl/jwtauthccl/authentication_jwt.go
Original file line number Diff line number Diff line change
@@ -23,6 +23,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/util/syncutil"
"github.com/cockroachdb/cockroach/pkg/util/uuid"
"github.com/cockroachdb/errors"
"github.com/cockroachdb/redact"
"github.com/lestrrat-go/jwx/jwk"
"github.com/lestrrat-go/jwx/jwt"
)
@@ -64,7 +65,7 @@ type jwtAuthenticator struct {
type jwtAuthenticatorConf struct {
audience []string
enabled bool
issuers []string
issuersConf issuerURLConf
jwks jwk.Set
claim string
jwksAutoFetchEnabled bool
@@ -84,7 +85,7 @@ func (authenticator *jwtAuthenticator) reloadConfigLocked(
conf := jwtAuthenticatorConf{
audience: mustParseValueOrArray(JWTAuthAudience.Get(&st.SV)),
enabled: JWTAuthEnabled.Get(&st.SV),
issuers: mustParseValueOrArray(JWTAuthIssuers.Get(&st.SV)),
issuersConf: mustParseJWTIssuersConf(JWTAuthIssuersConfig.Get(&st.SV)),
jwks: mustParseJWKS(JWTAuthJWKS.Get(&st.SV)),
claim: JWTAuthClaim.Get(&st.SV),
jwksAutoFetchEnabled: JWKSAutoFetchEnabled.Get(&st.SV),
@@ -134,7 +135,7 @@ func (authenticator *jwtAuthenticator) ValidateJWTLogin(
user username.SQLUsername,
tokenBytes []byte,
identMap *identmap.Conf,
) (detailedErrorMsg string, authError error) {
) (detailedErrorMsg redact.RedactableString, authError error) {
authenticator.mu.Lock()
defer authenticator.mu.Unlock()

@@ -154,27 +155,17 @@ func (authenticator *jwtAuthenticator) ValidateJWTLogin(
}

// Check for issuer match against configured issuers.
issuerUrl := ""
issuerMatch := false
for _, issuer := range authenticator.mu.conf.issuers {
if issuer == unverifiedToken.Issuer() {
issuerMatch = true
issuerUrl = issuer
break
}
}
if !issuerMatch {
return "", errors.WithDetailf(
errors.Newf("JWT authentication: invalid issuer"),
"token issued by %s", unverifiedToken.Issuer())
tokenIssuer := unverifiedToken.Issuer()
if err = authenticator.mu.conf.issuersConf.checkIssuerConfigured(tokenIssuer); err != nil {
return "", errors.WithDetailf(err, "token issued by %s", tokenIssuer)
}

var jwkSet jwk.Set
// If auto-fetch is enabled, fetch the JWKS remotely from the issuer's well known jwks url.
// If auto-fetch is enabled, fetch the JWKS remotely from the issuer's well known jwks URI.
if authenticator.mu.conf.jwksAutoFetchEnabled {
jwkSet, err = authenticator.remoteFetchJWKS(ctx, issuerUrl)
jwkSet, err = authenticator.remoteFetchJWKS(ctx, tokenIssuer)
if err != nil {
return fmt.Sprintf("unable to fetch jwks: %v", err),
return redact.Sprintf("unable to fetch jwks: %v", err),
errors.Newf("JWT authentication: unable to validate token")
}
} else {
@@ -306,16 +297,27 @@ func (authenticator *jwtAuthenticator) RetrieveIdentity(
return user, nil
}

// remoteFetchJWKS fetches the JWKS from the provided URI.
// remoteFetchJWKS fetches the JWKS URI from the provided issuer URL.
func (authenticator *jwtAuthenticator) remoteFetchJWKS(
ctx context.Context, issuerUrl string,
ctx context.Context, issuerURL string,
) (jwk.Set, error) {
jwksUrl, err := authenticator.getJWKSUrl(ctx, issuerUrl)
var jwksURI string
// if JWKS URI is configured in JWTAuthIssuersConfig use that instead of URL
// from issuer's well-known endpoint
err := authenticator.mu.conf.issuersConf.checkJWKSConfigured()
if err != nil {
return nil, err
jwksURI, err = authenticator.getJWKSURI(ctx, issuerURL)
if err != nil {
return nil, err
}
} else {
jwksURI, err = authenticator.mu.conf.issuersConf.getJWKSURI(issuerURL)
if err != nil {
return nil, err
}
}

body, err := getHttpResponse(ctx, jwksUrl, authenticator)
body, err := getHttpResponse(ctx, jwksURI, authenticator)
if err != nil {
return nil, err
}
@@ -326,8 +328,8 @@ func (authenticator *jwtAuthenticator) remoteFetchJWKS(
return jwkSet, nil
}

// getJWKSUrl returns the JWKS URI from the OpenID configuration endpoint.
func (authenticator *jwtAuthenticator) getJWKSUrl(
// getJWKSURI returns the JWKS URI from the OpenID configuration endpoint.
func (authenticator *jwtAuthenticator) getJWKSURI(
ctx context.Context, issuerUrl string,
) (string, error) {
type OIDCConfigResponse struct {
@@ -389,7 +391,7 @@ var ConfigureJWTAuth = func(
JWTAuthEnabled.SetOnChange(&st.SV, func(ctx context.Context) {
authenticator.reloadConfig(ambientCtx.AnnotateCtx(ctx), st)
})
JWTAuthIssuers.SetOnChange(&st.SV, func(ctx context.Context) {
JWTAuthIssuersConfig.SetOnChange(&st.SV, func(ctx context.Context) {
authenticator.reloadConfig(ambientCtx.AnnotateCtx(ctx), st)
})
JWTAuthJWKS.SetOnChange(&st.SV, func(ctx context.Context) {
152 changes: 130 additions & 22 deletions pkg/ccl/jwtauthccl/authentication_jwt_test.go
Original file line number Diff line number Diff line change
@@ -31,6 +31,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
"github.com/cockroachdb/errors"
"github.com/cockroachdb/errors/oserror"
"github.com/cockroachdb/redact"
"github.com/lestrrat-go/jwx/jwa"
"github.com/lestrrat-go/jwx/jwk"
"github.com/lestrrat-go/jwx/jwt"
@@ -170,13 +171,13 @@ func TestJWTSingleKey(t *testing.T) {
jwkPublicKey := serializePublicKey(t, publicKey)

// Configure issuer as it gets checked even before the token validity check.
JWTAuthIssuers.Override(ctx, &s.ClusterSettings().SV, issuer1)
JWTAuthIssuersConfig.Override(ctx, &s.ClusterSettings().SV, issuer1)

// When JWKSAutoFetchEnabled JWKS fetch should be attempted and fail for configured issuer.
JWKSAutoFetchEnabled.Override(ctx, &s.ClusterSettings().SV, true)
detailedErrorMsg, err := verifier.ValidateJWTLogin(ctx, s.ClusterSettings(), username.MakeSQLUsernameFromPreNormalizedString(invalidUsername), token, identMap)
require.ErrorContains(t, err, "JWT authentication: unable to validate token")
require.EqualValues(t, "unable to fetch jwks: Get \"issuer1/.well-known/openid-configuration\": unsupported protocol scheme \"\"", detailedErrorMsg)
require.EqualValues(t, redact.RedactableString(`unable to fetch jwks: Get "issuer1/.well-known/openid-configuration"›: ‹unsupported protocol scheme ""›`), detailedErrorMsg)

// Set the JWKS cluster setting.
JWKSAutoFetchEnabled.Override(ctx, &s.ClusterSettings().SV, false)
@@ -210,7 +211,7 @@ func TestJWTSingleKeyWithoutKeyAlgorithm(t *testing.T) {
jwkPublicKey := serializePublicKey(t, publicKey)

// Configure issuer as it gets checked even before the token validity check.
JWTAuthIssuers.Override(ctx, &s.ClusterSettings().SV, issuer1)
JWTAuthIssuersConfig.Override(ctx, &s.ClusterSettings().SV, issuer1)

// When JWKSAutoFetchEnabled, JWKS fetch should be attempted and fail for configured issuer.
JWKSAutoFetchEnabled.Override(ctx, &s.ClusterSettings().SV, true)
@@ -240,7 +241,7 @@ func TestJWTMultiKey(t *testing.T) {
// Make sure jwt auth is enabled.
JWTAuthEnabled.Override(ctx, &s.ClusterSettings().SV, true)
// Configure issuer as it gets checked even before the token validity check.
JWTAuthIssuers.Override(ctx, &s.ClusterSettings().SV, issuer1)
JWTAuthIssuersConfig.Override(ctx, &s.ClusterSettings().SV, issuer1)
keySet, key, key2 := createJWKS(t)
token := createJWT(t, username1, audience1, issuer1, timeutil.Now().Add(time.Hour), key2, jwa.ES384, "", "")
publicKey, err := key.PublicKey()
@@ -281,7 +282,7 @@ func TestExpiredToken(t *testing.T) {
// Make sure jwt auth is enabled and accepts valid signing keys.
JWTAuthEnabled.Override(ctx, &s.ClusterSettings().SV, true)
// Configure issuer as it gets checked even before the token validity check.
JWTAuthIssuers.Override(ctx, &s.ClusterSettings().SV, issuer1)
JWTAuthIssuersConfig.Override(ctx, &s.ClusterSettings().SV, issuer1)
keySet, key, _ := createJWKS(t)
token := createJWT(t, username1, audience1, issuer1, timeutil.Now().Add(-1*time.Second), key, jwa.RS256, "", "")
JWTAuthJWKS.Override(ctx, &s.ClusterSettings().SV, serializePublicKeySet(t, keySet))
@@ -312,7 +313,7 @@ func TestKeyIdMismatch(t *testing.T) {
JWTAuthJWKS.Override(ctx, &s.ClusterSettings().SV, serializePublicKeySet(t, keySet))
verifier := ConfigureJWTAuth(ctx, s.AmbientCtx(), s.ClusterSettings(), s.StorageClusterID())
// Configure issuer as it gets checked even before the token validity check.
JWTAuthIssuers.Override(ctx, &s.ClusterSettings().SV, issuer1)
JWTAuthIssuersConfig.Override(ctx, &s.ClusterSettings().SV, issuer1)

// When JWKSAutoFetchEnabled the jwks fetch should be attempted and fail.
_, err = verifier.ValidateJWTLogin(ctx, s.ClusterSettings(), username.MakeSQLUsernameFromPreNormalizedString(username1), token, identMap)
@@ -353,7 +354,7 @@ func TestIssuerCheck(t *testing.T) {
require.ErrorContains(t, err, "JWT authentication: invalid issuer")
require.EqualValues(t, "token issued by issuer1", errors.GetAllDetails(err)[0])

JWTAuthIssuers.Override(ctx, &s.ClusterSettings().SV, issuer2)
JWTAuthIssuersConfig.Override(ctx, &s.ClusterSettings().SV, issuer2)
// Validation fails with an issuer error when the issuer in the token is not in cluster's accepted issuers.
_, err = verifier.ValidateJWTLogin(ctx, s.ClusterSettings(), username.MakeSQLUsernameFromPreNormalizedString(invalidUsername), token1, identMap)
require.ErrorContains(t, err, "JWT authentication: invalid issuer")
@@ -365,7 +366,7 @@ func TestIssuerCheck(t *testing.T) {
require.EqualValues(t, "token issued for [test_user1] and login was for invalid_user", errors.GetAllDetails(err)[0])

// Set the cluster setting to accept issuer values of either "issuer" or "issuer2".
JWTAuthIssuers.Override(ctx, &s.ClusterSettings().SV, "[\""+issuer1+"\", \""+issuer2+"\"]")
JWTAuthIssuersConfig.Override(ctx, &s.ClusterSettings().SV, "[\""+issuer1+"\", \""+issuer2+"\"]")

// Validation succeeds when the issuer in the token is an element of the cluster's accepted issuers.
_, err = verifier.ValidateJWTLogin(ctx, s.ClusterSettings(), username.MakeSQLUsernameFromPreNormalizedString(invalidUsername), token1, identMap)
@@ -395,7 +396,7 @@ func TestSubjectCheck(t *testing.T) {
JWTAuthEnabled.Override(ctx, &s.ClusterSettings().SV, true)
JWTAuthJWKS.Override(ctx, &s.ClusterSettings().SV, serializePublicKeySet(t, keySet))
verifier := ConfigureJWTAuth(ctx, s.AmbientCtx(), s.ClusterSettings(), s.StorageClusterID())
JWTAuthIssuers.Override(ctx, &s.ClusterSettings().SV, issuer2)
JWTAuthIssuersConfig.Override(ctx, &s.ClusterSettings().SV, issuer2)

// Validation fails with a subject error when a user tries to log in with a user named
// "invalid" but the token is for the user "test2".
@@ -427,7 +428,7 @@ func TestClaimMissing(t *testing.T) {
JWTAuthEnabled.Override(ctx, &s.ClusterSettings().SV, true)
JWTAuthJWKS.Override(ctx, &s.ClusterSettings().SV, serializePublicKeySet(t, keySet))
verifier := ConfigureJWTAuth(ctx, s.AmbientCtx(), s.ClusterSettings(), s.StorageClusterID())
JWTAuthIssuers.Override(ctx, &s.ClusterSettings().SV, issuer2)
JWTAuthIssuersConfig.Override(ctx, &s.ClusterSettings().SV, issuer2)
JWTAuthClaim.Override(ctx, &s.ClusterSettings().SV, customClaimName)

// Validation fails with missing claim
@@ -454,7 +455,7 @@ func TestIntegerClaimValue(t *testing.T) {
JWTAuthEnabled.Override(ctx, &s.ClusterSettings().SV, true)
JWTAuthJWKS.Override(ctx, &s.ClusterSettings().SV, serializePublicKeySet(t, keySet))
verifier := ConfigureJWTAuth(ctx, s.AmbientCtx(), s.ClusterSettings(), s.StorageClusterID())
JWTAuthIssuers.Override(ctx, &s.ClusterSettings().SV, issuer2)
JWTAuthIssuersConfig.Override(ctx, &s.ClusterSettings().SV, issuer2)
JWTAuthClaim.Override(ctx, &s.ClusterSettings().SV, customClaimName)

// the integer claim is implicitly cast to a string
@@ -480,7 +481,7 @@ func TestSingleClaim(t *testing.T) {
JWTAuthEnabled.Override(ctx, &s.ClusterSettings().SV, true)
JWTAuthJWKS.Override(ctx, &s.ClusterSettings().SV, serializePublicKeySet(t, keySet))
verifier := ConfigureJWTAuth(ctx, s.AmbientCtx(), s.ClusterSettings(), s.StorageClusterID())
JWTAuthIssuers.Override(ctx, &s.ClusterSettings().SV, issuer2)
JWTAuthIssuersConfig.Override(ctx, &s.ClusterSettings().SV, issuer2)
JWTAuthClaim.Override(ctx, &s.ClusterSettings().SV, customClaimName)
JWTAuthAudience.Override(ctx, &s.ClusterSettings().SV, audience1)

@@ -521,7 +522,7 @@ func TestMultipleClaim(t *testing.T) {
JWTAuthEnabled.Override(ctx, &s.ClusterSettings().SV, true)
JWTAuthJWKS.Override(ctx, &s.ClusterSettings().SV, serializePublicKeySet(t, keySet))
verifier := ConfigureJWTAuth(ctx, s.AmbientCtx(), s.ClusterSettings(), s.StorageClusterID())
JWTAuthIssuers.Override(ctx, &s.ClusterSettings().SV, issuer2)
JWTAuthIssuersConfig.Override(ctx, &s.ClusterSettings().SV, issuer2)
JWTAuthClaim.Override(ctx, &s.ClusterSettings().SV, customClaimName)
JWTAuthAudience.Override(ctx, &s.ClusterSettings().SV, audience1)

@@ -570,7 +571,7 @@ func TestSubjectMappingCheck(t *testing.T) {
JWTAuthEnabled.Override(ctx, &s.ClusterSettings().SV, true)
JWTAuthJWKS.Override(ctx, &s.ClusterSettings().SV, serializePublicKeySet(t, keySet))
verifier := ConfigureJWTAuth(ctx, s.AmbientCtx(), s.ClusterSettings(), s.StorageClusterID())
JWTAuthIssuers.Override(ctx, &s.ClusterSettings().SV, issuer2)
JWTAuthIssuersConfig.Override(ctx, &s.ClusterSettings().SV, issuer2)

// Validation fails with a subject error when a user tries to log in when their user is mapped to username2
// but they try to log in with username1.
@@ -608,7 +609,7 @@ func TestSubjectReservedUser(t *testing.T) {
JWTAuthEnabled.Override(ctx, &s.ClusterSettings().SV, true)
JWTAuthJWKS.Override(ctx, &s.ClusterSettings().SV, serializePublicKeySet(t, keySet))
verifier := ConfigureJWTAuth(ctx, s.AmbientCtx(), s.ClusterSettings(), s.StorageClusterID())
JWTAuthIssuers.Override(ctx, &s.ClusterSettings().SV, "[\""+issuer1+"\", \""+issuer2+"\"]")
JWTAuthIssuersConfig.Override(ctx, &s.ClusterSettings().SV, "[\""+issuer1+"\", \""+issuer2+"\"]")

// You cannot log in as root or other reserved users using token based auth when mapped to root.
_, err = verifier.ValidateJWTLogin(ctx, s.ClusterSettings(), username.MakeSQLUsernameFromPreNormalizedString("root"), token, identMap)
@@ -636,7 +637,7 @@ func TestAudienceCheck(t *testing.T) {
// Make sure jwt auth is enabled and accepts jwk1 or jwk2 as valid signing keys.
JWTAuthEnabled.Override(ctx, &s.ClusterSettings().SV, true)
JWTAuthJWKS.Override(ctx, &s.ClusterSettings().SV, serializePublicKeySet(t, keySet))
JWTAuthIssuers.Override(ctx, &s.ClusterSettings().SV, issuer2)
JWTAuthIssuersConfig.Override(ctx, &s.ClusterSettings().SV, issuer2)

// Set audience field to audience2.
JWTAuthAudience.Override(ctx, &s.ClusterSettings().SV, audience2)
@@ -691,7 +692,7 @@ func createJWKSFromFile(t *testing.T, fileName string) jwk.Set {
return jwkSet
}

// test that jwks url is used when JWKSAutoFetchEnabled is true.
// test that jwks URI is used when JWKSAutoFetchEnabled is true.
func Test_JWKSFetchWorksWhenEnabled(t *testing.T) {
defer leaktest.AfterTest(t)()
// Intercept the call to getHttpResponse and return the mockGetHttpResponse
@@ -709,7 +710,7 @@ func Test_JWKSFetchWorksWhenEnabled(t *testing.T) {
require.NoError(t, err)

// Create key from a file. This key will be used to sign the token.
// Matching public key available in jwks url is used to verify token.
// Matching public key available in jwks URI is used to verify token.
keySet := createJWKSFromFile(t, "testdata/www.idp1apis.com_oauth2_v3_certs_private")
key, _ := keySet.Get(0)
validIssuer := "https://accounts.idp1.com"
@@ -718,7 +719,7 @@ func Test_JWKSFetchWorksWhenEnabled(t *testing.T) {
// Make sure jwt auth is enabled and accepts jwk1 or jwk2 as valid signing keys.
JWTAuthEnabled.Override(ctx, &s.ClusterSettings().SV, true)
//JWTAuthJWKS.Override(ctx, &s.ClusterSettings().SV, serializePublicKeySet(t, keySet))
JWTAuthIssuers.Override(ctx, &s.ClusterSettings().SV, validIssuer)
JWTAuthIssuersConfig.Override(ctx, &s.ClusterSettings().SV, validIssuer)

// Set audience field to audience2.
JWTAuthAudience.Override(ctx, &s.ClusterSettings().SV, audience2)
@@ -745,7 +746,7 @@ func Test_JWKSFetchWorksWhenEnabled(t *testing.T) {
require.NoError(t, err)
}

// test jwks url is used when JWKSAutoFetchEnabled and static jwks ignored.
// test jwks URI is used when JWKSAutoFetchEnabled and static jwks ignored.
func Test_JWKSFetchWorksWhenEnabledIgnoresTheStaticJWKS(t *testing.T) {
defer leaktest.AfterTest(t)()
// Intercept the call to getHttpResponse and return the mockGetHttpResponse
@@ -763,7 +764,7 @@ func Test_JWKSFetchWorksWhenEnabledIgnoresTheStaticJWKS(t *testing.T) {
require.NoError(t, err)

// Create key from a file. This key will be used to sign the token.
// Matching public key available in jwks url is used to verify token.
// Matching public key available in jwks URI is used to verify token.
keySetUsedForSigning := createJWKSFromFile(t, "testdata/www.idp1apis.com_oauth2_v3_certs_private")
key, _ := keySetUsedForSigning.Get(0)
validIssuer := "https://accounts.idp1.com"
@@ -775,7 +776,7 @@ func Test_JWKSFetchWorksWhenEnabledIgnoresTheStaticJWKS(t *testing.T) {
// Configure cluster setting with a key that is not used for signing.
keySetNotUsedForSigning, _, _ := createJWKS(t)
JWTAuthJWKS.Override(ctx, &s.ClusterSettings().SV, serializePublicKeySet(t, keySetNotUsedForSigning))
JWTAuthIssuers.Override(ctx, &s.ClusterSettings().SV, validIssuer)
JWTAuthIssuersConfig.Override(ctx, &s.ClusterSettings().SV, validIssuer)

// Set audience field to audience2.
JWTAuthAudience.Override(ctx, &s.ClusterSettings().SV, audience2)
@@ -824,3 +825,110 @@ func TestJWTAuthCanUseHTTPProxy(t *testing.T) {
require.NoError(t, err)
require.EqualValues(t, "proxied-http://my-server/.well-known/openid-configuration", string(res))
}

func TestJWTAuthWithIssuerJWKSConfAutoFetchJWKS(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)

ctx := context.Background()

// Initiate a test OIDC/JWKS server locally over HTTP.
testServer := httptest.NewUnstartedServer(nil)
testServerURL := "http://" + testServer.Listener.Addr().String()

mux := http.NewServeMux()
mux.HandleFunc(
"/.well-known/openid-configuration",
func(w http.ResponseWriter, r *http.Request) {
// Serve the response locally mocking openid config.
dataBytes := map[string]string{"invalid-key": "invalid-value"}
updatedBytes, _ := json.Marshal(dataBytes)
_, err := w.Write(updatedBytes)
require.NoError(t, err)
},
)
mux.HandleFunc(
"/jwks",
func(w http.ResponseWriter, r *http.Request) {
// Serve the JWKS response locally from testdata.
dataBytes, err := os.ReadFile("testdata/www.idp1apis.com_oauth2_v3_certs")
require.NoError(t, err)
_, err = w.Write(dataBytes)
require.NoError(t, err)
},
)

testServer.Config = &http.Server{
Handler: mux,
}
testServer.Start()
defer testServer.Close()

s := serverutils.StartServerOnly(t, base.TestServerArgs{})
defer s.Stopper().Stop(ctx)

// Create a key to sign the token using testdata.
// The same will be fetched through the JWKS URI to verify the provided token.
keySet := createJWKSFromFile(t, "testdata/www.idp1apis.com_oauth2_v3_certs_private")
key, _ := keySet.Get(0)
issuer := testServerURL
token := createJWT(
t, username1, audience1, issuer, timeutil.Now().Add(time.Hour), key, jwa.RS256, "", "")

JWTAuthEnabled.Override(ctx, &s.ClusterSettings().SV, true)
JWKSAutoFetchEnabled.Override(ctx, &s.ClusterSettings().SV, true)
JWTAuthAudience.Override(ctx, &s.ClusterSettings().SV, audience1)

verifier := ConfigureJWTAuth(ctx, s.AmbientCtx(), s.ClusterSettings(), s.StorageClusterID())
identMapString := ""
identMap, err := identmap.From(strings.NewReader(identMapString))
require.NoError(t, err)

for _, testCase := range []struct {
testName string
issuerConfSetting string
assertFn func(t require.TestingT, err error, msgAndArgs ...interface{})
expectedErr string
expectedErrDetails string
detailedErrMsg string
}{
{
testName: "fail if issuer not set",
issuerConfSetting: "",
assertFn: require.Error,
expectedErr: "JWT authentication: invalid issuer",
expectedErrDetails: "token issued by " + testServerURL,
},
{
testName: "fail if issuer provided without JWKS URI mapping",
issuerConfSetting: testServerURL,
assertFn: require.Error,
expectedErr: "JWT authentication: unable to validate token",
detailedErrMsg: "unable to fetch jwks: no JWKS URI found in OpenID configuration",
},
{
testName: "success if issuer to jwks URI provided",
issuerConfSetting: "{\"issuer_jwks_map\": {\"" + testServerURL + "\": \"" + testServerURL + "/jwks" + "\"}}",
assertFn: require.NoError,
},
} {
t.Run(testCase.testName, func(t *testing.T) {
// set the issuer configuration cluster setting
JWTAuthIssuersConfig.Override(ctx, &s.ClusterSettings().SV, testCase.issuerConfSetting)
detailedErrMsg, err := verifier.ValidateJWTLogin(
ctx,
s.ClusterSettings(),
username.MakeSQLUsernameFromPreNormalizedString(username1),
token,
identMap,
)

testCase.assertFn(t, err)
if err != nil {
require.Equal(t, testCase.expectedErr, err.Error())
require.Equal(t, testCase.expectedErrDetails, errors.FlattenDetails(err))
require.Equal(t, redact.RedactableString(testCase.detailedErrMsg), detailedErrMsg)
}
})
}
}
127 changes: 114 additions & 13 deletions pkg/ccl/jwtauthccl/settings.go
Original file line number Diff line number Diff line change
@@ -12,6 +12,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/settings"
"github.com/cockroachdb/errors"
"github.com/lestrrat-go/jwx/jwk"
"golang.org/x/exp/maps"
)

// All cluster settings necessary for the JWT authentication feature.
@@ -61,28 +62,123 @@ var JWTAuthJWKS = settings.RegisterStringSetting(
settings.WithValidateString(validateJWTAuthJWKS),
)

// JWTAuthIssuers is the list of "issuer" values that are accepted for JWT logins over the SQL interface.
var JWTAuthIssuers = settings.RegisterStringSetting(
// JWTAuthIssuersConfig contains the configuration of all JWT issuers whose
// tokens are allowed for JWT logins over the SQL interface. This can be set to
// one of the following values:
// 1. Simple string that Go can parse as a valid issuer URL.
// 2. String that can be parsed as valid JSON array of issuer URLs list.
// 3. String that can be parsed as valid JSON and deserialized into a map of
// issuer URLs to corresponding JWKS URIs.
// In the third case we will be overriding the JWKS URI present in the issuer's
// well-known endpoint.
// Example valid values:
// - 'https://accounts.google.com'
// - ['example.com/adfs','https://accounts.google.com']
// - '{
// "issuer_jwks_map": {
// "https://accounts.google.com": "https://www.googleapis.com/oauth2/v3/certs",
// "example.com/adfs": "https://example.com/adfs/discovery/keys"
// }
// }'
//
// When issuer_jwks_map is set, we directly use the JWKS URI to get the key set.
// In all other cases where JWKSAutoFetchEnabled is set we obtain the JWKS URI
// first from issuer's well-known endpoint and the use this endpoint.
var JWTAuthIssuersConfig = settings.RegisterStringSetting(
settings.ApplicationLevel,
JWTAuthIssuersSettingName,
"sets accepted issuer values for JWT logins over the SQL interface either as a string or as a JSON "+
"string with an array of issuer strings in it",
"sets accepted issuer values for JWT logins over the SQL interface which can "+
"be a single issuer URL string or a JSON string containing an array of "+
"issuer URLs or a JSON object containing map of issuer URLS to JWKS URIs",
"",
settings.WithValidateString(validateJWTAuthIssuers),
settings.WithValidateString(validateJWTAuthIssuersConf),
)

// JWKSAutoFetchEnabled enables or disables automatic fetching of JWKs from the issuer's well-known endpoint.
var JWKSAutoFetchEnabled = settings.RegisterBoolSetting(
settings.ApplicationLevel,
JWKSAutoFetchEnabledSettingName,
"enables or disables automatic fetching of JWKs from the issuer's well-known endpoint. "+
"If this is enabled, the server.jwt_authentication.jwks will be ignored.",
"enables or disables automatic fetching of JWKS from the issuer's well-known "+
"endpoint or JWKS URI set in JWTAuthIssuersConfig. If this is enabled, the "+
"server.jwt_authentication.jwks will be ignored.",
false,
settings.WithReportable(true),
)

func validateJWTAuthIssuers(values *settings.Values, s string) error {
// getJSONDecoder generates a new decoder from provided json string. This is
// necessary as currently the offset for decoder can't be reset after Decode().
func getJSONDecoder(s string) *json.Decoder {
decoder := json.NewDecoder(bytes.NewReader([]byte(s)))
decoder.DisallowUnknownFields()
return decoder
}

type issuerURLConf struct {
ijMap *issuerJWKSMap
issuers []string
}

func (conf *issuerURLConf) checkIssuerConfigured(issuer string) error {
issuerMatch := false
for idx := range conf.issuers {
if issuer == conf.issuers[idx] {
issuerMatch = true
break
}
}
if !issuerMatch {
return errors.Newf("JWT authentication: invalid issuer")
}
return nil
}

func (conf *issuerURLConf) checkJWKSConfigured() error {
if conf.ijMap == nil || len(conf.ijMap.Mappings) == 0 {
return errors.Newf("JWT authentication: no jwks mappings configured")
}
return nil
}

func (conf *issuerURLConf) getJWKSURI(issuer string) (jwksURI string, err error) {
var ok bool
if jwksURI, ok = conf.ijMap.Mappings[issuer]; !ok {
return "", errors.Newf("JWT authentication: no jwks uri set for issuer")
}
return jwksURI, nil
}

// issuerJWKSMap is a struct that defines a valid JSON body for the
// OIDCRedirectURL cluster setting in multi-region environments.
type issuerJWKSMap struct {
Mappings map[string]string `json:"issuer_jwks_map"`
}

// mustParseJWTIssuersConf will read in a string that's from the
// JWTAuthIssuersConfig setting. We know from the validation that runs on that
// setting that any value that's not valid JSON that deserializes into the
// issuerJWKSMap struct will be either a list of issuer URLs or a single issuer
// URL which will populate and return issuerURLConf.
func mustParseJWTIssuersConf(s string) issuerURLConf {
var ijMap = issuerJWKSMap{}
var issuers []string
decoder := getJSONDecoder(s)
err := decoder.Decode(&ijMap)
if err == nil {
issuers = append(issuers, maps.Keys(ijMap.Mappings)...)
return issuerURLConf{ijMap: &ijMap, issuers: issuers}
}

decoder = getJSONDecoder(s)
err = decoder.Decode(&issuers)
if err == nil {
return issuerURLConf{issuers: issuers}
}
return issuerURLConf{issuers: []string{s}}
}

func validateJWTAuthIssuersConf(values *settings.Values, s string) error {
var issuers []string
var ijMap = issuerJWKSMap{}

var jsonCheck json.RawMessage
if json.Unmarshal([]byte(s), &jsonCheck) != nil {
@@ -91,10 +187,15 @@ func validateJWTAuthIssuers(values *settings.Values, s string) error {
return nil
}

decoder := json.NewDecoder(bytes.NewReader([]byte(s)))
decoder.DisallowUnknownFields()
if err := decoder.Decode(&issuers); err != nil {
return errors.Wrap(err, "JWT authentication issuers JSON not valid")
decoder := getJSONDecoder(s)
issuerListErr := decoder.Decode(&issuers)
decoder = getJSONDecoder(s)
issuerJWKSMapErr := decoder.Decode(&ijMap)
if issuerListErr != nil && issuerJWKSMapErr != nil {
return errors.Wrap(
errors.Join(issuerListErr, issuerJWKSMapErr),
"JWT authentication: issuers JSON not valid",
)
}
return nil
}
@@ -130,7 +231,7 @@ func mustParseValueOrArray(rawString string) []string {
var jsonCheck json.RawMessage
if json.Unmarshal([]byte(rawString), &jsonCheck) != nil {
// If we know the string is *not* valid JSON, fall back to assuming basic
// string to use a single valid.
// string to use a single valid issuer.
return []string{rawString}
}

68 changes: 40 additions & 28 deletions pkg/ccl/jwtauthccl/settings_test.go
Original file line number Diff line number Diff line change
@@ -15,41 +15,53 @@ import (
func TestValidateAndParseJWTAuthIssuers(t *testing.T) {
defer leaktest.AfterTest(t)()
tests := []struct {
name string
setting string
wantErr bool
expectedIssuers []string
name string
setting string
wantErr bool
expectedIssuers []string
expectedIssuerJWKSMap map[string]string
}{
{"empty string",
"", false,
[]string{""}},
{"string constant",
"issuer1", false,
[]string{"issuer1"}},
{"odd string constant",
"issuer1{}`[]!@#%#^$&*", false,
[]string{"issuer1{}`[]!@#%#^$&*"}},
{"empty json",
"[]", false,
[]string{}},
{"single element json",
"[\"issuer 1\"]", false,
[]string{"issuer 1"}},
{"multiple element json",
"[\"issuer 1\", \"issuer 2\", \"issuer 3\", \"issuer 4\", \"issuer 5\"]", false,
[]string{"issuer 1", "issuer 2", "issuer 3", "issuer 4", "issuer 5"}},
{"json but invalid in this context",
"{\"redirect_urls\": {\"key\":\"http://example.com\"}}", true,
nil},
{name: "empty string",
expectedIssuers: []string{""}},
{name: "string constant",
setting: "issuer1",
expectedIssuers: []string{"issuer1"}},
{name: "odd string constant",
setting: "issuer1{}`[]!@#%#^$&*",
expectedIssuers: []string{"issuer1{}`[]!@#%#^$&*"}},
{name: "empty json",
setting: "[]",
expectedIssuers: []string{}},
{name: "single element json",
setting: "[\"issuer 1\"]",
expectedIssuers: []string{"issuer 1"}},
{name: "multiple element json",
setting: "[\"issuer 1\", \"issuer 2\", \"issuer 3\", \"issuer 4\", \"issuer 5\"]",
expectedIssuers: []string{"issuer 1", "issuer 2", "issuer 3", "issuer 4", "issuer 5"}},
{name: "json but invalid in this context",
setting: "{\"redirect_urls\": {\"key\":\"http://example.com\"}}", wantErr: true},
{name: "json object valid in this context single mapping",
setting: "{\"issuer_jwks_map\": {\"https://accounts.google.com\": \"https://www.googleapis.com/oauth2/v3/certs\"}}",
expectedIssuers: []string{"https://accounts.google.com"},
expectedIssuerJWKSMap: map[string]string{"https://accounts.google.com": "https://www.googleapis.com/oauth2/v3/certs"}},
{name: "json object valid in this context multiple mapping",
setting: "{\"issuer_jwks_map\": {\"https://accounts.google.com\": \"https://www.googleapis.com/oauth2/v3/certs\"" +
",\"example.com/adfs\": \"https://example.com/adfs/discovery/keys\"}}",
expectedIssuers: []string{"https://accounts.google.com", "example.com/adfs"},
expectedIssuerJWKSMap: map[string]string{"https://accounts.google.com": "https://www.googleapis.com/oauth2/v3/certs",
"example.com/adfs": "https://example.com/adfs/discovery/keys"}},
}
for _, tt := range tests {
t.Run(tt.name, func(t *testing.T) {
if err := validateJWTAuthIssuers(nil, tt.setting); (err != nil) != tt.wantErr {
if err := validateJWTAuthIssuersConf(nil, tt.setting); (err != nil) != tt.wantErr {
t.Errorf("validateJWTAuthIssuers() error = %v, wantErr %v", err, tt.wantErr)
}
if !tt.wantErr {
parsedIssuers := mustParseValueOrArray(tt.setting)
require.Equal(t, parsedIssuers, tt.expectedIssuers)
parsedIssuersConf := mustParseJWTIssuersConf(tt.setting)
require.ElementsMatch(t, parsedIssuersConf.issuers, tt.expectedIssuers)
if parsedIssuersConf.ijMap != nil {
require.Equal(t, parsedIssuersConf.ijMap.Mappings, tt.expectedIssuerJWKSMap)
}
}
})
}
11 changes: 1 addition & 10 deletions pkg/ccl/testccl/authccl/auth_test.go
Original file line number Diff line number Diff line change
@@ -48,7 +48,6 @@ import (
)

// The code in this file takes inspiration from pgwire/auth_test.go

// TestAuthenticationAndHBARules exercises the authentication code
// using datadriven testing.
//
@@ -109,7 +108,6 @@ import (
func TestAuthenticationAndHBARules(t *testing.T) {
defer leaktest.AfterTest(t)()
skip.UnderRace(t, "takes >1min under race")

testutils.RunTrueAndFalse(t, "insecure", func(t *testing.T, insecure bool) {
jwtRunTest(t, insecure)
})
@@ -153,12 +151,10 @@ func jwtRunTest(t *testing.T, insecure bool) {

maybeSocketDir, maybeSocketFile, cleanup := makeSocketFile(t)
defer cleanup()

// We really need to have the logs go to files, so that -show-logs
// does not break the "authlog" directives.
sc := log.ScopeWithoutShowLogs(t)
defer sc.Close(t)

// Enable logging channels.
log.TestingResetActive()
cfg := logconfig.DefaultConfig()
@@ -231,7 +227,7 @@ func jwtRunTest(t *testing.T, insecure bool) {
if len(a.Vals) != 1 {
t.Fatalf("wrong number of argumenets to jwt_cluster_setting issuers: %d", len(a.Vals))
}
jwtauthccl.JWTAuthIssuers.Override(ctx, sv, a.Vals[0])
jwtauthccl.JWTAuthIssuersConfig.Override(ctx, sv, a.Vals[0])
case "jwks":
if len(a.Vals) != 1 {
t.Fatalf("wrong number of argumenets to jwt_cluster_setting jwks: %d", len(a.Vals))
@@ -289,29 +285,25 @@ func jwtRunTest(t *testing.T, insecure bool) {
// Unix sockets not supported; assume the test succeeded.
return td.Expected, nil
}

// Prepare a connection string using the server's default.
// What is the user requested by the test?
user := username.RootUser
if td.HasArg("user") {
td.ScanArgs(t, "user", &user)
}

// Allow connections for non-root, non-testuser to force the
// use of client certificates.
forceCerts := false
if td.HasArg("force_certs") {
forceCerts = true
}

// We want the certs to be present in the filesystem for this test.
// However, certs are only generated for users "root" and "testuser" specifically.
sqlURL, cleanupFn := s.PGUrl(
t, serverutils.CertsDirPrefix(t.Name()), serverutils.User(user),
serverutils.ClientCerts(forceCerts || user == username.RootUser || user == username.TestUser),
)
defer cleanupFn()

var host, port string
if td.Cmd == "connect" {
host, port, err = net.SplitHostPort(s.AdvSQLAddr())
@@ -326,7 +318,6 @@ func jwtRunTest(t *testing.T, insecure bool) {
if err != nil {
t.Fatal(err)
}

// Here we make use of the fact that pq accepts connection
// strings using the alternate postgres configuration format,
// consisting of k=v pairs separated by spaces.
4 changes: 2 additions & 2 deletions pkg/sql/pgwire/auth_methods.go
Original file line number Diff line number Diff line change
@@ -694,7 +694,7 @@ type JWTVerifier interface {
_ username.SQLUsername,
_ []byte,
_ *identmap.Conf,
) (detailedErrorMsg string, authError error)
) (detailedErrorMsg redact.RedactableString, authError error)

// RetrieveIdentity retrieves the user identity from the JWT.
//
@@ -716,7 +716,7 @@ type noJWTConfigured struct{}

func (c *noJWTConfigured) ValidateJWTLogin(
_ context.Context, _ *cluster.Settings, _ username.SQLUsername, _ []byte, _ *identmap.Conf,
) (detailedErrorMsg string, authError error) {
) (detailedErrorMsg redact.RedactableString, authError error) {
return "", errors.New("JWT token authentication requires CCL features")
}

0 comments on commit 9577e2f

Please sign in to comment.